diff --git a/Cargo.lock b/Cargo.lock index b1ac42becd271..89b3fec5a38db 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8625,6 +8625,7 @@ dependencies = [ name = "risingwave_bench" version = "1.7.0-alpha" dependencies = [ + "anyhow", "async-trait", "aws-config", "aws-sdk-s3", @@ -8691,6 +8692,7 @@ dependencies = [ "console", "const-str", "expect-test", + "home", "madsim-tokio", "prometheus", "risingwave_cmd", @@ -8841,6 +8843,7 @@ name = "risingwave_common_proc_macro" version = "1.7.0-alpha" dependencies = [ "bae", + "itertools 0.12.0", "proc-macro-error 1.0.4", "proc-macro2", "quote", @@ -9288,6 +9291,7 @@ dependencies = [ "madsim-tonic", "maplit", "md5", + "memcomparable", "num-integer", "parking_lot 0.12.1", "parse-display", diff --git a/lints/Cargo.lock b/lints/Cargo.lock index 5cd984d3cac47..5251d536cd1aa 100644 --- a/lints/Cargo.lock +++ b/lints/Cargo.lock @@ -588,6 +588,7 @@ dependencies = [ name = "lints" version = "0.1.0" dependencies = [ + "anyhow", "clippy_utils", "dylint_linting", "dylint_testing", diff --git a/lints/Cargo.toml b/lints/Cargo.toml index 7974f6970bd90..14019f167147b 100644 --- a/lints/Cargo.toml +++ b/lints/Cargo.toml @@ -22,6 +22,7 @@ itertools = "0.12" dylint_testing = "2.6.0" # UI test dependencies +anyhow = "1" tracing = "0.1" [package.metadata.rust-analyzer] diff --git a/lints/src/format_error.rs b/lints/src/format_error.rs index b090388d72fca..0d1df649460e8 100644 --- a/lints/src/format_error.rs +++ b/lints/src/format_error.rs @@ -62,7 +62,8 @@ impl_lint_pass!(FormatError => [FORMAT_ERROR]); const TRACING_FIELD_DEBUG: [&str; 3] = ["tracing_core", "field", "debug"]; const TRACING_FIELD_DISPLAY: [&str; 3] = ["tracing_core", "field", "display"]; -const TRACING_MACRO_EVENT: [&str; 3] = ["tracing", "macros", "event"]; +const TRACING_MACROS_EVENT: [&str; 3] = ["tracing", "macros", "event"]; +const ANYHOW_MACROS_ANYHOW: [&str; 3] = ["anyhow", "macros", "anyhow"]; impl<'tcx> LateLintPass<'tcx> for FormatError { fn check_expr(&mut self, cx: &LateContext<'tcx>, expr: &'tcx Expr<'_>) { @@ -76,12 +77,14 @@ impl<'tcx> LateLintPass<'tcx> for FormatError { .or_else(|| match_function_call(cx, expr, &TRACING_FIELD_DISPLAY)) && let [arg_expr, ..] = args { - check_fmt_arg(cx, arg_expr); + check_fmt_arg_in_tracing_event(cx, arg_expr); } - // `{}`, `{:?}` in format macros. - let in_tracing_macro_event = macro_backtrace(expr.span) - .any(|macro_call| match_def_path(cx, macro_call.def_id, &TRACING_MACRO_EVENT)); + // Indirect `{}`, `{:?}` from other macros. + let in_tracing_event_macro = macro_backtrace(expr.span) + .any(|macro_call| match_def_path(cx, macro_call.def_id, &TRACING_MACROS_EVENT)); + let in_anyhow_macro = macro_backtrace(expr.span) + .any(|macro_call| match_def_path(cx, macro_call.def_id, &ANYHOW_MACROS_ANYHOW)); for macro_call in macro_backtrace(expr.span) { if is_format_macro(cx, macro_call.def_id) @@ -93,8 +96,14 @@ impl<'tcx> LateLintPass<'tcx> for FormatError { && let Some(arg) = format_args.arguments.all_args().get(index) && let Ok(arg_expr) = find_format_arg_expr(expr, arg) { - if in_tracing_macro_event { + if in_tracing_event_macro { check_fmt_arg_in_tracing_event(cx, arg_expr); + } else if in_anyhow_macro { + if format_args.template.len() == 1 { + check_fmt_arg_in_anyhow_error(cx, arg_expr); + } else { + check_fmt_arg_in_anyhow_context(cx, arg_expr); + } } else { check_fmt_arg(cx, arg_expr); } @@ -113,25 +122,44 @@ impl<'tcx> LateLintPass<'tcx> for FormatError { } fn check_fmt_arg(cx: &LateContext<'_>, arg_expr: &Expr<'_>) { - check_arg( + check_fmt_arg_with_help( cx, arg_expr, - arg_expr.span, "consider importing `thiserror_ext::AsReport` and using `.as_report()` instead", - ); + ) } fn check_fmt_arg_in_tracing_event(cx: &LateContext<'_>, arg_expr: &Expr<'_>) { // TODO: replace `` with the actual code snippet. - check_arg( + check_fmt_arg_with_help( + cx, + arg_expr, + "consider importing `thiserror_ext::AsReport` and recording the error as a field \ + with `error = %.as_report()` instead", + ); +} + +fn check_fmt_arg_in_anyhow_error(cx: &LateContext<'_>, arg_expr: &Expr<'_>) { + check_fmt_arg_with_help( cx, arg_expr, - arg_expr.span, - "consider importing `thiserror_ext::AsReport` and recording the error as a field -with `error = %.as_report()` instead", + "consider directly wrapping the error with `anyhow::anyhow!(..)` instead of formatting it", ); } +fn check_fmt_arg_in_anyhow_context(cx: &LateContext<'_>, arg_expr: &Expr<'_>) { + check_fmt_arg_with_help( + cx, + arg_expr, + "consider using `anyhow::Error::context`, `anyhow::Context::(with_)context` to \ + attach additional message to the error and make it an error source instead", + ); +} + +fn check_fmt_arg_with_help(cx: &LateContext<'_>, arg_expr: &Expr<'_>, help: &str) { + check_arg(cx, arg_expr, arg_expr.span, help); +} + fn check_to_string_call(cx: &LateContext<'_>, receiver: &Expr<'_>, to_string_span: Span) { check_arg( cx, diff --git a/lints/ui/format_error.rs b/lints/ui/format_error.rs index 1cdc112bd93aa..eeead1306ea3f 100644 --- a/lints/ui/format_error.rs +++ b/lints/ui/format_error.rs @@ -48,4 +48,11 @@ fn main() { let _ = (err.clone()).to_string(); let _ = err.to_string().to_string(); let _ = (&&err).to_string(); + + use anyhow::anyhow; + + let _ = anyhow!("{}", err); + let _ = anyhow!("{:?}", err); + let _ = anyhow!("some error occurred: {}", err); + let _ = anyhow!("some error occurred: {:?}", err); } diff --git a/lints/ui/format_error.stderr b/lints/ui/format_error.stderr index 5ee0c1bdd9ec1..8ec6e69b7fcf4 100644 --- a/lints/ui/format_error.stderr +++ b/lints/ui/format_error.stderr @@ -118,8 +118,7 @@ error: should not format error directly LL | info!("{}", err); | ^^^ | - = help: consider importing `thiserror_ext::AsReport` and recording the error as a field - with `error = %.as_report()` instead + = help: consider importing `thiserror_ext::AsReport` and recording the error as a field with `error = %.as_report()` instead error: should not format error directly --> $DIR/format_error.rs:32:20 @@ -127,8 +126,7 @@ error: should not format error directly LL | my_info!("{}", err); | ^^^ | - = help: consider importing `thiserror_ext::AsReport` and recording the error as a field - with `error = %.as_report()` instead + = help: consider importing `thiserror_ext::AsReport` and recording the error as a field with `error = %.as_report()` instead error: should not format error directly --> $DIR/format_error.rs:34:29 @@ -136,7 +134,7 @@ error: should not format error directly LL | tracing::field::display(&err); | ^^^^ | - = help: consider importing `thiserror_ext::AsReport` and using `.as_report()` instead + = help: consider importing `thiserror_ext::AsReport` and recording the error as a field with `error = %.as_report()` instead error: should not format error directly --> $DIR/format_error.rs:35:27 @@ -144,7 +142,7 @@ error: should not format error directly LL | tracing::field::debug(err.clone()); | ^^^^^^^^^^^ | - = help: consider importing `thiserror_ext::AsReport` and using `.as_report()` instead + = help: consider importing `thiserror_ext::AsReport` and recording the error as a field with `error = %.as_report()` instead error: should not format error directly --> $DIR/format_error.rs:37:5 @@ -152,7 +150,7 @@ error: should not format error directly LL | info!(%err, "233"); | ^^^^^^^^^^^^^^^^^^ | - = help: consider importing `thiserror_ext::AsReport` and using `.as_report()` instead + = help: consider importing `thiserror_ext::AsReport` and recording the error as a field with `error = %.as_report()` instead error: should not format error directly --> $DIR/format_error.rs:38:5 @@ -160,7 +158,7 @@ error: should not format error directly LL | info!(?err, "233"); | ^^^^^^^^^^^^^^^^^^ | - = help: consider importing `thiserror_ext::AsReport` and using `.as_report()` instead + = help: consider importing `thiserror_ext::AsReport` and recording the error as a field with `error = %.as_report()` instead error: should not format error directly --> $DIR/format_error.rs:39:23 @@ -168,8 +166,7 @@ error: should not format error directly LL | info!(%err, "{}", err); | ^^^ | - = help: consider importing `thiserror_ext::AsReport` and recording the error as a field - with `error = %.as_report()` instead + = help: consider importing `thiserror_ext::AsReport` and recording the error as a field with `error = %.as_report()` instead error: should not format error directly --> $DIR/format_error.rs:39:5 @@ -177,7 +174,7 @@ error: should not format error directly LL | info!(%err, "{}", err); | ^^^^^^^^^^^^^^^^^^^^^^ | - = help: consider importing `thiserror_ext::AsReport` and using `.as_report()` instead + = help: consider importing `thiserror_ext::AsReport` and recording the error as a field with `error = %.as_report()` instead error: should not format error directly --> $DIR/format_error.rs:40:13 @@ -185,7 +182,7 @@ error: should not format error directly LL | let _ = info_span!("span", %err); | ^^^^^^^^^^^^^^^^^^^^^^^^ | - = help: consider importing `thiserror_ext::AsReport` and using `.as_report()` instead + = help: consider importing `thiserror_ext::AsReport` and recording the error as a field with `error = %.as_report()` instead error: should not format error directly --> $DIR/format_error.rs:44:9 @@ -243,5 +240,37 @@ LL | let _ = (&&err).to_string(); | = help: consider importing `thiserror_ext::AsReport` and using `.to_report_string()` instead -error: aborting due to 30 previous errors +error: should not format error directly + --> $DIR/format_error.rs:54:27 + | +LL | let _ = anyhow!("{}", err); + | ^^^ + | + = help: consider directly wrapping the error with `anyhow::anyhow!(..)` instead of formatting it + +error: should not format error directly + --> $DIR/format_error.rs:55:29 + | +LL | let _ = anyhow!("{:?}", err); + | ^^^ + | + = help: consider directly wrapping the error with `anyhow::anyhow!(..)` instead of formatting it + +error: should not format error directly + --> $DIR/format_error.rs:56:48 + | +LL | let _ = anyhow!("some error occurred: {}", err); + | ^^^ + | + = help: consider using `anyhow::Error::context`, `anyhow::Context::(with_)context` to attach additional message to the error and make it an error source instead + +error: should not format error directly + --> $DIR/format_error.rs:57:50 + | +LL | let _ = anyhow!("some error occurred: {:?}", err); + | ^^^ + | + = help: consider using `anyhow::Error::context`, `anyhow::Context::(with_)context` to attach additional message to the error and make it an error source instead + +error: aborting due to 34 previous errors diff --git a/proto/plan_common.proto b/proto/plan_common.proto index c0aecddb2261f..82f9fbc63a0f8 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -61,7 +61,7 @@ message ColumnDesc { ColumnDescVersion version = 10; - AdditionalColumn additional_columns = 11; + AdditionalColumn additional_column = 11; } message ColumnCatalog { diff --git a/proto/telemetry.proto b/proto/telemetry.proto index 2da24cc823085..3fcf3911ea361 100644 --- a/proto/telemetry.proto +++ b/proto/telemetry.proto @@ -2,6 +2,8 @@ syntax = "proto3"; package telemetry; +option go_package = "risingwavelabs.com/risingwave/proto/telemetry"; + enum MetaBackend { META_BACKEND_UNSPECIFIED = 0; META_BACKEND_MEMORY = 1; diff --git a/src/batch/clippy.toml b/src/batch/clippy.toml index aeb91fb713fb1..df03c275d0dee 100644 --- a/src/batch/clippy.toml +++ b/src/batch/clippy.toml @@ -1,11 +1,6 @@ -disallowed-methods = [ -] +disallowed-methods = [] -disallowed-types = [ - { path = "risingwave_common::error::ErrorCode", reason = "Please use per-crate error type instead." }, - { path = "risingwave_common::error::RwError", reason = "Please use per-crate error type instead." }, - { path = "risingwave_common::error::Result", reason = "Please use per-crate error type instead." }, -] +disallowed-types = [] doc-valid-idents = [ "RisingWave", @@ -16,7 +11,7 @@ doc-valid-idents = [ "PostgreSQL", "MySQL", "TopN", - "VNode" + "VNode", ] avoid-breaking-exported-api = false upper-case-acronyms-aggressive = true diff --git a/src/batch/src/error.rs b/src/batch/src/error.rs index 1cbfc7247827c..5631707e2f422 100644 --- a/src/batch/src/error.rs +++ b/src/batch/src/error.rs @@ -18,7 +18,7 @@ use std::sync::Arc; pub use anyhow::anyhow; use risingwave_common::array::ArrayError; -use risingwave_common::error::{BoxedError, ErrorCode, RwError}; +use risingwave_common::error::BoxedError; use risingwave_common::util::value_encoding::error::ValueEncodingError; use risingwave_dml::error::DmlError; use risingwave_expr::ExprError; @@ -145,19 +145,6 @@ impl From for BatchError { } } -impl From for RwError { - fn from(s: BatchError) -> Self { - ErrorCode::BatchError(Box::new(s)).into() - } -} - -// TODO(error-handling): remove after eliminating RwError from connector. -impl From for BatchError { - fn from(s: RwError) -> Self { - Self::Internal(anyhow!(s)) - } -} - impl<'a> From<&'a BatchError> for Status { fn from(err: &'a BatchError) -> Self { err.to_status(tonic::Code::Internal, "batch") diff --git a/src/bench/Cargo.toml b/src/bench/Cargo.toml index 86e88b53a6149..63e8d89844c54 100644 --- a/src/bench/Cargo.toml +++ b/src/bench/Cargo.toml @@ -8,6 +8,7 @@ license = { workspace = true } repository = { workspace = true } [dependencies] +anyhow = "1" async-trait = "0.1" aws-config = { workspace = true } aws-sdk-s3 = { workspace = true } diff --git a/src/bench/s3_bench/main.rs b/src/bench/s3_bench/main.rs index 8eec3e6cfbeea..792c9c4743dbf 100644 --- a/src/bench/s3_bench/main.rs +++ b/src/bench/s3_bench/main.rs @@ -28,7 +28,6 @@ use futures::stream::{self, StreamExt}; use futures::{future, Future, FutureExt}; use itertools::Itertools; use rand::{Rng, SeedableRng}; -use risingwave_common::error::RwError; use tokio::join; use tokio::sync::RwLock; use tracing::debug; @@ -233,7 +232,7 @@ async fn multi_part_upload( let part_t = Instant::now(); let result = a.send().await.unwrap(); let part_ttl = part_t.elapsed(); - Ok::<_, RwError>((result, part_ttl)) + Ok::<_, anyhow::Error>((result, part_ttl)) }) .collect_vec(); let ttfb = t.elapsed(); @@ -318,7 +317,7 @@ async fn multi_part_get( .into_iter() .map(create_part_get) .map(|resp| async move { - let result: Result<(usize, Duration), RwError> = Ok(( + let result: anyhow::Result<(usize, Duration)> = Ok(( resp.await .unwrap() .body @@ -381,7 +380,7 @@ async fn run_case( cfg: Arc, client: Arc, objs: Arc>, -) -> Result<(), RwError> { +) -> anyhow::Result<()> { let (name, analysis) = match case.clone() { Case::Put { name, diff --git a/src/bench/sink_bench/main.rs b/src/bench/sink_bench/main.rs index 067460db830db..66572ca67cdf2 100644 --- a/src/bench/sink_bench/main.rs +++ b/src/bench/sink_bench/main.rs @@ -19,6 +19,7 @@ use core::str::FromStr; use std::collections::HashMap; +use anyhow::anyhow; use clap::Parser; use futures::prelude::future::Either; use futures::prelude::stream::{BoxStream, PollNext}; @@ -27,7 +28,6 @@ use futures::{FutureExt, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::ColumnId; -use risingwave_common::error::anyhow_error; use risingwave_connector::dispatch_sink; use risingwave_connector::parser::{ EncodingProperties, ParserConfig, ProtocolProperties, SpecificParserConfig, @@ -82,7 +82,7 @@ impl LogReader for MockRangeLogReader { .take() .unwrap() .send(self.throughput_metric.take().unwrap()) - .map_err(|_| anyhow_error!("Can't send throughput_metric"))?; + .map_err(|_| anyhow!("Can't send throughput_metric"))?; futures::future::pending().await }, item = self.upstreams.next() => { @@ -108,7 +108,7 @@ impl LogReader for MockRangeLogReader { }, )) } - _ => Err(anyhow_error!("Can't assert message type".to_string())), + _ => Err(anyhow!("Can't assert message type".to_string())), } } } @@ -390,7 +390,7 @@ fn mock_from_legacy_type( SINK_TYPE_APPEND_ONLY => SinkFormat::AppendOnly, SINK_TYPE_UPSERT => SinkFormat::Upsert, _ => { - return Err(SinkError::Config(risingwave_common::array::error::anyhow!( + return Err(SinkError::Config(anyhow!( "sink type unsupported: {}", r#type ))) diff --git a/src/cmd_all/Cargo.toml b/src/cmd_all/Cargo.toml index f5a08e6c4b688..bb57fbfe88a09 100644 --- a/src/cmd_all/Cargo.toml +++ b/src/cmd_all/Cargo.toml @@ -23,6 +23,7 @@ anyhow = "1" clap = { version = "4", features = ["cargo", "derive"] } console = "0.15" const-str = "0.5" +home = "0.5" prometheus = { version = "0.13" } risingwave_cmd = { workspace = true } risingwave_common = { workspace = true } diff --git a/src/cmd_all/src/bin/risingwave.rs b/src/cmd_all/src/bin/risingwave.rs index a1e3a1b5f7063..2c167fc1bdc20 100644 --- a/src/cmd_all/src/bin/risingwave.rs +++ b/src/cmd_all/src/bin/risingwave.rs @@ -17,9 +17,10 @@ use std::str::FromStr; use anyhow::Result; +use clap::error::ErrorKind; use clap::{command, ArgMatches, Args, Command, FromArgMatches}; use risingwave_cmd::{compactor, compute, ctl, frontend, meta}; -use risingwave_cmd_all::{PlaygroundOpts, StandaloneOpts}; +use risingwave_cmd_all::{PlaygroundOpts, SingleNodeOpts, StandaloneOpts}; use risingwave_common::git_sha; use risingwave_compactor::CompactorOpts; use risingwave_compute::ComputeNodeOpts; @@ -98,7 +99,14 @@ enum Component { Compactor, Ctl, Playground, + /// Used by cloud to bundle different components into a single node. + /// It exposes the low level configuration options of each node. Standalone, + /// Used by users to run a single node. + /// The low level configuration options are hidden. + /// We only expose high-level configuration options, + /// which map across multiple nodes. + SingleNode, } impl Component { @@ -117,6 +125,7 @@ impl Component { Self::Ctl => ctl(parse_opts(matches)), Self::Playground => playground(parse_opts(matches)), Self::Standalone => standalone(parse_opts(matches)), + Self::SingleNode => single_node(parse_opts(matches)), } } @@ -130,6 +139,7 @@ impl Component { Component::Ctl => vec!["risectl"], Component::Playground => vec!["play"], Component::Standalone => vec![], + Component::SingleNode => vec!["single-node", "single"], } } @@ -143,6 +153,7 @@ impl Component { Component::Ctl => CtlOpts::augment_args(cmd), Component::Playground => PlaygroundOpts::augment_args(cmd), Component::Standalone => StandaloneOpts::augment_args(cmd), + Component::SingleNode => SingleNodeOpts::augment_args(cmd), } } @@ -179,7 +190,23 @@ fn main() -> Result<()> { .subcommands(Component::commands()), ); - let matches = command.get_matches(); + let matches = match command.try_get_matches() { + Ok(m) => m, + Err(e) if e.kind() == ErrorKind::MissingSubcommand => { + // `$ ./risingwave` + // NOTE(kwannoel): This is a hack to make `risingwave` + // work as an alias of `risingwave single-process`. + // If invocation is not a multicall and there's no subcommand, + // we will try to invoke it as a single node. + let command = Component::SingleNode.augment_args(risingwave()); + let matches = command.get_matches(); + Component::SingleNode.start(&matches); + return Ok(()); + } + Err(e) => { + e.exit(); + } + }; let multicall = matches.subcommand().unwrap(); let argv_1 = multicall.1.subcommand(); @@ -207,3 +234,15 @@ fn standalone(opts: StandaloneOpts) { risingwave_rt::init_risingwave_logger(settings); risingwave_rt::main_okk(risingwave_cmd_all::standalone(opts)).unwrap(); } + +/// For single node, the internals are just a config mapping from its +/// high level options to standalone mode node-level options. +/// We will start a standalone instance, with all nodes in the same process. +fn single_node(opts: SingleNodeOpts) { + let opts = risingwave_cmd_all::map_single_node_opts_to_standalone_opts(&opts); + let settings = risingwave_rt::LoggerSettings::from_opts(&opts) + .with_target("risingwave_storage", Level::WARN) + .with_thread_name(true); + risingwave_rt::init_risingwave_logger(settings); + risingwave_rt::main_okk(risingwave_cmd_all::standalone(opts)).unwrap(); +} diff --git a/src/cmd_all/src/lib.rs b/src/cmd_all/src/lib.rs index 94d4fd7ae3929..54ee3243bc662 100644 --- a/src/cmd_all/src/lib.rs +++ b/src/cmd_all/src/lib.rs @@ -18,7 +18,10 @@ mod common; pub mod playground; mod standalone; +pub mod single_node; + pub use playground::*; +pub use single_node::*; pub use standalone::*; risingwave_expr_impl::enable!(); diff --git a/src/cmd_all/src/playground.rs b/src/cmd_all/src/playground.rs index 70039264f6ef7..1b03048d5d6e0 100644 --- a/src/cmd_all/src/playground.rs +++ b/src/cmd_all/src/playground.rs @@ -137,7 +137,7 @@ fn get_services(profile: &str) -> (Vec, bool) { } #[derive(Debug, Clone, Parser)] -#[command(about = "The quick way to start a RisingWave cluster for playing around")] +#[command(about = "The quick way to start an in-memory RisingWave cluster for playing around")] pub struct PlaygroundOpts { /// The profile to use. #[clap(short, long, env = "PLAYGROUND_PROFILE", default_value = "playground")] diff --git a/src/cmd_all/src/single_node.rs b/src/cmd_all/src/single_node.rs new file mode 100644 index 0000000000000..b89f861f6e4fd --- /dev/null +++ b/src/cmd_all/src/single_node.rs @@ -0,0 +1,229 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::LazyLock; + +use clap::Parser; +use home::home_dir; +use risingwave_common::config::{AsyncStackTraceOption, MetaBackend}; +use risingwave_compactor::CompactorOpts; +use risingwave_compute::{default_parallelism, default_total_memory_bytes, ComputeNodeOpts}; +use risingwave_frontend::FrontendOpts; +use risingwave_meta_node::MetaNodeOpts; + +use crate::ParsedStandaloneOpts; + +pub static DEFAULT_STORE_DIRECTORY: LazyLock = LazyLock::new(|| { + let mut home_path = home_dir().unwrap(); + home_path.push(".risingwave"); + let home_path = home_path.to_str().unwrap(); + home_path.to_string() +}); + +pub static DEFAULT_SINGLE_NODE_SQLITE_PATH: LazyLock = + LazyLock::new(|| format!("{}/meta_store/single_node.db", &*DEFAULT_STORE_DIRECTORY)); + +pub static DEFAULT_SINGLE_NODE_SQL_ENDPOINT: LazyLock = + LazyLock::new(|| format!("sqlite://{}?mode=rwc", *DEFAULT_SINGLE_NODE_SQLITE_PATH)); + +pub static DEFAULT_SINGLE_NODE_STATE_STORE_PATH: LazyLock = + LazyLock::new(|| format!("{}/state_store", DEFAULT_STORE_DIRECTORY.clone())); + +pub static DEFAULT_SINGLE_NODE_STATE_STORE_URL: LazyLock = LazyLock::new(|| { + format!( + "hummock+fs://{}", + DEFAULT_SINGLE_NODE_STATE_STORE_PATH.clone() + ) +}); + +#[derive(Eq, PartialOrd, PartialEq, Debug, Clone, Parser)] +#[command( + version, + about = "[default] The Single Node mode. Start all services in one process, with process-level options. This will be executed if no subcommand is specified" +)] +/// Here we define our own defaults for the single node mode. +pub struct SingleNodeOpts { + /// The address prometheus polls metrics from. + #[clap(long, env = "RW_SINGLE_NODE_PROMETHEUS_LISTENER_ADDR")] + prometheus_listener_addr: Option, + + /// The path to the cluster configuration file. + #[clap(long, env = "RW_SINGLE_NODE_CONFIG_PATH")] + config_path: Option, + + /// The store directory used by meta store and object store. + #[clap(long, env = "RW_SINGLE_NODE_STORE_DIRECTORY")] + store_directory: Option, + + /// The address of the meta node. + #[clap(long, env = "RW_SINGLE_NODE_META_ADDR")] + meta_addr: Option, + + /// The address of the compute node + #[clap(long, env = "RW_SINGLE_NODE_COMPUTE_ADDR")] + compute_addr: Option, + + /// The address of the frontend node + #[clap(long, env = "RW_SINGLE_NODE_FRONTEND_ADDR")] + frontend_addr: Option, + + /// The address of the compactor node + #[clap(long, env = "RW_SINGLE_NODE_COMPACTOR_ADDR")] + compactor_addr: Option, +} + +pub fn make_single_node_sql_endpoint(store_directory: &String) -> String { + format!( + "sqlite://{}/meta_store/single_node.db?mode=rwc", + store_directory + ) +} + +pub fn make_single_node_state_store_url(store_directory: &String) -> String { + format!("hummock+fs://{}/state_store", store_directory) +} + +pub fn map_single_node_opts_to_standalone_opts(opts: &SingleNodeOpts) -> ParsedStandaloneOpts { + let mut meta_opts = SingleNodeOpts::default_meta_opts(); + let mut compute_opts = SingleNodeOpts::default_compute_opts(); + let mut frontend_opts = SingleNodeOpts::default_frontend_opts(); + let mut compactor_opts = SingleNodeOpts::default_compactor_opts(); + if let Some(prometheus_listener_addr) = &opts.prometheus_listener_addr { + meta_opts.prometheus_listener_addr = Some(prometheus_listener_addr.clone()); + compute_opts.prometheus_listener_addr = prometheus_listener_addr.clone(); + frontend_opts.prometheus_listener_addr = prometheus_listener_addr.clone(); + compactor_opts.prometheus_listener_addr = prometheus_listener_addr.clone(); + } + if let Some(config_path) = &opts.config_path { + meta_opts.config_path = config_path.clone(); + compute_opts.config_path = config_path.clone(); + frontend_opts.config_path = config_path.clone(); + compactor_opts.config_path = config_path.clone(); + } + if let Some(store_directory) = &opts.store_directory { + let state_store_url = make_single_node_state_store_url(store_directory); + let meta_store_endpoint = make_single_node_sql_endpoint(store_directory); + meta_opts.state_store = Some(state_store_url); + meta_opts.sql_endpoint = Some(meta_store_endpoint); + } + if let Some(meta_addr) = &opts.meta_addr { + meta_opts.listen_addr = meta_addr.clone(); + meta_opts.advertise_addr = meta_addr.clone(); + + compute_opts.meta_address = meta_addr.parse().unwrap(); + frontend_opts.meta_addr = meta_addr.parse().unwrap(); + compactor_opts.meta_address = meta_addr.parse().unwrap(); + } + if let Some(compute_addr) = &opts.compute_addr { + compute_opts.listen_addr = compute_addr.clone(); + } + if let Some(frontend_addr) = &opts.frontend_addr { + frontend_opts.listen_addr = frontend_addr.clone(); + } + if let Some(compactor_addr) = &opts.compactor_addr { + compactor_opts.listen_addr = compactor_addr.clone(); + } + ParsedStandaloneOpts { + meta_opts: Some(meta_opts), + compute_opts: Some(compute_opts), + frontend_opts: Some(frontend_opts), + compactor_opts: Some(compactor_opts), + } +} + +impl SingleNodeOpts { + fn default_frontend_opts() -> FrontendOpts { + FrontendOpts { + listen_addr: "0.0.0.0:4566".to_string(), + advertise_addr: Some("0.0.0.0:4566".to_string()), + port: None, + meta_addr: "http://0.0.0.0:5690".parse().unwrap(), + prometheus_listener_addr: "0.0.0.0:1250".to_string(), + health_check_listener_addr: "0.0.0.0:6786".to_string(), + config_path: "".to_string(), + metrics_level: None, + enable_barrier_read: None, + } + } + + fn default_meta_opts() -> MetaNodeOpts { + MetaNodeOpts { + vpc_id: None, + security_group_id: None, + listen_addr: "0.0.0.0:5690".to_string(), + advertise_addr: "0.0.0.0:5690".to_string(), + dashboard_host: Some("0.0.0.0:5691".to_string()), + prometheus_listener_addr: Some("0.0.0.0:1250".to_string()), + etcd_endpoints: Default::default(), + etcd_auth: false, + etcd_username: Default::default(), + etcd_password: Default::default(), + sql_endpoint: Some(DEFAULT_SINGLE_NODE_SQL_ENDPOINT.clone()), + dashboard_ui_path: None, + prometheus_endpoint: None, + prometheus_selector: None, + connector_rpc_endpoint: None, + privatelink_endpoint_default_tags: None, + config_path: "".to_string(), + backend: Some(MetaBackend::Sql), + barrier_interval_ms: None, + sstable_size_mb: None, + block_size_kb: None, + bloom_false_positive: None, + state_store: Some(DEFAULT_SINGLE_NODE_STATE_STORE_URL.clone()), + data_directory: Some("hummock_001".to_string()), + do_not_config_object_storage_lifecycle: None, + backup_storage_url: None, + backup_storage_directory: None, + heap_profiling_dir: None, + } + } + + pub fn default_compute_opts() -> ComputeNodeOpts { + ComputeNodeOpts { + listen_addr: "0.0.0.0:5688".to_string(), + advertise_addr: Some("0.0.0.0:5688".to_string()), + prometheus_listener_addr: "0.0.0.0:1250".to_string(), + meta_address: "http://0.0.0.0:5690".parse().unwrap(), + connector_rpc_endpoint: None, + connector_rpc_sink_payload_format: None, + config_path: "".to_string(), + total_memory_bytes: default_total_memory_bytes(), + parallelism: default_parallelism(), + role: Default::default(), + metrics_level: None, + data_file_cache_dir: None, + meta_file_cache_dir: None, + async_stack_trace: Some(AsyncStackTraceOption::ReleaseVerbose), + heap_profiling_dir: None, + } + } + + fn default_compactor_opts() -> CompactorOpts { + CompactorOpts { + listen_addr: "0.0.0.0:6660".to_string(), + advertise_addr: Some("0.0.0.0:6660".to_string()), + port: None, + prometheus_listener_addr: "0.0.0.0:1250".to_string(), + meta_address: "http://0.0.0.0:5690".parse().unwrap(), + compaction_worker_threads_number: None, + config_path: "".to_string(), + metrics_level: None, + async_stack_trace: None, + heap_profiling_dir: None, + compactor_mode: None, + proxy_rpc_endpoint: "".to_string(), + } + } +} diff --git a/src/cmd_all/src/standalone.rs b/src/cmd_all/src/standalone.rs index 8765db18a07cd..33e61e5b41f2b 100644 --- a/src/cmd_all/src/standalone.rs +++ b/src/cmd_all/src/standalone.rs @@ -25,6 +25,11 @@ use tokio::signal; use crate::common::osstrs; #[derive(Eq, PartialOrd, PartialEq, Debug, Clone, Parser)] +#[command( + version, + about = "The Standalone mode allows users to start multiple services in one process, it exposes node-level options for each service", + hide = true +)] pub struct StandaloneOpts { /// Compute node options /// If missing, compute node won't start @@ -161,6 +166,9 @@ pub fn parse_standalone_opt_args(opts: &StandaloneOpts) -> ParsedStandaloneOpts } } +/// For `standalone` mode, we can configure and start multiple services in one process. +/// `standalone` mode is meant to be used by our cloud service and docker, +/// where we can configure and start multiple services in one process. pub async fn standalone( ParsedStandaloneOpts { meta_opts, diff --git a/src/common/proc_macro/Cargo.toml b/src/common/proc_macro/Cargo.toml index 0f86bff6a19d2..bbfe10baf3b07 100644 --- a/src/common/proc_macro/Cargo.toml +++ b/src/common/proc_macro/Cargo.toml @@ -22,6 +22,7 @@ quote = "1" proc-macro2 = { version = "1", default-features = false } syn = "1" bae = "0.1.7" +itertools = "0.12" [lints] workspace = true diff --git a/src/common/proc_macro/src/config_doc.rs b/src/common/proc_macro/src/config_doc.rs new file mode 100644 index 0000000000000..e63145087f157 --- /dev/null +++ b/src/common/proc_macro/src/config_doc.rs @@ -0,0 +1,157 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use itertools::Itertools; +use quote::quote; +use syn::{Attribute, Data, DataStruct, DeriveInput, Field, Fields}; + +pub fn generate_config_doc_fn(input: DeriveInput) -> proc_macro2::TokenStream { + let mut doc = StructFieldDocs::new(); + + let struct_name = input.ident; + match input.data { + Data::Struct(ref data) => doc.extract_field_docs(data), + _ => panic!("This macro only supports structs"), + }; + + let vec_fields = doc.token_vec_fields(); + let call_nested_fields = doc.token_call_nested_fields(); + quote! { + impl #struct_name { + pub fn config_docs(name: String, docs: &mut std::collections::BTreeMap>) { + docs.insert(name.clone(), #vec_fields); + #call_nested_fields; + } + } + } +} + +fn extract_comment(attrs: &Vec) -> String { + attrs + .iter() + .filter_map(|attr| { + if let Ok(meta) = attr.parse_meta() { + if meta.path().is_ident("doc") { + if let syn::Meta::NameValue(syn::MetaNameValue { + lit: syn::Lit::Str(comment), + .. + }) = meta + { + return Some(comment.value()); + } + } + } + None + }) + .join(" ") +} + +fn is_nested_config_field(field: &Field) -> bool { + field.attrs.iter().any(|attr| { + if let Some(attr_name) = attr.path.get_ident() { + attr_name == "config_doc" && attr.tokens.to_string() == "(nested)" + } else { + false + } + }) +} + +fn is_omitted_config_field(field: &Field) -> bool { + field.attrs.iter().any(|attr| { + if let Some(attr_name) = attr.path.get_ident() { + attr_name == "config_doc" && attr.tokens.to_string() == "(omitted)" + } else { + false + } + }) +} + +fn field_name(f: &Field) -> String { + f.ident + .as_ref() + .expect("field name should not be empty") + .to_string() +} + +struct StructFieldDocs { + // Fields that require recursively retrieving their field docs. + nested_fields: Vec<(String, syn::Type)>, + + fields: Vec<(String, String)>, +} + +impl StructFieldDocs { + fn new() -> Self { + Self { + nested_fields: vec![], + fields: vec![], + } + } + + fn extract_field_docs(&mut self, data: &DataStruct) { + match &data.fields { + Fields::Named(fields) => { + self.fields = fields + .named + .iter() + .filter_map(|field| { + if is_omitted_config_field(field) { + return None; + } + if is_nested_config_field(field) { + self.nested_fields + .push((field_name(field), field.ty.clone())); + return None; + } + let field_name = field.ident.as_ref()?.to_string(); + let rustdoc = extract_comment(&field.attrs); + Some((field_name, rustdoc)) + }) + .collect_vec(); + } + _ => unreachable!("field should be named"), + } + } + + fn token_vec_fields(&self) -> proc_macro2::TokenStream { + let token_fields: Vec = self + .fields + .iter() + .map(|(name, doc)| { + quote! { (#name.to_string(), #doc.to_string()) } + }) + .collect(); + + quote! { + vec![#(#token_fields),*] + } + } + + fn token_call_nested_fields(&self) -> proc_macro2::TokenStream { + let tokens: Vec = self + .nested_fields + .iter() + .map(|(ident, ty)| { + quote! { + if name.is_empty() { + #ty::config_docs(#ident.to_string(), docs); + } else { + #ty::config_docs(format!("{}.{}", name, #ident), docs); + } + } + }) + .collect(); + quote! { #(#tokens)* } + } +} diff --git a/src/common/proc_macro/src/lib.rs b/src/common/proc_macro/src/lib.rs index 77807f8291ff2..cb52bf9786fef 100644 --- a/src/common/proc_macro/src/lib.rs +++ b/src/common/proc_macro/src/lib.rs @@ -23,6 +23,7 @@ use quote::quote; use syn::parse_macro_input; mod config; +mod config_doc; mod estimate_size; mod session_config; @@ -263,3 +264,44 @@ pub fn session_config(input: TokenStream) -> TokenStream { let input = parse_macro_input!(input); session_config::derive_config(input).into() } + +/// This proc macro recursively extracts rustdoc comments from the fields in a struct and generates a method +/// that produces docs for each field. +/// Unlike rustdoc, this tool focuses solely on extracting rustdoc for struct fields, without methods. +/// +/// Example: +/// +/// ```ignore +/// #[derive(ConfigDoc)] +/// pub struct Foo { +/// /// Description for `a`. +/// a: i32, +/// +/// #[config_doc(nested)] +/// b: Bar, +/// +/// #[config_doc(omitted)] +/// dummy: (), +/// } +/// ``` +/// +/// The `#[config_doc(nested)]` attribute indicates that the field is a nested config that will be documented in a separate section. +/// Fields marked with `#[config_doc(omitted)]` will simply be omitted from the doc. +/// +/// Here is the method generated by this macro: +/// +/// ```ignore +/// impl Foo { +/// pub fn config_docs(name: String, docs: &mut std::collections::BTreeMap>) +/// } +/// ``` +/// +/// In `test_example_up_to_date`, we further process the output of this method to generate a markdown in src/config/docs.md. +#[proc_macro_derive(ConfigDoc, attributes(config_doc))] +pub fn config_doc(input: TokenStream) -> TokenStream { + let input = parse_macro_input!(input); + + let gen = config_doc::generate_config_doc_fn(input); + + gen.into() +} diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index 09fd3db09561f..f82e96a80c0e2 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -103,7 +103,7 @@ pub struct ColumnDesc { pub type_name: String, pub generated_or_default_column: Option, pub description: Option, - pub additional_columns: AdditionalColumn, + pub additional_column: AdditionalColumn, pub version: ColumnDescVersion, } @@ -117,7 +117,7 @@ impl ColumnDesc { type_name: String::new(), generated_or_default_column: None, description: None, - additional_columns: AdditionalColumn { column_type: None }, + additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, } } @@ -131,7 +131,7 @@ impl ColumnDesc { type_name: String::new(), generated_or_default_column: None, description: None, - additional_columns: AdditionalColumn { column_type: None }, + additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, } } @@ -150,7 +150,7 @@ impl ColumnDesc { type_name: String::new(), generated_or_default_column: None, description: None, - additional_columns: additional_column_type, + additional_column: additional_column_type, version: ColumnDescVersion::Pr13707, } } @@ -170,7 +170,7 @@ impl ColumnDesc { type_name: self.type_name.clone(), generated_or_default_column: self.generated_or_default_column.clone(), description: self.description.clone(), - additional_columns: Some(self.additional_columns.clone()), + additional_column: Some(self.additional_column.clone()), version: self.version as i32, } } @@ -198,7 +198,7 @@ impl ColumnDesc { type_name: "".to_string(), generated_or_default_column: None, description: None, - additional_columns: AdditionalColumn { column_type: None }, + additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, } } @@ -221,7 +221,7 @@ impl ColumnDesc { type_name: type_name.to_string(), generated_or_default_column: None, description: None, - additional_columns: AdditionalColumn { column_type: None }, + additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, } } @@ -239,7 +239,7 @@ impl ColumnDesc { type_name: field.type_name.clone(), description: None, generated_or_default_column: None, - additional_columns: AdditionalColumn { column_type: None }, + additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, } } @@ -265,8 +265,8 @@ impl ColumnDesc { impl From for ColumnDesc { fn from(prost: PbColumnDesc) -> Self { - let additional_columns = prost - .get_additional_columns() + let additional_column = prost + .get_additional_column() .unwrap_or(&AdditionalColumn { column_type: None }) .clone(); let version = prost.version(); @@ -283,7 +283,7 @@ impl From for ColumnDesc { field_descs, generated_or_default_column: prost.generated_or_default_column, description: prost.description.clone(), - additional_columns, + additional_column, version, } } @@ -305,7 +305,7 @@ impl From<&ColumnDesc> for PbColumnDesc { type_name: c.type_name.clone(), generated_or_default_column: c.generated_or_default_column.clone(), description: c.description.clone(), - additional_columns: c.additional_columns.clone().into(), + additional_column: c.additional_column.clone().into(), version: c.version as i32, } } diff --git a/src/common/src/catalog/test_utils.rs b/src/common/src/catalog/test_utils.rs index ca154b9bf0b0b..9930a5717b849 100644 --- a/src/common/src/catalog/test_utils.rs +++ b/src/common/src/catalog/test_utils.rs @@ -35,7 +35,7 @@ impl ColumnDescTestExt for ColumnDesc { column_type: Some(data_type), column_id, name: name.to_string(), - additional_columns: Some(AdditionalColumn { column_type: None }), + additional_column: Some(AdditionalColumn { column_type: None }), version: ColumnDescVersion::Pr13707 as i32, ..Default::default() } @@ -60,7 +60,7 @@ impl ColumnDescTestExt for ColumnDesc { field_descs: fields, generated_or_default_column: None, description: None, - additional_columns: Some(AdditionalColumn { column_type: None }), + additional_column: Some(AdditionalColumn { column_type: None }), version: ColumnDescVersion::Pr13707 as i32, } } diff --git a/src/common/src/config.rs b/src/common/src/config.rs index ea4354f9b23ad..971fb28d208c2 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -25,6 +25,7 @@ use std::num::NonZeroUsize; use anyhow::Context; use clap::ValueEnum; use educe::Educe; +use risingwave_common_proc_macro::ConfigDoc; pub use risingwave_common_proc_macro::OverrideConfig; use risingwave_pb::meta::SystemParams; use serde::{Deserialize, Serialize, Serializer}; @@ -132,29 +133,36 @@ impl OverrideConfig for NoOverride { /// [`RwConfig`] corresponds to the whole config file `risingwave.toml`. Each field corresponds to a /// section. -#[derive(Educe, Clone, Serialize, Deserialize, Default)] +#[derive(Educe, Clone, Serialize, Deserialize, Default, ConfigDoc)] #[educe(Debug)] pub struct RwConfig { #[serde(default)] + #[config_doc(nested)] pub server: ServerConfig, #[serde(default)] + #[config_doc(nested)] pub meta: MetaConfig, #[serde(default)] + #[config_doc(nested)] pub batch: BatchConfig, #[serde(default)] + #[config_doc(nested)] pub streaming: StreamingConfig, #[serde(default)] + #[config_doc(nested)] pub storage: StorageConfig, #[serde(default)] #[educe(Debug(ignore))] + #[config_doc(nested)] pub system: SystemConfig, #[serde(flatten)] + #[config_doc(omitted)] pub unrecognized: Unrecognized, } @@ -171,7 +179,7 @@ pub enum MetaBackend { } /// The section `[meta]` in `risingwave.toml`. -#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] +#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)] pub struct MetaConfig { /// Objects within `min_sst_retention_time_sec` won't be deleted by hummock full GC, even they /// are dangling. @@ -311,6 +319,7 @@ pub struct MetaConfig { pub compaction_task_max_progress_interval_secs: u64, #[serde(default)] + #[config_doc(nested)] pub compaction_config: CompactionConfig, #[serde(default = "default::meta::hybird_partition_vnode_count")] @@ -322,6 +331,7 @@ pub struct MetaConfig { pub event_log_channel_max_size: u32, #[serde(default, with = "meta_prefix")] + #[config_doc(omitted)] pub developer: MetaDeveloperConfig, } @@ -392,7 +402,7 @@ impl<'de> Deserialize<'de> for DefaultParallelism { /// The subsections `[meta.developer]`. /// /// It is put at [`MetaConfig::developer`]. -#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] +#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)] pub struct MetaDeveloperConfig { /// The number of traces to be cached in-memory by the tracing collector /// embedded in the meta node. @@ -412,7 +422,7 @@ pub struct MetaDeveloperConfig { } /// The section `[server]` in `risingwave.toml`. -#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] +#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)] pub struct ServerConfig { /// The interval for periodic heartbeat from worker to the meta service. #[serde(default = "default::server::heartbeat_interval_ms")] @@ -437,11 +447,12 @@ pub struct ServerConfig { pub grpc_max_reset_stream: u32, #[serde(default, flatten)] + #[config_doc(omitted)] pub unrecognized: Unrecognized, } /// The section `[batch]` in `risingwave.toml`. -#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] +#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)] pub struct BatchConfig { /// The thread number of the batch task runtime in the compute node. The default value is /// decided by `tokio`. @@ -449,6 +460,7 @@ pub struct BatchConfig { pub worker_threads_num: Option, #[serde(default, with = "batch_prefix")] + #[config_doc(omitted)] pub developer: BatchDeveloperConfig, #[serde(default)] @@ -462,6 +474,7 @@ pub struct BatchConfig { pub statement_timeout_in_sec: u32, #[serde(default, flatten)] + #[config_doc(omitted)] pub unrecognized: Unrecognized, #[serde(default = "default::batch::frontend_compute_runtime_worker_threads")] @@ -470,7 +483,7 @@ pub struct BatchConfig { } /// The section `[streaming]` in `risingwave.toml`. -#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] +#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)] pub struct StreamingConfig { /// The maximum number of barriers in-flight in the compute nodes. #[serde(default = "default::streaming::in_flight_barrier_nums")] @@ -486,6 +499,7 @@ pub struct StreamingConfig { pub async_stack_trace: AsyncStackTraceOption, #[serde(default, with = "streaming_prefix")] + #[config_doc(omitted)] pub developer: StreamingDeveloperConfig, /// Max unique user stream errors per actor @@ -493,6 +507,7 @@ pub struct StreamingConfig { pub unique_user_stream_errors: usize, #[serde(default, flatten)] + #[config_doc(omitted)] pub unrecognized: Unrecognized, } @@ -533,7 +548,7 @@ impl PartialOrd for MetricLevel { } /// The section `[storage]` in `risingwave.toml`. -#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] +#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)] pub struct StorageConfig { /// parallelism while syncing share buffers into L0 SST. Should NOT be 0. #[serde(default = "default::storage::share_buffers_sync_parallelism")] @@ -652,6 +667,7 @@ pub struct StorageConfig { pub compactor_fast_max_compact_task_size: u64, #[serde(default, flatten)] + #[config_doc(omitted)] pub unrecognized: Unrecognized, /// The spill threshold for mem table. @@ -662,7 +678,7 @@ pub struct StorageConfig { pub object_store: ObjectStoreConfig, } -#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] +#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)] pub struct CacheRefillConfig { /// SSTable levels to refill. #[serde(default = "default::cache_refill::data_refill_levels")] @@ -695,13 +711,14 @@ pub struct CacheRefillConfig { pub recent_filter_rotate_interval_ms: usize, #[serde(default, flatten)] + #[config_doc(omitted)] pub unrecognized: Unrecognized, } /// The subsection `[storage.data_file_cache]` and `[storage.meta_file_cache]` in `risingwave.toml`. /// /// It's put at [`StorageConfig::data_file_cache`] and [`StorageConfig::meta_file_cache`]. -#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] +#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)] pub struct FileCacheConfig { #[serde(default = "default::file_cache::dir")] pub dir: String, @@ -746,6 +763,7 @@ pub struct FileCacheConfig { pub compression: String, #[serde(default, flatten)] + #[config_doc(omitted)] pub unrecognized: Unrecognized, } @@ -782,7 +800,7 @@ pub enum CompactorMode { Shared, } -#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] +#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)] pub struct HeapProfilingConfig { /// Enable to auto dump heap profile when memory usage is high #[serde(default = "default::heap_profiling::enable_auto")] @@ -800,7 +818,7 @@ pub struct HeapProfilingConfig { /// The subsections `[streaming.developer]`. /// /// It is put at [`StreamingConfig::developer`]. -#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] +#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)] pub struct StreamingDeveloperConfig { /// Set to true to enable per-executor row count metrics. This will produce a lot of timeseries /// and might affect the prometheus performance. If you only need actor input and output @@ -855,7 +873,7 @@ pub struct StreamingDeveloperConfig { /// The subsections `[batch.developer]`. /// /// It is put at [`BatchConfig::developer`]. -#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] +#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)] pub struct BatchDeveloperConfig { /// The capacity of the chunks in the channel that connects between `ConnectorSource` and /// `SourceExecutor`. @@ -877,7 +895,7 @@ macro_rules! define_system_config { /// The section `[system]` in `risingwave.toml`. All these fields are used to initialize the system /// parameters persisted in Meta store. Most fields are for testing purpose only and should not be /// documented. - #[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] + #[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)] pub struct SystemConfig { $( #[doc = $doc] @@ -1640,7 +1658,7 @@ pub fn extract_storage_memory_config(s: &RwConfig) -> StorageMemoryConfig { } } -#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] +#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)] pub struct CompactionConfig { #[serde(default = "default::compaction_config::max_bytes_for_level_base")] pub max_bytes_for_level_base: u64, @@ -1678,6 +1696,8 @@ pub struct CompactionConfig { #[cfg(test)] mod tests { + use std::collections::BTreeMap; + use super::*; /// This test ensures that `config/example.toml` is up-to-date with the default values specified @@ -1693,5 +1713,89 @@ mod tests { let expected = format!("{HEADER}\n\n{default}"); actual.assert_eq(&expected); + + let expected = rw_config_to_markdown(); + let actual = expect_test::expect_file!["../../config/docs.md"]; + actual.assert_eq(&expected); + } + + #[derive(Debug)] + struct ConfigItemDoc { + desc: String, + default: String, + } + + fn rw_config_to_markdown() -> String { + let mut config_rustdocs = BTreeMap::>::new(); + RwConfig::config_docs("".to_string(), &mut config_rustdocs); + + // Section -> Config Name -> ConfigItemDoc + let mut configs: BTreeMap> = config_rustdocs + .into_iter() + .map(|(k, v)| { + let docs: BTreeMap = v + .into_iter() + .map(|(name, desc)| { + ( + name, + ConfigItemDoc { + desc, + default: "".to_string(), // unset + }, + ) + }) + .collect(); + (k, docs) + }) + .collect(); + + let toml_doc: BTreeMap = + toml::from_str(&toml::to_string(&RwConfig::default()).unwrap()).unwrap(); + toml_doc.into_iter().for_each(|(name, value)| { + set_default_values("".to_string(), name, value, &mut configs); + }); + + let mut markdown = "# RisingWave System Configurations\n\n".to_string() + + "This page is automatically generated by `./risedev generate-example-config`\n"; + for (section, configs) in configs { + if configs.is_empty() { + continue; + } + markdown.push_str(&format!("\n## {}\n\n", section)); + markdown.push_str("| Config | Description | Default |\n"); + markdown.push_str("|--------|-------------|---------|\n"); + for (config, doc) in configs { + markdown.push_str(&format!( + "| {} | {} | {} |\n", + config, doc.desc, doc.default + )); + } + } + markdown + } + + fn set_default_values( + section: String, + name: String, + value: toml::Value, + configs: &mut BTreeMap>, + ) { + // Set the default value if it's a config name-value pair, otherwise it's a sub-section (Table) that should be recursively processed. + if let toml::Value::Table(table) = value { + let section_configs: BTreeMap = + table.clone().into_iter().collect(); + let sub_section = if section.is_empty() { + name + } else { + format!("{}.{}", section, name) + }; + section_configs + .into_iter() + .for_each(|(k, v)| set_default_values(sub_section.clone(), k, v, configs)) + } else if let Some(t) = configs.get_mut(§ion) { + if let Some(item_doc) = t.get_mut(&name) { + item_doc.default = format!("{}", value); + } + } } } diff --git a/src/common/src/error.rs b/src/common/src/error.rs index 1f3a782914299..80f46fa7a9e92 100644 --- a/src/common/src/error.rs +++ b/src/common/src/error.rs @@ -13,21 +13,11 @@ // limitations under the License. use std::collections::HashSet; -use std::convert::Infallible; use std::fmt::{Debug, Display, Formatter}; -use std::io::Error as IoError; use std::time::{Duration, SystemTime}; -use memcomparable::Error as MemComparableError; -use risingwave_error::tonic::{ToTonicStatus, TonicStatusWrapper}; -use risingwave_pb::PbFieldNotFound; use thiserror::Error; -use thiserror_ext::{Box, Macro}; -use tokio::task::JoinError; - -use crate::array::ArrayError; -use crate::session_config::SessionConfigError; -use crate::util::value_encoding::error::ValueEncodingError; +use thiserror_ext::Macro; /// Re-export `risingwave_error` for easy access. pub mod v2 { @@ -39,6 +29,7 @@ const ERROR_SUPPRESSOR_RESET_DURATION: Duration = Duration::from_millis(60 * 60 pub trait Error = std::error::Error + Send + Sync + 'static; pub type BoxedError = Box; +#[doc(hidden)] // Used by macros only. pub use anyhow::anyhow as anyhow_error; #[derive(Debug, Clone, Copy, Default)] @@ -102,228 +93,6 @@ impl Display for NoFunction { } } -#[derive(Error, Debug, Box)] -#[thiserror_ext(newtype(name = RwError, backtrace, report_debug))] -pub enum ErrorCode { - #[error("internal error: {0}")] - InternalError(String), - // TODO: unify with the above - #[error(transparent)] - Uncategorized( - #[from] - #[backtrace] - anyhow::Error, - ), - #[error("connector error: {0}")] - ConnectorError( - #[source] - #[backtrace] - BoxedError, - ), - #[error(transparent)] - NotImplemented(#[from] NotImplemented), - // Tips: Use this only if it's intended to reject the query - #[error("Not supported: {0}\nHINT: {1}")] - NotSupported(String, String), - #[error(transparent)] - NoFunction(#[from] NoFunction), - #[error(transparent)] - IoError(#[from] IoError), - #[error("Storage error: {0}")] - StorageError( - #[backtrace] - #[source] - BoxedError, - ), - #[error("Expr error: {0}")] - ExprError( - #[source] - #[backtrace] - BoxedError, - ), - // TODO(error-handling): there's a limitation that `#[transparent]` can't be used with `#[backtrace]` if no `#[from]` - // So we emulate a transparent error with "{0}" display here. - #[error("{0}")] - BatchError( - #[source] - #[backtrace] - // `BatchError` - BoxedError, - ), - #[error("Array error: {0}")] - ArrayError( - #[from] - #[backtrace] - ArrayError, - ), - #[error("Stream error: {0}")] - StreamError( - #[backtrace] - #[source] - BoxedError, - ), - // TODO(error-handling): there's a limitation that `#[transparent]` can't be used with `#[backtrace]` if no `#[from]` - // So we emulate a transparent error with "{0}" display here. - #[error("{0}")] - RpcError( - #[source] - #[backtrace] - // `tonic::transport::Error`, `TonicStatusWrapper`, or `RpcError` - BoxedError, - ), - // TODO: use a new type for bind error - // TODO(error-handling): should prefer use error types than strings. - #[error("Bind error: {0}")] - BindError(String), - // TODO: only keep this one - #[error("Failed to bind expression: {expr}: {error}")] - BindErrorRoot { - expr: String, - #[source] - #[backtrace] - error: BoxedError, - }, - #[error("Catalog error: {0}")] - CatalogError( - #[source] - #[backtrace] - BoxedError, - ), - #[error("Protocol error: {0}")] - ProtocolError(String), - #[error("Scheduler error: {0}")] - SchedulerError( - #[source] - #[backtrace] - BoxedError, - ), - #[error("Task not found")] - TaskNotFound, - #[error("Session not found")] - SessionNotFound, - #[error("Item not found: {0}")] - ItemNotFound(String), - #[error("Invalid input syntax: {0}")] - InvalidInputSyntax(String), - #[error("Can not compare in memory: {0}")] - MemComparableError(#[from] MemComparableError), - #[error("Error while de/se values: {0}")] - ValueEncodingError( - #[from] - #[backtrace] - ValueEncodingError, - ), - #[error("Invalid value `{config_value}` for `{config_entry}`")] - InvalidConfigValue { - config_entry: String, - config_value: String, - }, - #[error("Invalid Parameter Value: {0}")] - InvalidParameterValue(String), - #[error("Sink error: {0}")] - SinkError( - #[source] - #[backtrace] - BoxedError, - ), - #[error("Permission denied: {0}")] - PermissionDenied(String), - #[error("Failed to get/set session config: {0}")] - SessionConfig( - #[from] - #[backtrace] - SessionConfigError, - ), -} - -impl RwError { - pub fn uncategorized(err: impl Into) -> Self { - Self::from(ErrorCode::Uncategorized(err.into())) - } -} - -impl From for tonic::Status { - fn from(err: RwError) -> Self { - use tonic::Code; - - let code = match err.inner() { - ErrorCode::ExprError(_) => Code::InvalidArgument, - ErrorCode::PermissionDenied(_) => Code::PermissionDenied, - ErrorCode::InternalError(_) => Code::Internal, - _ => Code::Internal, - }; - - err.to_status_unnamed(code) - } -} - -impl From for RwError { - fn from(status: TonicStatusWrapper) -> Self { - use tonic::Code; - - let message = status.inner().message(); - - // TODO(error-handling): `message` loses the source chain. - match status.inner().code() { - Code::InvalidArgument => ErrorCode::InvalidParameterValue(message.to_string()), - Code::NotFound | Code::AlreadyExists => ErrorCode::CatalogError(status.into()), - Code::PermissionDenied => ErrorCode::PermissionDenied(message.to_string()), - Code::Cancelled => ErrorCode::SchedulerError(status.into()), - _ => ErrorCode::RpcError(status.into()), - } - .into() - } -} - -impl From for RwError { - fn from(status: tonic::Status) -> Self { - // Always wrap the status. - Self::from(TonicStatusWrapper::new(status)) - } -} - -impl From for RwError { - fn from(join_error: JoinError) -> Self { - Self::uncategorized(join_error) - } -} - -impl From for RwError { - fn from(addr_parse_error: std::net::AddrParseError) -> Self { - Self::uncategorized(addr_parse_error) - } -} - -impl From for RwError { - fn from(x: Infallible) -> Self { - match x {} - } -} - -impl From for RwError { - fn from(e: String) -> Self { - ErrorCode::InternalError(e).into() - } -} - -impl From for RwError { - fn from(err: PbFieldNotFound) -> Self { - ErrorCode::InternalError(format!( - "Failed to decode prost: field not found `{}`", - err.0 - )) - .into() - } -} - -impl From for RwError { - fn from(err: tonic::transport::Error) -> Self { - ErrorCode::RpcError(err.into()).into() - } -} - -pub type Result = std::result::Result; - /// Util macro for generating error when condition check failed. /// /// # Case 1: Expression only. @@ -462,13 +231,10 @@ mod tests { use anyhow::anyhow; use super::*; - use crate::error::ErrorCode::Uncategorized; - #[test] - fn test_display_internal_error() { - let internal_error = ErrorCode::InternalError("some thing bad happened!".to_string()); - println!("{:?}", RwError::from(internal_error)); - } + #[derive(Error, Debug)] + #[error(transparent)] + struct MyError(#[from] anyhow::Error); #[test] fn test_ensure() { @@ -478,37 +244,31 @@ mod tests { let err_msg = "a < 0"; let error = (|| { ensure!(a < 0); - Ok::<_, RwError>(()) + Ok::<_, MyError>(()) })() .unwrap_err(); - assert_eq!( - RwError::from(Uncategorized(anyhow!(err_msg))).to_string(), - error.to_string(), - ); + assert_eq!(MyError(anyhow!(err_msg)).to_string(), error.to_string(),); } { let err_msg = "error msg without args"; let error = (|| { ensure!(a < 0, "error msg without args"); - Ok::<_, RwError>(()) + Ok::<_, MyError>(()) })() .unwrap_err(); - assert_eq!( - RwError::from(Uncategorized(anyhow!(err_msg))).to_string(), - error.to_string() - ); + assert_eq!(MyError(anyhow!(err_msg)).to_string(), error.to_string()); } { let error = (|| { ensure!(a < 0, "error msg with args: {}", "xx"); - Ok::<_, RwError>(()) + Ok::<_, MyError>(()) })() .unwrap_err(); assert_eq!( - RwError::from(Uncategorized(anyhow!("error msg with args: {}", "xx"))).to_string(), + MyError(anyhow!("error msg with args: {}", "xx")).to_string(), error.to_string() ); } @@ -516,7 +276,7 @@ mod tests { #[test] fn test_ensure_eq() { - fn ensure_a_equals_b() -> Result<()> { + fn ensure_a_equals_b() -> Result<(), MyError> { let a = 1; let b = 2; ensure_eq!(a, b); @@ -525,31 +285,4 @@ mod tests { let err = ensure_a_equals_b().unwrap_err(); assert_eq!(err.to_string(), "a == b assertion failed (a is 1, b is 2)"); } - - #[test] - fn test_into() { - use tonic::{Code, Status}; - fn check_grpc_error(ec: ErrorCode, grpc_code: Code) { - assert_eq!(Status::from(RwError::from(ec)).code(), grpc_code); - } - - check_grpc_error(ErrorCode::TaskNotFound, Code::Internal); - check_grpc_error(ErrorCode::InternalError(String::new()), Code::Internal); - check_grpc_error( - ErrorCode::NotImplemented(not_implemented!("test")), - Code::Internal, - ); - } - - #[test] - #[ignore] // it's not a good practice to include error source in `Display`, see #13248 - fn test_internal_sources() { - use anyhow::Context; - - let res: Result<()> = Err(anyhow::anyhow!("inner")) - .context("outer") - .map_err(Into::into); - - assert_eq!(res.unwrap_err().to_string(), "internal error: outer: inner"); - } } diff --git a/src/common/src/lib.rs b/src/common/src/lib.rs index 20428599b1039..980897d5636e7 100644 --- a/src/common/src/lib.rs +++ b/src/common/src/lib.rs @@ -80,7 +80,6 @@ pub mod test_utils; pub mod transaction; pub mod types; pub mod vnode_mapping; - pub mod test_prelude { pub use super::array::{DataChunkTestExt, StreamChunkTestExt}; pub use super::catalog::test_utils::ColumnDescTestExt; diff --git a/src/compute/src/lib.rs b/src/compute/src/lib.rs index b713aa9951622..3673997d2a128 100644 --- a/src/compute/src/lib.rs +++ b/src/compute/src/lib.rs @@ -225,14 +225,14 @@ pub fn start(opts: ComputeNodeOpts) -> Pin + Send>> }) } -fn default_total_memory_bytes() -> usize { +pub fn default_total_memory_bytes() -> usize { (system_memory_available_bytes() as f64 * DEFAULT_MEMORY_PROPORTION) as usize } -fn default_parallelism() -> usize { +pub fn default_parallelism() -> usize { total_cpu_available().ceil() as usize } -fn default_role() -> Role { +pub fn default_role() -> Role { Role::Both } diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index d9e9e608abc92..21fbe90b514e0 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -32,7 +32,6 @@ use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{ ColumnDesc, ColumnId, ConflictBehavior, Field, Schema, TableId, INITIAL_TABLE_VERSION_ID, }; -use risingwave_common::error::{Result, RwError}; use risingwave_common::row::OwnedRow; use risingwave_common::system_param::local_manager::LocalSystemParamsManager; use risingwave_common::test_prelude::DataChunkTestExt; @@ -297,7 +296,7 @@ async fn test_table_materialize() -> StreamResult<()> { barrier_tx_clone .send(Barrier::new_test_barrier(curr_epoch)) .unwrap(); - Ok::<_, RwError>(()) + anyhow::Ok(()) }); // Poll `Materialize`, should output the same insertion stream chunk. @@ -379,7 +378,7 @@ async fn test_table_materialize() -> StreamResult<()> { barrier_tx_clone .send(Barrier::new_test_barrier(curr_epoch)) .unwrap(); - Ok::<_, RwError>(()) + anyhow::Ok(()) }); // Poll `Materialize`, should output the same deletion stream chunk. @@ -429,7 +428,7 @@ async fn test_table_materialize() -> StreamResult<()> { } #[tokio::test] -async fn test_row_seq_scan() -> Result<()> { +async fn test_row_seq_scan() -> StreamResult<()> { // In this test we test if the memtable can be correctly scanned for K-V pair insertions. let memory_state_store = MemoryStateStore::new(); diff --git a/src/config/docs.md b/src/config/docs.md new file mode 100644 index 0000000000000..36fd40ce2d13a --- /dev/null +++ b/src/config/docs.md @@ -0,0 +1,153 @@ +# RisingWave System Configurations + +This page is automatically generated by `./risedev generate-example-config` + +## batch + +| Config | Description | Default | +|--------|-------------|---------| +| distributed_query_limit | | | +| enable_barrier_read | | false | +| frontend_compute_runtime_worker_threads | frontend compute runtime worker threads | 4 | +| statement_timeout_in_sec | Timeout for a batch query in seconds. | 3600 | +| worker_threads_num | The thread number of the batch task runtime in the compute node. The default value is decided by `tokio`. | | + +## meta + +| Config | Description | Default | +|--------|-------------|---------| +| backend | | "Mem" | +| collect_gc_watermark_spin_interval_sec | The spin interval when collecting global GC watermark in hummock. | 5 | +| compaction_task_max_heartbeat_interval_secs | | 30 | +| compaction_task_max_progress_interval_secs | | 600 | +| cut_table_size_limit | | 1073741824 | +| dangerous_max_idle_secs | After specified seconds of idle (no mview or flush), the process will be exited. It is mainly useful for playgrounds. | | +| default_parallelism | The default global parallelism for all streaming jobs, if user doesn't specify the parallelism, this value will be used. `FULL` means use all available parallelism units, otherwise it's a number. | "Full" | +| disable_automatic_parallelism_control | Whether to disable adaptive-scaling feature. | false | +| disable_recovery | Whether to enable fail-on-recovery. Should only be used in e2e tests. | false | +| do_not_config_object_storage_lifecycle | Whether config object storage bucket lifecycle to purge stale data. | false | +| enable_committed_sst_sanity_check | Enable sanity check when SSTs are committed. | false | +| enable_compaction_deterministic | Whether to enable deterministic compaction scheduling, which will disable all auto scheduling of compaction tasks. Should only be used in e2e tests. | false | +| enable_hummock_data_archive | If enabled, SSTable object file and version delta will be retained. SSTable object file need to be deleted via full GC. version delta need to be manually deleted. | false | +| event_log_channel_max_size | Keeps the latest N events per channel. | 10 | +| event_log_enabled | | true | +| full_gc_interval_sec | Interval of automatic hummock full GC. | 86400 | +| hummock_version_checkpoint_interval_sec | Interval of hummock version checkpoint. | 30 | +| hybird_partition_vnode_count | | 4 | +| max_heartbeat_interval_secs | Maximum allowed heartbeat interval in seconds. | 300 | +| meta_leader_lease_secs | | 30 | +| min_delta_log_num_for_hummock_version_checkpoint | The minimum delta log number a new checkpoint should compact, otherwise the checkpoint attempt is rejected. | 10 | +| min_sst_retention_time_sec | Objects within `min_sst_retention_time_sec` won't be deleted by hummock full GC, even they are dangling. | 86400 | +| min_table_split_write_throughput | If the size of one table is smaller than `min_table_split_write_throughput`, we would not split it to an single group. | 4194304 | +| move_table_size_limit | | 10737418240 | +| node_num_monitor_interval_sec | | 10 | +| partition_vnode_count | | 16 | +| periodic_compaction_interval_sec | Schedule compaction for all compaction groups with this interval. | 60 | +| periodic_space_reclaim_compaction_interval_sec | Schedule space_reclaim compaction for all compaction groups with this interval. | 3600 | +| periodic_split_compact_group_interval_sec | | 10 | +| periodic_tombstone_reclaim_compaction_interval_sec | | 600 | +| periodic_ttl_reclaim_compaction_interval_sec | Schedule ttl_reclaim compaction for all compaction groups with this interval. | 1800 | +| split_group_size_limit | | 68719476736 | +| table_write_throughput_threshold | | 16777216 | +| unrecognized | | | +| vacuum_interval_sec | Interval of invoking a vacuum job, to remove stale metadata from meta store and objects from object store. | 30 | +| vacuum_spin_interval_ms | The spin interval inside a vacuum job. It avoids the vacuum job monopolizing resources of meta node. | 10 | + +## meta.compaction_config + +| Config | Description | Default | +|--------|-------------|---------| +| compaction_filter_mask | | 6 | +| enable_emergency_picker | | true | +| level0_max_compact_file_number | | 100 | +| level0_overlapping_sub_level_compact_level_count | | 12 | +| level0_stop_write_threshold_sub_level_number | | 300 | +| level0_sub_level_compact_level_count | | 3 | +| level0_tier_compact_file_number | | 12 | +| max_bytes_for_level_base | | 536870912 | +| max_bytes_for_level_multiplier | | 5 | +| max_compaction_bytes | | 2147483648 | +| max_space_reclaim_bytes | | 536870912 | +| max_sub_compaction | | 4 | +| sub_level_max_compaction_bytes | | 134217728 | +| target_file_size_base | | 33554432 | +| tombstone_reclaim_ratio | | 40 | + +## server + +| Config | Description | Default | +|--------|-------------|---------| +| connection_pool_size | | 16 | +| grpc_max_reset_stream | | 200 | +| heap_profiling | Enable heap profile dump when memory usage is high. | | +| heartbeat_interval_ms | The interval for periodic heartbeat from worker to the meta service. | 1000 | +| metrics_level | Used for control the metrics level, similar to log level. | "Info" | +| telemetry_enabled | | true | + +## storage + +| Config | Description | Default | +|--------|-------------|---------| +| block_cache_capacity_mb | Capacity of sstable block cache. | | +| cache_refill | | | +| check_compaction_result | | false | +| compact_iter_recreate_timeout_ms | | 600000 | +| compactor_fast_max_compact_delete_ratio | | 40 | +| compactor_fast_max_compact_task_size | | 2147483648 | +| compactor_max_sst_key_count | | 2097152 | +| compactor_max_sst_size | | 536870912 | +| compactor_max_task_multiplier | Compactor calculates the maximum number of tasks that can be executed on the node based on worker_num and compactor_max_task_multiplier. max_pull_task_count = worker_num * compactor_max_task_multiplier | 2.5 | +| compactor_memory_available_proportion | The percentage of memory available when compactor is deployed separately. non_reserved_memory_bytes = system_memory_available_bytes * compactor_memory_available_proportion | 0.8 | +| compactor_memory_limit_mb | | | +| data_file_cache | | | +| disable_remote_compactor | | false | +| enable_fast_compaction | | false | +| high_priority_ratio_in_percent | | | +| imm_merge_threshold | The threshold for the number of immutable memtables to merge to a new imm. | 0 | +| max_concurrent_compaction_task_number | | 16 | +| max_prefetch_block_number | max prefetch block number | 16 | +| max_preload_io_retry_times | | 3 | +| max_preload_wait_time_mill | | 0 | +| max_sub_compaction | Max sub compaction task numbers | 4 | +| max_version_pinning_duration_sec | | 10800 | +| mem_table_spill_threshold | The spill threshold for mem table. | 0 | +| meta_cache_capacity_mb | Capacity of sstable meta cache. | | +| meta_file_cache | | | +| min_sst_size_for_streaming_upload | Whether to enable streaming upload for sstable. | 33554432 | +| object_store | | | +| prefetch_buffer_capacity_mb | max memory usage for large query | | +| share_buffer_compaction_worker_threads_number | Worker threads number of dedicated tokio runtime for share buffer compaction. 0 means use tokio's default value (number of CPU core). | 4 | +| share_buffer_upload_concurrency | Number of tasks shared buffer can upload in parallel. | 8 | +| share_buffers_sync_parallelism | parallelism while syncing share buffers into L0 SST. Should NOT be 0. | 1 | +| shared_buffer_capacity_mb | Maximum shared buffer size, writes attempting to exceed the capacity will stall until there is enough space. | | +| shared_buffer_flush_ratio | The shared buffer will start flushing data to object when the ratio of memory usage to the shared buffer capacity exceed such ratio. | 0.800000011920929 | +| sstable_id_remote_fetch_number | Number of SST ids fetched from meta per RPC | 10 | +| write_conflict_detection_enabled | Whether to enable write conflict detection | true | + +## streaming + +| Config | Description | Default | +|--------|-------------|---------| +| actor_runtime_worker_threads_num | The thread number of the streaming actor runtime in the compute node. The default value is decided by `tokio`. | | +| async_stack_trace | Enable async stack tracing through `await-tree` for risectl. | "ReleaseVerbose" | +| in_flight_barrier_nums | The maximum number of barriers in-flight in the compute nodes. | 10000 | +| unique_user_stream_errors | Max unique user stream errors per actor | 10 | + +## system + +| Config | Description | Default | +|--------|-------------|---------| +| backup_storage_directory | Remote directory for storing snapshots. | | +| backup_storage_url | Remote storage url for storing snapshots. | | +| barrier_interval_ms | The interval of periodic barrier. | 1000 | +| block_size_kb | Size of each block in bytes in SST. | 64 | +| bloom_false_positive | False positive probability of bloom filter. | 0.001 | +| checkpoint_frequency | There will be a checkpoint for every n barriers. | 1 | +| data_directory | Remote directory for storing data and metadata objects. | | +| enable_tracing | Whether to enable distributed tracing. | false | +| max_concurrent_creating_streaming_jobs | Max number of concurrent creating streaming jobs. | 1 | +| parallel_compact_size_mb | | 512 | +| pause_on_next_bootstrap | Whether to pause all data sources on next bootstrap. | false | +| sstable_size_mb | Target size of the Sstable. | 256 | +| state_store | | | +| wasm_storage_url | | "fs://.risingwave/data" | diff --git a/src/connector/src/aws_utils.rs b/src/connector/src/aws_utils.rs index 4f9b1bd08ae57..cf70a90e07cda 100644 --- a/src/connector/src/aws_utils.rs +++ b/src/connector/src/aws_utils.rs @@ -15,11 +15,9 @@ use std::collections::HashMap; use std::time::Duration; +use anyhow::Context; use aws_config::timeout::TimeoutConfig; -use aws_sdk_s3::error::DisplayErrorContext; use aws_sdk_s3::{client as s3_client, config as s3_config}; -use risingwave_common::error::ErrorCode::InternalError; -use risingwave_common::error::{Result, RwError}; use url::Url; use crate::common::AwsAuthProps; @@ -108,10 +106,10 @@ pub fn s3_client( pub async fn load_file_descriptor_from_s3( location: &Url, config: &AwsAuthProps, -) -> Result> { +) -> anyhow::Result> { let bucket = location .domain() - .ok_or_else(|| RwError::from(InternalError(format!("Illegal file path {}", location))))?; + .with_context(|| format!("illegal file path {}", location))?; let key = location.path().replace('/', ""); let sdk_config = config.build_config().await?; let s3_client = s3_client(&sdk_config, Some(default_conn_config())); @@ -121,18 +119,12 @@ pub async fn load_file_descriptor_from_s3( .key(&key) .send() .await - .map_err(|e| { - RwError::from(InternalError(format!( - "get file {} err:{}", - location, - DisplayErrorContext(e) - ))) - })?; + .with_context(|| format!("failed to get file from s3 at `{}`", location))?; let body = response .body .collect() .await - .map_err(|e| RwError::from(InternalError(format!("Read file from s3 {}", e))))?; + .with_context(|| format!("failed to read file from s3 at `{}`", location))?; Ok(body.into_bytes().to_vec()) } diff --git a/src/connector/src/common.rs b/src/connector/src/common.rs index 25179bb209866..88f6c9f9ea313 100644 --- a/src/connector/src/common.rs +++ b/src/connector/src/common.rs @@ -17,15 +17,14 @@ use std::collections::HashMap; use std::io::Write; use std::time::Duration; -use anyhow::{anyhow, Ok}; +use anyhow::{anyhow, Context, Ok}; use async_nats::jetstream::consumer::DeliverPolicy; use async_nats::jetstream::{self}; use aws_sdk_kinesis::Client as KinesisClient; use pulsar::authentication::oauth2::{OAuth2Authentication, OAuth2Params}; use pulsar::{Authentication, Pulsar, TokioExecutor}; use rdkafka::ClientConfig; -use risingwave_common::error::ErrorCode::InvalidParameterValue; -use risingwave_common::error::{anyhow_error, RwError}; +use risingwave_common::bail; use serde_derive::Deserialize; use serde_with::json::JsonString; use serde_with::{serde_as, DisplayFromStr}; @@ -400,10 +399,7 @@ impl PulsarCommon { } "file" => {} _ => { - return Err(RwError::from(InvalidParameterValue(String::from( - "invalid credentials_url, only file url and s3 url are supported", - ))) - .into()); + bail!("invalid credentials_url, only file url and s3 url are supported",); } } @@ -546,9 +542,7 @@ impl NatsCommon { connect_options = connect_options.user_and_password(v_user.into(), v_password.into()) } else { - return Err(anyhow_error!( - "nats connect mode is user_and_password, but user or password is empty" - )); + bail!("nats connect mode is user_and_password, but user or password is empty"); } } @@ -558,16 +552,12 @@ impl NatsCommon { .credentials(&self.create_credential(v_nkey, v_jwt)?) .expect("failed to parse static creds") } else { - return Err(anyhow_error!( - "nats connect mode is credential, but nkey or jwt is empty" - )); + bail!("nats connect mode is credential, but nkey or jwt is empty"); } } "plain" => {} _ => { - return Err(anyhow_error!( - "nats connect mode only accept user_and_password/credential/plain" - )); + bail!("nats connect mode only accept user_and_password/credential/plain"); } }; @@ -580,7 +570,8 @@ impl NatsCommon { .collect::, _>>()?, ) .await - .map_err(|e| SinkError::Nats(anyhow_error!("build nats client error: {:?}", e)))?; + .context("build nats client error") + .map_err(SinkError::Nats)?; Ok(client) } diff --git a/src/connector/src/error.rs b/src/connector/src/error.rs index 86ba8a6cc39bc..d25658dfaf5e5 100644 --- a/src/connector/src/error.rs +++ b/src/connector/src/error.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::{ErrorCode, RwError}; use thiserror::Error; #[derive(Error, Debug)] @@ -60,9 +59,3 @@ pub enum ConnectorError { anyhow::Error, ), } - -impl From for RwError { - fn from(s: ConnectorError) -> Self { - ErrorCode::ConnectorError(Box::new(s)).into() - } -} diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index bc1a2784ec3f6..c1da30f788b3e 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -15,8 +15,8 @@ use std::collections::{HashMap, HashSet}; use std::sync::LazyLock; +use risingwave_common::bail; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId}; -use risingwave_common::error::Result; use risingwave_common::types::{DataType, StructType}; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType as PbDataType; @@ -86,7 +86,7 @@ pub fn build_additional_column_catalog( inner_field_name: Option<&str>, data_type: Option<&str>, reject_unknown_connector: bool, -) -> Result { +) -> anyhow::Result { let compatible_columns = match ( COMPATIBLE_ADDITIONAL_COLUMNS.get(connector_name), reject_unknown_connector, @@ -94,19 +94,18 @@ pub fn build_additional_column_catalog( (Some(compat_cols), _) => compat_cols, (None, false) => &COMMON_COMPATIBLE_ADDITIONAL_COLUMNS, (None, true) => { - return Err(format!( + bail!( "additional column is not supported for connector {}, acceptable connectors: {:?}", connector_name, COMPATIBLE_ADDITIONAL_COLUMNS.keys(), - ) - .into()) + ); } }; if !compatible_columns.contains(additional_col_type) { - return Err(format!( + bail!( "additional column type {} is not supported for connector {}, acceptable column types: {:?}", additional_col_type, connector_name, compatible_columns - ).into()); + ); } let column_name = column_alias.unwrap_or_else(|| { diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index 10e000a4fdab7..5e876d2ce9324 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -15,11 +15,10 @@ use std::fmt::Debug; use std::sync::Arc; +use anyhow::Context; use apache_avro::types::Value; use apache_avro::{from_avro_datum, Reader, Schema}; -use risingwave_common::error::ErrorCode::{InternalError, ProtocolError}; -use risingwave_common::error::{Result, RwError}; -use risingwave_common::try_match_expand; +use risingwave_common::{bail, try_match_expand}; use risingwave_pb::plan_common::ColumnDesc; use super::schema_resolver::ConfluentSchemaResolver; @@ -41,7 +40,7 @@ pub struct AvroAccessBuilder { } impl AccessBuilder for AvroAccessBuilder { - async fn generate_accessor(&mut self, payload: Vec) -> Result> { + async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { self.value = self.parse_avro_value(&payload, Some(&*self.schema)).await?; Ok(AccessImpl::Avro(AvroAccess::new( self.value.as_ref().unwrap(), @@ -51,7 +50,7 @@ impl AccessBuilder for AvroAccessBuilder { } impl AvroAccessBuilder { - pub fn new(config: AvroParserConfig, encoding_type: EncodingType) -> Result { + pub fn new(config: AvroParserConfig, encoding_type: EncodingType) -> anyhow::Result { let AvroParserConfig { schema, key_schema, @@ -60,9 +59,7 @@ impl AvroAccessBuilder { } = config; Ok(Self { schema: match encoding_type { - EncodingType::Key => key_schema.ok_or(RwError::from(ProtocolError( - "Avro with empty key schema".to_string(), - )))?, + EncodingType::Key => key_schema.context("Avro with empty key schema")?, EncodingType::Value => schema, }, schema_resolver, @@ -108,7 +105,7 @@ pub struct AvroParserConfig { } impl AvroParserConfig { - pub async fn new(encoding_properties: EncodingProperties) -> Result { + pub async fn new(encoding_properties: EncodingProperties) -> anyhow::Result { let avro_config = try_match_expand!(encoding_properties, EncodingProperties::Avro)?; let schema_location = &avro_config.row_schema_location; let enable_upsert = avro_config.enable_upsert; @@ -126,9 +123,7 @@ impl AvroParserConfig { )?) } else { if let Some(name) = &avro_config.key_record_name { - return Err(RwError::from(ProtocolError(format!( - "key.message = {name} not used", - )))); + bail!("key.message = {name} not used"); } None }; @@ -151,15 +146,12 @@ impl AvroParserConfig { }) } else { if enable_upsert { - return Err(RwError::from(InternalError( - "avro upsert without schema registry is not supported".to_string(), - ))); + bail!("avro upsert without schema registry is not supported"); } let url = url.first().unwrap(); let schema_content = bytes_from_url(url, avro_config.aws_auth_props.as_ref()).await?; - let schema = Schema::parse_reader(&mut schema_content.as_slice()).map_err(|e| { - RwError::from(InternalError(format!("Avro schema parse error {}", e))) - })?; + let schema = Schema::parse_reader(&mut schema_content.as_slice()) + .context("failed to parse avro schema")?; Ok(Self { schema: Arc::new(schema), key_schema: None, @@ -264,7 +256,7 @@ mod test { println!("schema = {:?}", schema.unwrap()); } - async fn new_avro_conf_from_local(file_name: &str) -> Result { + async fn new_avro_conf_from_local(file_name: &str) -> anyhow::Result { let schema_path = "file://".to_owned() + &test_data_path(file_name); let info = StreamSourceInfo { row_schema_location: schema_path.clone(), @@ -277,7 +269,7 @@ mod test { AvroParserConfig::new(parser_config.encoding_config).await } - async fn new_avro_parser_from_local(file_name: &str) -> Result { + async fn new_avro_parser_from_local(file_name: &str) -> anyhow::Result { let conf = new_avro_conf_from_local(file_name).await?; Ok(PlainParser { diff --git a/src/connector/src/parser/avro/schema_resolver.rs b/src/connector/src/parser/avro/schema_resolver.rs index dca3305784bc9..ef2dd9fc5f731 100644 --- a/src/connector/src/parser/avro/schema_resolver.rs +++ b/src/connector/src/parser/avro/schema_resolver.rs @@ -14,10 +14,9 @@ use std::sync::Arc; +use anyhow::Context; use apache_avro::Schema; use moka::future::Cache; -use risingwave_common::error::ErrorCode::ProtocolError; -use risingwave_common::error::{Result, RwError}; use crate::schema::schema_registry::{Client, ConfluentSchema}; @@ -28,9 +27,12 @@ pub struct ConfluentSchemaResolver { } impl ConfluentSchemaResolver { - async fn parse_and_cache_schema(&self, raw_schema: ConfluentSchema) -> Result> { - let schema = Schema::parse_str(&raw_schema.content) - .map_err(|e| RwError::from(ProtocolError(format!("Avro schema parse error {}", e))))?; + async fn parse_and_cache_schema( + &self, + raw_schema: ConfluentSchema, + ) -> anyhow::Result> { + let schema = + Schema::parse_str(&raw_schema.content).context("failed to parse avro schema")?; let schema = Arc::new(schema); self.writer_schemas .insert(raw_schema.id, Arc::clone(&schema)) @@ -46,7 +48,7 @@ impl ConfluentSchemaResolver { } } - pub async fn get_by_subject_name(&self, subject_name: &str) -> Result> { + pub async fn get_by_subject_name(&self, subject_name: &str) -> anyhow::Result> { let raw_schema = self.get_raw_schema_by_subject_name(subject_name).await?; self.parse_and_cache_schema(raw_schema).await } @@ -54,7 +56,7 @@ impl ConfluentSchemaResolver { pub async fn get_raw_schema_by_subject_name( &self, subject_name: &str, - ) -> Result { + ) -> anyhow::Result { self.confluent_client .get_schema_by_subject(subject_name) .await @@ -62,7 +64,7 @@ impl ConfluentSchemaResolver { } // get the writer schema by id - pub async fn get(&self, schema_id: i32) -> Result> { + pub async fn get(&self, schema_id: i32) -> anyhow::Result> { // TODO: use `get_with` if let Some(schema) = self.writer_schemas.get(&schema_id).await { Ok(schema) diff --git a/src/connector/src/parser/avro/util.rs b/src/connector/src/parser/avro/util.rs index a6c5c6fbef5d1..8d2d4265883e6 100644 --- a/src/connector/src/parser/avro/util.rs +++ b/src/connector/src/parser/avro/util.rs @@ -61,7 +61,7 @@ fn avro_field_to_column_desc( type_name: schema_name.to_string(), generated_or_default_column: None, description: None, - additional_columns: Some(AdditionalColumn { column_type: None }), + additional_column: Some(AdditionalColumn { column_type: None }), version: ColumnDescVersion::Pr13707 as i32, }) } @@ -71,7 +71,7 @@ fn avro_field_to_column_desc( column_type: Some(data_type.to_protobuf()), column_id: *index, name: name.to_owned(), - additional_columns: Some(AdditionalColumn { column_type: None }), + additional_column: Some(AdditionalColumn { column_type: None }), version: ColumnDescVersion::Pr13707 as i32, ..Default::default() }) diff --git a/src/connector/src/parser/bytes_parser.rs b/src/connector/src/parser/bytes_parser.rs index 07420a2d36ab1..44c035fa3ff5d 100644 --- a/src/connector/src/parser/bytes_parser.rs +++ b/src/connector/src/parser/bytes_parser.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_common::try_match_expand; use super::unified::bytes::BytesAccess; @@ -26,7 +25,7 @@ pub struct BytesAccessBuilder { impl AccessBuilder for BytesAccessBuilder { #[allow(clippy::unused_async)] - async fn generate_accessor(&mut self, payload: Vec) -> Result> { + async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { Ok(AccessImpl::Bytes(BytesAccess::new( &self.column_name, payload, @@ -35,7 +34,7 @@ impl AccessBuilder for BytesAccessBuilder { } impl BytesAccessBuilder { - pub fn new(encoding_properties: EncodingProperties) -> Result { + pub fn new(encoding_properties: EncodingProperties) -> anyhow::Result { let config = try_match_expand!(encoding_properties, EncodingProperties::Bytes)?; Ok(Self { column_name: config.column_name, diff --git a/src/connector/src/parser/canal/simd_json_parser.rs b/src/connector/src/parser/canal/simd_json_parser.rs index fdb317c96c20a..09a00c490a9f5 100644 --- a/src/connector/src/parser/canal/simd_json_parser.rs +++ b/src/connector/src/parser/canal/simd_json_parser.rs @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use anyhow::Context; use itertools::Itertools; -use risingwave_common::error::ErrorCode::{self, ProtocolError}; -use risingwave_common::error::{Result, RwError}; +use risingwave_common::bail; use simd_json::prelude::{MutableObject, ValueAsScalar, ValueObjectAccess}; use simd_json::BorrowedValue; @@ -44,7 +44,7 @@ impl CanalJsonParser { rw_columns: Vec, source_ctx: SourceContextRef, config: &JsonProperties, - ) -> Result { + ) -> anyhow::Result { Ok(Self { rw_columns, source_ctx, @@ -57,29 +57,22 @@ impl CanalJsonParser { &self, mut payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> Result<()> { + ) -> anyhow::Result<()> { let mut event: BorrowedValue<'_> = - simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..]) - .map_err(|e| RwError::from(ProtocolError(e.to_string())))?; + simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..])?; - let is_ddl = event.get(IS_DDL).and_then(|v| v.as_bool()).ok_or_else(|| { - RwError::from(ProtocolError( - "isDdl field not found in canal json".to_owned(), - )) - })?; + let is_ddl = event + .get(IS_DDL) + .and_then(|v| v.as_bool()) + .context("field `isDdl` not found in canal json")?; if is_ddl { - return Err(RwError::from(ProtocolError( - "received a DDL message, please set `canal.instance.filter.query.dml` to true." - .to_string(), - ))); + bail!("received a DDL message, please set `canal.instance.filter.query.dml` to true."); } let op = match event.get(OP).and_then(|v| v.as_str()) { Some(CANAL_INSERT_EVENT | CANAL_UPDATE_EVENT) => ChangeEventOperation::Upsert, Some(CANAL_DELETE_EVENT) => ChangeEventOperation::Delete, - _ => Err(RwError::from(ProtocolError( - "op field not found in canal json".to_owned(), - )))?, + _ => bail!("op field not found in canal json"), }; let events = event @@ -88,11 +81,7 @@ impl CanalJsonParser { BorrowedValue::Array(array) => Some(array), _ => None, }) - .ok_or_else(|| { - RwError::from(ProtocolError( - "'data' is missing for creating event".to_string(), - )) - })?; + .context("field `data` is missing for creating event")?; let mut errors = Vec::new(); for event in events.drain(..) { @@ -106,11 +95,12 @@ impl CanalJsonParser { if errors.is_empty() { Ok(()) } else { - Err(RwError::from(ErrorCode::InternalError(format!( + // TODO(error-handling): multiple errors + bail!( "failed to parse {} row(s) in a single canal json message: {}", errors.len(), - errors.iter().join(", ") - )))) + errors.iter().format(", ") + ); } } } @@ -133,7 +123,7 @@ impl ByteStreamSourceParser for CanalJsonParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { + ) -> anyhow::Result<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/csv_parser.rs b/src/connector/src/parser/csv_parser.rs index 77f0cdd30e41f..7bb67c9d7b510 100644 --- a/src/connector/src/parser/csv_parser.rs +++ b/src/connector/src/parser/csv_parser.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::ErrorCode::ProtocolError; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::types::{Date, Decimal, Time, Timestamp, Timestamptz}; use super::unified::{AccessError, AccessResult}; @@ -46,7 +44,7 @@ impl CsvParser { rw_columns: Vec, csv_props: CsvProperties, source_ctx: SourceContextRef, - ) -> Result { + ) -> anyhow::Result { let CsvProperties { delimiter, has_header, @@ -60,15 +58,14 @@ impl CsvParser { }) } - fn read_row(&self, buf: &[u8]) -> Result> { + fn read_row(&self, buf: &[u8]) -> anyhow::Result> { let mut reader_builder = csv::ReaderBuilder::default(); reader_builder.delimiter(self.delimiter).has_headers(false); let record = reader_builder .from_reader(buf) .records() .next() - .transpose() - .map_err(|err| RwError::from(ProtocolError(err.to_string())))?; + .transpose()?; Ok(record .map(|record| record.iter().map(|field| field.to_string()).collect()) .unwrap_or_default()) @@ -105,7 +102,7 @@ impl CsvParser { &mut self, payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> Result<()> { + ) -> anyhow::Result<()> { let mut fields = self.read_row(&payload)?; if let Some(headers) = &mut self.headers { @@ -161,7 +158,7 @@ impl ByteStreamSourceParser for CsvParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { + ) -> anyhow::Result<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/debezium/avro_parser.rs b/src/connector/src/parser/debezium/avro_parser.rs index e0a486fe18c98..6320ef5cdf3ec 100644 --- a/src/connector/src/parser/debezium/avro_parser.rs +++ b/src/connector/src/parser/debezium/avro_parser.rs @@ -17,8 +17,6 @@ use std::sync::Arc; use apache_avro::types::Value; use apache_avro::{from_avro_datum, Schema}; -use risingwave_common::error::ErrorCode::ProtocolError; -use risingwave_common::error::{Result, RwError}; use risingwave_common::try_match_expand; use risingwave_pb::catalog::PbSchemaRegistryNameStrategy; use risingwave_pb::plan_common::ColumnDesc; @@ -50,13 +48,10 @@ pub struct DebeziumAvroAccessBuilder { // TODO: reduce encodingtype match impl AccessBuilder for DebeziumAvroAccessBuilder { - async fn generate_accessor(&mut self, payload: Vec) -> Result> { + async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { let (schema_id, mut raw_payload) = extract_schema_id(&payload)?; let schema = self.schema_resolver.get(schema_id).await?; - self.value = Some( - from_avro_datum(schema.as_ref(), &mut raw_payload, None) - .map_err(|e| RwError::from(ProtocolError(e.to_string())))?, - ); + self.value = Some(from_avro_datum(schema.as_ref(), &mut raw_payload, None)?); self.key_schema = match self.encoding_type { EncodingType::Key => Some(schema), EncodingType::Value => None, @@ -72,19 +67,19 @@ impl AccessBuilder for DebeziumAvroAccessBuilder { } impl DebeziumAvroAccessBuilder { - pub fn new(config: DebeziumAvroParserConfig, encoding_type: EncodingType) -> Result { + pub fn new( + config: DebeziumAvroParserConfig, + encoding_type: EncodingType, + ) -> anyhow::Result { let DebeziumAvroParserConfig { outer_schema, schema_resolver, .. } = config; - let resolver = apache_avro::schema::ResolvedSchema::try_from(&*outer_schema) - .map_err(|e| RwError::from(ProtocolError(e.to_string())))?; + let resolver = apache_avro::schema::ResolvedSchema::try_from(&*outer_schema)?; // todo: to_resolved may cause stackoverflow if there's a loop in the schema - let schema = resolver - .to_resolved(&outer_schema) - .map_err(|e| RwError::from(ProtocolError(e.to_string())))?; + let schema = resolver.to_resolved(&outer_schema)?; Ok(Self { schema, schema_resolver, @@ -104,7 +99,7 @@ pub struct DebeziumAvroParserConfig { } impl DebeziumAvroParserConfig { - pub async fn new(encoding_config: EncodingProperties) -> Result { + pub async fn new(encoding_config: EncodingProperties) -> anyhow::Result { let avro_config = try_match_expand!(encoding_config, EncodingProperties::Avro)?; let schema_location = &avro_config.row_schema_location; let client_config = &avro_config.client_config; @@ -353,7 +348,7 @@ mod tests { #[ignore] #[tokio::test] - async fn test_debezium_avro_parser() -> Result<()> { + async fn test_debezium_avro_parser() -> anyhow::Result<()> { let props = convert_args!(hashmap!( "kafka.topic" => "dbserver1.inventory.customers" )); diff --git a/src/connector/src/parser/debezium/debezium_parser.rs b/src/connector/src/parser/debezium/debezium_parser.rs index 8ff32804a96d4..f69c19e691e97 100644 --- a/src/connector/src/parser/debezium/debezium_parser.rs +++ b/src/connector/src/parser/debezium/debezium_parser.rs @@ -12,8 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::ErrorCode::ProtocolError; -use risingwave_common::error::{Result, RwError}; +use risingwave_common::bail; use super::simd_json_parser::DebeziumJsonAccessBuilder; use super::{DebeziumAvroAccessBuilder, DebeziumAvroParserConfig}; @@ -38,7 +37,7 @@ pub struct DebeziumParser { async fn build_accessor_builder( config: EncodingProperties, encoding_type: EncodingType, -) -> Result { +) -> anyhow::Result { match config { EncodingProperties::Avro(_) => { let config = DebeziumAvroParserConfig::new(config).await?; @@ -52,9 +51,7 @@ async fn build_accessor_builder( EncodingProperties::Protobuf(_) => { Ok(AccessBuilderImpl::new_default(config, encoding_type).await?) } - _ => Err(RwError::from(ProtocolError( - "unsupported encoding for Debezium".to_string(), - ))), + _ => bail!("unsupported encoding for Debezium"), } } @@ -63,7 +60,7 @@ impl DebeziumParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> Result { + ) -> anyhow::Result { let (key_config, key_type) = extract_key_config!(props); let key_builder = build_accessor_builder(key_config, key_type).await?; let payload_builder = @@ -76,7 +73,7 @@ impl DebeziumParser { }) } - pub async fn new_for_test(rw_columns: Vec) -> Result { + pub async fn new_for_test(rw_columns: Vec) -> anyhow::Result { let props = SpecificParserConfig { key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { @@ -92,7 +89,7 @@ impl DebeziumParser { key: Option>, payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> Result { + ) -> anyhow::Result { // tombetone messages are handled implicitly by these accessors let key_accessor = match key { None => None, @@ -140,7 +137,7 @@ impl ByteStreamSourceParser for DebeziumParser { _key: Option>, _payload: Option>, _writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { + ) -> anyhow::Result<()> { unreachable!("should call `parse_one_with_txn` instead") } @@ -149,7 +146,7 @@ impl ByteStreamSourceParser for DebeziumParser { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result { + ) -> anyhow::Result { self.parse_inner(key, payload, writer).await } } diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index 21129a816e0a3..74c8a4ee8cfbc 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -14,8 +14,8 @@ use std::fmt::Debug; -use risingwave_common::error::ErrorCode::{self, ProtocolError}; -use risingwave_common::error::{Result, RwError}; +use anyhow::Context; +use risingwave_common::bail; use risingwave_common::types::DataType; use simd_json::prelude::MutableObject; use simd_json::BorrowedValue; @@ -36,7 +36,10 @@ pub struct DebeziumMongoJsonParser { } impl DebeziumMongoJsonParser { - pub fn new(rw_columns: Vec, source_ctx: SourceContextRef) -> Result { + pub fn new( + rw_columns: Vec, + source_ctx: SourceContextRef, + ) -> anyhow::Result { let id_column = rw_columns .iter() .find(|desc| { @@ -49,25 +52,16 @@ impl DebeziumMongoJsonParser { | DataType::Int64 ) }) - .ok_or_else(|| RwError::from(ProtocolError( - "Debezium Mongo needs a `_id` column with supported types (Varchar Jsonb int32 int64) in table".into(), - )))?.clone(); + .context("Debezium Mongo needs a `_id` column with supported types (Varchar Jsonb int32 int64) in table")?.clone(); let payload_column = rw_columns .iter() .find(|desc| desc.name == "payload" && matches!(desc.data_type, DataType::Jsonb)) - .ok_or_else(|| { - RwError::from(ProtocolError( - "Debezium Mongo needs a `payload` column with supported types Jsonb in table" - .into(), - )) - })? + .context("Debezium Mongo needs a `payload` column with supported types Jsonb in table")? .clone(); // _rw_{connector}_file/partition & _rw_{connector}_offset are created automatically. if rw_columns.iter().filter(|desc| desc.is_visible()).count() != 2 { - return Err(RwError::from(ProtocolError( - "Debezium Mongo needs no more columns except `_id` and `payload` in table".into(), - ))); + bail!("Debezium Mongo needs no more columns except `_id` and `payload` in table"); } Ok(Self { @@ -84,13 +78,11 @@ impl DebeziumMongoJsonParser { &self, mut payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> Result<()> { - let mut event: BorrowedValue<'_> = simd_json::to_borrowed_value(&mut payload) - .map_err(|e| RwError::from(ProtocolError(e.to_string())))?; + ) -> anyhow::Result<()> { + let mut event: BorrowedValue<'_> = simd_json::to_borrowed_value(&mut payload)?; // Event can be configured with and without the "payload" field present. // See https://github.com/risingwavelabs/risingwave/issues/10178 - let payload = if let Some(payload) = event.get_mut("payload") { std::mem::take(payload) } else { @@ -123,7 +115,7 @@ impl ByteStreamSourceParser for DebeziumMongoJsonParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { + ) -> anyhow::Result<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index 87aead865c6db..3ad3609c78063 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -14,7 +14,6 @@ use std::fmt::Debug; -use risingwave_common::error::{ErrorCode, Result, RwError}; use simd_json::prelude::MutableObject; use simd_json::BorrowedValue; @@ -28,18 +27,17 @@ pub struct DebeziumJsonAccessBuilder { } impl DebeziumJsonAccessBuilder { - pub fn new() -> Result { + pub fn new() -> anyhow::Result { Ok(Self { value: None }) } } impl AccessBuilder for DebeziumJsonAccessBuilder { #[allow(clippy::unused_async)] - async fn generate_accessor(&mut self, payload: Vec) -> Result> { + async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { self.value = Some(payload); let mut event: BorrowedValue<'_> = - simd_json::to_borrowed_value(self.value.as_mut().unwrap()) - .map_err(|e| RwError::from(ErrorCode::ProtocolError(e.to_string())))?; + simd_json::to_borrowed_value(self.value.as_mut().unwrap())?; let payload = if let Some(payload) = event.get_mut("payload") { std::mem::take(payload) @@ -567,7 +565,7 @@ mod tests { column_type: SourceColumnType::Normal, is_pk: false, is_hidden_addition_col: false, - additional_column_type: AdditionalColumn { column_type: None }, + additional_column: AdditionalColumn { column_type: None }, }, SourceColumnDesc::simple("o_enum", DataType::Varchar, ColumnId::from(8)), SourceColumnDesc::simple("o_char", DataType::Varchar, ColumnId::from(9)), diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 8480138e02c07..74c82d14d8065 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -14,12 +14,11 @@ use std::collections::HashMap; +use anyhow::Context as _; use apache_avro::Schema; use itertools::{Either, Itertools}; use jst::{convert_avro, Context}; -use risingwave_common::error::ErrorCode::{self, InternalError, ProtocolError}; -use risingwave_common::error::{Result, RwError}; -use risingwave_common::try_match_expand; +use risingwave_common::{bail, try_match_expand}; use risingwave_pb::plan_common::ColumnDesc; use super::avro::schema_resolver::ConfluentSchemaResolver; @@ -44,7 +43,7 @@ pub struct JsonAccessBuilder { impl AccessBuilder for JsonAccessBuilder { #[allow(clippy::unused_async)] - async fn generate_accessor(&mut self, payload: Vec) -> Result> { + async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { if payload.is_empty() { self.value = Some("{}".into()); } else { @@ -52,8 +51,7 @@ impl AccessBuilder for JsonAccessBuilder { } let value = simd_json::to_borrowed_value( &mut self.value.as_mut().unwrap()[self.payload_start_idx..], - ) - .map_err(|e| RwError::from(ProtocolError(e.to_string())))?; + )?; Ok(AccessImpl::Json(JsonAccess::new_with_options( value, // Debezium and Canal have their special json access builder and will not @@ -64,7 +62,7 @@ impl AccessBuilder for JsonAccessBuilder { } impl JsonAccessBuilder { - pub fn new(use_schema_registry: bool) -> Result { + pub fn new(use_schema_registry: bool) -> anyhow::Result { Ok(Self { value: None, payload_start_idx: if use_schema_registry { 5 } else { 0 }, @@ -86,7 +84,7 @@ impl JsonParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> Result { + ) -> anyhow::Result { let json_config = try_match_expand!(props.encoding_config, EncodingProperties::Json)?; let payload_start_idx = if json_config.use_schema_registry { 5 @@ -100,7 +98,7 @@ impl JsonParser { }) } - pub fn new_for_test(rw_columns: Vec) -> Result { + pub fn new_for_test(rw_columns: Vec) -> anyhow::Result { Ok(Self { rw_columns, source_ctx: Default::default(), @@ -113,9 +111,8 @@ impl JsonParser { &self, mut payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> Result<()> { - let value = simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..]) - .map_err(|e| RwError::from(ProtocolError(e.to_string())))?; + ) -> anyhow::Result<()> { + let value = simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..])?; let values = if let simd_json::BorrowedValue::Array(arr) = value { Either::Left(arr.into_iter()) } else { @@ -134,11 +131,12 @@ impl JsonParser { if errors.is_empty() { Ok(()) } else { - Err(RwError::from(ErrorCode::InternalError(format!( + // TODO(error-handling): multiple errors + bail!( "failed to parse {} row(s) in a single json message: {}", errors.len(), - errors.iter().join(", ") - )))) + errors.iter().format(", ") + ); } } } @@ -165,8 +163,7 @@ pub async fn schema_to_columns( }; let context = Context::default(); let avro_schema = convert_avro(&json_schema, context).to_string(); - let schema = Schema::parse_str(&avro_schema) - .map_err(|e| RwError::from(InternalError(format!("Avro schema parse error {}", e))))?; + let schema = Schema::parse_str(&avro_schema).context("failed to parse avro schema")?; avro_schema_to_column_descs(&schema) } @@ -188,7 +185,7 @@ impl ByteStreamSourceParser for JsonParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { + ) -> anyhow::Result<()> { only_parse_payload!(self, payload, writer) } } @@ -582,7 +579,7 @@ mod tests { column_type: SourceColumnType::Normal, is_pk: true, is_hidden_addition_col: false, - additional_column_type: AdditionalColumn { + additional_column: AdditionalColumn { column_type: Some(AdditionalColumnType::Key(AdditionalColumnKey {})), }, }; diff --git a/src/connector/src/parser/maxwell/maxwell_parser.rs b/src/connector/src/parser/maxwell/maxwell_parser.rs index 06c2231e5d104..aa661585fa1ff 100644 --- a/src/connector/src/parser/maxwell/maxwell_parser.rs +++ b/src/connector/src/parser/maxwell/maxwell_parser.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::{ErrorCode, Result, RwError}; +use risingwave_common::bail; use crate::only_parse_payload; use crate::parser::unified::maxwell::MaxwellChangeEvent; @@ -35,7 +35,7 @@ impl MaxwellParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> Result { + ) -> anyhow::Result { match props.encoding_config { EncodingProperties::Json(_) => { let payload_builder = @@ -47,9 +47,7 @@ impl MaxwellParser { source_ctx, }) } - _ => Err(RwError::from(ErrorCode::ProtocolError( - "unsupported encoding for Maxwell".to_string(), - ))), + _ => bail!("unsupported encoding for Maxwell"), } } @@ -57,7 +55,7 @@ impl MaxwellParser { &mut self, payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> Result<()> { + ) -> anyhow::Result<()> { let payload_accessor = self.payload_builder.generate_accessor(payload).await?; let row_op = MaxwellChangeEvent::new(payload_accessor); @@ -83,7 +81,7 @@ impl ByteStreamSourceParser for MaxwellParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { + ) -> anyhow::Result<()> { // restrict the behaviours since there is no corresponding // key/value test for maxwell yet. only_parse_payload!(self, payload, writer) diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 7884eb62bf139..aa36c610a4d03 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -26,9 +26,8 @@ use futures_async_stream::try_stream; pub use json_parser::*; pub use protobuf::*; use risingwave_common::array::{ArrayBuilderImpl, Op, StreamChunk}; +use risingwave_common::bail; use risingwave_common::catalog::{KAFKA_TIMESTAMP_COLUMN_NAME, TABLE_NAME_COLUMN_NAME}; -use risingwave_common::error::ErrorCode::ProtocolError; -use risingwave_common::error::{Result, RwError}; use risingwave_common::log::LogSuppresser; use risingwave_common::types::{Datum, Scalar, ScalarImpl}; use risingwave_common::util::iter_util::ZipEqFast; @@ -318,7 +317,7 @@ impl SourceStreamChunkRowWriter<'_> { mut f: impl FnMut(&SourceColumnDesc) -> AccessResult, ) -> AccessResult<()> { let mut wrapped_f = |desc: &SourceColumnDesc| { - match (&desc.column_type, &desc.additional_column_type.column_type) { + match (&desc.column_type, &desc.additional_column.column_type) { (&SourceColumnType::Offset | &SourceColumnType::RowId, _) => { // SourceColumnType is for CDC source only. Ok(A::output_for( @@ -536,7 +535,7 @@ pub trait ByteStreamSourceParser: Send + Debug + Sized + 'static { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a; + ) -> impl Future> + Send + 'a; /// Parse one record from the given `payload`, either write rows to the `writer` or interpret it /// as a transaction control message. @@ -550,7 +549,7 @@ pub trait ByteStreamSourceParser: Send + Debug + Sized + 'static { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a { + ) -> impl Future> + Send + 'a { self.parse_one(key, payload, writer) .map_ok(|_| ParseResult::Rows) } @@ -589,7 +588,7 @@ const MAX_ROWS_FOR_TRANSACTION: usize = 4096; // TODO: when upsert is disabled, how to filter those empty payload // Currently, an err is returned for non upsert with empty payload -#[try_stream(ok = StreamChunk, error = RwError)] +#[try_stream(ok = StreamChunk, error = anyhow::Error)] async fn into_chunk_stream(mut parser: P, data_stream: BoxSourceStream) { let columns = parser.columns().to_vec(); @@ -686,7 +685,7 @@ async fn into_chunk_stream(mut parser: P, data_stream "failed to parse message, skipping" ); } - parser.source_ctx().report_user_source_error(error); + parser.source_ctx().report_user_source_error(&*error); } } @@ -729,7 +728,7 @@ async fn into_chunk_stream(mut parser: P, data_stream } pub trait AccessBuilder { - async fn generate_accessor(&mut self, payload: Vec) -> Result>; + async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result>; } #[derive(Debug)] @@ -749,7 +748,7 @@ pub enum AccessBuilderImpl { } impl AccessBuilderImpl { - pub async fn new_default(config: EncodingProperties, kv: EncodingType) -> Result { + pub async fn new_default(config: EncodingProperties, kv: EncodingType) -> anyhow::Result { let accessor = match config { EncodingProperties::Avro(_) => { let config = AvroParserConfig::new(config).await?; @@ -770,7 +769,10 @@ impl AccessBuilderImpl { Ok(accessor) } - pub async fn generate_accessor(&mut self, payload: Vec) -> Result> { + pub async fn generate_accessor( + &mut self, + payload: Vec, + ) -> anyhow::Result> { let accessor = match self { Self::Avro(builder) => builder.generate_accessor(payload).await?, Self::Protobuf(builder) => builder.generate_accessor(payload).await?, @@ -816,7 +818,10 @@ impl ByteStreamSourceParserImpl { } impl ByteStreamSourceParserImpl { - pub async fn create(parser_config: ParserConfig, source_ctx: SourceContextRef) -> Result { + pub async fn create( + parser_config: ParserConfig, + source_ctx: SourceContextRef, + ) -> anyhow::Result { let CommonParserConfig { rw_columns } = parser_config.common; let protocol = &parser_config.specific.protocol_config; let encode = &parser_config.specific.encoding_config; @@ -960,7 +965,10 @@ pub enum ProtocolProperties { impl SpecificParserConfig { // The validity of (format, encode) is ensured by `extract_format_encode` - pub fn new(info: &StreamSourceInfo, with_properties: &HashMap) -> Result { + pub fn new( + info: &StreamSourceInfo, + with_properties: &HashMap, + ) -> anyhow::Result { let source_struct = extract_source_struct(info)?; let format = source_struct.format; let encode = source_struct.encode; @@ -1016,9 +1024,7 @@ impl SpecificParserConfig { (SourceFormat::Plain, SourceEncode::Protobuf) | (SourceFormat::Upsert, SourceEncode::Protobuf) => { if info.row_schema_location.is_empty() { - return Err( - ProtocolError("protobuf file location not provided".to_string()).into(), - ); + bail!("protobuf file location not provided"); } let mut config = ProtobufProperties { message_name: info.proto_message_name.clone(), @@ -1081,10 +1087,7 @@ impl SpecificParserConfig { } (SourceFormat::Native, SourceEncode::Native) => EncodingProperties::Native, (format, encode) => { - return Err(RwError::from(ProtocolError(format!( - "Unsupported format {:?} encode {:?}", - format, encode - )))); + bail!("Unsupported format {:?} encode {:?}", format, encode); } }; Ok(Self { diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 75912155a0dd6..5fdb9fbf3c6ca 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -12,8 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::ErrorCode::ProtocolError; -use risingwave_common::error::{Result, RwError}; +use risingwave_common::bail; use super::{ AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, @@ -44,7 +43,7 @@ impl PlainParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> Result { + ) -> anyhow::Result { let key_builder = if let Some(key_column_name) = get_key_column_name(&rw_columns) { Some(AccessBuilderImpl::Bytes(BytesAccessBuilder::new( EncodingProperties::Bytes(BytesProperties { @@ -62,11 +61,7 @@ impl PlainParser { | EncodingProperties::Bytes(_) => { AccessBuilderImpl::new_default(props.encoding_config, EncodingType::Value).await? } - _ => { - return Err(RwError::from(ProtocolError( - "unsupported encoding for Plain".to_string(), - ))); - } + _ => bail!("Unsupported encoding for Plain"), }; let transaction_meta_builder = Some(AccessBuilderImpl::DebeziumJson( @@ -86,7 +81,7 @@ impl PlainParser { key: Option>, payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> Result { + ) -> anyhow::Result { // if the message is transaction metadata, parse it and return if let Some(msg_meta) = writer.row_meta && let SourceMeta::DebeziumCdc(cdc_meta) = msg_meta.meta @@ -150,7 +145,7 @@ impl ByteStreamSourceParser for PlainParser { _key: Option>, _payload: Option>, _writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { + ) -> anyhow::Result<()> { unreachable!("should call `parse_one_with_txn` instead") } @@ -159,7 +154,7 @@ impl ByteStreamSourceParser for PlainParser { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result { + ) -> anyhow::Result { self.parse_inner(key, payload, writer).await } } diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index 13da5c5b86b2d..922705e3d3f8f 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -14,16 +14,15 @@ use std::sync::Arc; +use anyhow::Context; use itertools::Itertools; use prost_reflect::{ Cardinality, DescriptorPool, DynamicMessage, FieldDescriptor, Kind, MessageDescriptor, ReflectMessage, Value, }; use risingwave_common::array::{ListValue, StructValue}; -use risingwave_common::error::ErrorCode::ProtocolError; -use risingwave_common::error::{Result, RwError}; -use risingwave_common::try_match_expand; use risingwave_common::types::{DataType, Datum, Decimal, JsonbVal, ScalarImpl, F32, F64}; +use risingwave_common::{bail, try_match_expand}; use risingwave_pb::plan_common::{AdditionalColumn, ColumnDesc, ColumnDescVersion}; use thiserror::Error; use thiserror_ext::{AsReport, Macro}; @@ -48,7 +47,7 @@ pub struct ProtobufAccessBuilder { impl AccessBuilder for ProtobufAccessBuilder { #[allow(clippy::unused_async)] - async fn generate_accessor(&mut self, payload: Vec) -> Result> { + async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { let payload = if self.confluent_wire_type { resolve_pb_header(&payload)? } else { @@ -56,7 +55,7 @@ impl AccessBuilder for ProtobufAccessBuilder { }; let message = DynamicMessage::decode(self.message_descriptor.clone(), payload) - .map_err(|e| ProtocolError(format!("parse message failed: {}", e)))?; + .context("failed to parse message")?; Ok(AccessImpl::Protobuf(ProtobufAccess::new( message, @@ -66,7 +65,7 @@ impl AccessBuilder for ProtobufAccessBuilder { } impl ProtobufAccessBuilder { - pub fn new(config: ProtobufParserConfig) -> Result { + pub fn new(config: ProtobufParserConfig) -> anyhow::Result { let ProtobufParserConfig { confluent_wire_type, message_descriptor, @@ -90,17 +89,15 @@ pub struct ProtobufParserConfig { } impl ProtobufParserConfig { - pub async fn new(encoding_properties: EncodingProperties) -> Result { + pub async fn new(encoding_properties: EncodingProperties) -> anyhow::Result { let protobuf_config = try_match_expand!(encoding_properties, EncodingProperties::Protobuf)?; let location = &protobuf_config.row_schema_location; let message_name = &protobuf_config.message_name; let url = handle_sr_list(location.as_str())?; - if let Some(name) = protobuf_config.key_message_name { + if protobuf_config.key_message_name.is_some() { // https://docs.confluent.io/platform/7.5/control-center/topics/schema.html#c3-schemas-best-practices-key-value-pairs - return Err(RwError::from(ProtocolError(format!( - "key.message = {name} not used. Protobuf key unsupported." - )))); + bail!("protobuf key is not supported"); } let schema_bytes = if protobuf_config.use_schema_registry { let schema_value = get_subject_by_strategy( @@ -118,18 +115,14 @@ impl ProtobufParserConfig { bytes_from_url(url, protobuf_config.aws_auth_props.as_ref()).await? }; - let pool = DescriptorPool::decode(schema_bytes.as_slice()).map_err(|e| { - ProtocolError(format!( - "cannot build descriptor pool from schema: {}, error: {}", - location, e - )) - })?; + let pool = DescriptorPool::decode(schema_bytes.as_slice()) + .with_context(|| format!("cannot build descriptor pool from schema `{}`", location))?; - let message_descriptor = pool.get_message_by_name(message_name).ok_or_else(|| { - ProtocolError(format!( - "Cannot find message {} in schema: {}.\nDescriptor pool is {:?}", - message_name, location, pool - )) + let message_descriptor = pool.get_message_by_name(message_name).with_context(|| { + format!( + "cannot find message `{}` in schema `{}`", + message_name, location, + ) })?; Ok(Self { @@ -140,7 +133,7 @@ impl ProtobufParserConfig { } /// Maps the protobuf schema to relational schema. - pub fn map_to_columns(&self) -> Result> { + pub fn map_to_columns(&self) -> anyhow::Result> { let mut columns = Vec::with_capacity(self.message_descriptor.fields().len()); let mut index = 0; let mut parse_trace: Vec = vec![]; @@ -160,16 +153,15 @@ impl ProtobufParserConfig { field_descriptor: &FieldDescriptor, index: &mut i32, parse_trace: &mut Vec, - ) -> Result { - let field_type = - protobuf_type_mapping(field_descriptor, parse_trace).map_err(RwError::uncategorized)?; + ) -> anyhow::Result { + let field_type = protobuf_type_mapping(field_descriptor, parse_trace)?; if let Kind::Message(m) = field_descriptor.kind() { let field_descs = if let DataType::List { .. } = field_type { vec![] } else { m.fields() .map(|f| Self::pb_field_to_col_desc(&f, index, parse_trace)) - .collect::>>()? + .try_collect()? }; *index += 1; Ok(ColumnDesc { @@ -180,7 +172,7 @@ impl ProtobufParserConfig { type_name: m.full_name().to_string(), generated_or_default_column: None, description: None, - additional_columns: Some(AdditionalColumn { column_type: None }), + additional_column: Some(AdditionalColumn { column_type: None }), version: ColumnDescVersion::Pr13707 as i32, }) } else { @@ -189,7 +181,7 @@ impl ProtobufParserConfig { column_id: *index, name: field_descriptor.name().to_string(), column_type: Some(field_type.to_protobuf()), - additional_columns: Some(AdditionalColumn { column_type: None }), + additional_column: Some(AdditionalColumn { column_type: None }), version: ColumnDescVersion::Pr13707 as i32, ..Default::default() }) @@ -533,7 +525,7 @@ fn protobuf_type_mapping( /// Wire format for Confluent pb header is: /// | 0 | 1-4 | 5-x | x+1-end /// | magic-byte | schema-id | message-indexes | protobuf-payload -pub(crate) fn resolve_pb_header(payload: &[u8]) -> Result<&[u8]> { +pub(crate) fn resolve_pb_header(payload: &[u8]) -> anyhow::Result<&[u8]> { // there's a message index array at the front of payload // if it is the first message in proto def, the array is just and `0` // TODO: support parsing more complex index array @@ -544,9 +536,7 @@ pub(crate) fn resolve_pb_header(payload: &[u8]) -> Result<&[u8]> { match remained.first() { Some(0) => Ok(&remained[1..]), Some(i) => Ok(&remained[(*i as usize)..]), - None => Err(RwError::from(ProtocolError( - "The proto payload is empty".to_owned(), - ))), + None => bail!("The proto payload is empty"), } } @@ -585,7 +575,7 @@ mod test { static PRE_GEN_PROTO_DATA: &[u8] = b"\x08\x7b\x12\x0c\x74\x65\x73\x74\x20\x61\x64\x64\x72\x65\x73\x73\x1a\x09\x74\x65\x73\x74\x20\x63\x69\x74\x79\x20\xc8\x03\x2d\x19\x04\x9e\x3f\x32\x0a\x32\x30\x32\x31\x2d\x30\x31\x2d\x30\x31"; #[tokio::test] - async fn test_simple_schema() -> Result<()> { + async fn test_simple_schema() -> anyhow::Result<()> { let location = schema_dir() + "/simple-schema"; println!("location: {}", location); let message_name = "test.TestRecord"; @@ -630,7 +620,7 @@ mod test { } #[tokio::test] - async fn test_complex_schema() -> Result<()> { + async fn test_complex_schema() -> anyhow::Result<()> { let location = schema_dir() + "/complex-schema"; let message_name = "test.User"; @@ -922,7 +912,7 @@ mod test { static ANY_GEN_PROTO_DATA: &[u8] = b"\x08\xb9\x60\x12\x32\x0a\x24\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x53\x74\x72\x69\x6e\x67\x56\x61\x6c\x75\x65\x12\x0a\x0a\x08\x4a\x6f\x68\x6e\x20\x44\x6f\x65"; #[tokio::test] - async fn test_any_schema() -> Result<()> { + async fn test_any_schema() -> anyhow::Result<()> { let conf = create_recursive_pb_parser_config("/any-schema.pb", "test.TestAny").await; println!("Current conf: {:#?}", conf); @@ -983,7 +973,7 @@ mod test { static ANY_GEN_PROTO_DATA_1: &[u8] = b"\x08\xb9\x60\x12\x2b\x0a\x23\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x49\x6e\x74\x33\x32\x56\x61\x6c\x75\x65\x12\x04\x08\xd2\xfe\x06"; #[tokio::test] - async fn test_any_schema_1() -> Result<()> { + async fn test_any_schema_1() -> anyhow::Result<()> { let conf = create_recursive_pb_parser_config("/any-schema.pb", "test.TestAny").await; println!("Current conf: {:#?}", conf); @@ -1052,7 +1042,7 @@ mod test { static ANY_RECURSIVE_GEN_PROTO_DATA: &[u8] = b"\x08\xb9\x60\x12\x84\x01\x0a\x21\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x41\x6e\x79\x56\x61\x6c\x75\x65\x12\x5f\x0a\x30\x0a\x24\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x53\x74\x72\x69\x6e\x67\x56\x61\x6c\x75\x65\x12\x08\x0a\x06\x31\x31\x34\x35\x31\x34\x12\x2b\x0a\x23\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x49\x6e\x74\x33\x32\x56\x61\x6c\x75\x65\x12\x04\x08\xd2\xfe\x06"; #[tokio::test] - async fn test_any_recursive() -> Result<()> { + async fn test_any_recursive() -> anyhow::Result<()> { let conf = create_recursive_pb_parser_config("/any-schema.pb", "test.TestAny").await; println!("Current conf: {:#?}", conf); diff --git a/src/connector/src/parser/protobuf/schema_resolver.rs b/src/connector/src/parser/protobuf/schema_resolver.rs index 47720ae13b9f6..919413a0c50dc 100644 --- a/src/connector/src/parser/protobuf/schema_resolver.rs +++ b/src/connector/src/parser/protobuf/schema_resolver.rs @@ -15,13 +15,12 @@ use std::iter; use std::path::Path; +use anyhow::Context; use itertools::Itertools; use protobuf_native::compiler::{ SimpleErrorCollector, SourceTreeDescriptorDatabase, VirtualSourceTree, }; use protobuf_native::MessageLite; -use risingwave_common::error::ErrorCode::{InternalError, ProtocolError}; -use risingwave_common::error::{Result, RwError}; use crate::schema::schema_registry::Client; @@ -55,9 +54,11 @@ const WELL_KNOWN_TYPES: &[(&str, &[u8])] = embed_wkts![ pub(super) async fn compile_file_descriptor_from_schema_registry( subject_name: &str, client: &Client, -) -> Result> { - let (primary_subject, dependency_subjects) = - client.get_subject_and_references(subject_name).await?; +) -> anyhow::Result> { + let (primary_subject, dependency_subjects) = client + .get_subject_and_references(subject_name) + .await + .with_context(|| format!("failed to resolve subject `{subject_name}`"))?; // Compile .proto files into a file descriptor set. let mut source_tree = VirtualSourceTree::new(); @@ -81,12 +82,13 @@ pub(super) async fn compile_file_descriptor_from_schema_registry( db.as_mut() .build_file_descriptor_set(&[Path::new(&primary_subject.name)]) } - .map_err(|_| { - RwError::from(ProtocolError(format!( + .with_context(|| { + format!( "build_file_descriptor_set failed. Errors:\n{}", error_collector.as_mut().join("\n") - ))) + ) })?; - fds.serialize() - .map_err(|_| RwError::from(InternalError("serialize descriptor set failed".to_owned()))) + + let serialized = fds.serialize().context("serialize descriptor set failed")?; + Ok(serialized) } diff --git a/src/connector/src/parser/unified/avro.rs b/src/connector/src/parser/unified/avro.rs index 1cebb0fd60dac..f00e26a32faed 100644 --- a/src/connector/src/parser/unified/avro.rs +++ b/src/connector/src/parser/unified/avro.rs @@ -22,7 +22,6 @@ use chrono::Datelike; use itertools::Itertools; use num_bigint::{BigInt, Sign}; use risingwave_common::array::{ListValue, StructValue}; -use risingwave_common::error::Result as RwResult; use risingwave_common::log::LogSuppresser; use risingwave_common::types::{ DataType, Date, Datum, Interval, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, @@ -334,7 +333,7 @@ pub(crate) fn avro_decimal_to_rust_decimal( avro_decimal: AvroDecimal, _precision: usize, scale: usize, -) -> RwResult { +) -> AccessResult { let negative = !avro_decimal.is_positive(); let bytes = avro_decimal.to_vec_unsigned(); @@ -430,7 +429,6 @@ pub(crate) fn unix_epoch_days() -> i32 { #[cfg(test)] mod tests { use apache_avro::Decimal as AvroDecimal; - use risingwave_common::error::{ErrorCode, RwError}; use risingwave_common::types::{Decimal, Timestamptz}; use super::*; @@ -483,13 +481,13 @@ mod tests { value: Value, value_schema: &Schema, shape: &DataType, - ) -> RwResult { + ) -> anyhow::Result { AvroParseOptions { schema: Some(value_schema), relax_numeric: true, } .parse(&value, Some(shape)) - .map_err(|err| RwError::from(ErrorCode::InternalError(format!("{:?}", err)))) + .map_err(Into::into) } #[test] diff --git a/src/connector/src/parser/unified/upsert.rs b/src/connector/src/parser/unified/upsert.rs index 8fbed9dc2ac52..9129f0d16d864 100644 --- a/src/connector/src/parser/unified/upsert.rs +++ b/src/connector/src/parser/unified/upsert.rs @@ -105,7 +105,7 @@ where } fn access_field(&self, desc: &SourceColumnDesc) -> super::AccessResult { - match desc.additional_column_type.column_type { + match desc.additional_column.column_type { Some(AdditionalColumnType::Key(_)) => { if let Some(key_as_column_name) = &self.key_column_name && &desc.name == key_as_column_name diff --git a/src/connector/src/parser/unified/util.rs b/src/connector/src/parser/unified/util.rs index b7b00667e7de4..948190edf685a 100644 --- a/src/connector/src/parser/unified/util.rs +++ b/src/connector/src/parser/unified/util.rs @@ -12,9 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::{ErrorCode, RwError}; - -use super::{Access, AccessError, AccessResult, ChangeEvent}; +use super::{Access, AccessResult, ChangeEvent}; use crate::parser::unified::ChangeEventOperation; use crate::parser::SourceStreamChunkRowWriter; use crate::source::SourceColumnDesc; @@ -45,9 +43,3 @@ pub fn apply_row_accessor_on_stream_chunk_writer( ) -> AccessResult<()> { writer.insert(|column| accessor.access(&[&column.name], Some(&column.data_type))) } - -impl From for RwError { - fn from(val: AccessError) -> Self { - ErrorCode::InternalError(format!("AccessError: {:?}", val)).into() - } -} diff --git a/src/connector/src/parser/upsert_parser.rs b/src/connector/src/parser/upsert_parser.rs index fbd82f6f3c167..b8abf4785bb4a 100644 --- a/src/connector/src/parser/upsert_parser.rs +++ b/src/connector/src/parser/upsert_parser.rs @@ -12,8 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::ErrorCode::ProtocolError; -use risingwave_common::error::{Result, RwError}; +use risingwave_common::bail; use risingwave_pb::plan_common::additional_column::ColumnType as AdditionalColumnType; use super::bytes_parser::BytesAccessBuilder; @@ -38,23 +37,21 @@ pub struct UpsertParser { async fn build_accessor_builder( config: EncodingProperties, encoding_type: EncodingType, -) -> Result { +) -> anyhow::Result { match config { EncodingProperties::Json(_) | EncodingProperties::Protobuf(_) | EncodingProperties::Avro(_) => { Ok(AccessBuilderImpl::new_default(config, encoding_type).await?) } - _ => Err(RwError::from(ProtocolError( - "unsupported encoding for Upsert".to_string(), - ))), + _ => bail!("unsupported encoding for Upsert"), } } pub fn get_key_column_name(columns: &[SourceColumnDesc]) -> Option { columns.iter().find_map(|column| { if matches!( - column.additional_column_type.column_type, + column.additional_column.column_type, Some(AdditionalColumnType::Key(_)) ) { Some(column.name.clone()) @@ -69,7 +66,7 @@ impl UpsertParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> Result { + ) -> anyhow::Result { // check whether columns has Key as AdditionalColumnType, if so, the key accessor should be // bytes let key_builder = if let Some(key_column_name) = get_key_column_name(&rw_columns) { @@ -98,7 +95,7 @@ impl UpsertParser { key: Option>, payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> Result<()> { + ) -> anyhow::Result<()> { let mut row_op: UpsertChangeEvent, AccessImpl<'_, '_>> = UpsertChangeEvent::default(); let mut change_event_op = ChangeEventOperation::Delete; @@ -136,7 +133,7 @@ impl ByteStreamSourceParser for UpsertParser { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { + ) -> anyhow::Result<()> { self.parse_inner(key, payload, writer).await } } diff --git a/src/connector/src/parser/util.rs b/src/connector/src/parser/util.rs index 81819d7b0953d..ccb0e47310d9a 100644 --- a/src/connector/src/parser/util.rs +++ b/src/connector/src/parser/util.rs @@ -13,10 +13,10 @@ // limitations under the License. use std::collections::HashMap; +use anyhow::Context; use bytes::Bytes; use reqwest::Url; -use risingwave_common::error::ErrorCode::{InvalidParameterValue, ProtocolError}; -use risingwave_common::error::{Result, RwError}; +use risingwave_common::bail; use risingwave_common::types::Datum; use risingwave_pb::data::DataType as PbDataType; @@ -25,7 +25,7 @@ use crate::common::AwsAuthProps; use crate::source::SourceMeta; /// get kafka topic name -pub(super) fn get_kafka_topic(props: &HashMap) -> Result<&String> { +pub(super) fn get_kafka_topic(props: &HashMap) -> anyhow::Result<&String> { const KAFKA_TOPIC_KEY1: &str = "kafka.topic"; const KAFKA_TOPIC_KEY2: &str = "topic"; @@ -36,30 +36,28 @@ pub(super) fn get_kafka_topic(props: &HashMap) -> Result<&String return Ok(topic); } - Err(RwError::from(ProtocolError(format!( + // config + bail!( "Must specify '{}' or '{}'", - KAFKA_TOPIC_KEY1, KAFKA_TOPIC_KEY2, - )))) + KAFKA_TOPIC_KEY1, + KAFKA_TOPIC_KEY2 + ) } /// download bytes from http(s) url -pub(super) async fn download_from_http(location: &Url) -> Result { - let res = reqwest::get(location.clone()).await.map_err(|e| { - InvalidParameterValue(format!( - "failed to make request to URL: {}, err: {}", - location, e - )) - })?; - if !res.status().is_success() { - return Err(RwError::from(InvalidParameterValue(format!( - "Http request err, URL: {}, status code: {}", - location, - res.status() - )))); - } - res.bytes() +pub(super) async fn download_from_http(location: &Url) -> anyhow::Result { + let res = reqwest::get(location.clone()) + .await + .with_context(|| format!("failed to make request to {location}"))? + .error_for_status() + .with_context(|| format!("http request failed for {location}"))?; + + let bytes = res + .bytes() .await - .map_err(|e| InvalidParameterValue(format!("failed to read HTTP body: {}", e)).into()) + .with_context(|| format!("failed to read HTTP body of {location}"))?; + + Ok(bytes) } // For parser that doesn't support key currently @@ -69,9 +67,7 @@ macro_rules! only_parse_payload { if let Some(payload) = $payload { $self.parse_inner(payload, $writer).await } else { - Err(RwError::from(ErrorCode::InternalError( - "Empty payload with nonempty key".into(), - ))) + risingwave_common::bail!("empty payload with non-empty key") } }; } @@ -96,20 +92,23 @@ macro_rules! extract_key_config { /// * local file, for on-premise or testing. /// * http/https, for common usage. /// * s3 file location format: -pub(super) async fn bytes_from_url(url: &Url, config: Option<&AwsAuthProps>) -> Result> { +pub(super) async fn bytes_from_url( + url: &Url, + config: Option<&AwsAuthProps>, +) -> anyhow::Result> { match (url.scheme(), config) { // TODO(Tao): support local file only when it's compiled in debug mode. ("file", _) => { let path = url .to_file_path() - .map_err(|()| InvalidParameterValue(format!("illegal path: {url}")))?; - Ok(std::fs::read(path)?) + .ok() + .with_context(|| format!("illegal path: {url}"))?; + Ok(std::fs::read(&path) + .with_context(|| format!("failed to read file from `{}`", path.display()))?) } ("https" | "http", _) => Ok(download_from_http(url).await?.into()), ("s3", Some(config)) => load_file_descriptor_from_s3(url, config).await, - (scheme, _) => Err(RwError::from(InvalidParameterValue(format!( - "path scheme {scheme} is not supported", - )))), + (scheme, _) => bail!("path scheme `{scheme}` is not supported"), } } diff --git a/src/connector/src/schema/mod.rs b/src/connector/src/schema/mod.rs index 1401cebd48921..0f64ee85e57dd 100644 --- a/src/connector/src/schema/mod.rs +++ b/src/connector/src/schema/mod.rs @@ -30,12 +30,6 @@ pub struct InvalidOptionError { // source: Option, } -impl From for risingwave_common::error::RwError { - fn from(value: InvalidOptionError) -> Self { - anyhow::anyhow!(value).into() - } -} - #[derive(Debug, thiserror::Error)] pub enum SchemaFetchError { #[error(transparent)] @@ -52,6 +46,6 @@ pub enum SchemaFetchError { YetToMigrate( #[source] #[backtrace] - risingwave_common::error::RwError, + anyhow::Error, ), } diff --git a/src/connector/src/schema/schema_registry/client.rs b/src/connector/src/schema/schema_registry/client.rs index 1b29116f311b8..21b0d4a7f6586 100644 --- a/src/connector/src/schema/schema_registry/client.rs +++ b/src/connector/src/schema/schema_registry/client.rs @@ -68,12 +68,6 @@ pub struct ConcurrentRequestError { context: String, } -impl From for risingwave_common::error::RwError { - fn from(value: ConcurrentRequestError) -> Self { - anyhow::anyhow!(value).into() - } -} - type SrResult = Result; impl Client { diff --git a/src/connector/src/schema/schema_registry/util.rs b/src/connector/src/schema/schema_registry/util.rs index c2fce4e5b7c17..407534b1a5671 100644 --- a/src/connector/src/schema/schema_registry/util.rs +++ b/src/connector/src/schema/schema_registry/util.rs @@ -51,12 +51,6 @@ pub enum WireFormatError { NoSchemaId, } -impl From for risingwave_common::error::RwError { - fn from(value: WireFormatError) -> Self { - anyhow::anyhow!(value).into() - } -} - /// extract the magic number and `schema_id` at the front of payload /// /// 0 -> magic number diff --git a/src/connector/src/sink/deltalake.rs b/src/connector/src/sink/deltalake.rs index 0ea787d2735d8..c092b9c995870 100644 --- a/src/connector/src/sink/deltalake.rs +++ b/src/connector/src/sink/deltalake.rs @@ -28,7 +28,6 @@ use deltalake::DeltaTable; use risingwave_common::array::{to_deltalake_record_batch_with_schema, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::Schema; -use risingwave_common::error::anyhow_error; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::connector_service::sink_metadata::Metadata::Serialized; @@ -268,7 +267,7 @@ impl Sink for DeltaLakeSink { .await, self.param.clone(), writer_param.vnode_bitmap.ok_or_else(|| { - SinkError::Remote(anyhow_error!( + SinkError::Remote(anyhow!( "sink needs coordination should not have singleton input" )) })?, diff --git a/src/connector/src/sink/iceberg/mod.rs b/src/connector/src/sink/iceberg/mod.rs index 0e4f28f64639c..6b759845373fa 100644 --- a/src/connector/src/sink/iceberg/mod.rs +++ b/src/connector/src/sink/iceberg/mod.rs @@ -42,7 +42,6 @@ use itertools::Itertools; use risingwave_common::array::{to_iceberg_record_batch_with_schema, Op, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::Schema; -use risingwave_common::error::anyhow_error; use risingwave_pb::connector_service::sink_metadata::Metadata::Serialized; use risingwave_pb::connector_service::sink_metadata::SerializedMetadata; use risingwave_pb::connector_service::SinkMetadata; @@ -503,7 +502,7 @@ impl Sink for IcebergSink { .await, self.param.clone(), writer_param.vnode_bitmap.ok_or_else(|| { - SinkError::Remote(anyhow_error!( + SinkError::Remote(anyhow!( "sink needs coordination should not have singleton input" )) })?, diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 45f185196040b..26e946fa06d5e 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -48,7 +48,6 @@ use anyhow::anyhow; use async_trait::async_trait; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; -use risingwave_common::error::{anyhow_error, ErrorCode, RwError}; use risingwave_common::metrics::{ LabelGuardedHistogram, LabelGuardedIntCounter, LabelGuardedIntGauge, }; @@ -536,13 +535,13 @@ pub enum SinkError { impl From for SinkError { fn from(value: icelake::Error) -> Self { - SinkError::Iceberg(anyhow_error!("{}", value)) + SinkError::Iceberg(anyhow!(value)) } } impl From for SinkError { fn from(value: RpcError) -> Self { - SinkError::Remote(anyhow_error!("{}", value)) + SinkError::Remote(anyhow!(value)) } } @@ -554,7 +553,7 @@ impl From for SinkError { impl From for SinkError { fn from(value: DeltaTableError) -> Self { - SinkError::DeltaLake(anyhow_error!("{}", value)) + SinkError::DeltaLake(anyhow!(value)) } } @@ -563,9 +562,3 @@ impl From for SinkError { SinkError::Redis(format!("{}", value)) } } - -impl From for RwError { - fn from(e: SinkError) -> Self { - ErrorCode::SinkError(Box::new(e)).into() - } -} diff --git a/src/connector/src/sink/nats.rs b/src/connector/src/sink/nats.rs index 01bac5e6b048c..9f906b49fbd21 100644 --- a/src/connector/src/sink/nats.rs +++ b/src/connector/src/sink/nats.rs @@ -18,7 +18,6 @@ use anyhow::anyhow; use async_nats::jetstream::context::Context; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; -use risingwave_common::error::anyhow_error; use serde_derive::Deserialize; use serde_with::serde_as; use tokio_retry::strategy::{jitter, ExponentialBackoff}; @@ -111,7 +110,7 @@ impl Sink for NatsSink { match self.config.common.build_client().await { Ok(_client) => {} Err(error) => { - return Err(SinkError::Nats(anyhow_error!( + return Err(SinkError::Nats(anyhow!( "validate nats sink error: {:?}", error ))); @@ -135,7 +134,7 @@ impl NatsSinkWriter { .common .build_context() .await - .map_err(|e| SinkError::Nats(anyhow_error!("nats sink error: {:?}", e)))?; + .map_err(|e| SinkError::Nats(anyhow!("nats sink error: {:?}", e)))?; Ok::<_, SinkError>(Self { config: config.clone(), context, @@ -160,13 +159,13 @@ impl NatsSinkWriter { self.context .publish(self.config.common.subject.clone(), item.into()) .await - .map_err(|e| SinkError::Nats(anyhow_error!("nats sink error: {:?}", e)))?; + .map_err(|e| SinkError::Nats(anyhow!("nats sink error: {:?}", e)))?; } Ok::<_, SinkError>(()) }, ) .await - .map_err(|e| SinkError::Nats(anyhow_error!("nats sink error: {:?}", e))) + .map_err(|e| SinkError::Nats(anyhow!("nats sink error: {:?}", e))) } } diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index 6c4e12c5997a0..943dd46a565b2 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -27,7 +27,6 @@ use jni::JavaVM; use prost::Message; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::{ColumnDesc, ColumnId}; -use risingwave_common::error::anyhow_error; use risingwave_common::types::DataType; use risingwave_jni_core::jvm_runtime::JVM; use risingwave_jni_core::{ @@ -162,7 +161,7 @@ async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> anyhow::Res && param.downstream_pk.len() > 1 && param.properties.get(ES_OPTION_DELIMITER).is_none() { - return Err(anyhow_error!( + return Err(anyhow!( "Es sink only support single pk or pk with delimiter option" )); } @@ -188,7 +187,7 @@ async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> anyhow::Res if (sink_name==ElasticSearchSink::SINK_NAME) | matches!(list.as_ref(), DataType::Int16 | DataType::Int32 | DataType::Int64 | DataType::Float32 | DataType::Float64 | DataType::Varchar){ Ok(()) } else{ - Err(SinkError::Remote(anyhow_error!( + Err(SinkError::Remote(anyhow!( "Remote sink only support list, got {:?}: {:?}", col.name, col.data_type, @@ -199,14 +198,14 @@ async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> anyhow::Res if sink_name==ElasticSearchSink::SINK_NAME{ Ok(()) }else{ - Err(SinkError::Remote(anyhow_error!( + Err(SinkError::Remote(anyhow!( "Only Es sink support struct, got {:?}: {:?}", col.name, col.data_type, ))) } }, - DataType::Serial | DataType::Int256 => Err(SinkError::Remote(anyhow_error!( + DataType::Serial | DataType::Int256 => Err(SinkError::Remote(anyhow!( "remote sink supports Int16, Int32, Int64, Float32, Float64, Boolean, Decimal, Time, Date, Interval, Jsonb, Timestamp, Timestamptz, Bytea, List and Varchar, (Es sink support Struct) got {:?}: {:?}", col.name, col.data_type, @@ -516,7 +515,7 @@ impl Sink for CoordinatedRemoteSink { .await, self.param.clone(), writer_param.vnode_bitmap.ok_or_else(|| { - SinkError::Remote(anyhow_error!( + SinkError::Remote(anyhow!( "sink needs coordination should not have singleton input" )) })?, @@ -602,9 +601,7 @@ impl SinkWriter for CoordinatedRemoteSinkWriter { .inc_by(cardinality as _); let epoch = self.epoch.ok_or_else(|| { - SinkError::Remote(anyhow_error!( - "epoch has not been initialize, call `begin_epoch`" - )) + SinkError::Remote(anyhow!("epoch has not been initialize, call `begin_epoch`")) })?; let batch_id = self.batch_id; self.stream_handle @@ -626,16 +623,14 @@ impl SinkWriter for CoordinatedRemoteSinkWriter { async fn barrier(&mut self, is_checkpoint: bool) -> Result> { let epoch = self.epoch.ok_or_else(|| { - SinkError::Remote(anyhow_error!( - "epoch has not been initialize, call `begin_epoch`" - )) + SinkError::Remote(anyhow!("epoch has not been initialize, call `begin_epoch`")) })?; if is_checkpoint { // TODO: add metrics to measure commit time let rsp = self.stream_handle.commit(epoch).await?; rsp.metadata .ok_or_else(|| { - SinkError::Remote(anyhow_error!( + SinkError::Remote(anyhow!( "get none metadata in commit response for coordinated sink writer" )) }) diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 941723bea2848..b0e556c52b3be 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -26,7 +26,7 @@ use itertools::Itertools; use parking_lot::Mutex; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::TableId; -use risingwave_common::error::{ErrorSuppressor, RwError}; +use risingwave_common::error::ErrorSuppressor; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; use risingwave_common::types::{JsonbVal, Scalar}; use risingwave_pb::catalog::{PbSource, PbStreamSourceInfo}; @@ -34,6 +34,7 @@ use risingwave_pb::plan_common::ExternalTableDesc; use risingwave_pb::source::ConnectorSplit; use risingwave_rpc_client::ConnectorClient; use serde::de::DeserializeOwned; +use thiserror_ext::AsReport; use super::cdc::DebeziumCdcMeta; use super::datagen::DatagenMeta; @@ -214,11 +215,11 @@ impl SourceContext { ctx } - pub(crate) fn report_user_source_error(&self, e: RwError) { + pub(crate) fn report_user_source_error(&self, e: &(impl AsReport + ?Sized)) { if self.source_info.fragment_id == u32::MAX { return; } - let mut err_str = e.inner().to_string(); + let mut err_str = e.to_report_string(); if let Some(suppressor) = &self.error_suppressor && suppressor.lock().suppress_error(&err_str) { @@ -346,11 +347,11 @@ pub fn extract_source_struct(info: &PbStreamSourceInfo) -> Result Ok(SourceStruct::new(format, encode)) } -pub type BoxSourceStream = BoxStream<'static, Result>>; +pub type BoxSourceStream = BoxStream<'static, anyhow::Result>>; -pub trait ChunkSourceStream = Stream> + Send + 'static; -pub type BoxChunkSourceStream = BoxStream<'static, Result>; -pub type BoxTryStream = BoxStream<'static, Result>; +pub trait ChunkSourceStream = Stream> + Send + 'static; +pub type BoxChunkSourceStream = BoxStream<'static, anyhow::Result>; +pub type BoxTryStream = BoxStream<'static, anyhow::Result>; /// [`SplitReader`] is a new abstraction of the external connector read interface which is /// responsible for parsing, it is used to read messages from the outside and transform them into a @@ -366,7 +367,7 @@ pub trait SplitReader: Sized + Send { parser_config: ParserConfig, source_ctx: SourceContextRef, columns: Option>, - ) -> Result; + ) -> anyhow::Result; fn into_stream(self) -> BoxChunkSourceStream; } diff --git a/src/connector/src/source/common.rs b/src/connector/src/source/common.rs index 6693bce671944..5d39e303e7590 100644 --- a/src/connector/src/source/common.rs +++ b/src/connector/src/source/common.rs @@ -15,18 +15,15 @@ use futures::{Stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; -use risingwave_common::error::RwError; use crate::parser::ParserConfig; use crate::source::{SourceContextRef, SourceMessage, SplitReader}; pub(crate) trait CommonSplitReader: SplitReader + 'static { - fn into_data_stream( - self, - ) -> impl Stream, anyhow::Error>> + Send; + fn into_data_stream(self) -> impl Stream>> + Send; } -#[try_stream(boxed, ok = StreamChunk, error = RwError)] +#[try_stream(boxed, ok = StreamChunk, error = anyhow::Error)] pub(crate) async fn into_chunk_stream( reader: impl CommonSplitReader, parser_config: ParserConfig, diff --git a/src/connector/src/source/datagen/source/generator.rs b/src/connector/src/source/datagen/source/generator.rs index b298d8eafcfa1..18376f3ce73ab 100644 --- a/src/connector/src/source/datagen/source/generator.rs +++ b/src/connector/src/source/datagen/source/generator.rs @@ -13,11 +13,9 @@ // limitations under the License. use std::time::{Duration, SystemTime, UNIX_EPOCH}; -use anyhow::Result; use futures_async_stream::try_stream; use risingwave_common::array::stream_chunk_builder::StreamChunkBuilder; use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::error::RwError; use risingwave_common::field_generator::FieldGeneratorImpl; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; @@ -61,7 +59,7 @@ impl DatagenEventGenerator { split_id: SplitId, split_num: u64, split_index: u64, - ) -> Result { + ) -> anyhow::Result { let partition_rows_per_second = if rows_per_second % split_num > split_index { rows_per_second / split_num + 1 } else { @@ -158,7 +156,7 @@ impl DatagenEventGenerator { } } - #[try_stream(ok = StreamChunk, error = RwError)] + #[try_stream(ok = StreamChunk, error = anyhow::Error)] pub async fn into_native_stream(mut self) { let mut interval = tokio::time::interval(Duration::from_secs(1)); const MAX_ROWS_PER_YIELD: u64 = 1024; diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs index 49c6be470d573..8ed913cad64a4 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -12,13 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{Ok, Result}; use async_trait::async_trait; use futures::TryStreamExt; use futures_async_stream::try_stream; use opendal::Operator; use risingwave_common::array::StreamChunk; -use risingwave_common::error::RwError; use tokio::io::BufReader; use tokio_util::io::{ReaderStream, StreamReader}; @@ -52,7 +50,7 @@ impl SplitReader for OpendalReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> Result { + ) -> anyhow::Result { let connector = Src::new_enumerator(properties)?; let opendal_reader = OpendalReader { connector, @@ -69,7 +67,7 @@ impl SplitReader for OpendalReader { } impl OpendalReader { - #[try_stream(boxed, ok = StreamChunk, error = RwError)] + #[try_stream(boxed, ok = StreamChunk, error = anyhow::Error)] async fn into_chunk_stream(self) { for split in self.splits { let actor_id = self.source_ctx.source_info.actor_id.to_string(); diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index d1e03b91fc6b9..b8e7a2a71b0cd 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::pin::pin; -use anyhow::{anyhow, Result}; +use anyhow::anyhow; use async_trait::async_trait; use aws_sdk_s3::client as s3_client; use aws_sdk_s3::operation::get_object::GetObjectError; @@ -26,7 +26,6 @@ use aws_smithy_types::byte_stream::ByteStream; use futures_async_stream::try_stream; use io::StreamReader; use risingwave_common::array::StreamChunk; -use risingwave_common::error::RwError; use tokio::io::BufReader; use tokio_util::io; use tokio_util::io::ReaderStream; @@ -185,7 +184,7 @@ impl SplitReader for S3FileReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> Result { + ) -> anyhow::Result { let config = AwsAuthProps::from(&props); let sdk_config = config.build_config().await?; @@ -211,7 +210,7 @@ impl SplitReader for S3FileReader { } impl S3FileReader { - #[try_stream(boxed, ok = StreamChunk, error = RwError)] + #[try_stream(boxed, ok = StreamChunk, error = anyhow::Error)] async fn into_chunk_stream(self) { for split in self.splits { let actor_id = self.source_ctx.source_info.actor_id.to_string(); diff --git a/src/connector/src/source/kafka/source/reader.rs b/src/connector/src/source/kafka/source/reader.rs index 2a0ef8babe992..691590e361cde 100644 --- a/src/connector/src/source/kafka/source/reader.rs +++ b/src/connector/src/source/kafka/source/reader.rs @@ -198,7 +198,7 @@ impl CommonSplitReader for KafkaSplitReader { // ingest kafka message header can be expensive, do it only when required let require_message_header = self.parser_config.common.rw_columns.iter().any(|col_desc| { matches!( - col_desc.additional_column_type.column_type, + col_desc.additional_column.column_type, Some(AdditionalColumnType::Headers(_) | AdditionalColumnType::HeaderInner(_)) ) }); diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index 049515d6091a8..a5584f6af83d5 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -37,10 +37,10 @@ pub struct SourceColumnDesc { /// `is_hidden_addition_col` is used to indicate whether the column is a hidden addition column. pub is_hidden_addition_col: bool, - /// `additional_column_type` and `column_type` are orthogonal - /// `additional_column_type` is used to indicate the column is from which part of the message + /// `additional_column` and `column_type` are orthogonal + /// `additional_column` is used to indicate the column is from which part of the message /// `column_type` is used to indicate the type of the column, only used in cdc scenario - pub additional_column_type: AdditionalColumn, + pub additional_column: AdditionalColumn, } /// `SourceColumnType` is used to indicate the type of a column emitted by the Source. @@ -91,7 +91,7 @@ impl SourceColumnDesc { column_type: SourceColumnType::Normal, is_pk: false, is_hidden_addition_col: false, - additional_column_type: AdditionalColumn { column_type: None }, + additional_column: AdditionalColumn { column_type: None }, } } @@ -131,7 +131,7 @@ impl From<&ColumnDesc> for SourceColumnDesc { column_type, is_pk: false, is_hidden_addition_col: false, - additional_column_type: c.additional_columns.clone(), + additional_column: c.additional_column.clone(), } } } @@ -146,7 +146,7 @@ impl From<&SourceColumnDesc> for ColumnDesc { type_name: "".to_string(), generated_or_default_column: None, description: None, - additional_columns: s.additional_column_type.clone(), + additional_column: s.additional_column.clone(), version: ColumnDescVersion::Pr13707, } } diff --git a/src/connector/src/source/nexmark/source/reader.rs b/src/connector/src/source/nexmark/source/reader.rs index 5df614f7a4545..e7621e5325524 100644 --- a/src/connector/src/source/nexmark/source/reader.rs +++ b/src/connector/src/source/nexmark/source/reader.rs @@ -14,7 +14,6 @@ use std::time::Duration; -use anyhow::Result; use async_trait::async_trait; use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; @@ -23,7 +22,6 @@ use nexmark::event::EventType; use nexmark::EventGenerator; use risingwave_common::array::stream_chunk_builder::StreamChunkBuilder; use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::error::RwError; use risingwave_common::estimate_size::EstimateSize; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; @@ -66,7 +64,7 @@ impl SplitReader for NexmarkSplitReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> Result { + ) -> anyhow::Result { tracing::debug!("Splits for nexmark found! {:?}", splits); assert!(splits.len() == 1); // TODO: currently, assume there's only one split in one reader @@ -165,7 +163,7 @@ impl NexmarkSplitReader { } } - #[try_stream(boxed, ok = StreamChunk, error = RwError)] + #[try_stream(boxed, ok = StreamChunk, error = anyhow::Error)] async fn into_native_stream(mut self) { let start_time = Instant::now(); let start_offset = self.generator.global_offset(); @@ -210,14 +208,12 @@ impl NexmarkSplitReader { #[cfg(test)] mod tests { - use anyhow::Result; - use super::*; use crate::source::nexmark::{NexmarkProperties, NexmarkSplitEnumerator}; use crate::source::{SourceEnumeratorContext, SplitEnumerator}; #[tokio::test] - async fn test_nexmark_split_reader() -> Result<()> { + async fn test_nexmark_split_reader() -> anyhow::Result<()> { let props = NexmarkProperties { split_num: 2, min_event_gap_in_ns: 0, @@ -251,7 +247,7 @@ mod tests { } #[tokio::test] - async fn test_nexmark_event_num() -> Result<()> { + async fn test_nexmark_event_num() -> anyhow::Result<()> { let max_chunk_size = 32; let event_num = max_chunk_size * 128 + 1; let props = NexmarkProperties { diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index 07265f1e4acfc..7181710b70868 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::time::{SystemTime, UNIX_EPOCH}; -use anyhow::{anyhow, ensure, Result}; +use anyhow::{anyhow, Context}; use arrow_array::{Int32Array, Int64Array, RecordBatch}; use async_trait::async_trait; use futures::StreamExt; @@ -30,7 +30,7 @@ use pulsar::message::proto::MessageIdData; use pulsar::{Consumer, ConsumerBuilder, ConsumerOptions, Pulsar, SubType, TokioExecutor}; use risingwave_common::array::{DataChunk, StreamChunk}; use risingwave_common::catalog::ROWID_PREFIX; -use risingwave_common::error::RwError; +use risingwave_common::{bail, ensure}; use crate::error::ConnectorError; use crate::parser::ParserConfig; @@ -57,7 +57,7 @@ impl SplitReader for PulsarSplitReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> Result { + ) -> anyhow::Result { ensure!(splits.len() == 1, "only support single split"); let split = splits.into_iter().next().unwrap(); let topic = split.topic.to_string(); @@ -107,19 +107,15 @@ pub struct PulsarBrokerReader { } // {ledger_id}:{entry_id}:{partition}:{batch_index} -fn parse_message_id(id: &str) -> Result { +fn parse_message_id(id: &str) -> anyhow::Result { let splits = id.split(':').collect_vec(); if splits.len() < 2 || splits.len() > 4 { - return Err(anyhow!("illegal message id string {}", id)); + bail!("illegal message id string {}", id); } - let ledger_id = splits[0] - .parse::() - .map_err(|e| anyhow!("illegal ledger id {}", e))?; - let entry_id = splits[1] - .parse::() - .map_err(|e| anyhow!("illegal entry id {}", e))?; + let ledger_id = splits[0].parse::().context("illegal ledger id")?; + let entry_id = splits[1].parse::().context("illegal entry id")?; let mut message_id = MessageIdData { ledger_id, @@ -132,16 +128,12 @@ fn parse_message_id(id: &str) -> Result { }; if splits.len() > 2 { - let partition = splits[2] - .parse::() - .map_err(|e| anyhow!("illegal partition {}", e))?; + let partition = splits[2].parse::().context("illegal partition")?; message_id.partition = Some(partition); } if splits.len() == 4 { - let batch_index = splits[3] - .parse::() - .map_err(|e| anyhow!("illegal batch index {}", e))?; + let batch_index = splits[3].parse::().context("illegal batch index")?; message_id.batch_index = Some(batch_index); } @@ -159,7 +151,7 @@ impl SplitReader for PulsarBrokerReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> Result { + ) -> anyhow::Result { ensure!(splits.len() == 1, "only support single split"); let split = splits.into_iter().next().unwrap(); let pulsar = props @@ -287,7 +279,7 @@ impl PulsarIcebergReader { } } - async fn scan(&self) -> Result { + async fn scan(&self) -> anyhow::Result { let table = self.create_iceberg_table().await?; let schema = table.current_table_metadata().current_schema()?; tracing::debug!("Created iceberg pulsar table, schema is: {:?}", schema,); @@ -302,18 +294,14 @@ impl PulsarIcebergReader { .fields() .iter() .find(|f| f.name == META_COLUMN_PARTITION) - .ok_or_else(|| { - ConnectorError::Pulsar(anyhow!( - "Partition field not found in partition spec" - )) - })?; + .context("Partition field not found in partition spec")?; (s.clone(), field.clone()) } _ => { - return Err(ConnectorError::Pulsar(anyhow!( + bail!( "Partition type is not struct in iceberg table: {}", table.table_name() - )))?; + ); } }; @@ -339,19 +327,16 @@ impl PulsarIcebergReader { .await?) } - async fn create_iceberg_table(&self) -> Result { + async fn create_iceberg_table(&self) -> anyhow::Result
{ let catalog = load_catalog(&self.build_iceberg_configs()?) .await - .map_err(|e| ConnectorError::Pulsar(anyhow!("Unable to load iceberg catalog: {e}")))?; + .context("Unable to load iceberg catalog")?; let table_id = TableIdentifier::new(vec![self.split.topic.topic_str_without_partition()?]) - .map_err(|e| ConnectorError::Pulsar(anyhow!("Unable to parse table name: {e}")))?; + .context("Unable to parse table name")?; - let table = catalog - .load_table(&table_id) - .await - .map_err(|err| ConnectorError::Pulsar(anyhow!(err)))?; + let table = catalog.load_table(&table_id).await?; Ok(table) } @@ -371,7 +356,7 @@ impl PulsarIcebergReader { } } - #[try_stream(ok = StreamChunk, error = RwError)] + #[try_stream(ok = StreamChunk, error = anyhow::Error)] async fn into_stream(self) { let (props, mut split, parser_config, source_ctx) = ( self.props.clone(), @@ -410,7 +395,7 @@ impl PulsarIcebergReader { } } - fn build_iceberg_configs(&self) -> Result> { + fn build_iceberg_configs(&self) -> anyhow::Result> { let mut iceberg_configs = HashMap::new(); let bucket = @@ -466,58 +451,49 @@ impl PulsarIcebergReader { fn convert_record_batch_to_source_with_state( &self, record_batch: &RecordBatch, - ) -> Result<(StreamChunk, HashMap)> { + ) -> anyhow::Result<(StreamChunk, HashMap)> { let mut offsets = Vec::with_capacity(record_batch.num_rows()); let ledger_id_array = record_batch .column_by_name(META_COLUMN_LEDGER_ID) - .ok_or_else(|| ConnectorError::Pulsar(anyhow!("Ledger id not found in iceberg table")))? + .context("Ledger id not found in iceberg table")? .as_any() .downcast_ref::() - .ok_or_else(|| { - ConnectorError::Pulsar(anyhow!("Ledger id is not i64 in iceberg table")) - })?; + .context("Ledger id is not i64 in iceberg table")?; let entry_id_array = record_batch .column_by_name(META_COLUMN_ENTRY_ID) - .ok_or_else(|| ConnectorError::Pulsar(anyhow!("Entry id not found in iceberg table")))? + .context("Entry id not found in iceberg table")? .as_any() .downcast_ref::() - .ok_or_else(|| { - ConnectorError::Pulsar(anyhow!("Entry id is not i64 in iceberg table")) - })?; + .context("Entry id is not i64 in iceberg table")?; let partition_array = record_batch .column_by_name(META_COLUMN_PARTITION) .map(|arr| { - arr.as_any().downcast_ref::().ok_or_else(|| { - ConnectorError::Pulsar(anyhow!("Partition is not i32 in iceberg table")) - }) + arr.as_any() + .downcast_ref::() + .context("Partition is not i32 in iceberg table") }) .transpose()?; let batch_index_array = record_batch .column_by_name(META_COLUMN_BATCH_INDEX) .map(|arr| { - arr.as_any().downcast_ref::().ok_or_else(|| { - ConnectorError::Pulsar(anyhow!("Batch index is not i64 in iceberg table")) - }) + arr.as_any() + .downcast_ref::() + .context("Batch index is not i64 in iceberg table") }) .transpose()?; - let field_indices = self + let field_indices: Vec<_> = self .parser_config .common .rw_columns .iter() .filter(|col| col.name != ROWID_PREFIX) - .map(|col| { - record_batch - .schema() - .index_of(col.name.as_str()) - .map_err(|e| anyhow!(e)) - }) - .collect::>>()?; + .map(|col| record_batch.schema().index_of(col.name.as_str())) + .try_collect()?; for row in 0..record_batch.num_rows() { let offset = format!( diff --git a/src/connector/src/source/reader/desc.rs b/src/connector/src/source/reader/desc.rs index e049be8bbe940..a842b091ab928 100644 --- a/src/connector/src/source/reader/desc.rs +++ b/src/connector/src/source/reader/desc.rs @@ -15,9 +15,8 @@ use std::collections::HashMap; use std::sync::Arc; +use risingwave_common::bail; use risingwave_common::catalog::{ColumnDesc, ColumnId}; -use risingwave_common::error::ErrorCode::ProtocolError; -use risingwave_common::error::{Result, RwError}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::catalog::PbStreamSourceInfo; use risingwave_pb::plan_common::additional_column::ColumnType; @@ -139,7 +138,7 @@ impl SourceDescBuilder { // Check if partition/file/offset columns are included explicitly. for col in &self.columns { - match col.column_desc.as_ref().unwrap().get_additional_columns() { + match col.column_desc.as_ref().unwrap().get_additional_column() { Ok(AdditionalColumn { column_type: Some(ColumnType::Partition(_) | ColumnType::Filename(_)), }) => { @@ -177,7 +176,7 @@ impl SourceDescBuilder { columns } - pub fn build(self) -> Result { + pub fn build(self) -> anyhow::Result { let columns = self.column_catalogs_to_source_column_descs(); let psrser_config = SpecificParserConfig::new(&self.source_info, &self.with_properties)?; @@ -202,7 +201,7 @@ impl SourceDescBuilder { #[deprecated = "will be replaced by new fs source (list + fetch)"] #[expect(deprecated)] - pub fn build_fs_source_desc(&self) -> Result { + pub fn build_fs_source_desc(&self) -> anyhow::Result { let parser_config = SpecificParserConfig::new(&self.source_info, &self.with_properties)?; match ( @@ -214,10 +213,11 @@ impl SourceDescBuilder { EncodingProperties::Csv(_) | EncodingProperties::Json(_), ) => {} (format, encode) => { - return Err(RwError::from(ProtocolError(format!( + bail!( "Unsupported combination of format {:?} and encode {:?}", - format, encode - )))); + format, + encode, + ); } } diff --git a/src/connector/src/source/reader/fs_reader.rs b/src/connector/src/source/reader/fs_reader.rs index bb8cc39389e47..5b199c8d8955b 100644 --- a/src/connector/src/source/reader/fs_reader.rs +++ b/src/connector/src/source/reader/fs_reader.rs @@ -17,12 +17,10 @@ use std::collections::HashMap; use std::sync::Arc; -use anyhow::anyhow; +use anyhow::Context; use futures::stream::pending; use futures::StreamExt; use risingwave_common::catalog::ColumnId; -use risingwave_common::error::ErrorCode::ConnectorError; -use risingwave_common::error::Result; use crate::dispatch_source_prop; use crate::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; @@ -46,13 +44,12 @@ impl FsSourceReader { columns: Vec, connector_node_addr: Option, parser_config: SpecificParserConfig, - ) -> Result { + ) -> anyhow::Result { // Store the connector node address to properties for later use. let mut source_props: HashMap = HashMap::from_iter(properties.clone()); connector_node_addr .map(|addr| source_props.insert("connector_node_addr".to_string(), addr)); - let config = ConnectorProperties::extract(source_props, false) - .map_err(|e| ConnectorError(e.into()))?; + let config = ConnectorProperties::extract(source_props, false)?; Ok(Self { config, @@ -62,19 +59,22 @@ impl FsSourceReader { }) } - fn get_target_columns(&self, column_ids: Vec) -> Result> { + fn get_target_columns( + &self, + column_ids: Vec, + ) -> anyhow::Result> { column_ids .iter() .map(|id| { self.columns .iter() .find(|c| c.column_id == *id) - .ok_or_else(|| { - anyhow!("Failed to find column id: {} in source: {:?}", id, self).into() + .with_context(|| { + format!("Failed to find column id: {} in source: {:?}", id, self) }) .map(|col| col.clone()) }) - .collect::>>() + .try_collect() } pub async fn to_stream( @@ -82,7 +82,7 @@ impl FsSourceReader { state: ConnectorState, column_ids: Vec, source_ctx: Arc, - ) -> Result { + ) -> anyhow::Result { let config = self.config.clone(); let columns = self.get_target_columns(column_ids)?; diff --git a/src/connector/src/source/reader/reader.rs b/src/connector/src/source/reader/reader.rs index 571825b05e5fc..ba9bd4dded4d8 100644 --- a/src/connector/src/source/reader/reader.rs +++ b/src/connector/src/source/reader/reader.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::sync::Arc; -use anyhow::anyhow; +use anyhow::Context; use futures::future::try_join_all; use futures::stream::pending; use futures::StreamExt; @@ -23,8 +23,6 @@ use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::bail; use risingwave_common::catalog::ColumnId; -use risingwave_common::error::ErrorCode::ConnectorError; -use risingwave_common::error::{Result, RwError}; use rw_futures_util::select_all; use crate::dispatch_source_prop; @@ -53,9 +51,8 @@ impl SourceReader { columns: Vec, connector_message_buffer_size: usize, parser_config: SpecificParserConfig, - ) -> Result { - let config = ConnectorProperties::extract(properties, false) - .map_err(|e| ConnectorError(e.into()))?; + ) -> anyhow::Result { + let config = ConnectorProperties::extract(properties, false)?; Ok(Self { config, @@ -65,22 +62,25 @@ impl SourceReader { }) } - fn get_target_columns(&self, column_ids: Vec) -> Result> { + fn get_target_columns( + &self, + column_ids: Vec, + ) -> anyhow::Result> { column_ids .iter() .map(|id| { self.columns .iter() .find(|c| c.column_id == *id) - .ok_or_else(|| { - anyhow!("Failed to find column id: {} in source: {:?}", id, self).into() + .with_context(|| { + format!("Failed to find column id: {} in source: {:?}", id, self) }) .cloned() }) - .collect::>>() + .try_collect() } - pub fn get_source_list(&self) -> Result> { + pub fn get_source_list(&self) -> anyhow::Result> { let config = self.config.clone(); match config { ConnectorProperties::Gcs(prop) => { @@ -107,7 +107,7 @@ impl SourceReader { state: ConnectorState, column_ids: Vec, source_ctx: Arc, - ) -> Result { + ) -> anyhow::Result { let Some(splits) = state else { return Ok(pending().boxed()); }; @@ -165,7 +165,7 @@ impl SourceReader { } } -#[try_stream(boxed, ok = FsPageItem, error = RwError)] +#[try_stream(boxed, ok = FsPageItem, error = anyhow::Error)] async fn build_opendal_fs_list_stream(lister: OpendalEnumerator) { let matcher = lister.get_matcher(); let mut object_metadata_iter = lister.list().await?; @@ -186,7 +186,7 @@ async fn build_opendal_fs_list_stream(lister: OpendalEnumera } Err(err) => { tracing::error!("list object fail, err {}", err); - return Err(err.into()); + return Err(err); } } } diff --git a/src/expr/core/src/error.rs b/src/expr/core/src/error.rs index 83e57c117185b..6688824093d2d 100644 --- a/src/expr/core/src/error.rs +++ b/src/expr/core/src/error.rs @@ -15,7 +15,6 @@ use std::fmt::{Debug, Display}; use risingwave_common::array::{ArrayError, ArrayRef}; -use risingwave_common::error::{ErrorCode, RwError}; use risingwave_common::types::DataType; use risingwave_pb::PbFieldNotFound; use thiserror::Error; @@ -138,12 +137,6 @@ pub struct CryptographyError { static_assertions::const_assert_eq!(std::mem::size_of::(), 40); -impl From for RwError { - fn from(s: ExprError) -> Self { - ErrorCode::ExprError(Box::new(s)).into() - } -} - impl From for ExprError { fn from(e: chrono::ParseError) -> Self { Self::Parse(e.to_report_string().into()) diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index bf4dd37c233b6..af6e430864323 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -42,6 +42,7 @@ itertools = "0.12" linkme = { version = "0.3", features = ["used_linker"] } maplit = "1" md5 = "0.7.0" +memcomparable = "0.2" num-integer = "0.1" parking_lot = "0.12" parse-display = "0.8" diff --git a/src/frontend/src/binder/bind_context.rs b/src/frontend/src/binder/bind_context.rs index 7a39ed1cee63b..0dc03464fbe62 100644 --- a/src/frontend/src/binder/bind_context.rs +++ b/src/frontend/src/binder/bind_context.rs @@ -18,10 +18,11 @@ use std::rc::Rc; use parse_display::Display; use risingwave_common::catalog::Field; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; use risingwave_sqlparser::ast::TableAlias; +use crate::error::{ErrorCode, Result}; + type LiteResult = std::result::Result; use crate::binder::{BoundQuery, ShareId, COLUMN_GROUP_PREFIX}; diff --git a/src/frontend/src/binder/bind_param.rs b/src/frontend/src/binder/bind_param.rs index 5ed887aa02ebb..9d51947b35063 100644 --- a/src/frontend/src/binder/bind_param.rs +++ b/src/frontend/src/binder/bind_param.rs @@ -15,11 +15,12 @@ use bytes::Bytes; use pgwire::types::{Format, FormatIterator}; use risingwave_common::bail; -use risingwave_common::error::{BoxedError, ErrorCode, Result}; +use risingwave_common::error::BoxedError; use risingwave_common::types::{Datum, ScalarImpl}; use super::statement::RewriteExprsRecursive; use super::BoundStatement; +use crate::error::{ErrorCode, Result}; use crate::expr::{Expr, ExprImpl, ExprRewriter, Literal}; /// Rewrites parameter expressions to literals. diff --git a/src/frontend/src/binder/create.rs b/src/frontend/src/binder/create.rs index baa78fb322619..e08eb1bcbe533 100644 --- a/src/frontend/src/binder/create.rs +++ b/src/frontend/src/binder/create.rs @@ -14,8 +14,8 @@ use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, Field}; -use risingwave_common::error::Result; +use crate::error::Result; use crate::Binder; impl Binder { diff --git a/src/frontend/src/binder/delete.rs b/src/frontend/src/binder/delete.rs index 31b02566c1f43..78fd623cb9d2c 100644 --- a/src/frontend/src/binder/delete.rs +++ b/src/frontend/src/binder/delete.rs @@ -13,12 +13,12 @@ // limitations under the License. use risingwave_common::catalog::{Schema, TableVersionId}; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_sqlparser::ast::{Expr, ObjectName, SelectItem}; use super::statement::RewriteExprsRecursive; use super::{Binder, BoundBaseTable}; use crate::catalog::TableId; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::ExprImpl; use crate::user::UserId; diff --git a/src/frontend/src/binder/expr/binary_op.rs b/src/frontend/src/binder/expr/binary_op.rs index a2efee44fe343..d8a2ddae1b1a5 100644 --- a/src/frontend/src/binder/expr/binary_op.rs +++ b/src/frontend/src/binder/expr/binary_op.rs @@ -13,11 +13,11 @@ // limitations under the License. use risingwave_common::bail_not_implemented; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::{DataType, JsonbVal}; use risingwave_sqlparser::ast::{BinaryOperator, Expr}; use crate::binder::Binder; +use crate::error::{ErrorCode, Result}; use crate::expr::{Expr as _, ExprImpl, ExprType, FunctionCall}; impl Binder { diff --git a/src/frontend/src/binder/expr/column.rs b/src/frontend/src/binder/expr/column.rs index c2fd536c1ccc4..29f1a0f1c64e5 100644 --- a/src/frontend/src/binder/expr/column.rs +++ b/src/frontend/src/binder/expr/column.rs @@ -12,11 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; use risingwave_sqlparser::ast::Ident; use crate::binder::Binder; +use crate::error::{ErrorCode, Result}; use crate::expr::{CorrelatedInputRef, ExprImpl, ExprType, FunctionCall, InputRef, Literal}; impl Binder { diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index bb6e4ee14c335..477c493ef2ab9 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -21,7 +21,6 @@ use bk_tree::{metrics, BKTree}; use itertools::Itertools; use risingwave_common::array::ListValue; use risingwave_common::catalog::{INFORMATION_SCHEMA_SCHEMA_NAME, PG_CATALOG_SCHEMA_NAME}; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::session_config::USER_NAME_WILD_CARD; use risingwave_common::types::{data_types, DataType, ScalarImpl, Timestamptz}; use risingwave_common::{bail_not_implemented, current_cluster_version, no_function}; @@ -39,6 +38,7 @@ use thiserror_ext::AsReport; use crate::binder::bind_context::Clause; use crate::binder::{Binder, BoundQuery, BoundSetExpr, UdfContext}; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{ AggCall, Expr, ExprImpl, ExprType, FunctionCall, FunctionCallWithLambda, Literal, Now, OrderBy, Subquery, SubqueryKind, TableFunction, TableFunctionType, UserDefinedFunction, WindowFunction, diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index a50eec922143b..2baf9cb1f84e7 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -14,7 +14,6 @@ use itertools::Itertools; use risingwave_common::catalog::{ColumnDesc, ColumnId}; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::zip_eq_fast; use risingwave_common::{bail_not_implemented, not_implemented}; @@ -26,6 +25,7 @@ use risingwave_sqlparser::ast::{ use crate::binder::expr::function::SYS_FUNCTION_WITHOUT_ARGS; use crate::binder::Binder; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{Expr as _, ExprImpl, ExprType, FunctionCall, InputRef, Parameter, SubqueryKind}; mod binary_op; @@ -703,7 +703,7 @@ pub fn bind_struct_field(column_def: &StructField) -> Result { type_name: "".to_string(), generated_or_default_column: None, description: None, - additional_columns: AdditionalColumn { column_type: None }, + additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, }) } diff --git a/src/frontend/src/binder/expr/order_by.rs b/src/frontend/src/binder/expr/order_by.rs index b4497137a1af7..201461b4b13de 100644 --- a/src/frontend/src/binder/expr/order_by.rs +++ b/src/frontend/src/binder/expr/order_by.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_common::util::sort_util::OrderType; use risingwave_sqlparser::ast::OrderByExpr; +use crate::error::Result; use crate::expr::OrderByExpr as BoundOrderByExpr; use crate::Binder; diff --git a/src/frontend/src/binder/expr/subquery.rs b/src/frontend/src/binder/expr/subquery.rs index eed3f1657baee..51819116771f1 100644 --- a/src/frontend/src/binder/expr/subquery.rs +++ b/src/frontend/src/binder/expr/subquery.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::{ErrorCode, Result}; use risingwave_sqlparser::ast::Query; use crate::binder::Binder; +use crate::error::{ErrorCode, Result}; use crate::expr::{ExprImpl, Subquery, SubqueryKind}; impl Binder { diff --git a/src/frontend/src/binder/expr/value.rs b/src/frontend/src/binder/expr/value.rs index fa72b7b77d5c6..0b6eace27083f 100644 --- a/src/frontend/src/binder/expr/value.rs +++ b/src/frontend/src/binder/expr/value.rs @@ -14,12 +14,12 @@ use itertools::Itertools; use risingwave_common::bail_not_implemented; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::{DataType, DateTimeField, Decimal, Interval, ScalarImpl}; use risingwave_sqlparser::ast::{DateTimeField as AstDateTimeField, Expr, Value}; use thiserror_ext::AsReport; use crate::binder::Binder; +use crate::error::{ErrorCode, Result}; use crate::expr::{align_types, Expr as _, ExprImpl, ExprType, FunctionCall, Literal}; impl Binder { diff --git a/src/frontend/src/binder/for_system.rs b/src/frontend/src/binder/for_system.rs index 547786c3864b7..09fb0c8da6b3d 100644 --- a/src/frontend/src/binder/for_system.rs +++ b/src/frontend/src/binder/for_system.rs @@ -14,13 +14,13 @@ use std::sync::Arc; -use risingwave_common::error::Result; use risingwave_connector::sink::catalog::SinkCatalog; use risingwave_sqlparser::ast::ObjectName; use crate::binder::BindFor; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::view_catalog::ViewCatalog; +use crate::error::Result; use crate::Binder; pub struct BoundSink { diff --git a/src/frontend/src/binder/insert.rs b/src/frontend/src/binder/insert.rs index 492d0abb5d91f..cc7ddc06b8aff 100644 --- a/src/frontend/src/binder/insert.rs +++ b/src/frontend/src/binder/insert.rs @@ -16,7 +16,6 @@ use std::collections::{BTreeMap, HashMap, HashSet}; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, Schema, TableVersionId}; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_sqlparser::ast::{Ident, ObjectName, Query, SelectItem}; @@ -25,6 +24,7 @@ use super::statement::RewriteExprsRecursive; use super::BoundQuery; use crate::binder::Binder; use crate::catalog::TableId; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{ExprImpl, InputRef}; use crate::user::UserId; diff --git a/src/frontend/src/binder/mod.rs b/src/frontend/src/binder/mod.rs index f1c7d97c57fa2..2505ca886f19c 100644 --- a/src/frontend/src/binder/mod.rs +++ b/src/frontend/src/binder/mod.rs @@ -17,7 +17,6 @@ use std::sync::Arc; use itertools::Itertools; use parking_lot::RwLock; -use risingwave_common::error::Result; use risingwave_common::session_config::{ConfigMap, SearchPath}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqDebug; @@ -25,6 +24,8 @@ use risingwave_sqlparser::ast::{ Expr as AstExpr, FunctionArg, FunctionArgExpr, SelectItem, SetExpr, Statement, }; +use crate::error::Result; + mod bind_context; mod bind_param; mod create; @@ -51,7 +52,6 @@ pub use relation::{ BoundBaseTable, BoundJoin, BoundShare, BoundSource, BoundSystemTable, BoundWatermark, BoundWindowTableFunction, Relation, ResolveQualifiedNameError, WindowTableFunctionKind, }; -use risingwave_common::error::ErrorCode; pub use select::{BoundDistinct, BoundSelect}; pub use set_expr::*; pub use statement::BoundStatement; @@ -62,6 +62,7 @@ use crate::catalog::catalog_service::CatalogReadGuard; use crate::catalog::function_catalog::FunctionCatalog; use crate::catalog::schema_catalog::SchemaCatalog; use crate::catalog::{CatalogResult, TableId, ViewId}; +use crate::error::ErrorCode; use crate::expr::ExprImpl; use crate::session::{AuthContext, SessionImpl}; diff --git a/src/frontend/src/binder/query.rs b/src/frontend/src/binder/query.rs index 1d3823fb14814..fe2008f50f3eb 100644 --- a/src/frontend/src/binder/query.rs +++ b/src/frontend/src/binder/query.rs @@ -17,7 +17,6 @@ use std::rc::Rc; use risingwave_common::bail_not_implemented; use risingwave_common::catalog::Schema; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_sqlparser::ast::{Cte, Expr, Fetch, OrderByExpr, Query, Value, With}; @@ -26,6 +25,7 @@ use thiserror_ext::AsReport; use super::statement::RewriteExprsRecursive; use super::BoundValues; use crate::binder::{Binder, BoundSetExpr}; +use crate::error::{ErrorCode, Result}; use crate::expr::{CorrelatedId, Depth, ExprImpl, ExprRewriter}; /// A validated sql query, including order and union. diff --git a/src/frontend/src/binder/relation/join.rs b/src/frontend/src/binder/relation/join.rs index 0e083b3457d47..c2f820aff7744 100644 --- a/src/frontend/src/binder/relation/join.rs +++ b/src/frontend/src/binder/relation/join.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::plan_common::JoinType; use risingwave_sqlparser::ast::{ BinaryOperator, Expr, Ident, JoinConstraint, JoinOperator, TableFactor, TableWithJoins, Value, @@ -21,6 +20,7 @@ use risingwave_sqlparser::ast::{ use crate::binder::bind_context::BindContext; use crate::binder::statement::RewriteExprsRecursive; use crate::binder::{Binder, Clause, Relation, COLUMN_GROUP_PREFIX}; +use crate::error::{ErrorCode, Result}; use crate::expr::ExprImpl; #[derive(Debug, Clone)] diff --git a/src/frontend/src/binder/relation/mod.rs b/src/frontend/src/binder/relation/mod.rs index 9b6f0c6163375..69eb6787d47a0 100644 --- a/src/frontend/src/binder/relation/mod.rs +++ b/src/frontend/src/binder/relation/mod.rs @@ -18,7 +18,6 @@ use std::ops::Deref; use itertools::{EitherOrBoth, Itertools}; use risingwave_common::bail; use risingwave_common::catalog::{Field, TableId, DEFAULT_SCHEMA_NAME}; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_sqlparser::ast::{ Expr as ParserExpr, FunctionArg, FunctionArgExpr, Ident, ObjectName, TableAlias, TableFactor, }; @@ -28,6 +27,7 @@ use thiserror_ext::AsReport; use super::bind_context::ColumnBinding; use super::statement::RewriteExprsRecursive; use crate::binder::Binder; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{ExprImpl, InputRef}; mod join; diff --git a/src/frontend/src/binder/relation/subquery.rs b/src/frontend/src/binder/relation/subquery.rs index da2cb6816ef0f..b29c65dd465f1 100644 --- a/src/frontend/src/binder/relation/subquery.rs +++ b/src/frontend/src/binder/relation/subquery.rs @@ -12,11 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_sqlparser::ast::{Query, TableAlias}; use crate::binder::statement::RewriteExprsRecursive; use crate::binder::{Binder, BoundQuery, UNNAMED_SUBQUERY}; +use crate::error::Result; #[derive(Debug, Clone)] pub struct BoundSubquery { diff --git a/src/frontend/src/binder/relation/table_function.rs b/src/frontend/src/binder/relation/table_function.rs index d4c0df1f82e60..f7d5c803ea635 100644 --- a/src/frontend/src/binder/relation/table_function.rs +++ b/src/frontend/src/binder/relation/table_function.rs @@ -19,7 +19,6 @@ use risingwave_common::bail_not_implemented; use risingwave_common::catalog::{ Field, Schema, PG_CATALOG_SCHEMA_NAME, RW_INTERNAL_TABLE_FUNCTION_NAME, }; -use risingwave_common::error::ErrorCode; use risingwave_common::types::DataType; use risingwave_sqlparser::ast::{Function, FunctionArg, ObjectName, TableAlias}; @@ -29,6 +28,7 @@ use crate::binder::bind_context::Clause; use crate::catalog::system_catalog::pg_catalog::{ PG_GET_KEYWORDS_FUNC_NAME, PG_KEYWORDS_TABLE_NAME, }; +use crate::error::ErrorCode; use crate::expr::{Expr, ExprImpl}; impl Binder { diff --git a/src/frontend/src/binder/relation/table_or_source.rs b/src/frontend/src/binder/relation/table_or_source.rs index 8c16f14d7ce71..a459efd39f016 100644 --- a/src/frontend/src/binder/relation/table_or_source.rs +++ b/src/frontend/src/binder/relation/table_or_source.rs @@ -17,7 +17,6 @@ use std::sync::Arc; use itertools::Itertools; use risingwave_common::bail_not_implemented; use risingwave_common::catalog::{is_system_schema, Field}; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::session_config::USER_NAME_WILD_CARD; use risingwave_sqlparser::ast::{Statement, TableAlias}; use risingwave_sqlparser::parser::Parser; @@ -32,6 +31,7 @@ use crate::catalog::system_catalog::SystemTableCatalog; use crate::catalog::table_catalog::{TableCatalog, TableType}; use crate::catalog::view_catalog::ViewCatalog; use crate::catalog::{CatalogError, IndexCatalog, TableId}; +use crate::error::{ErrorCode, Result, RwError}; #[derive(Debug, Clone)] pub struct BoundBaseTable { diff --git a/src/frontend/src/binder/relation/watermark.rs b/src/frontend/src/binder/relation/watermark.rs index e85452d3302d6..43f6341981c0a 100644 --- a/src/frontend/src/binder/relation/watermark.rs +++ b/src/frontend/src/binder/relation/watermark.rs @@ -13,12 +13,12 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::error::ErrorCode; use risingwave_common::types::DataType; use risingwave_sqlparser::ast::{FunctionArg, TableAlias}; use super::{Binder, Relation, Result}; use crate::binder::statement::RewriteExprsRecursive; +use crate::error::ErrorCode; use crate::expr::{ExprImpl, InputRef}; const ERROR_1ST_ARG: &str = "The 1st arg of watermark function should be a table name (incl. source, CTE, view) but not complex structure (subquery, join, another table function). Consider using an intermediate CTE or view as workaround."; diff --git a/src/frontend/src/binder/relation/window_table_function.rs b/src/frontend/src/binder/relation/window_table_function.rs index 005490773475d..530988ad01a8a 100644 --- a/src/frontend/src/binder/relation/window_table_function.rs +++ b/src/frontend/src/binder/relation/window_table_function.rs @@ -16,12 +16,12 @@ use std::str::FromStr; use itertools::Itertools; use risingwave_common::catalog::Field; -use risingwave_common::error::ErrorCode; use risingwave_common::types::DataType; use risingwave_sqlparser::ast::{FunctionArg, TableAlias}; use super::{Binder, Relation, Result}; use crate::binder::statement::RewriteExprsRecursive; +use crate::error::ErrorCode; use crate::expr::{ExprImpl, InputRef}; #[derive(Copy, Clone, Debug)] diff --git a/src/frontend/src/binder/select.rs b/src/frontend/src/binder/select.rs index eb56ae77e2404..b9689e007297e 100644 --- a/src/frontend/src/binder/select.rs +++ b/src/frontend/src/binder/select.rs @@ -17,7 +17,6 @@ use std::fmt::Debug; use itertools::Itertools; use risingwave_common::catalog::{Field, Schema, PG_CATALOG_SCHEMA_NAME, RW_CATALOG_SCHEMA_NAME}; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::aggregate::AggKind; @@ -38,6 +37,7 @@ use crate::catalog::system_catalog::rw_catalog::{ RW_TABLE_STATS_COLUMNS, RW_TABLE_STATS_KEY_SIZE_INDEX, RW_TABLE_STATS_TABLE_ID_INDEX, RW_TABLE_STATS_TABLE_NAME, RW_TABLE_STATS_VALUE_SIZE_INDEX, }; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{ AggCall, CorrelatedId, CorrelatedInputRef, Depth, Expr as _, ExprImpl, ExprType, FunctionCall, InputRef, diff --git a/src/frontend/src/binder/set_expr.rs b/src/frontend/src/binder/set_expr.rs index 099a2a13a44e8..99ec66ac0b725 100644 --- a/src/frontend/src/binder/set_expr.rs +++ b/src/frontend/src/binder/set_expr.rs @@ -14,12 +14,12 @@ use risingwave_common::bail_not_implemented; use risingwave_common::catalog::Schema; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_sqlparser::ast::{SetExpr, SetOperator}; use super::statement::RewriteExprsRecursive; use crate::binder::{BindContext, Binder, BoundQuery, BoundSelect, BoundValues}; +use crate::error::{ErrorCode, Result}; use crate::expr::{align_types, CorrelatedId, Depth}; /// Part of a validated query, without order or limit clause. It may be composed of smaller diff --git a/src/frontend/src/binder/statement.rs b/src/frontend/src/binder/statement.rs index 916a60a7d1fb0..764ebfca8783f 100644 --- a/src/frontend/src/binder/statement.rs +++ b/src/frontend/src/binder/statement.rs @@ -14,12 +14,12 @@ use risingwave_common::bail_not_implemented; use risingwave_common::catalog::Field; -use risingwave_common::error::Result; use risingwave_sqlparser::ast::Statement; use super::delete::BoundDelete; use super::update::BoundUpdate; use crate::binder::{Binder, BoundInsert, BoundQuery}; +use crate::error::Result; use crate::expr::ExprRewriter; #[derive(Debug, Clone)] diff --git a/src/frontend/src/binder/struct_field.rs b/src/frontend/src/binder/struct_field.rs index 30ce1e87df8c9..7a5f0f91f625e 100644 --- a/src/frontend/src/binder/struct_field.rs +++ b/src/frontend/src/binder/struct_field.rs @@ -13,11 +13,11 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::{DataType, Scalar}; use risingwave_sqlparser::ast::{Expr, Ident}; use crate::binder::Binder; +use crate::error::{ErrorCode, Result}; use crate::expr::{Expr as RwExpr, ExprImpl, ExprType, FunctionCall, Literal}; impl Binder { diff --git a/src/frontend/src/binder/update.rs b/src/frontend/src/binder/update.rs index c19ceeb56e481..cbe51f9ec69bd 100644 --- a/src/frontend/src/binder/update.rs +++ b/src/frontend/src/binder/update.rs @@ -19,13 +19,13 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::bail_not_implemented; use risingwave_common::catalog::{Schema, TableVersionId}; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_sqlparser::ast::{Assignment, AssignmentValue, Expr, ObjectName, SelectItem}; use super::statement::RewriteExprsRecursive; use super::{Binder, BoundBaseTable}; use crate::catalog::TableId; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{Expr as _, ExprImpl, InputRef}; use crate::user::UserId; use crate::TableCatalog; diff --git a/src/frontend/src/binder/values.rs b/src/frontend/src/binder/values.rs index 04dbb701640d8..93f9e3f4f8a18 100644 --- a/src/frontend/src/binder/values.rs +++ b/src/frontend/src/binder/values.rs @@ -15,7 +15,6 @@ use itertools::Itertools; use risingwave_common::bail_not_implemented; use risingwave_common::catalog::{Field, Schema}; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_sqlparser::ast::Values; @@ -23,6 +22,7 @@ use risingwave_sqlparser::ast::Values; use super::bind_context::Clause; use super::statement::RewriteExprsRecursive; use crate::binder::Binder; +use crate::error::{ErrorCode, Result}; use crate::expr::{align_types, CorrelatedId, Depth, ExprImpl}; #[derive(Debug, Clone)] diff --git a/src/frontend/src/catalog/catalog_service.rs b/src/frontend/src/catalog/catalog_service.rs index a785ed9ac0282..5228dd4683f5a 100644 --- a/src/frontend/src/catalog/catalog_service.rs +++ b/src/frontend/src/catalog/catalog_service.rs @@ -18,7 +18,6 @@ use anyhow::anyhow; use parking_lot::lock_api::ArcRwLockReadGuard; use parking_lot::{RawRwLock, RwLock}; use risingwave_common::catalog::{CatalogVersion, FunctionId, IndexId}; -use risingwave_common::error::Result; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::catalog::{ PbComment, PbCreateType, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, @@ -36,6 +35,7 @@ use tokio::sync::watch::Receiver; use super::root_catalog::Catalog; use super::{DatabaseId, TableId}; +use crate::error::Result; use crate::user::UserId; pub type CatalogReadGuard = ArcRwLockReadGuard; diff --git a/src/frontend/src/catalog/connection_catalog.rs b/src/frontend/src/catalog/connection_catalog.rs index 29f72006b4cd0..58595dfbdfd62 100644 --- a/src/frontend/src/catalog/connection_catalog.rs +++ b/src/frontend/src/catalog/connection_catalog.rs @@ -16,7 +16,6 @@ use std::collections::BTreeMap; use std::sync::Arc; use anyhow::anyhow; -use risingwave_common::error::{Result, RwError}; use risingwave_connector::source::kafka::private_link::insert_privatelink_broker_rewrite_map; use risingwave_connector::source::KAFKA_CONNECTOR; use risingwave_pb::catalog::connection::private_link_service::PrivateLinkProvider; @@ -24,6 +23,7 @@ use risingwave_pb::catalog::connection::Info; use risingwave_pb::catalog::{connection, PbConnection}; use crate::catalog::{ConnectionId, OwnedByUserCatalog}; +use crate::error::{Result, RwError}; use crate::user::UserId; #[derive(Clone, Debug, PartialEq)] diff --git a/src/frontend/src/catalog/mod.rs b/src/frontend/src/catalog/mod.rs index 3ee9d1996fb3f..dcced97f28911 100644 --- a/src/frontend/src/catalog/mod.rs +++ b/src/frontend/src/catalog/mod.rs @@ -21,9 +21,10 @@ use risingwave_common::catalog::{ is_row_id_column_name, is_system_schema, ROWID_PREFIX, RW_RESERVED_COLUMN_NAME_PREFIX, }; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_connector::sink::catalog::SinkCatalog; use thiserror::Error; + +use crate::error::{ErrorCode, Result, RwError}; pub(crate) mod catalog_service; pub(crate) mod connection_catalog; diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs index f8d969029300d..f4703b3148768 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs @@ -14,11 +14,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 crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_ACTORS: BuiltinTable = BuiltinTable { name: "rw_actors", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_columns.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_columns.rs index b2675de869231..25184f48ae7c1 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_columns.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_columns.rs @@ -16,11 +16,11 @@ use std::sync::LazyLock; 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 crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub static RW_COLUMNS: LazyLock = LazyLock::new(|| BuiltinTable { name: "rw_columns", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs index 6ec1cbbd72b4d..528847be10436 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs @@ -14,11 +14,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 crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_CONNECTIONS: BuiltinTable = BuiltinTable { name: "rw_connections", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs index 0344133a2f4fa..85f19afdb1640 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs @@ -14,13 +14,13 @@ 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_pb::user::grant_privilege::Object; use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; use crate::catalog::OwnedByUserCatalog; +use crate::error::Result; pub const RW_DATABASES: BuiltinTable = BuiltinTable { name: "rw_databases", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_ddl_progress.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_ddl_progress.rs index a926ca7c2581b..7efa1edb2d10a 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_ddl_progress.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_ddl_progress.rs @@ -14,12 +14,12 @@ 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::util::epoch::Epoch; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_DDL_PROGRESS: BuiltinTable = BuiltinTable { name: "rw_ddl_progress", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs index 5aff8aa621092..eb1e7d6fd6214 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs @@ -15,11 +15,11 @@ use std::iter; 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 crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_DESCRIPTION: BuiltinTable = BuiltinTable { name: "rw_description", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_event_logs.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_event_logs.rs index 246e9ee282849..9572ecfbc2617 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_event_logs.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_event_logs.rs @@ -14,13 +14,13 @@ 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, Timestamptz}; use risingwave_pb::meta::event_log::Event; use serde_json::json; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_EVENT_LOGS: BuiltinTable = BuiltinTable { name: "rw_event_logs", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs index e2d37edd4b780..d18bac5a9b1f4 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs @@ -17,12 +17,12 @@ use std::sync::LazyLock; use itertools::Itertools; use risingwave_common::array::ListValue; 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_pb::stream_plan::FragmentTypeFlag; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl, SystemCatalogColumnsDef}; +use crate::error::Result; pub static RW_FRAGMENTS_COLUMNS: LazyLock>> = LazyLock::new(|| { vec![ diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs index da4f7de4f6438..d31784d55989d 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs @@ -17,7 +17,6 @@ use std::sync::LazyLock; use itertools::Itertools; use risingwave_common::array::ListValue; 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_pb::user::grant_privilege::Object; @@ -25,6 +24,7 @@ use risingwave_pb::user::grant_privilege::Object; use crate::catalog::system_catalog::{ get_acl_items, BuiltinTable, SysCatalogReaderImpl, SystemCatalogColumnsDef, }; +use crate::error::Result; pub static RW_FUNCTIONS_COLUMNS: LazyLock>> = LazyLock::new(|| { vec![ diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_branched_objects.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_branched_objects.rs index 9fa7acb4341a8..c63c5c6872faa 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_branched_objects.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_branched_objects.rs @@ -13,11 +13,11 @@ // limitations under the License. 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 crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_HUMMOCK_BRANCHED_OBJECTS: BuiltinTable = BuiltinTable { name: "rw_hummock_branched_objects", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_assignment.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_assignment.rs index 7d9863450316b..47cea4b382816 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_assignment.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_assignment.rs @@ -13,13 +13,13 @@ // limitations under the License. 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_pb::hummock::CompactTaskAssignment; use serde_json::json; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_HUMMOCK_COMPACT_TASK_ASSIGNMENT: BuiltinTable = BuiltinTable { name: "rw_hummock_compact_task_assignment", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_progress.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_progress.rs index 1c86f4dcb83c7..0f16d9f3b7ab9 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_progress.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_progress.rs @@ -13,12 +13,12 @@ // limitations under the License. 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_pb::hummock::CompactTaskProgress; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_HUMMOCK_COMPACT_TASK_PROGRESS: BuiltinTable = BuiltinTable { name: "rw_hummock_compact_task_progress", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compaction_group_configs.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compaction_group_configs.rs index 0fd1c7bdd4970..2ffad904c9eca 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compaction_group_configs.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compaction_group_configs.rs @@ -14,12 +14,12 @@ 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 serde_json::json; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_HUMMOCK_COMPACTION_GROUP_CONFIGS: BuiltinTable = BuiltinTable { name: "rw_hummock_compaction_group_configs", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_meta_configs.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_meta_configs.rs index 4ca29681cfda3..9bf0303e2e8ca 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_meta_configs.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_meta_configs.rs @@ -14,11 +14,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 crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_HUMMOCK_META_CONFIGS: BuiltinTable = BuiltinTable { name: "rw_hummock_meta_configs", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_snapshots.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_snapshots.rs index 1785087577ecf..48a22923087a6 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_snapshots.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_snapshots.rs @@ -14,11 +14,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 crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_HUMMOCK_PINNED_SNAPSHOTS: BuiltinTable = BuiltinTable { name: "rw_hummock_pinned_snapshots", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_versions.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_versions.rs index 0e160c4a1b51a..07f0d786f4206 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_versions.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_versions.rs @@ -14,11 +14,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 crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_HUMMOCK_PINNED_VERSIONS: BuiltinTable = BuiltinTable { name: "rw_hummock_pinned_versions", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs index 941b912262d1a..6b0e68d7590b5 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs @@ -13,13 +13,13 @@ // limitations under the License. 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_hummock_sdk::version::HummockVersion; use serde_json::json; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_HUMMOCK_CURRENT_VERSION: BuiltinTable = BuiltinTable { name: "rw_hummock_current_version", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs index 1d338f0c30d33..7e99e4040f981 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs @@ -13,12 +13,12 @@ // limitations under the License. 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 serde_json::json; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_HUMMOCK_VERSION_DELTAS: BuiltinTable = BuiltinTable { name: "rw_hummock_version_deltas", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs index 3d5f3f42a5988..7c82b0aa86df3 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs @@ -17,11 +17,11 @@ use std::sync::LazyLock; use itertools::Itertools; use risingwave_common::array::ListValue; 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 crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl, SystemCatalogColumnsDef}; +use crate::error::Result; pub static RW_INDEXES_COLUMNS: LazyLock>> = LazyLock::new(|| { vec![ diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_internal_tables.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_internal_tables.rs index c950c26dc010b..d7a9412a5943d 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_internal_tables.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_internal_tables.rs @@ -14,12 +14,12 @@ 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_pb::user::grant_privilege::Object; use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_INTERNAL_TABLES: BuiltinTable = BuiltinTable { name: "rw_internal_tables", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs index fc9d17b644749..9b9b7de05a844 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs @@ -14,12 +14,12 @@ 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_pb::user::grant_privilege::Object; use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_MATERIALIZED_VIEWS: BuiltinTable = BuiltinTable { name: "rw_materialized_views", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs index a8b62003a47bd..e79e0870fc673 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs @@ -14,12 +14,12 @@ 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, Timestamp}; use risingwave_common::util::epoch::Epoch; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_META_SNAPSHOT: BuiltinTable = BuiltinTable { name: "rw_meta_snapshot", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs index 28ce4717642ed..d883c45d83a77 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs @@ -14,11 +14,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 crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_PARALLEL_UNITS: BuiltinTable = BuiltinTable { name: "rw_parallel_units", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs index e354661800c0e..3040213bc91af 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs @@ -13,12 +13,12 @@ // limitations under the License. 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 serde_json::json; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; // TODO: `rw_relation_info` contains some extra streaming meta info that's only meaningful for // streaming jobs, we'd better query relation infos from `rw_relations` and move these streaming diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs index c632455c99c11..e603fab62a565 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs @@ -14,13 +14,13 @@ 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_pb::user::grant_privilege::Object; use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; use crate::catalog::OwnedByUserCatalog; +use crate::error::Result; pub const RW_SCHEMAS: BuiltinTable = BuiltinTable { name: "rw_schemas", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs index 54d18a3ec5904..9735820c9ad9f 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs @@ -14,12 +14,12 @@ 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_pb::user::grant_privilege::Object; use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; use crate::handler::create_source::UPSTREAM_SOURCE_KEY; pub const RW_SINKS: BuiltinTable = BuiltinTable { diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs index fa862017315e8..43f5ddf630d31 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs @@ -17,7 +17,6 @@ use std::sync::LazyLock; use itertools::Itertools; use risingwave_common::array::ListValue; 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_pb::user::grant_privilege::Object; @@ -25,6 +24,7 @@ use risingwave_pb::user::grant_privilege::Object; use crate::catalog::system_catalog::{ get_acl_items, BuiltinTable, SysCatalogReaderImpl, SystemCatalogColumnsDef, }; +use crate::error::Result; use crate::handler::create_source::UPSTREAM_SOURCE_KEY; pub static RW_SOURCES_COLUMNS: LazyLock>> = LazyLock::new(|| { diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_system_tables.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_system_tables.rs index d5c9ffda266f0..79176bc96ec36 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_system_tables.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_system_tables.rs @@ -16,12 +16,12 @@ use std::sync::LazyLock; 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_pb::user::grant_privilege::Object; use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; /// `rw_system_tables` stores all system tables in the database. pub static RW_SYSTEM_TABLES: LazyLock = LazyLock::new(|| BuiltinTable { diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_fragments.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_fragments.rs index dec52e39bd568..85de321f344b5 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_fragments.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_fragments.rs @@ -14,13 +14,13 @@ 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 crate::catalog::system_catalog::{ extract_parallelism_from_table_state, BuiltinTable, SysCatalogReaderImpl, }; +use crate::error::Result; pub const RW_TABLE_FRAGMENTS: BuiltinTable = BuiltinTable { name: "rw_table_fragments", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_stats.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_stats.rs index c3653fb56540b..eeae9167d1af8 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_stats.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_stats.rs @@ -13,11 +13,11 @@ // limitations under the License. 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 crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl, SystemCatalogColumnsDef}; +use crate::error::Result; pub const RW_TABLE_STATS_TABLE_NAME: &str = "rw_table_stats"; pub const RW_TABLE_STATS_TABLE_ID_INDEX: usize = 0; diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs index 475312d318665..45db055e6f8f6 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs @@ -14,12 +14,12 @@ 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_pb::user::grant_privilege::Object; use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_TABLES: BuiltinTable = BuiltinTable { name: "rw_tables", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs index 9fdd82bdaafd5..d207d222578c5 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs @@ -16,12 +16,12 @@ use std::sync::LazyLock; use itertools::Itertools; use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::error::Result; use risingwave_common::for_all_base_types; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; macro_rules! impl_pg_type_data { ($( { $enum:ident | $oid:literal | $oid_array:literal | $name:ident | $input:ident | $len:literal } )*) => { diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_user_secrets.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_user_secrets.rs index 0d9cd63a82c45..b481d60a89b4b 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_user_secrets.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_user_secrets.rs @@ -16,11 +16,11 @@ use std::sync::LazyLock; use itertools::Itertools; use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::{ErrorCode, Result}; use crate::user::user_authentication::encrypted_raw_password; /// `rw_user_secret` stores all user encrypted passwords in the database, which is only readable by diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_users.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_users.rs index 9c1046e5f2c6c..fb59dbab44da7 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_users.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_users.rs @@ -14,11 +14,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 crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_USERS: BuiltinTable = BuiltinTable { name: "rw_users", diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs index 879892ee8cd2b..c9a21e0f7e81b 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs @@ -14,12 +14,12 @@ 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_pb::user::grant_privilege::Object; use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; pub const RW_VIEWS: BuiltinTable = BuiltinTable { name: "rw_views", 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 c3f270cd422fe..c1bbd92478895 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 @@ -14,11 +14,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, Timestamptz}; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::error::Result; /// `rw_worker_nodes` contains all information about the compute nodes in the cluster. pub const RW_WORKER_NODES: BuiltinTable = BuiltinTable { diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index dfd605c11c7b0..fbb77a0ca0bb5 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -19,7 +19,6 @@ use itertools::Itertools; use risingwave_common::catalog::{ ColumnCatalog, ConflictBehavior, TableDesc, TableId, TableVersionId, }; -use risingwave_common::error::{ErrorCode, RwError}; use risingwave_common::util::epoch::Epoch; use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::table::{OptionalAssociatedSourceId, PbTableType, PbTableVersion}; @@ -28,6 +27,7 @@ use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; use risingwave_pb::plan_common::DefaultColumnDesc; use super::{ColumnId, DatabaseId, FragmentId, OwnedByUserCatalog, SchemaId, SinkId}; +use crate::error::{ErrorCode, RwError}; use crate::expr::ExprImpl; use crate::optimizer::property::Cardinality; use crate::user::UserId; @@ -683,7 +683,7 @@ mod tests { type_name: ".test.Country".to_string(), description: None, generated_or_default_column: None, - additional_columns: AdditionalColumn { column_type: None }, + additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, }, is_hidden: false diff --git a/src/frontend/src/error.rs b/src/frontend/src/error.rs new file mode 100644 index 0000000000000..5c1daa024afb1 --- /dev/null +++ b/src/frontend/src/error.rs @@ -0,0 +1,231 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use risingwave_batch::error::BatchError; +use risingwave_common::array::ArrayError; +use risingwave_common::error::{BoxedError, NoFunction, NotImplemented}; +use risingwave_common::session_config::SessionConfigError; +use risingwave_common::util::value_encoding::error::ValueEncodingError; +use risingwave_connector::sink::SinkError; +use risingwave_expr::ExprError; +use risingwave_pb::PbFieldNotFound; +use risingwave_rpc_client::error::{RpcError, TonicStatusWrapper}; +use thiserror::Error; +use thiserror_ext::Box; +use tokio::task::JoinError; + +/// The error type for the frontend crate, acting as the top-level error type for the +/// entire RisingWave project. +// TODO(error-handling): this is migrated from the `common` crate, and there could +// be some further refactoring to do: +// - Some variants are never constructed. +// - Some variants store a type-erased `BoxedError` to resolve the reverse dependency. +// It's not necessary anymore as the error type is now defined at the top-level. +#[derive(Error, Debug, Box)] +#[thiserror_ext(newtype(name = RwError, backtrace, report_debug))] +pub enum ErrorCode { + #[error("internal error: {0}")] + InternalError(String), + // TODO: unify with the above + #[error(transparent)] + Uncategorized( + #[from] + #[backtrace] + anyhow::Error, + ), + #[error("connector error: {0}")] + ConnectorError( + #[source] + #[backtrace] + BoxedError, + ), + #[error(transparent)] + NotImplemented(#[from] NotImplemented), + // Tips: Use this only if it's intended to reject the query + #[error("Not supported: {0}\nHINT: {1}")] + NotSupported(String, String), + #[error(transparent)] + NoFunction(#[from] NoFunction), + #[error(transparent)] + IoError(#[from] std::io::Error), + #[error("Storage error: {0}")] + StorageError( + #[backtrace] + #[source] + BoxedError, + ), + #[error("Expr error: {0}")] + ExprError( + #[source] + #[backtrace] + BoxedError, + ), + // TODO(error-handling): there's a limitation that `#[transparent]` can't be used with `#[backtrace]` if no `#[from]` + // So we emulate a transparent error with "{0}" display here. + #[error("{0}")] + BatchError( + #[source] + #[backtrace] + // `BatchError` + BoxedError, + ), + #[error("Array error: {0}")] + ArrayError( + #[from] + #[backtrace] + ArrayError, + ), + #[error("Stream error: {0}")] + StreamError( + #[backtrace] + #[source] + BoxedError, + ), + // TODO(error-handling): there's a limitation that `#[transparent]` can't be used with `#[backtrace]` if no `#[from]` + // So we emulate a transparent error with "{0}" display here. + #[error("{0}")] + RpcError( + #[source] + #[backtrace] + // `tonic::transport::Error`, `TonicStatusWrapper`, or `RpcError` + BoxedError, + ), + // TODO: use a new type for bind error + // TODO(error-handling): should prefer use error types than strings. + #[error("Bind error: {0}")] + BindError(String), + // TODO: only keep this one + #[error("Failed to bind expression: {expr}: {error}")] + BindErrorRoot { + expr: String, + #[source] + #[backtrace] + error: BoxedError, + }, + #[error("Catalog error: {0}")] + CatalogError( + #[source] + #[backtrace] + BoxedError, + ), + #[error("Protocol error: {0}")] + ProtocolError(String), + #[error("Scheduler error: {0}")] + SchedulerError( + #[source] + #[backtrace] + BoxedError, + ), + #[error("Task not found")] + TaskNotFound, + #[error("Session not found")] + SessionNotFound, + #[error("Item not found: {0}")] + ItemNotFound(String), + #[error("Invalid input syntax: {0}")] + InvalidInputSyntax(String), + #[error("Can not compare in memory: {0}")] + MemComparableError(#[from] memcomparable::Error), + #[error("Error while de/se values: {0}")] + ValueEncodingError( + #[from] + #[backtrace] + ValueEncodingError, + ), + #[error("Invalid value `{config_value}` for `{config_entry}`")] + InvalidConfigValue { + config_entry: String, + config_value: String, + }, + #[error("Invalid Parameter Value: {0}")] + InvalidParameterValue(String), + #[error("Sink error: {0}")] + SinkError( + #[source] + #[backtrace] + BoxedError, + ), + #[error("Permission denied: {0}")] + PermissionDenied(String), + #[error("Failed to get/set session config: {0}")] + SessionConfig( + #[from] + #[backtrace] + SessionConfigError, + ), +} + +/// The result type for the frontend crate. +pub type Result = std::result::Result; + +impl From for RwError { + fn from(status: TonicStatusWrapper) -> Self { + use tonic::Code; + + let message = status.inner().message(); + + // TODO(error-handling): `message` loses the source chain. + match status.inner().code() { + Code::InvalidArgument => ErrorCode::InvalidParameterValue(message.to_string()), + Code::NotFound | Code::AlreadyExists => ErrorCode::CatalogError(status.into()), + Code::PermissionDenied => ErrorCode::PermissionDenied(message.to_string()), + Code::Cancelled => ErrorCode::SchedulerError(status.into()), + _ => ErrorCode::RpcError(status.into()), + } + .into() + } +} + +impl From for RwError { + fn from(r: RpcError) -> Self { + match r { + RpcError::GrpcStatus(status) => TonicStatusWrapper::into(*status), + _ => ErrorCode::RpcError(r.into()).into(), + } + } +} + +impl From for RwError { + fn from(s: ExprError) -> Self { + ErrorCode::ExprError(Box::new(s)).into() + } +} + +impl From for RwError { + fn from(e: SinkError) -> Self { + ErrorCode::SinkError(Box::new(e)).into() + } +} + +impl From for RwError { + fn from(err: PbFieldNotFound) -> Self { + ErrorCode::InternalError(format!( + "Failed to decode prost: field not found `{}`", + err.0 + )) + .into() + } +} + +impl From for RwError { + fn from(s: BatchError) -> Self { + ErrorCode::BatchError(Box::new(s)).into() + } +} + +impl From for RwError { + fn from(join_error: JoinError) -> Self { + ErrorCode::Uncategorized(join_error.into()).into() + } +} diff --git a/src/frontend/src/expr/agg_call.rs b/src/frontend/src/expr/agg_call.rs index bd18365746756..0f9493a694952 100644 --- a/src/frontend/src/expr/agg_call.rs +++ b/src/frontend/src/expr/agg_call.rs @@ -12,11 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_common::types::DataType; use risingwave_expr::aggregate::AggKind; use super::{infer_type, Expr, ExprImpl, Literal, OrderBy}; +use crate::error::Result; use crate::utils::Condition; #[derive(Clone, Eq, PartialEq, Hash)] diff --git a/src/frontend/src/expr/function_call.rs b/src/frontend/src/expr/function_call.rs index ff82eca8ea4ec..5868c74355ccb 100644 --- a/src/frontend/src/expr/function_call.rs +++ b/src/frontend/src/expr/function_call.rs @@ -14,13 +14,13 @@ use itertools::Itertools; use risingwave_common::catalog::Schema; -use risingwave_common::error::{ErrorCode, Result as RwResult}; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::iter_util::ZipEqFast; use thiserror::Error; use thiserror_ext::AsReport; use super::{cast_ok, infer_some_all, infer_type, CastContext, Expr, ExprImpl, Literal}; +use crate::error::{ErrorCode, Result as RwResult}; use crate::expr::{ExprDisplay, ExprType, ExprVisitor, ImpureAnalyzer}; #[derive(Clone, Eq, PartialEq, Hash)] diff --git a/src/frontend/src/expr/input_ref.rs b/src/frontend/src/expr/input_ref.rs index 64d564e9dd570..073d48b6d7c47 100644 --- a/src/frontend/src/expr/input_ref.rs +++ b/src/frontend/src/expr/input_ref.rs @@ -125,7 +125,7 @@ impl InputRef { pub(super) fn from_expr_proto( column_index: usize, ret_type: DataType, - ) -> risingwave_common::error::Result { + ) -> crate::error::Result { Ok(Self { index: column_index, data_type: ret_type, diff --git a/src/frontend/src/expr/literal.rs b/src/frontend/src/expr/literal.rs index 03cb6b59f310d..d44a1b859d289 100644 --- a/src/frontend/src/expr/literal.rs +++ b/src/frontend/src/expr/literal.rs @@ -93,7 +93,7 @@ impl Literal { pub(super) fn from_expr_proto( proto: &risingwave_pb::expr::ExprNode, - ) -> risingwave_common::error::Result { + ) -> crate::error::Result { let data_type = proto.get_return_type()?; Ok(Self { data: value_encoding_to_literal(&proto.rex_node, &data_type.into())?, @@ -126,7 +126,7 @@ pub fn literal_to_value_encoding(d: &Datum) -> RexNode { fn value_encoding_to_literal( proto: &Option, ty: &DataType, -) -> risingwave_common::error::Result { +) -> crate::error::Result { if let Some(rex_node) = proto { if let RexNode::Constant(prost_datum) = rex_node { let datum = Datum::from_protobuf(prost_datum, ty)?; diff --git a/src/frontend/src/expr/mod.rs b/src/frontend/src/expr/mod.rs index b38aaf735c2bd..1241ef6e4e288 100644 --- a/src/frontend/src/expr/mod.rs +++ b/src/frontend/src/expr/mod.rs @@ -17,13 +17,14 @@ use fixedbitset::FixedBitSet; use futures::FutureExt; use paste::paste; use risingwave_common::array::ListValue; -use risingwave_common::error::{ErrorCode, Result as RwResult}; use risingwave_common::types::{DataType, Datum, JsonbVal, Scalar}; use risingwave_expr::aggregate::AggKind; use risingwave_expr::expr::build_from_prost; use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::{ExprNode, ProjectSetSelectItem}; +use crate::error::{ErrorCode, Result as RwResult}; + mod agg_call; mod correlated_input_ref; mod function_call; diff --git a/src/frontend/src/expr/type_inference/cast.rs b/src/frontend/src/expr/type_inference/cast.rs index dae2e1f46b84b..aa7e1c8ee9192 100644 --- a/src/frontend/src/expr/type_inference/cast.rs +++ b/src/frontend/src/expr/type_inference/cast.rs @@ -17,10 +17,10 @@ use std::sync::LazyLock; use itertools::Itertools as _; use parse_display::Display; -use risingwave_common::error::ErrorCode; use risingwave_common::types::{DataType, DataTypeName}; use risingwave_common::util::iter_util::ZipEqFast; +use crate::error::ErrorCode; use crate::expr::{Expr as _, ExprImpl, InputRef, Literal}; /// Find the least restrictive type. Used by `VALUES`, `CASE`, `UNION`, etc. diff --git a/src/frontend/src/expr/type_inference/func.rs b/src/frontend/src/expr/type_inference/func.rs index 337c901a47eda..f80a131e71734 100644 --- a/src/frontend/src/expr/type_inference/func.rs +++ b/src/frontend/src/expr/type_inference/func.rs @@ -15,7 +15,6 @@ use itertools::Itertools as _; use num_integer::Integer as _; use risingwave_common::bail_no_function; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::hash::VirtualNode; use risingwave_common::types::{DataType, StructType}; use risingwave_common::util::iter_util::ZipEqFast; @@ -23,6 +22,7 @@ use risingwave_expr::aggregate::AggKind; pub use risingwave_expr::sig::*; use super::{align_types, cast_ok_base, CastContext}; +use crate::error::{ErrorCode, Result}; use crate::expr::type_inference::cast::align_array_and_element; use crate::expr::{cast_ok, is_row_function, Expr as _, ExprImpl, ExprType, FunctionCall}; diff --git a/src/frontend/src/expr/user_defined_function.rs b/src/frontend/src/expr/user_defined_function.rs index 0724b55254617..323d74b04be08 100644 --- a/src/frontend/src/expr/user_defined_function.rs +++ b/src/frontend/src/expr/user_defined_function.rs @@ -35,7 +35,7 @@ impl UserDefinedFunction { pub(super) fn from_expr_proto( udf: &risingwave_pb::expr::UserDefinedFunction, return_type: DataType, - ) -> risingwave_common::error::Result { + ) -> crate::error::Result { let args: Vec<_> = udf .get_children() .iter() diff --git a/src/frontend/src/expr/window_function.rs b/src/frontend/src/expr/window_function.rs index 5d9debcfcd4f5..fd9c38961132b 100644 --- a/src/frontend/src/expr/window_function.rs +++ b/src/frontend/src/expr/window_function.rs @@ -14,12 +14,12 @@ use itertools::Itertools; use risingwave_common::bail_not_implemented; -use risingwave_common::error::{ErrorCode, RwError}; use risingwave_common::types::DataType; use risingwave_expr::sig::FUNCTION_REGISTRY; use risingwave_expr::window_function::{Frame, WindowFuncKind}; use super::{Expr, ExprImpl, OrderBy, RwResult}; +use crate::error::{ErrorCode, RwError}; /// A window function performs a calculation across a set of table rows that are somehow related to /// the current row, according to the window spec `OVER (PARTITION BY .. ORDER BY ..)`. diff --git a/src/frontend/src/handler/alter_owner.rs b/src/frontend/src/handler/alter_owner.rs index 67e24fae7280d..85a2409eb1d08 100644 --- a/src/frontend/src/handler/alter_owner.rs +++ b/src/frontend/src/handler/alter_owner.rs @@ -16,8 +16,6 @@ use std::sync::Arc; use pgwire::pg_response::StatementType; use risingwave_common::acl::AclMode; -use risingwave_common::error::ErrorCode::PermissionDenied; -use risingwave_common::error::Result; use risingwave_pb::ddl_service::alter_owner_request::Object; use risingwave_pb::user::grant_privilege; use risingwave_sqlparser::ast::{Ident, ObjectName}; @@ -25,6 +23,8 @@ use risingwave_sqlparser::ast::{Ident, ObjectName}; use super::{HandlerArgs, RwPgResponse}; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::{CatalogError, OwnedByUserCatalog}; +use crate::error::ErrorCode::PermissionDenied; +use crate::error::Result; use crate::session::SessionImpl; use crate::user::user_catalog::UserCatalog; use crate::Binder; diff --git a/src/frontend/src/handler/alter_parallelism.rs b/src/frontend/src/handler/alter_parallelism.rs index 2e1f1facea0f2..5f0155e9dd46a 100644 --- a/src/frontend/src/handler/alter_parallelism.rs +++ b/src/frontend/src/handler/alter_parallelism.rs @@ -14,7 +14,6 @@ use pgwire::pg_response::StatementType; use risingwave_common::bail; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::meta::table_parallelism::{ AdaptiveParallelism, FixedParallelism, PbParallelism, }; @@ -27,6 +26,7 @@ use super::{HandlerArgs, RwPgResponse}; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::table_catalog::TableType; use crate::catalog::CatalogError; +use crate::error::{ErrorCode, Result}; use crate::Binder; pub async fn handle_alter_parallelism( diff --git a/src/frontend/src/handler/alter_rename.rs b/src/frontend/src/handler/alter_rename.rs index a244de4c47daf..6e255e8d06c21 100644 --- a/src/frontend/src/handler/alter_rename.rs +++ b/src/frontend/src/handler/alter_rename.rs @@ -15,13 +15,13 @@ use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::acl::AclMode; use risingwave_common::catalog::is_system_schema; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::user::grant_privilege::Object; use risingwave_sqlparser::ast::ObjectName; use super::{HandlerArgs, RwPgResponse}; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::table_catalog::TableType; +use crate::error::{ErrorCode, Result}; use crate::Binder; pub async fn handle_rename_table( diff --git a/src/frontend/src/handler/alter_set_schema.rs b/src/frontend/src/handler/alter_set_schema.rs index 5faf292ab2f1b..eac41426ce574 100644 --- a/src/frontend/src/handler/alter_set_schema.rs +++ b/src/frontend/src/handler/alter_set_schema.rs @@ -13,12 +13,12 @@ // limitations under the License. use pgwire::pg_response::StatementType; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::ddl_service::alter_set_schema_request::Object; use risingwave_sqlparser::ast::{ObjectName, OperateFunctionArg}; use super::{HandlerArgs, RwPgResponse}; use crate::catalog::root_catalog::SchemaPath; +use crate::error::{ErrorCode, Result}; use crate::{bind_data_type, Binder}; // Steps for validation: diff --git a/src/frontend/src/handler/alter_source_column.rs b/src/frontend/src/handler/alter_source_column.rs index bc9781689c414..fcabedc1149c4 100644 --- a/src/frontend/src/handler/alter_source_column.rs +++ b/src/frontend/src/handler/alter_source_column.rs @@ -15,7 +15,6 @@ use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::catalog::ColumnId; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_connector::source::{extract_source_struct, SourceEncode, SourceStruct}; use risingwave_sqlparser::ast::{ AlterSourceOperation, ColumnDef, CreateSourceStatement, ObjectName, Statement, @@ -25,6 +24,7 @@ use risingwave_sqlparser::parser::Parser; use super::create_table::bind_sql_columns; use super::{HandlerArgs, RwPgResponse}; use crate::catalog::root_catalog::SchemaPath; +use crate::error::{ErrorCode, Result, RwError}; use crate::Binder; // Note for future drop column: diff --git a/src/frontend/src/handler/alter_source_with_sr.rs b/src/frontend/src/handler/alter_source_with_sr.rs index ee46d63ead406..a8e6892e5a908 100644 --- a/src/frontend/src/handler/alter_source_with_sr.rs +++ b/src/frontend/src/handler/alter_source_with_sr.rs @@ -16,7 +16,6 @@ use itertools::Itertools; use pgwire::pg_response::StatementType; use risingwave_common::bail_not_implemented; use risingwave_common::catalog::ColumnCatalog; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::catalog::StreamSourceInfo; use risingwave_pb::plan_common::{EncodeType, FormatType}; use risingwave_sqlparser::ast::{ @@ -30,6 +29,7 @@ use super::create_source::{bind_columns_from_source, validate_compatibility}; use super::util::is_cdc_connector; use super::{HandlerArgs, RwPgResponse}; use crate::catalog::root_catalog::SchemaPath; +use crate::error::{ErrorCode, Result}; use crate::{Binder, WithOptions}; fn format_type_to_format(from: FormatType) -> Option { diff --git a/src/frontend/src/handler/alter_system.rs b/src/frontend/src/handler/alter_system.rs index 511a9eb96ec42..9a186aead78f9 100644 --- a/src/frontend/src/handler/alter_system.rs +++ b/src/frontend/src/handler/alter_system.rs @@ -13,12 +13,12 @@ // limitations under the License. use pgwire::pg_response::StatementType; -use risingwave_common::error::Result; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_sqlparser::ast::{Ident, SetVariableValue}; use super::variable::set_var_to_param_str; use super::{HandlerArgs, RwPgResponse}; +use crate::error::Result; // Warn user if barrier_interval_ms is set above 5mins. const NOTICE_BARRIER_INTERVAL_MS: u32 = 300000; diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index b765c84b6e4fa..8446f3ffeb83d 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -18,7 +18,6 @@ use anyhow::Context; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::bail_not_implemented; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_sqlparser::ast::{ AlterTableOperation, ColumnOption, ConnectorSchema, Encode, ObjectName, Statement, @@ -31,6 +30,7 @@ use super::util::SourceSchemaCompatExt; use super::{HandlerArgs, RwPgResponse}; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::table_catalog::TableType; +use crate::error::{ErrorCode, Result, RwError}; use crate::session::SessionImpl; use crate::{Binder, TableCatalog, WithOptions}; diff --git a/src/frontend/src/handler/alter_user.rs b/src/frontend/src/handler/alter_user.rs index 56aad93cc3550..810b71c6dcec1 100644 --- a/src/frontend/src/handler/alter_user.rs +++ b/src/frontend/src/handler/alter_user.rs @@ -13,8 +13,6 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::ErrorCode::{InternalError, PermissionDenied}; -use risingwave_common::error::Result; use risingwave_pb::user::update_user_request::UpdateField; use risingwave_pb::user::UserInfo; use risingwave_sqlparser::ast::{AlterUserStatement, ObjectName, UserOption, UserOptions}; @@ -22,6 +20,8 @@ use risingwave_sqlparser::ast::{AlterUserStatement, ObjectName, UserOption, User use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::CatalogError; +use crate::error::ErrorCode::{InternalError, PermissionDenied}; +use crate::error::Result; use crate::handler::HandlerArgs; use crate::user::user_authentication::encrypted_password; use crate::user::user_catalog::UserCatalog; diff --git a/src/frontend/src/handler/cancel_job.rs b/src/frontend/src/handler/cancel_job.rs index 4a5d5a40d8c91..f124a2a030bd1 100644 --- a/src/frontend/src/handler/cancel_job.rs +++ b/src/frontend/src/handler/cancel_job.rs @@ -16,11 +16,11 @@ use itertools::Itertools; use pgwire::pg_field_descriptor::PgFieldDescriptor; use pgwire::pg_response::{PgResponse, StatementType}; use pgwire::types::Row; -use risingwave_common::error::Result; use risingwave_common::types::DataType; use risingwave_pb::meta::cancel_creating_jobs_request::{CreatingJobIds, PbJobs}; use risingwave_sqlparser::ast::JobIdents; +use crate::error::Result; use crate::handler::{HandlerArgs, RwPgResponse}; pub(super) async fn handle_cancel( diff --git a/src/frontend/src/handler/comment.rs b/src/frontend/src/handler/comment.rs index f3da7c042be80..b35a65112ffa6 100644 --- a/src/frontend/src/handler/comment.rs +++ b/src/frontend/src/handler/comment.rs @@ -13,11 +13,11 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::catalog::PbComment; use risingwave_sqlparser::ast::{CommentObject, ObjectName}; use super::{HandlerArgs, RwPgResponse}; +use crate::error::{ErrorCode, Result}; use crate::Binder; pub async fn handle_comment( diff --git a/src/frontend/src/handler/create_connection.rs b/src/frontend/src/handler/create_connection.rs index bc60682d528b6..40fd6f0d3729e 100644 --- a/src/frontend/src/handler/create_connection.rs +++ b/src/frontend/src/handler/create_connection.rs @@ -15,8 +15,6 @@ use std::collections::HashMap; use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::ErrorCode::ProtocolError; -use risingwave_common::error::{Result, RwError}; use risingwave_connector::source::kafka::PRIVATELINK_CONNECTION; use risingwave_pb::catalog::connection::private_link_service::PrivateLinkProvider; use risingwave_pb::ddl_service::create_connection_request; @@ -24,6 +22,8 @@ use risingwave_sqlparser::ast::CreateConnectionStatement; use super::RwPgResponse; use crate::binder::Binder; +use crate::error::ErrorCode::ProtocolError; +use crate::error::{Result, RwError}; use crate::handler::HandlerArgs; pub(crate) const CONNECTION_TYPE_PROP: &str = "type"; diff --git a/src/frontend/src/handler/create_database.rs b/src/frontend/src/handler/create_database.rs index 590a8d5315973..c5f94b54684d7 100644 --- a/src/frontend/src/handler/create_database.rs +++ b/src/frontend/src/handler/create_database.rs @@ -13,13 +13,13 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::ErrorCode::PermissionDenied; -use risingwave_common::error::Result; use risingwave_sqlparser::ast::ObjectName; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::CatalogError; +use crate::error::ErrorCode::PermissionDenied; +use crate::error::Result; use crate::handler::HandlerArgs; pub async fn handle_create_database( diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index 9e0e2849e265e..af9d9f52b1752 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -22,7 +22,6 @@ use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::acl::AclMode; use risingwave_common::catalog::{IndexId, TableDesc, TableId}; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::catalog::{PbIndex, PbStreamJobStatus, PbTable}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; @@ -33,6 +32,7 @@ use risingwave_sqlparser::ast::{Ident, ObjectName, OrderByExpr}; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::root_catalog::SchemaPath; +use crate::error::{ErrorCode, Result}; use crate::expr::{Expr, ExprImpl, ExprRewriter, InputRef}; use crate::handler::privilege::ObjectCheckItem; use crate::handler::HandlerArgs; diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index 857dadbc100be..a6a48a2c2dba9 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -16,8 +16,6 @@ use either::Either; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::acl::AclMode; -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_pb::stream_plan::StreamScanType; @@ -27,6 +25,8 @@ use super::privilege::resolve_relation_privileges; use super::RwPgResponse; use crate::binder::{Binder, BoundQuery, BoundSetExpr}; use crate::catalog::check_valid_column_name; +use crate::error::ErrorCode::ProtocolError; +use crate::error::{ErrorCode, Result, RwError}; use crate::handler::privilege::resolve_query_privileges; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::generic::GenericPlanRef; diff --git a/src/frontend/src/handler/create_schema.rs b/src/frontend/src/handler/create_schema.rs index faea4fd23ac54..806fbce8af61d 100644 --- a/src/frontend/src/handler/create_schema.rs +++ b/src/frontend/src/handler/create_schema.rs @@ -15,13 +15,13 @@ use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::acl::AclMode; use risingwave_common::catalog::RESERVED_PG_SCHEMA_PREFIX; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::user::grant_privilege::Object; use risingwave_sqlparser::ast::ObjectName; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::{CatalogError, OwnedByUserCatalog}; +use crate::error::{ErrorCode, Result}; use crate::handler::privilege::ObjectCheckItem; use crate::handler::HandlerArgs; diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index e2e9388e81866..de8e93e04a784 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -23,7 +23,6 @@ use itertools::Itertools; use maplit::{convert_args, hashmap}; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::catalog::{ConnectionId, DatabaseId, SchemaId, UserId}; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::types::{DataType, Datum}; use risingwave_common::util::value_encoding::DatumFromProtoExt; use risingwave_common::{bail, catalog}; @@ -49,6 +48,7 @@ use super::create_source::UPSTREAM_SOURCE_KEY; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::catalog_service::CatalogReadGuard; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{ExprImpl, InputRef, Literal}; use crate::handler::alter_table_column::fetch_table_catalog_for_alter; use crate::handler::create_table::{generate_stream_graph_for_table, ColumnIdGenerator}; diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 5f25d12650f0c..0ab4d9ec3a416 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -25,8 +25,6 @@ use risingwave_common::catalog::{ is_column_ids_dedup, ColumnCatalog, ColumnDesc, TableId, INITIAL_SOURCE_VERSION_ID, KAFKA_TIMESTAMP_COLUMN_NAME, }; -use risingwave_common::error::ErrorCode::{self, InvalidInputSyntax, NotSupported, ProtocolError}; -use risingwave_common::error::{Result, RwError}; use risingwave_common::types::DataType; use risingwave_connector::parser::additional_columns::{ build_additional_column_catalog, COMPATIBLE_ADDITIONAL_COLUMNS, @@ -66,6 +64,8 @@ use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::source_catalog::SourceCatalog; use crate::catalog::ColumnId; +use crate::error::ErrorCode::{self, InvalidInputSyntax, NotSupported, ProtocolError}; +use crate::error::{Result, RwError}; use crate::expr::Expr; use crate::handler::create_table::{ bind_pk_on_relation, bind_sql_column_constraints, bind_sql_columns, bind_sql_pk_names, @@ -685,7 +685,7 @@ pub(crate) async fn bind_source_pk( // return the key column names if exists columns.iter().find_map(|catalog| { if matches!( - catalog.column_desc.additional_columns.column_type, + catalog.column_desc.additional_column.column_type, Some(AdditionalColumnType::Key(_)) ) { Some(catalog.name().to_string()) @@ -697,7 +697,7 @@ pub(crate) async fn bind_source_pk( let additional_column_names = columns .iter() .filter_map(|col| { - if col.column_desc.additional_columns.column_type.is_some() { + if col.column_desc.additional_column.column_type.is_some() { Some(col.name().to_string()) } else { None @@ -848,7 +848,7 @@ fn check_and_add_timestamp_column( if is_kafka_connector(with_properties) { if columns.iter().any(|col| { matches!( - col.column_desc.additional_columns.column_type, + col.column_desc.additional_column.column_type, Some(AdditionalColumnType::Timestamp(_)) ) }) { diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index ee871bc68702f..8fc30c2c30e19 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -27,7 +27,6 @@ use risingwave_common::catalog::{ CdcTableDesc, ColumnCatalog, ColumnDesc, TableId, TableVersionId, DEFAULT_SCHEMA_NAME, INITIAL_SOURCE_VERSION_ID, INITIAL_TABLE_VERSION_ID, USER_COLUMN_ID_OFFSET, }; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_common::util::value_encoding::DatumToProtoExt; use risingwave_connector::source; @@ -56,6 +55,7 @@ use crate::catalog::root_catalog::SchemaPath; use crate::catalog::source_catalog::SourceCatalog; use crate::catalog::table_catalog::TableVersion; use crate::catalog::{check_valid_column_name, ColumnId}; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{Expr, ExprImpl, ExprRewriter, InlineNowProcTime}; use crate::handler::create_source::{ bind_all_columns, bind_columns_from_source, bind_source_pk, bind_source_watermark, @@ -214,7 +214,7 @@ pub fn bind_sql_columns(column_defs: &[ColumnDef]) -> Result> type_name: "".to_string(), generated_or_default_column: None, description: None, - additional_columns: AdditionalColumn { column_type: None }, + additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, }, is_hidden: false, diff --git a/src/frontend/src/handler/create_table_as.rs b/src/frontend/src/handler/create_table_as.rs index 576781b5c5594..94a3d541bdcf0 100644 --- a/src/frontend/src/handler/create_table_as.rs +++ b/src/frontend/src/handler/create_table_as.rs @@ -15,13 +15,13 @@ use either::Either; use pgwire::pg_response::StatementType; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc}; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::ddl_service::TableJobType; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ColumnDef, ObjectName, Query, Statement}; use super::{HandlerArgs, RwPgResponse}; use crate::binder::BoundStatement; +use crate::error::{ErrorCode, Result}; use crate::handler::create_table::{gen_create_table_plan_without_bind, ColumnIdGenerator}; use crate::handler::query::handle_query; use crate::{build_graph, Binder, OptimizerContext}; diff --git a/src/frontend/src/handler/create_user.rs b/src/frontend/src/handler/create_user.rs index 7a06fb6092339..434927a21827b 100644 --- a/src/frontend/src/handler/create_user.rs +++ b/src/frontend/src/handler/create_user.rs @@ -13,8 +13,6 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::ErrorCode::PermissionDenied; -use risingwave_common::error::Result; use risingwave_pb::user::grant_privilege::{Action, ActionWithGrantOption, Object}; use risingwave_pb::user::{GrantPrivilege, UserInfo}; use risingwave_sqlparser::ast::{CreateUserStatement, UserOption, UserOptions}; @@ -22,6 +20,8 @@ use risingwave_sqlparser::ast::{CreateUserStatement, UserOption, UserOptions}; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::{CatalogError, DatabaseId}; +use crate::error::ErrorCode::PermissionDenied; +use crate::error::Result; use crate::handler::HandlerArgs; use crate::user::user_authentication::encrypted_password; use crate::user::user_catalog::UserCatalog; diff --git a/src/frontend/src/handler/create_view.rs b/src/frontend/src/handler/create_view.rs index 06b03adb7effa..673fc149dd8c6 100644 --- a/src/frontend/src/handler/create_view.rs +++ b/src/frontend/src/handler/create_view.rs @@ -17,13 +17,13 @@ use either::Either; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::Result; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::catalog::PbView; use risingwave_sqlparser::ast::{Ident, ObjectName, Query, Statement}; use super::RwPgResponse; use crate::binder::Binder; +use crate::error::Result; use crate::handler::HandlerArgs; use crate::optimizer::OptimizerContext; @@ -70,7 +70,7 @@ pub async fn handle_create_view( schema.fields().to_vec() } else { if columns.len() != schema.fields().len() { - return Err(risingwave_common::error::ErrorCode::InternalError( + return Err(crate::error::ErrorCode::InternalError( "view has different number of columns than the query's columns".to_string(), ) .into()); diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index 82b772f87c277..ef1a601cca590 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -19,13 +19,13 @@ use pgwire::pg_field_descriptor::PgFieldDescriptor; use pgwire::pg_response::{PgResponse, StatementType}; use pgwire::types::Row; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc}; -use risingwave_common::error::Result; use risingwave_common::types::DataType; use risingwave_sqlparser::ast::{display_comma_separated, ObjectName}; use super::RwPgResponse; use crate::binder::{Binder, Relation}; use crate::catalog::CatalogError; +use crate::error::Result; use crate::handler::util::col_descs_to_rows; use crate::handler::HandlerArgs; diff --git a/src/frontend/src/handler/drop_connection.rs b/src/frontend/src/handler/drop_connection.rs index d15908c267ec9..b90ae44990740 100644 --- a/src/frontend/src/handler/drop_connection.rs +++ b/src/frontend/src/handler/drop_connection.rs @@ -13,12 +13,12 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::Result; use risingwave_sqlparser::ast::ObjectName; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::root_catalog::SchemaPath; +use crate::error::Result; use crate::handler::HandlerArgs; pub async fn handle_drop_connection( diff --git a/src/frontend/src/handler/drop_database.rs b/src/frontend/src/handler/drop_database.rs index 0d2c7cd6d0b2d..0856840066197 100644 --- a/src/frontend/src/handler/drop_database.rs +++ b/src/frontend/src/handler/drop_database.rs @@ -13,11 +13,11 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_sqlparser::ast::{DropMode, ObjectName}; use super::RwPgResponse; use crate::binder::Binder; +use crate::error::{ErrorCode, Result}; use crate::handler::HandlerArgs; pub async fn handle_drop_database( diff --git a/src/frontend/src/handler/drop_index.rs b/src/frontend/src/handler/drop_index.rs index 428154f7ab0c5..9dc524f13df5c 100644 --- a/src/frontend/src/handler/drop_index.rs +++ b/src/frontend/src/handler/drop_index.rs @@ -13,8 +13,6 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::ErrorCode::PermissionDenied; -use risingwave_common::error::Result; use risingwave_sqlparser::ast::ObjectName; use super::RwPgResponse; @@ -22,6 +20,8 @@ use crate::binder::Binder; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::table_catalog::TableType; use crate::catalog::CatalogError; +use crate::error::ErrorCode::PermissionDenied; +use crate::error::Result; use crate::handler::HandlerArgs; pub async fn handle_drop_index( diff --git a/src/frontend/src/handler/drop_mv.rs b/src/frontend/src/handler/drop_mv.rs index bf3d106ae739d..f727eb5c8dd51 100644 --- a/src/frontend/src/handler/drop_mv.rs +++ b/src/frontend/src/handler/drop_mv.rs @@ -13,7 +13,6 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::Result; use risingwave_sqlparser::ast::ObjectName; use super::RwPgResponse; @@ -21,6 +20,7 @@ use crate::binder::Binder; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::table_catalog::TableType; use crate::catalog::CatalogError; +use crate::error::Result; use crate::handler::HandlerArgs; pub async fn handle_drop_mv( diff --git a/src/frontend/src/handler/drop_schema.rs b/src/frontend/src/handler/drop_schema.rs index ee8c06b064163..f0b889e6ea118 100644 --- a/src/frontend/src/handler/drop_schema.rs +++ b/src/frontend/src/handler/drop_schema.rs @@ -15,12 +15,12 @@ use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::bail_not_implemented; use risingwave_common::catalog::is_system_schema; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_sqlparser::ast::{DropMode, ObjectName}; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::CatalogError; +use crate::error::{ErrorCode, Result}; use crate::handler::HandlerArgs; pub async fn handle_drop_schema( diff --git a/src/frontend/src/handler/drop_sink.rs b/src/frontend/src/handler/drop_sink.rs index 7cc7573663d79..1dcc9d5ec3abf 100644 --- a/src/frontend/src/handler/drop_sink.rs +++ b/src/frontend/src/handler/drop_sink.rs @@ -13,13 +13,13 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::Result; use risingwave_pb::ddl_service::ReplaceTablePlan; use risingwave_sqlparser::ast::ObjectName; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::root_catalog::SchemaPath; +use crate::error::Result; use crate::handler::create_sink::{insert_merger_to_union, reparse_table_for_sink}; use crate::handler::HandlerArgs; diff --git a/src/frontend/src/handler/drop_source.rs b/src/frontend/src/handler/drop_source.rs index 0d242e4d1057c..0003d3a6ce5d0 100644 --- a/src/frontend/src/handler/drop_source.rs +++ b/src/frontend/src/handler/drop_source.rs @@ -13,12 +13,12 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::Result; use risingwave_sqlparser::ast::ObjectName; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::root_catalog::SchemaPath; +use crate::error::Result; use crate::handler::HandlerArgs; pub async fn handle_drop_source( diff --git a/src/frontend/src/handler/drop_table.rs b/src/frontend/src/handler/drop_table.rs index 16bbfacabd7e5..d0fbc3d4e0f78 100644 --- a/src/frontend/src/handler/drop_table.rs +++ b/src/frontend/src/handler/drop_table.rs @@ -13,13 +13,13 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::Result; use risingwave_sqlparser::ast::ObjectName; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::table_catalog::TableType; +use crate::error::Result; use crate::handler::HandlerArgs; pub async fn handle_drop_table( diff --git a/src/frontend/src/handler/drop_user.rs b/src/frontend/src/handler/drop_user.rs index 983544ee76b7b..5cd55ce23bb9f 100644 --- a/src/frontend/src/handler/drop_user.rs +++ b/src/frontend/src/handler/drop_user.rs @@ -13,12 +13,12 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_sqlparser::ast::{DropMode, ObjectName}; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::CatalogError; +use crate::error::{ErrorCode, Result}; use crate::handler::HandlerArgs; pub async fn handle_drop_user( diff --git a/src/frontend/src/handler/drop_view.rs b/src/frontend/src/handler/drop_view.rs index eed64327acf80..e11ae5ea36f8e 100644 --- a/src/frontend/src/handler/drop_view.rs +++ b/src/frontend/src/handler/drop_view.rs @@ -13,12 +13,12 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::Result; use risingwave_sqlparser::ast::ObjectName; use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::root_catalog::SchemaPath; +use crate::error::Result; use crate::handler::HandlerArgs; pub async fn handle_drop_view( diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index e5489bf258f8b..c25bf7678bd04 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -17,7 +17,6 @@ use pgwire::pg_field_descriptor::PgFieldDescriptor; use pgwire::pg_response::{PgResponse, StatementType}; use pgwire::types::Row; use risingwave_common::bail_not_implemented; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; use risingwave_sqlparser::ast::{ExplainOptions, ExplainType, Statement}; use thiserror_ext::AsReport; @@ -29,6 +28,7 @@ use super::create_table::ColumnIdGenerator; use super::query::gen_batch_plan_by_statement; use super::util::SourceSchemaCompatExt; use super::RwPgResponse; +use crate::error::{ErrorCode, Result}; use crate::handler::create_table::handle_create_table_plan; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::generic::GenericPlanRef; diff --git a/src/frontend/src/handler/extended_handle.rs b/src/frontend/src/handler/extended_handle.rs index 6074027799e26..b497b1164d144 100644 --- a/src/frontend/src/handler/extended_handle.rs +++ b/src/frontend/src/handler/extended_handle.rs @@ -19,12 +19,12 @@ use std::sync::Arc; use bytes::Bytes; use pgwire::types::Format; use risingwave_common::bail_not_implemented; -use risingwave_common::error::Result; use risingwave_common::types::DataType; use risingwave_sqlparser::ast::{CreateSink, Query, Statement}; use super::query::BoundResult; use super::{handle, query, HandlerArgs, RwPgResponse}; +use crate::error::Result; use crate::session::SessionImpl; /// Except for Query,Insert,Delete,Update statement, we store other statement as `PureStatement`. diff --git a/src/frontend/src/handler/flush.rs b/src/frontend/src/handler/flush.rs index 14e783d1f0005..784fbb393c916 100644 --- a/src/frontend/src/handler/flush.rs +++ b/src/frontend/src/handler/flush.rs @@ -13,9 +13,9 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::Result; use super::RwPgResponse; +use crate::error::Result; use crate::handler::HandlerArgs; use crate::session::SessionImpl; diff --git a/src/frontend/src/handler/handle_privilege.rs b/src/frontend/src/handler/handle_privilege.rs index cfb8b15eddbd7..99e18106ed931 100644 --- a/src/frontend/src/handler/handle_privilege.rs +++ b/src/frontend/src/handler/handle_privilege.rs @@ -13,7 +13,6 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::user::grant_privilege::{ActionWithGrantOption, PbObject}; use risingwave_pb::user::PbGrantPrivilege; use risingwave_sqlparser::ast::{GrantObjects, Privileges, Statement}; @@ -22,6 +21,7 @@ use super::RwPgResponse; use crate::binder::Binder; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::table_catalog::TableType; +use crate::error::{ErrorCode, Result}; use crate::handler::HandlerArgs; use crate::session::SessionImpl; use crate::user::user_privilege::{ diff --git a/src/frontend/src/handler/kill_process.rs b/src/frontend/src/handler/kill_process.rs index 348c2ce3c509f..1e1d1b03715c3 100644 --- a/src/frontend/src/handler/kill_process.rs +++ b/src/frontend/src/handler/kill_process.rs @@ -13,8 +13,8 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::{ErrorCode, Result}; +use crate::error::{ErrorCode, Result}; use crate::handler::{HandlerArgs, RwPgResponse}; pub(super) async fn handle_kill( diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 5becea016e108..3cdc4b191da92 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -23,12 +23,12 @@ use pgwire::pg_response::{PgResponse, PgResponseBuilder, RowSetResult}; use pgwire::pg_server::BoxedError; use pgwire::types::{Format, Row}; use risingwave_common::bail_not_implemented; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_sqlparser::ast::*; use self::util::{DataChunkToRowSetAdapter, SourceSchemaCompatExt}; use self::variable::handle_set_time_zone; use crate::catalog::table_catalog::TableType; +use crate::error::{ErrorCode, Result}; use crate::handler::cancel_job::handle_cancel; use crate::handler::kill_process::handle_kill; use crate::scheduler::{DistributedQueryStream, LocalQueryStream}; diff --git a/src/frontend/src/handler/privilege.rs b/src/frontend/src/handler/privilege.rs index 44118fddf6758..d26d1d6d4785c 100644 --- a/src/frontend/src/handler/privilege.rs +++ b/src/frontend/src/handler/privilege.rs @@ -13,12 +13,12 @@ // limitations under the License. use risingwave_common::acl::AclMode; -use risingwave_common::error::ErrorCode::PermissionDenied; -use risingwave_common::error::Result; use risingwave_pb::user::grant_privilege::PbObject; use crate::binder::{BoundQuery, BoundStatement, Relation}; use crate::catalog::OwnedByUserCatalog; +use crate::error::ErrorCode::PermissionDenied; +use crate::error::Result; use crate::session::SessionImpl; use crate::user::UserId; diff --git a/src/frontend/src/handler/query.rs b/src/frontend/src/handler/query.rs index d84db9cd8260f..321b318fa0068 100644 --- a/src/frontend/src/handler/query.rs +++ b/src/frontend/src/handler/query.rs @@ -23,7 +23,6 @@ use pgwire::pg_response::{PgResponse, StatementType}; use pgwire::types::Format; use postgres_types::FromSql; use risingwave_common::catalog::Schema; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::session_config::QueryMode; use risingwave_common::types::{DataType, Datum}; use risingwave_sqlparser::ast::{SetExpr, Statement}; @@ -32,6 +31,7 @@ use super::extended_handle::{PortalResult, PrepareStatement, PreparedResult}; use super::{PgResponseStream, RwPgResponse}; use crate::binder::{Binder, BoundStatement}; use crate::catalog::TableId; +use crate::error::{ErrorCode, Result, RwError}; use crate::handler::flush::do_flush; use crate::handler::privilege::resolve_privileges; use crate::handler::util::{to_pg_field, DataChunkToRowSetAdapter}; diff --git a/src/frontend/src/handler/show.rs b/src/frontend/src/handler/show.rs index d5addc3094e48..4a98b6c7cd33d 100644 --- a/src/frontend/src/handler/show.rs +++ b/src/frontend/src/handler/show.rs @@ -22,7 +22,6 @@ use pgwire::pg_server::Session; use pgwire::types::Row; use risingwave_common::bail_not_implemented; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, DEFAULT_SCHEMA_NAME}; -use risingwave_common::error::Result; use risingwave_common::types::DataType; use risingwave_common::util::addr::HostAddr; use risingwave_connector::source::kafka::PRIVATELINK_CONNECTION; @@ -36,6 +35,7 @@ use serde_json; use super::RwPgResponse; use crate::binder::{Binder, Relation}; use crate::catalog::{CatalogError, IndexCatalog}; +use crate::error::Result; use crate::handler::util::{col_descs_to_rows, indexes_to_rows}; use crate::handler::HandlerArgs; use crate::session::SessionImpl; diff --git a/src/frontend/src/handler/transaction.rs b/src/frontend/src/handler/transaction.rs index 538e67c264724..452cfe0ed9299 100644 --- a/src/frontend/src/handler/transaction.rs +++ b/src/frontend/src/handler/transaction.rs @@ -15,10 +15,10 @@ use pgwire::pg_response::StatementType; use pgwire::types::Row; use risingwave_common::bail_not_implemented; -use risingwave_common::error::Result; use risingwave_sqlparser::ast::{TransactionAccessMode, TransactionMode, Value}; use super::{HandlerArgs, RwPgResponse}; +use crate::error::Result; use crate::session::transaction::AccessMode; use crate::utils::infer_stmt_row_desc::infer_show_variable; diff --git a/src/frontend/src/handler/util.rs b/src/frontend/src/handler/util.rs index c225ccec33dc2..6e91cf53f0b32 100644 --- a/src/frontend/src/handler/util.rs +++ b/src/frontend/src/handler/util.rs @@ -28,7 +28,6 @@ use pgwire::types::{Format, FormatIterator, Row}; use pin_project_lite::pin_project; use risingwave_common::array::DataChunk; use risingwave_common::catalog::{ColumnCatalog, Field}; -use risingwave_common::error::{ErrorCode, Result as RwResult}; use risingwave_common::row::Row as _; use risingwave_common::types::{DataType, ScalarRefImpl, Timestamptz}; use risingwave_common::util::iter_util::ZipEqFast; @@ -36,6 +35,7 @@ use risingwave_connector::source::KAFKA_CONNECTOR; use risingwave_sqlparser::ast::{display_comma_separated, CompatibleSourceSchema, ConnectorSchema}; use crate::catalog::IndexCatalog; +use crate::error::{ErrorCode, Result as RwResult}; use crate::handler::create_source::UPSTREAM_SOURCE_KEY; use crate::session::{current, SessionImpl}; diff --git a/src/frontend/src/handler/variable.rs b/src/frontend/src/handler/variable.rs index dd633449050ce..884947c88b763 100644 --- a/src/frontend/src/handler/variable.rs +++ b/src/frontend/src/handler/variable.rs @@ -17,7 +17,6 @@ use itertools::Itertools; use pgwire::pg_protocol::ParameterStatus; use pgwire::pg_response::{PgResponse, StatementType}; use pgwire::types::Row; -use risingwave_common::error::Result; use risingwave_common::session_config::{ConfigReporter, SESSION_CONFIG_LIST_SEP}; use risingwave_common::system_param::is_mutable; use risingwave_common::types::{DataType, ScalarRefImpl}; @@ -25,6 +24,7 @@ use risingwave_sqlparser::ast::{Ident, SetTimeZoneValue, SetVariableValue, Value use risingwave_sqlparser::keywords::Keyword; use super::RwPgResponse; +use crate::error::Result; use crate::handler::HandlerArgs; use crate::utils::infer_stmt_row_desc::infer_show_variable; diff --git a/src/frontend/src/handler/wait.rs b/src/frontend/src/handler/wait.rs index 2edc541964a12..8e931133b2ace 100644 --- a/src/frontend/src/handler/wait.rs +++ b/src/frontend/src/handler/wait.rs @@ -13,9 +13,9 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; -use risingwave_common::error::Result; use super::RwPgResponse; +use crate::error::Result; use crate::handler::HandlerArgs; use crate::session::SessionImpl; diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 805221327f016..9dc64983671d3 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -61,6 +61,7 @@ use risingwave_common::util::meta_addr::MetaAddressStrategy; pub use stream_fragmenter::build_graph; mod utils; pub use utils::{explain_stream_graph, WithOptions}; +pub(crate) mod error; mod meta_client; pub mod test_utils; mod user; diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index db5dc8ceca7d2..6a1ceac758e46 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -14,10 +14,10 @@ use itertools::Itertools; use risingwave_common::bail; -use risingwave_common::error::Result; use super::plan_node::RewriteExprsRecursive; use super::plan_visitor::has_logical_max_one_row; +use crate::error::Result; use crate::expr::{InlineNowProcTime, NowProcTimeFinder}; use crate::optimizer::heuristic_optimizer::{ApplyOrder, HeuristicOptimizer}; use crate::optimizer::plan_node::{ diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index e27220feb279a..f222a41fb48e7 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -49,7 +49,6 @@ use risingwave_common::bail; use risingwave_common::catalog::{ ColumnCatalog, ColumnDesc, ColumnId, ConflictBehavior, Field, Schema, TableId, }; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_connector::sink::catalog::SinkFormatDesc; @@ -69,6 +68,7 @@ use self::plan_visitor::{has_batch_exchange, CardinalityVisitor, StreamKeyChecke use self::property::{Cardinality, RequiredDist}; use self::rule::*; use crate::catalog::table_catalog::{TableType, TableVersion}; +use crate::error::{ErrorCode, Result}; use crate::expr::TimestamptzExprFinder; use crate::optimizer::plan_node::generic::{SourceNodeKind, Union}; use crate::optimizer::plan_node::{ diff --git a/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs index d1cafc37ff25d..43b76891f3566 100644 --- a/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs +++ b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs @@ -12,8 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::RwError; - +use crate::error::RwError; use crate::expr::{Expr, ExprImpl, ExprRewriter, Literal}; pub(crate) struct ConstEvalRewriter { diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index 209cd74a76278..7bc932f1c07cf 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::DeleteNode; @@ -21,6 +20,7 @@ use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::{GenericPlanNode, PhysicalPlanRef}; use crate::optimizer::plan_node::{utils, ToLocalBatch}; diff --git a/src/frontend/src/optimizer/plan_node/batch_exchange.rs b/src/frontend/src/optimizer/plan_node/batch_exchange.rs index cd0f6ef8e1e0b..70d48b5923ba5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/batch_exchange.rs @@ -13,13 +13,13 @@ // limitations under the License. use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{ExchangeNode, MergeSortExchangeNode}; use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, DistributionDisplay, Order, OrderDisplay}; diff --git a/src/frontend/src/optimizer/plan_node/batch_expand.rs b/src/frontend/src/optimizer/plan_node/batch_expand.rs index 8c122d3bbbec9..f6738d9fdef89 100644 --- a/src/frontend/src/optimizer/plan_node/batch_expand.rs +++ b/src/frontend/src/optimizer/plan_node/batch_expand.rs @@ -13,7 +13,6 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::expand_node::Subset; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ExpandNode; @@ -21,6 +20,7 @@ use risingwave_pb::batch_plan::ExpandNode; use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable}; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ PlanBase, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, ToLocalBatch, diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index 9af73d2e88308..ff89eacd485c0 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -12,13 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::FilterNode; use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; +use crate::error::Result; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{PlanBase, ToLocalBatch}; diff --git a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs index ad2b555211322..adf5d102374df 100644 --- a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::GroupTopNNode; @@ -21,6 +20,7 @@ use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, RequiredDist}; diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs index b6b2509e09722..af5f4c4accbb6 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -13,7 +13,6 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashAggNode; @@ -24,6 +23,7 @@ use super::{ ExprRewritable, PlanBase, PlanNodeType, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; +use crate::error::Result; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::ToLocalBatch; diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs index e6e97180ef721..8b92f722490d5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -13,7 +13,6 @@ // limitations under the License. use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashJoinNode; use risingwave_pb::plan_common::JoinType; @@ -25,6 +24,7 @@ use super::{ EqJoinPredicate, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchPb, ToDistributedBatch, }; +use crate::error::Result; use crate::expr::{Expr, ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::IndicesDisplay; diff --git a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs index 1a7e8b35c0f65..e1db75da3220d 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HopWindowNode; @@ -21,6 +20,7 @@ use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; +use crate::error::Result; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::ToLocalBatch; diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index a0c4df9ef75ca..2a7098fc38374 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -13,7 +13,6 @@ // limitations under the License. use pretty_xmlish::XmlNode; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::InsertNode; use risingwave_pb::plan_common::{DefaultColumns, IndexAndExpr}; @@ -22,6 +21,7 @@ use super::batch::prelude::*; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; +use crate::error::Result; use crate::expr::Expr; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::GenericPlanNode; diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index 2425c98b12e1d..1dfe9215be1ea 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::LimitNode; @@ -22,6 +21,7 @@ use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{BatchExchange, ToLocalBatch}; use crate::optimizer::property::{Distribution, Order, RequiredDist}; diff --git a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs index 1567e1ffd0959..a20b4ad316cd7 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -14,7 +14,6 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{ColumnId, TableDesc}; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{DistributedLookupJoinNode, LocalLookupJoinNode}; @@ -22,6 +21,7 @@ use super::batch::prelude::*; use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::ExprRewritable; +use crate::error::Result; use crate::expr::{Expr, ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::IndicesDisplay; diff --git a/src/frontend/src/optimizer/plan_node/batch_max_one_row.rs b/src/frontend/src/optimizer/plan_node/batch_max_one_row.rs index 24a1e82907cba..94b8b3e6e0483 100644 --- a/src/frontend/src/optimizer/plan_node/batch_max_one_row.rs +++ b/src/frontend/src/optimizer/plan_node/batch_max_one_row.rs @@ -13,7 +13,6 @@ // limitations under the License. use pretty_xmlish::XmlNode; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::MaxOneRowNode; @@ -23,6 +22,7 @@ use super::utils::Distill; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::ToLocalBatch; diff --git a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs index 4a3abd55fc7d9..401166b1a298a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs @@ -13,7 +13,6 @@ // limitations under the License. use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::NestedLoopJoinNode; @@ -21,6 +20,7 @@ use super::batch::prelude::*; use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchPb, ToDistributedBatch}; +use crate::error::Result; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::IndicesDisplay; diff --git a/src/frontend/src/optimizer/plan_node/batch_over_window.rs b/src/frontend/src/optimizer/plan_node/batch_over_window.rs index 67ba824386898..1712e87bbabbf 100644 --- a/src/frontend/src/optimizer/plan_node/batch_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_over_window.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortOverWindowNode; @@ -25,6 +24,7 @@ use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, ToLocalBatch, }; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::property::{Order, RequiredDist}; diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index 1cb1afe20e717..8b69a4b5297a5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -13,7 +13,6 @@ // limitations under the License. use pretty_xmlish::XmlNode; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ProjectNode; use risingwave_pb::expr::ExprNode; @@ -23,6 +22,7 @@ use super::utils::{childless_record, Distill}; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; +use crate::error::Result; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::ToLocalBatch; diff --git a/src/frontend/src/optimizer/plan_node/batch_project_set.rs b/src/frontend/src/optimizer/plan_node/batch_project_set.rs index b20e8651f309b..1efb569cbb408 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -13,13 +13,13 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ProjectSetNode; use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable}; +use crate::error::Result; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ 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 245a2465631f0..91ac83ef1bfe5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -16,7 +16,6 @@ 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; @@ -26,6 +25,7 @@ use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, ToDistributedBatch}; use crate::catalog::ColumnId; +use crate::error::Result; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ToLocalBatch, TryToBatchPb}; diff --git a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs index 7bacf2f9463b3..a112e36c3a98f 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; @@ -20,6 +19,7 @@ use super::batch::prelude::*; use super::generic::{self, GenericPlanRef, PlanAggCall}; use super::utils::impl_distill_by_unit; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; +use crate::error::Result; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{BatchExchange, ToLocalBatch}; diff --git a/src/frontend/src/optimizer/plan_node/batch_sort.rs b/src/frontend/src/optimizer/plan_node/batch_sort.rs index e1e03e40fa7b7..413103ee98c89 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort.rs @@ -13,7 +13,6 @@ // limitations under the License. use pretty_xmlish::XmlNode; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortNode; @@ -21,6 +20,7 @@ use super::batch::prelude::*; use super::batch::BatchPlanRef; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, OrderDisplay}; diff --git a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs index 9a98fb65cc224..7393c10edc8f0 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; use risingwave_pb::expr::ExprNode; @@ -21,6 +20,7 @@ use super::batch::prelude::*; use super::generic::{self, GenericPlanRef, PlanAggCall}; use super::utils::impl_distill_by_unit; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; +use crate::error::Result; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprVisitor, InputRef}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::ToLocalBatch; diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index b95e2727821e4..a3c58d416bcdd 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -15,7 +15,6 @@ use std::rc::Rc; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SourceNode; @@ -25,6 +24,7 @@ use super::{ generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch, ToLocalBatch, }; use crate::catalog::source_catalog::SourceCatalog; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::property::{Distribution, Order}; 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 index 7ab3af13ac191..6068c1131626b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sys_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sys_seq_scan.rs @@ -16,7 +16,6 @@ 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; @@ -26,6 +25,7 @@ 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::error::Result; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::ToLocalBatch; diff --git a/src/frontend/src/optimizer/plan_node/batch_table_function.rs b/src/frontend/src/optimizer/plan_node/batch_table_function.rs index c7d9e54fa4bf6..3bda5cd4eb324 100644 --- a/src/frontend/src/optimizer/plan_node/batch_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/batch_table_function.rs @@ -13,13 +13,13 @@ // limitations under the License. use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TableFunctionNode; use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchPb, ToDistributedBatch}; +use crate::error::Result; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::logical_table_function::LogicalTableFunction; diff --git a/src/frontend/src/optimizer/plan_node/batch_topn.rs b/src/frontend/src/optimizer/plan_node/batch_topn.rs index 8d8c2a1e27813..3b0072821e389 100644 --- a/src/frontend/src/optimizer/plan_node/batch_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_topn.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TopNNode; @@ -22,6 +21,7 @@ use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; +use crate::error::Result; use crate::optimizer::plan_node::batch::BatchPlanRef; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{BatchLimit, ToLocalBatch}; diff --git a/src/frontend/src/optimizer/plan_node/batch_union.rs b/src/frontend/src/optimizer/plan_node/batch_union.rs index 4cc6e2bcba9c7..2c2a8f54d4b85 100644 --- a/src/frontend/src/optimizer/plan_node/batch_union.rs +++ b/src/frontend/src/optimizer/plan_node/batch_union.rs @@ -12,13 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UnionNode; use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanRef, ToBatchPb, ToDistributedBatch}; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{PlanBase, PlanTreeNode, ToLocalBatch}; use crate::optimizer::property::{Distribution, Order, RequiredDist}; diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index a6c675c996d90..65e888c15e328 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -14,7 +14,6 @@ use itertools::Itertools; use risingwave_common::catalog::Schema; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UpdateNode; @@ -24,6 +23,7 @@ use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; +use crate::error::Result; use crate::expr::{Expr, ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::GenericPlanNode; diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index a9baef2fff9f1..56b8f240e6620 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -13,7 +13,6 @@ // limitations under the License. use pretty_xmlish::XmlNode; -use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::values_node::ExprTuple; use risingwave_pb::batch_plan::ValuesNode; @@ -24,6 +23,7 @@ use super::{ ExprRewritable, LogicalValues, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchPb, ToDistributedBatch, }; +use crate::error::Result; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::ToLocalBatch; diff --git a/src/frontend/src/optimizer/plan_node/derive.rs b/src/frontend/src/optimizer/plan_node/derive.rs index 45985ddba113a..fc41f2a4808a8 100644 --- a/src/frontend/src/optimizer/plan_node/derive.rs +++ b/src/frontend/src/optimizer/plan_node/derive.rs @@ -17,10 +17,10 @@ use std::collections::HashSet; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, Schema, USER_COLUMN_ID_OFFSET}; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use super::PlanRef; +use crate::error::{ErrorCode, Result}; use crate::optimizer::property::Order; pub(crate) fn derive_columns( diff --git a/src/frontend/src/optimizer/plan_node/generic/hop_window.rs b/src/frontend/src/optimizer/plan_node/generic/hop_window.rs index 30dc877ca67f5..6fb3e8609f6c8 100644 --- a/src/frontend/src/optimizer/plan_node/generic/hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/generic/hop_window.rs @@ -17,13 +17,13 @@ use std::num::NonZeroUsize; use itertools::Itertools; use pretty_xmlish::{Pretty, StrAssocArr}; use risingwave_common::catalog::{Field, Schema}; -use risingwave_common::error::Result; use risingwave_common::types::{DataType, Interval}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_expr::ExprError; use super::super::utils::IndicesDisplay; use super::{impl_distill_unit_from_fields, GenericPlanNode, GenericPlanRef}; +use crate::error::Result; use crate::expr::{ExprImpl, ExprType, FunctionCall, InputRef, InputRefDisplay, Literal}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::batch::BatchPlanRef; diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 1bd75c2b3341a..d9a8d78c54aa8 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -14,7 +14,6 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::{DataType, Datum, ScalarImpl}; use risingwave_common::util::sort_util::ColumnOrder; use risingwave_common::{bail_not_implemented, not_implemented}; @@ -28,6 +27,7 @@ use super::{ PlanTreeNodeUnary, PredicatePushdown, StreamHashAgg, StreamProject, StreamSimpleAgg, StreamStatelessSimpleAgg, ToBatch, ToStream, }; +use crate::error::{ErrorCode, Result}; use crate::expr::{ AggCall, Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor, FunctionCall, InputRef, Literal, OrderBy, WindowFunction, diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index 49007fd080c15..dd30dac956383 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -15,7 +15,6 @@ // use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::Schema; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_pb::plan_common::JoinType; use super::generic::{ @@ -26,6 +25,7 @@ use super::{ ColPrunable, Logical, LogicalJoin, LogicalProject, PlanBase, PlanRef, PlanTreeNodeBinary, PredicatePushdown, ToBatch, ToStream, }; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{CorrelatedId, Expr, ExprImpl, ExprRewriter, ExprVisitor, InputRef}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ diff --git a/src/frontend/src/optimizer/plan_node/logical_cdc_scan.rs b/src/frontend/src/optimizer/plan_node/logical_cdc_scan.rs index a65d297b0681a..43ce91bd0dc21 100644 --- a/src/frontend/src/optimizer/plan_node/logical_cdc_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_cdc_scan.rs @@ -16,7 +16,6 @@ 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}; @@ -25,6 +24,7 @@ use super::{ ToStream, }; use crate::catalog::ColumnId; +use crate::error::Result; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; diff --git a/src/frontend/src/optimizer/plan_node/logical_dedup.rs b/src/frontend/src/optimizer/plan_node/logical_dedup.rs index 3bf1d49b61e26..a9902bc87ac11 100644 --- a/src/frontend/src/optimizer/plan_node/logical_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/logical_dedup.rs @@ -14,7 +14,6 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_common::error::Result; use risingwave_common::util::column_index_mapping::ColIndexMapping; use super::generic::TopNLimit; @@ -25,6 +24,7 @@ use super::{ PredicatePushdown, PredicatePushdownContext, RewriteStreamContext, StreamDedup, StreamGroupTopN, ToBatch, ToStream, ToStreamContext, }; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::property::{Order, RequiredDist}; use crate::utils::Condition; diff --git a/src/frontend/src/optimizer/plan_node/logical_delete.rs b/src/frontend/src/optimizer/plan_node/logical_delete.rs index 480272db5005e..651147958dee3 100644 --- a/src/frontend/src/optimizer/plan_node/logical_delete.rs +++ b/src/frontend/src/optimizer/plan_node/logical_delete.rs @@ -13,7 +13,6 @@ // limitations under the License. use risingwave_common::catalog::TableVersionId; -use risingwave_common::error::Result; use super::utils::impl_distill_by_unit; use super::{ @@ -21,6 +20,7 @@ use super::{ LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, diff --git a/src/frontend/src/optimizer/plan_node/logical_except.rs b/src/frontend/src/optimizer/plan_node/logical_except.rs index a8c813ec05e5a..d4a76bb98f091 100644 --- a/src/frontend/src/optimizer/plan_node/logical_except.rs +++ b/src/frontend/src/optimizer/plan_node/logical_except.rs @@ -14,12 +14,12 @@ use itertools::Itertools; use risingwave_common::catalog::Schema; -use risingwave_common::error::Result; use super::utils::impl_distill_by_unit; use super::{ ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, }; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index 4e666fd7d6c90..c1524277c99a1 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -13,7 +13,6 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::error::Result; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; @@ -21,6 +20,7 @@ use super::{ gen_filter_and_pushdown, generic, BatchExpand, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamExpand, ToBatch, ToStream, }; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, diff --git a/src/frontend/src/optimizer/plan_node/logical_filter.rs b/src/frontend/src/optimizer/plan_node/logical_filter.rs index 1460e7f0bf72f..6bae6cc7efb22 100644 --- a/src/frontend/src/optimizer/plan_node/logical_filter.rs +++ b/src/frontend/src/optimizer/plan_node/logical_filter.rs @@ -15,7 +15,6 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::bail; -use risingwave_common::error::Result; use risingwave_common::types::DataType; use super::generic::GenericPlanRef; @@ -24,6 +23,7 @@ use super::{ generic, ColPrunable, ExprRewritable, Logical, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; +use crate::error::Result; use crate::expr::{ assert_input_ref, ExprImpl, ExprRewriter, ExprType, ExprVisitor, FunctionCall, InputRef, }; diff --git a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs index 9072ac6d4113f..37f1dba5c2720 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -14,7 +14,6 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_common::error::Result; use risingwave_common::types::Interval; use super::generic::{GenericPlanNode, GenericPlanRef}; @@ -24,6 +23,7 @@ use super::{ LogicalFilter, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, ToBatch, ToStream, }; +use crate::error::Result; use crate::expr::{ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ diff --git a/src/frontend/src/optimizer/plan_node/logical_insert.rs b/src/frontend/src/optimizer/plan_node/logical_insert.rs index 487bc3c2f6ec6..347fdfdef664b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_insert.rs +++ b/src/frontend/src/optimizer/plan_node/logical_insert.rs @@ -14,7 +14,6 @@ use pretty_xmlish::XmlNode; use risingwave_common::catalog::TableVersionId; -use risingwave_common::error::Result; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; @@ -23,6 +22,7 @@ use super::{ LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; +use crate::error::Result; use crate::expr::{ExprImpl, ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ diff --git a/src/frontend/src/optimizer/plan_node/logical_intersect.rs b/src/frontend/src/optimizer/plan_node/logical_intersect.rs index ce697ff7f5939..f5719eede5954 100644 --- a/src/frontend/src/optimizer/plan_node/logical_intersect.rs +++ b/src/frontend/src/optimizer/plan_node/logical_intersect.rs @@ -14,12 +14,12 @@ use itertools::Itertools; use risingwave_common::catalog::Schema; -use risingwave_common::error::Result; use super::utils::impl_distill_by_unit; use super::{ ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, }; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ generic, ColumnPruningContext, PlanTreeNode, PredicatePushdownContext, RewriteStreamContext, diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index 0b83e2491ce99..62fb521fa9d0f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -17,7 +17,6 @@ use std::collections::HashMap; use fixedbitset::FixedBitSet; use itertools::{EitherOrBoth, Itertools}; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_pb::plan_common::JoinType; use risingwave_pb::stream_plan::StreamScanType; @@ -29,6 +28,7 @@ use super::{ generic, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeBinary, PredicatePushdown, StreamHashJoin, StreamProject, ToBatch, ToStream, }; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{CollectInputRef, Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor, InputRef}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::DynamicFilter; diff --git a/src/frontend/src/optimizer/plan_node/logical_limit.rs b/src/frontend/src/optimizer/plan_node/logical_limit.rs index 1789e1d0cf939..a0d1275ea0af8 100644 --- a/src/frontend/src/optimizer/plan_node/logical_limit.rs +++ b/src/frontend/src/optimizer/plan_node/logical_limit.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use super::utils::impl_distill_by_unit; @@ -20,6 +19,7 @@ use super::{ gen_filter_and_pushdown, generic, BatchLimit, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ ColumnPruningContext, LogicalTopN, PredicatePushdownContext, RewriteStreamContext, diff --git a/src/frontend/src/optimizer/plan_node/logical_max_one_row.rs b/src/frontend/src/optimizer/plan_node/logical_max_one_row.rs index 5f104c71c4c92..db9c07724fc96 100644 --- a/src/frontend/src/optimizer/plan_node/logical_max_one_row.rs +++ b/src/frontend/src/optimizer/plan_node/logical_max_one_row.rs @@ -13,7 +13,6 @@ // limitations under the License. use pretty_xmlish::XmlNode; -use risingwave_common::error::Result; use super::generic::DistillUnit; use super::utils::Distill; @@ -21,6 +20,7 @@ use super::{ gen_filter_and_pushdown, generic, BatchMaxOneRow, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, diff --git a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs index ca25b6fd92aef..fd56d3b1afcde 100644 --- a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs @@ -18,7 +18,6 @@ use std::collections::{BTreeMap, BTreeSet, VecDeque}; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::Schema; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_pb::plan_common::JoinType; use super::utils::{childless_record, Distill}; @@ -27,6 +26,7 @@ use super::{ PlanNodeType, PlanRef, PlanTreeNodeBinary, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{ExprImpl, ExprRewriter, ExprType, ExprVisitor, FunctionCall}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ diff --git a/src/frontend/src/optimizer/plan_node/logical_now.rs b/src/frontend/src/optimizer/plan_node/logical_now.rs index 22121070c0eb2..f9c33eb3d9cc1 100644 --- a/src/frontend/src/optimizer/plan_node/logical_now.rs +++ b/src/frontend/src/optimizer/plan_node/logical_now.rs @@ -15,7 +15,6 @@ use pretty_xmlish::XmlNode; use risingwave_common::bail; use risingwave_common::catalog::{Field, Schema}; -use risingwave_common::error::Result; use risingwave_common::types::DataType; use super::generic::GenericPlanRef; @@ -24,6 +23,7 @@ use super::{ ColPrunable, ColumnPruningContext, ExprRewritable, Logical, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, RewriteStreamContext, StreamNow, ToBatch, ToStream, ToStreamContext, }; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::column_names_pretty; use crate::optimizer::property::FunctionalDependencySet; diff --git a/src/frontend/src/optimizer/plan_node/logical_over_window.rs b/src/frontend/src/optimizer/plan_node/logical_over_window.rs index a1d3909b54d08..af1fc50c9057f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_over_window.rs @@ -14,7 +14,6 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::types::{DataType, Datum, ScalarImpl}; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_common::{bail_not_implemented, not_implemented}; @@ -28,6 +27,7 @@ use super::{ LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamEowcOverWindow, StreamEowcSort, StreamOverWindow, ToBatch, ToStream, }; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{ Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor, FunctionCall, InputRef, WindowFunction, }; diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index 42f7e699325e4..a0f26548da2b1 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -15,13 +15,13 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::XmlNode; -use risingwave_common::error::Result; use super::utils::{childless_record, Distill}; use super::{ gen_filter_and_pushdown, generic, BatchProject, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProject, ToBatch, ToStream, }; +use crate::error::Result; use crate::expr::{collect_input_refs, ExprImpl, ExprRewriter, ExprVisitor, InputRef}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::GenericPlanRef; diff --git a/src/frontend/src/optimizer/plan_node/logical_project_set.rs b/src/frontend/src/optimizer/plan_node/logical_project_set.rs index be0995de0cc40..53e7995c1517e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project_set.rs @@ -14,7 +14,6 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_common::error::Result; use risingwave_common::types::DataType; use super::utils::impl_distill_by_unit; @@ -23,6 +22,7 @@ use super::{ LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, ToBatch, ToStream, }; +use crate::error::Result; use crate::expr::{ collect_input_refs, Expr, ExprImpl, ExprRewriter, ExprVisitor, FunctionCall, InputRef, TableFunction, diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index b451dea88c630..bacb0952cece8 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -20,7 +20,6 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{ColumnDesc, TableDesc}; -use risingwave_common::error::Result; use risingwave_common::util::sort_util::ColumnOrder; use super::generic::{GenericPlanNode, GenericPlanRef}; @@ -30,6 +29,7 @@ use super::{ PredicatePushdown, StreamTableScan, ToBatch, ToStream, }; use crate::catalog::{ColumnId, IndexCatalog}; +use crate::error::Result; use crate::expr::{CorrelatedInputRef, ExprImpl, ExprRewriter, ExprVisitor, InputRef}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs index c810bb5c761cf..e2d6eb39214ed 100644 --- a/src/frontend/src/optimizer/plan_node/logical_share.rs +++ b/src/frontend/src/optimizer/plan_node/logical_share.rs @@ -16,13 +16,13 @@ use std::cell::RefCell; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::bail_not_implemented; -use risingwave_common::error::Result; use super::utils::{childless_record, Distill}; use super::{ generic, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; +use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index fbc9fe7a40c8c..fa7ad908d01d4 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -23,7 +23,6 @@ use risingwave_common::bail_not_implemented; use risingwave_common::catalog::{ ColumnCatalog, ColumnDesc, Field, Schema, KAFKA_TIMESTAMP_COLUMN_NAME, }; -use risingwave_common::error::Result; use risingwave_connector::source::DataType; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; use risingwave_pb::plan_common::GeneratedColumnDesc; @@ -37,6 +36,7 @@ use super::{ StreamSource, ToBatch, ToStream, }; use crate::catalog::source_catalog::SourceCatalog; +use crate::error::Result; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor, InputRef}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; diff --git a/src/frontend/src/optimizer/plan_node/logical_sys_scan.rs b/src/frontend/src/optimizer/plan_node/logical_sys_scan.rs index 93a48d147eee4..a85afa8afa039 100644 --- a/src/frontend/src/optimizer/plan_node/logical_sys_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_sys_scan.rs @@ -18,7 +18,6 @@ use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::bail_not_implemented; use risingwave_common::catalog::{ColumnDesc, TableDesc}; -use risingwave_common::error::Result; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::{childless_record, Distill}; @@ -26,6 +25,7 @@ use super::{ generic, BatchFilter, BatchProject, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, }; +use crate::error::Result; use crate::expr::{CorrelatedInputRef, ExprImpl, ExprRewriter, ExprVisitor, InputRef}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; diff --git a/src/frontend/src/optimizer/plan_node/logical_table_function.rs b/src/frontend/src/optimizer/plan_node/logical_table_function.rs index 7a8fabe31e398..c8adf434dab88 100644 --- a/src/frontend/src/optimizer/plan_node/logical_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/logical_table_function.rs @@ -14,7 +14,6 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{Field, Schema}; -use risingwave_common::error::Result; use risingwave_common::types::DataType; use super::utils::{childless_record, Distill}; @@ -22,6 +21,7 @@ use super::{ ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalProject, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, }; +use crate::error::Result; use crate::expr::{Expr, ExprRewriter, ExprVisitor, TableFunction}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; diff --git a/src/frontend/src/optimizer/plan_node/logical_topn.rs b/src/frontend/src/optimizer/plan_node/logical_topn.rs index 3bba28974668c..a454b1d5a89bb 100644 --- a/src/frontend/src/optimizer/plan_node/logical_topn.rs +++ b/src/frontend/src/optimizer/plan_node/logical_topn.rs @@ -15,7 +15,6 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::bail_not_implemented; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::util::sort_util::ColumnOrder; use super::generic::TopNLimit; @@ -25,6 +24,7 @@ use super::{ PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamGroupTopN, StreamProject, ToBatch, ToStream, }; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ diff --git a/src/frontend/src/optimizer/plan_node/logical_union.rs b/src/frontend/src/optimizer/plan_node/logical_union.rs index 15ad5c3b5313d..99de2748dca4e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_union.rs +++ b/src/frontend/src/optimizer/plan_node/logical_union.rs @@ -17,13 +17,13 @@ use std::collections::BTreeMap; use itertools::Itertools; use risingwave_common::catalog::Schema; -use risingwave_common::error::Result; use risingwave_common::types::{DataType, Scalar}; use super::utils::impl_distill_by_unit; use super::{ ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, }; +use crate::error::Result; use crate::expr::{ExprImpl, InputRef, Literal}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::GenericPlanRef; diff --git a/src/frontend/src/optimizer/plan_node/logical_update.rs b/src/frontend/src/optimizer/plan_node/logical_update.rs index 89657fc60d4be..127b6ed8b317b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_update.rs +++ b/src/frontend/src/optimizer/plan_node/logical_update.rs @@ -13,7 +13,6 @@ // limitations under the License. use risingwave_common::catalog::TableVersionId; -use risingwave_common::error::Result; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; @@ -22,6 +21,7 @@ use super::{ LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; +use crate::error::Result; use crate::expr::{ExprImpl, ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index 0f63b06282f05..cdf9af3c93541 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -18,7 +18,6 @@ use std::vec; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{Field, Schema}; -use risingwave_common::error::Result; use risingwave_common::types::{DataType, ScalarImpl}; use super::generic::GenericPlanRef; @@ -27,6 +26,7 @@ use super::{ BatchValues, ColPrunable, ExprRewritable, Logical, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, StreamValues, ToBatch, ToStream, }; +use crate::error::Result; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprVisitor, Literal}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 248bb59f4edde..e4dfb0e8f2fe1 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -39,7 +39,6 @@ use itertools::Itertools; use paste::paste; use pretty_xmlish::{Pretty, PrettyConfig}; use risingwave_common::catalog::Schema; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::batch_plan::PlanNode as BatchPlanPb; use risingwave_pb::stream_plan::StreamNode as StreamPlanPb; use serde::Serialize; @@ -50,6 +49,7 @@ use self::generic::{GenericPlanRef, PhysicalPlanRef}; use self::stream::StreamPlanRef; use self::utils::Distill; use super::property::{Distribution, FunctionalDependencySet, Order}; +use crate::error::{ErrorCode, Result}; /// A marker trait for different conventions, used for enforcing type safety. /// diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index b7f739646a451..d5d7bde249a17 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -15,13 +15,13 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::XmlNode; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{self, GenericPlanRef, PlanAggCall}; use super::stream::prelude::*; use super::utils::{childless_record, plan_node_name, watermark_pretty, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use crate::error::{ErrorCode, Result}; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::stream::StreamPlanRef; diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index abd390dc54602..3abc7ace0e494 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -18,7 +18,6 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{ColumnCatalog, ConflictBehavior, TableId, OBJECT_ID_PLACEHOLDER}; -use risingwave_common::error::Result; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; @@ -29,6 +28,7 @@ use super::stream::StreamPlanRef; use super::utils::{childless_record, Distill}; use super::{reorganize_elements_id, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::catalog::table_catalog::{CreateType, TableCatalog, TableType, TableVersion}; +use crate::error::Result; use crate::optimizer::plan_node::derive::derive_pk; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::GenericPlanRef; diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index e4af8eb1be54e..ffa25e3127a81 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -24,7 +24,6 @@ use risingwave_common::catalog::{ColumnCatalog, Field, TableId}; use risingwave_common::constants::log_store::v2::{ KV_LOG_STORE_PREDEFINED_COLUMNS, PK_ORDERING, VNODE_COLUMN_INDEX, }; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::session_config::sink_decouple::SinkDecouple; use risingwave_common::types::{DataType, StructType}; use risingwave_common::util::iter_util::ZipEqDebug; @@ -46,6 +45,7 @@ use super::generic::{self, GenericPlanRef}; use super::stream::prelude::*; use super::utils::{childless_record, Distill, IndicesDisplay, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, StreamNode, StreamProject}; +use crate::error::{ErrorCode, Result}; use crate::expr::{ExprImpl, FunctionCall, InputRef}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::PlanTreeNodeUnary; diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index 6b5b63a37987a..39d9ff5e7018d 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -279,7 +279,7 @@ impl<'a> IndicesDisplay<'a> { } } -pub(crate) fn sum_affected_row(dml: PlanRef) -> error::Result { +pub(crate) fn sum_affected_row(dml: PlanRef) -> Result { let dml = RequiredDist::single().enforce_if_not_satisfies(dml, &Order::any())?; // Accumulate the affected rows. let sum_agg = PlanAggCall { @@ -316,12 +316,12 @@ macro_rules! plan_node_name { }; } pub(crate) use plan_node_name; -use risingwave_common::error; use risingwave_common::types::DataType; use risingwave_expr::aggregate::AggKind; use super::generic::{self, GenericPlanRef}; use super::pretty_config; +use crate::error::Result; use crate::expr::InputRef; use crate::optimizer::plan_node::generic::Agg; use crate::optimizer::plan_node::{BatchSimpleAgg, PlanAggCall}; diff --git a/src/frontend/src/optimizer/plan_visitor/apply_visitor.rs b/src/frontend/src/optimizer/plan_visitor/apply_visitor.rs index a5d7d7412ab18..ce1cf18a6176f 100644 --- a/src/frontend/src/optimizer/plan_visitor/apply_visitor.rs +++ b/src/frontend/src/optimizer/plan_visitor/apply_visitor.rs @@ -12,9 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::{ErrorCode, RwError}; - use super::{DefaultBehavior, Merge}; +use crate::error::{ErrorCode, RwError}; use crate::optimizer::plan_node::{LogicalApply, PlanTreeNodeBinary}; use crate::optimizer::plan_visitor::PlanVisitor; use crate::PlanRef; diff --git a/src/frontend/src/optimizer/property/distribution.rs b/src/frontend/src/optimizer/property/distribution.rs index 028357cf80fb6..f0667e5eeb8e3 100644 --- a/src/frontend/src/optimizer/property/distribution.rs +++ b/src/frontend/src/optimizer/property/distribution.rs @@ -50,7 +50,6 @@ use fixedbitset::FixedBitSet; use generic::PhysicalPlanRef; use itertools::Itertools; use risingwave_common::catalog::{FieldDisplay, Schema, TableId}; -use risingwave_common::error::Result; use risingwave_common::hash::ParallelUnitId; use risingwave_pb::batch_plan::exchange_info::{ ConsistentHashInfo, Distribution as DistributionPb, DistributionMode, HashInfo, @@ -60,6 +59,7 @@ use risingwave_pb::batch_plan::ExchangeInfo; use super::super::plan_node::*; use crate::catalog::catalog_service::CatalogReader; use crate::catalog::FragmentId; +use crate::error::Result; use crate::optimizer::property::Order; use crate::optimizer::PlanRef; use crate::scheduler::worker_node_manager::WorkerNodeSelector; diff --git a/src/frontend/src/optimizer/property/order.rs b/src/frontend/src/optimizer/property/order.rs index 93e5aa268ac47..1a657e190fac4 100644 --- a/src/frontend/src/optimizer/property/order.rs +++ b/src/frontend/src/optimizer/property/order.rs @@ -17,11 +17,11 @@ use std::fmt; use itertools::Itertools; use pretty_xmlish::Pretty; use risingwave_common::catalog::Schema; -use risingwave_common::error::Result; use risingwave_common::util::sort_util::{ColumnOrder, ColumnOrderDisplay}; use risingwave_pb::common::PbColumnOrder; use super::super::plan_node::*; +use crate::error::Result; use crate::optimizer::PlanRef; // TODO(rc): use this type to replace all `Vec` diff --git a/src/frontend/src/planner/delete.rs b/src/frontend/src/planner/delete.rs index c859ce2cb7cd0..9e777774eee41 100644 --- a/src/frontend/src/planner/delete.rs +++ b/src/frontend/src/planner/delete.rs @@ -13,10 +13,10 @@ // limitations under the License. use fixedbitset::FixedBitSet; -use risingwave_common::error::Result; use super::Planner; use crate::binder::BoundDelete; +use crate::error::Result; use crate::optimizer::plan_node::{generic, LogicalDelete, LogicalProject}; use crate::optimizer::property::{Order, RequiredDist}; use crate::optimizer::{PlanRef, PlanRoot}; diff --git a/src/frontend/src/planner/insert.rs b/src/frontend/src/planner/insert.rs index 9189f17cb4d14..98a89202ff292 100644 --- a/src/frontend/src/planner/insert.rs +++ b/src/frontend/src/planner/insert.rs @@ -13,9 +13,9 @@ // limitations under the License. use fixedbitset::FixedBitSet; -use risingwave_common::error::Result; use crate::binder::BoundInsert; +use crate::error::Result; use crate::optimizer::plan_node::{generic, LogicalInsert, LogicalProject, PlanRef}; use crate::optimizer::property::{Order, RequiredDist}; use crate::optimizer::PlanRoot; diff --git a/src/frontend/src/planner/mod.rs b/src/frontend/src/planner/mod.rs index 21a17b22b59be..d2f695faa5ab2 100644 --- a/src/frontend/src/planner/mod.rs +++ b/src/frontend/src/planner/mod.rs @@ -14,9 +14,8 @@ use std::collections::HashMap; -use risingwave_common::error::Result; - use crate::binder::{BoundStatement, ShareId}; +use crate::error::Result; use crate::optimizer::{OptimizerContextRef, PlanRoot}; mod delete; diff --git a/src/frontend/src/planner/query.rs b/src/frontend/src/planner/query.rs index 0b33b4fb88dce..8be6a41178056 100644 --- a/src/frontend/src/planner/query.rs +++ b/src/frontend/src/planner/query.rs @@ -13,9 +13,9 @@ // limitations under the License. use fixedbitset::FixedBitSet; -use risingwave_common::error::Result; use crate::binder::BoundQuery; +use crate::error::Result; use crate::optimizer::plan_node::{LogicalLimit, LogicalTopN}; use crate::optimizer::property::{Order, RequiredDist}; use crate::optimizer::PlanRoot; diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index b411277761e7f..3f64a8fde4405 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -17,13 +17,13 @@ use std::rc::Rc; use itertools::Itertools; use risingwave_common::bail_not_implemented; use risingwave_common::catalog::{Field, Schema}; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::{DataType, Interval, ScalarImpl}; use crate::binder::{ BoundBaseTable, BoundJoin, BoundShare, BoundSource, BoundSystemTable, BoundWatermark, BoundWindowTableFunction, Relation, WindowTableFunctionKind, }; +use crate::error::{ErrorCode, Result}; use crate::expr::{Expr, ExprImpl, ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::generic::SourceNodeKind; use crate::optimizer::plan_node::{ diff --git a/src/frontend/src/planner/select.rs b/src/frontend/src/planner/select.rs index 086ba88238d6f..a9e7dd3526ed1 100644 --- a/src/frontend/src/planner/select.rs +++ b/src/frontend/src/planner/select.rs @@ -17,7 +17,6 @@ use std::collections::HashMap; use itertools::Itertools; use risingwave_common::bail_not_implemented; use risingwave_common::catalog::Schema; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::ColumnOrder; @@ -25,6 +24,7 @@ use risingwave_expr::ExprError; use risingwave_pb::plan_common::JoinType; use crate::binder::{BoundDistinct, BoundSelect}; +use crate::error::{ErrorCode, Result}; use crate::expr::{ CorrelatedId, Expr, ExprImpl, ExprRewriter, ExprType, FunctionCall, InputRef, Subquery, SubqueryKind, diff --git a/src/frontend/src/planner/set_expr.rs b/src/frontend/src/planner/set_expr.rs index 735712c9b055e..e2ff43a2c211b 100644 --- a/src/frontend/src/planner/set_expr.rs +++ b/src/frontend/src/planner/set_expr.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; use risingwave_common::util::sort_util::ColumnOrder; use crate::binder::BoundSetExpr; +use crate::error::Result; use crate::expr::ExprImpl; use crate::optimizer::plan_node::PlanRef; use crate::planner::Planner; diff --git a/src/frontend/src/planner/set_operation.rs b/src/frontend/src/planner/set_operation.rs index b9ba678755ab1..1050c28bd11fb 100644 --- a/src/frontend/src/planner/set_operation.rs +++ b/src/frontend/src/planner/set_operation.rs @@ -11,9 +11,8 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; - use crate::binder::{BoundSetExpr, BoundSetOperation}; +use crate::error::Result; use crate::optimizer::plan_node::{LogicalExcept, LogicalIntersect, LogicalUnion}; use crate::planner::Planner; use crate::PlanRef; diff --git a/src/frontend/src/planner/statement.rs b/src/frontend/src/planner/statement.rs index 4f6351b0ca6cb..0eed65e2df7e6 100644 --- a/src/frontend/src/planner/statement.rs +++ b/src/frontend/src/planner/statement.rs @@ -12,9 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; - use crate::binder::BoundStatement; +use crate::error::Result; use crate::optimizer::PlanRoot; use crate::planner::Planner; diff --git a/src/frontend/src/planner/update.rs b/src/frontend/src/planner/update.rs index d0fb16a13d1d4..83e36f17084db 100644 --- a/src/frontend/src/planner/update.rs +++ b/src/frontend/src/planner/update.rs @@ -14,10 +14,10 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_common::error::Result; use super::Planner; use crate::binder::BoundUpdate; +use crate::error::Result; use crate::optimizer::plan_node::{generic, LogicalProject, LogicalUpdate}; use crate::optimizer::property::{Order, RequiredDist}; use crate::optimizer::{PlanRef, PlanRoot}; diff --git a/src/frontend/src/planner/values.rs b/src/frontend/src/planner/values.rs index b2dc96a57c12e..ac03498956b86 100644 --- a/src/frontend/src/planner/values.rs +++ b/src/frontend/src/planner/values.rs @@ -12,9 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::Result; - use crate::binder::BoundValues; +use crate::error::Result; use crate::optimizer::plan_node::{LogicalValues, PlanRef}; use crate::planner::Planner; diff --git a/src/frontend/src/scheduler/error.rs b/src/frontend/src/scheduler/error.rs index 25e40fd106f07..f68f72b8727e7 100644 --- a/src/frontend/src/scheduler/error.rs +++ b/src/frontend/src/scheduler/error.rs @@ -13,13 +13,13 @@ // limitations under the License. use risingwave_batch::error::BatchError; -use risingwave_common::error::{ErrorCode, RwError}; use risingwave_common::session_config::QueryMode; use risingwave_rpc_client::error::RpcError; use thiserror::Error; use tonic::{Code, Status}; use crate::catalog::FragmentId; +use crate::error::{ErrorCode, RwError}; use crate::scheduler::plan_fragmenter::QueryId; #[derive(Error, Debug)] diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index aa5f159febe91..2cb642dc9054c 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -28,7 +28,6 @@ use risingwave_batch::executor::ExecutorBuilder; use risingwave_batch::task::{ShutdownToken, TaskId}; use risingwave_common::array::DataChunk; use risingwave_common::bail; -use risingwave_common::error::RwError; use risingwave_common::hash::ParallelUnitMapping; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::tracing::{InstrumentStream, TracingContext}; @@ -47,6 +46,7 @@ use tracing::debug; use super::plan_fragmenter::{PartitionInfo, QueryStage, QueryStageRef}; use crate::catalog::{FragmentId, TableId}; +use crate::error::RwError; use crate::optimizer::plan_node::PlanNodeType; use crate::scheduler::plan_fragmenter::{ExecutionPlanNode, Query, StageId}; use crate::scheduler::task_context::FrontendBatchTaskContext; diff --git a/src/frontend/src/scheduler/mod.rs b/src/frontend/src/scheduler/mod.rs index 2406aa3f7de7e..bb27231c21fdc 100644 --- a/src/frontend/src/scheduler/mod.rs +++ b/src/frontend/src/scheduler/mod.rs @@ -19,8 +19,8 @@ use std::time::Duration; use futures::Stream; use risingwave_common::array::DataChunk; -use risingwave_common::error::Result; +use crate::error::Result; use crate::session::SessionImpl; mod distributed; diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index f422c0b52b49b..7be803c8f7be0 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -25,7 +25,6 @@ use itertools::Itertools; use pgwire::pg_server::SessionId; use risingwave_common::buffer::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::TableDesc; -use risingwave_common::error::RwError; use risingwave_common::hash::{ParallelUnitId, ParallelUnitMapping, VirtualNode}; use risingwave_common::util::scan_range::ScanRange; use risingwave_connector::source::kafka::KafkaSplitEnumerator; @@ -43,6 +42,7 @@ use uuid::Uuid; use super::SchedulerError; use crate::catalog::catalog_service::CatalogReader; use crate::catalog::TableId; +use crate::error::RwError; use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; use crate::optimizer::plan_node::{PlanNodeId, PlanNodeType}; use crate::optimizer::property::Distribution; diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index aad2c79558cba..9419999f8e479 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -42,7 +42,6 @@ use risingwave_common::catalog::{ DEFAULT_DATABASE_NAME, DEFAULT_SUPER_USER, DEFAULT_SUPER_USER_ID, }; use risingwave_common::config::{load_config, BatchConfig, MetaConfig, MetricLevel}; -use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::session_config::{ConfigMap, ConfigReporter, VisibilityMode}; use risingwave_common::system_param::local_manager::LocalSystemParamsManager; use risingwave_common::telemetry::manager::TelemetryManager; @@ -79,6 +78,7 @@ use crate::catalog::root_catalog::Catalog; use crate::catalog::{ check_schema_writable, CatalogError, DatabaseId, OwnedByUserCatalog, SchemaId, }; +use crate::error::{ErrorCode, Result, RwError}; use crate::handler::extended_handle::{ handle_bind, handle_execute, handle_parse, Portal, PrepareStatement, }; diff --git a/src/frontend/src/session/transaction.rs b/src/frontend/src/session/transaction.rs index 3ec6ffa1346b7..682b89a4dcb98 100644 --- a/src/frontend/src/session/transaction.rs +++ b/src/frontend/src/session/transaction.rs @@ -16,11 +16,11 @@ use std::sync::atomic::{AtomicU64, Ordering}; use std::sync::{Arc, Weak}; use parking_lot::{MappedMutexGuard, Mutex, MutexGuard}; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::util::epoch::Epoch; use super::SessionImpl; use crate::catalog::catalog_service::CatalogWriter; +use crate::error::{ErrorCode, Result}; use crate::scheduler::ReadSnapshot; use crate::user::user_service::UserInfoWriter; diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index 344aa103deeb4..a3d18a2c6dc17 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -22,7 +22,6 @@ use std::rc::Rc; use educe::Educe; use risingwave_common::catalog::TableId; -use risingwave_common::error::Result; use risingwave_pb::plan_common::JoinType; use risingwave_pb::stream_plan::{ DispatchStrategy, DispatcherType, ExchangeNode, FragmentTypeFlag, NoOpNode, @@ -30,6 +29,7 @@ use risingwave_pb::stream_plan::{ }; use self::rewrite::build_delta_join_without_arrange; +use crate::error::Result; use crate::optimizer::plan_node::reorganize_elements_id; use crate::optimizer::PlanRef; use crate::scheduler::SchedulerResult; diff --git a/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs b/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs index b556323efb03e..8b6a2cb06c871 100644 --- a/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs +++ b/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs @@ -15,7 +15,6 @@ use std::rc::Rc; use itertools::Itertools; -use risingwave_common::error::Result; use risingwave_pb::plan_common::PbField; use risingwave_pb::stream_plan::lookup_node::ArrangementTableId; use risingwave_pb::stream_plan::stream_node::NodeBody; @@ -24,6 +23,7 @@ use risingwave_pb::stream_plan::{ }; use super::super::{BuildFragmentGraphState, StreamFragment, StreamFragmentEdge}; +use crate::error::Result; use crate::stream_fragmenter::build_and_add_fragment; fn build_no_shuffle_exchange_for_delta_join( diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index c7fbea9d401f7..55772ba9ed068 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -28,7 +28,6 @@ use risingwave_common::catalog::{ FunctionId, IndexId, TableId, DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME, DEFAULT_SUPER_USER, DEFAULT_SUPER_USER_ID, NON_RESERVED_USER_ID, PG_CATALOG_SCHEMA_NAME, RW_CATALOG_SCHEMA_NAME, }; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; @@ -63,6 +62,7 @@ use tempfile::{Builder, NamedTempFile}; use crate::catalog::catalog_service::CatalogWriter; use crate::catalog::root_catalog::Catalog; use crate::catalog::{ConnectionId, DatabaseId, SchemaId}; +use crate::error::{ErrorCode, Result}; use crate::handler::RwPgResponse; use crate::meta_client::FrontendMetaClient; use crate::session::{AuthContext, FrontendEnv, SessionImpl}; diff --git a/src/frontend/src/user/user_privilege.rs b/src/frontend/src/user/user_privilege.rs index 6a33047e2ab9e..37e443cb44f0e 100644 --- a/src/frontend/src/user/user_privilege.rs +++ b/src/frontend/src/user/user_privilege.rs @@ -16,11 +16,12 @@ use itertools::Itertools; use risingwave_common::acl; use risingwave_common::acl::{AclMode, AclModeSet}; use risingwave_common::catalog::DEFAULT_SUPER_USER_ID; -use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::user::grant_privilege::{ActionWithGrantOption, PbAction, PbObject}; use risingwave_pb::user::PbGrantPrivilege; use risingwave_sqlparser::ast::{Action, GrantObjects, Privileges}; +use crate::error::{ErrorCode, Result}; + pub fn check_privilege_type(privilege: &Privileges, objects: &GrantObjects) -> Result<()> { match privilege { Privileges::All { .. } => Ok(()), diff --git a/src/frontend/src/user/user_service.rs b/src/frontend/src/user/user_service.rs index cff0b6f3daa0b..ffc155d982688 100644 --- a/src/frontend/src/user/user_service.rs +++ b/src/frontend/src/user/user_service.rs @@ -17,12 +17,12 @@ use std::sync::Arc; use anyhow::anyhow; use parking_lot::lock_api::ArcRwLockReadGuard; use parking_lot::{RawRwLock, RwLock}; -use risingwave_common::error::Result; use risingwave_pb::user::update_user_request::UpdateField; use risingwave_pb::user::{GrantPrivilege, UserInfo}; use risingwave_rpc_client::MetaClient; use tokio::sync::watch::Receiver; +use crate::error::Result; use crate::user::user_manager::UserInfoManager; use crate::user::{UserId, UserInfoVersion}; diff --git a/src/frontend/src/utils/condition.rs b/src/frontend/src/utils/condition.rs index cf56fc4bdef32..b1db9d7846c49 100644 --- a/src/frontend/src/utils/condition.rs +++ b/src/frontend/src/utils/condition.rs @@ -21,10 +21,10 @@ use std::sync::LazyLock; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{Schema, TableDesc}; -use risingwave_common::error::Result; use risingwave_common::types::{DataType, DefaultOrd, ScalarImpl}; use risingwave_common::util::scan_range::{is_full_range, ScanRange}; +use crate::error::Result; use crate::expr::{ collect_input_refs, factorization_expr, fold_boolean_constant, push_down_not, to_conjunctions, try_get_bool_constant, ExprDisplay, ExprImpl, ExprMutator, ExprRewriter, ExprType, ExprVisitor, diff --git a/src/frontend/src/utils/with_options.rs b/src/frontend/src/utils/with_options.rs index f831c7c9e2e31..184596a37a3f8 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 risingwave_common::error::{ErrorCode, Result as RwResult, RwError}; use risingwave_connector::source::kafka::{ insert_privatelink_broker_rewrite_map, CONNECTION_NAME_KEY, PRIVATELINK_ENDPOINT_KEY, }; @@ -28,6 +27,7 @@ use risingwave_sqlparser::ast::{ use crate::catalog::connection_catalog::resolve_private_link_connection; use crate::catalog::ConnectionId; +use crate::error::{ErrorCode, Result as RwResult, RwError}; use crate::handler::create_source::UPSTREAM_SOURCE_KEY; use crate::session::SessionImpl; diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index 1fe65aaaf1633..2e770fb841ada 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -39,10 +39,10 @@ use crate::manager::MetaOpts; #[command(version, about = "The central metadata management service")] pub struct MetaNodeOpts { #[clap(long, env = "RW_VPC_ID")] - vpc_id: Option, + pub vpc_id: Option, #[clap(long, env = "RW_VPC_SECURITY_GROUP_ID")] - security_group_id: Option, + pub security_group_id: Option, // TODO: use `SocketAddr` #[clap(long, env = "RW_LISTEN_ADDR", default_value = "127.0.0.1:5690")] @@ -54,10 +54,10 @@ pub struct MetaNodeOpts { /// It will serve as a unique identifier in cluster /// membership and leader election. Must be specified for etcd backend. #[clap(long, env = "RW_ADVERTISE_ADDR")] - advertise_addr: String, + pub advertise_addr: String, #[clap(long, env = "RW_DASHBOARD_HOST")] - dashboard_host: Option, + pub dashboard_host: Option, /// We will start a http server at this address via `MetricsManager`. /// Then the prometheus instance will poll the metrics from this address. @@ -65,38 +65,38 @@ pub struct MetaNodeOpts { pub prometheus_listener_addr: Option, #[clap(long, env = "RW_ETCD_ENDPOINTS", default_value_t = String::from(""))] - etcd_endpoints: String, + pub etcd_endpoints: String, /// Enable authentication with etcd. By default disabled. #[clap(long, env = "RW_ETCD_AUTH")] - etcd_auth: bool, + pub etcd_auth: bool, /// Username of etcd, required when --etcd-auth is enabled. #[clap(long, env = "RW_ETCD_USERNAME", default_value = "")] - etcd_username: String, + pub etcd_username: String, /// Password of etcd, required when --etcd-auth is enabled. #[clap(long, env = "RW_ETCD_PASSWORD", default_value = "")] - etcd_password: Secret, + pub etcd_password: Secret, /// Endpoint of the SQL service, make it non-option when SQL service is required. #[clap(long, env = "RW_SQL_ENDPOINT")] - sql_endpoint: Option, + pub sql_endpoint: Option, #[clap(long, env = "RW_DASHBOARD_UI_PATH")] - dashboard_ui_path: Option, + pub dashboard_ui_path: Option, /// The HTTP REST-API address of the Prometheus instance associated to this cluster. /// This address is used to serve PromQL queries to Prometheus. /// It is also used by Grafana Dashboard Service to fetch metrics and visualize them. #[clap(long, env = "RW_PROMETHEUS_ENDPOINT")] - prometheus_endpoint: Option, + pub prometheus_endpoint: Option, /// The additional selector used when querying Prometheus. /// /// The format is same as PromQL. Example: `instance="foo",namespace="bar"` #[clap(long, env = "RW_PROMETHEUS_SELECTOR")] - prometheus_selector: Option, + pub prometheus_selector: Option, /// Endpoint of the connector node, there will be a sidecar connector node /// colocated with Meta node in the cloud environment @@ -117,52 +117,52 @@ pub struct MetaNodeOpts { #[clap(long, env = "RW_BACKEND", value_enum)] #[override_opts(path = meta.backend)] - backend: Option, + pub backend: Option, /// The interval of periodic barrier. #[clap(long, env = "RW_BARRIER_INTERVAL_MS")] #[override_opts(path = system.barrier_interval_ms)] - barrier_interval_ms: Option, + pub barrier_interval_ms: Option, /// Target size of the Sstable. #[clap(long, env = "RW_SSTABLE_SIZE_MB")] #[override_opts(path = system.sstable_size_mb)] - sstable_size_mb: Option, + pub sstable_size_mb: Option, /// Size of each block in bytes in SST. #[clap(long, env = "RW_BLOCK_SIZE_KB")] #[override_opts(path = system.block_size_kb)] - block_size_kb: Option, + pub block_size_kb: Option, /// False positive probability of bloom filter. #[clap(long, env = "RW_BLOOM_FALSE_POSITIVE")] #[override_opts(path = system.bloom_false_positive)] - bloom_false_positive: Option, + pub bloom_false_positive: Option, /// State store url #[clap(long, env = "RW_STATE_STORE")] #[override_opts(path = system.state_store)] - state_store: Option, + pub state_store: Option, /// Remote directory for storing data and metadata objects. #[clap(long, env = "RW_DATA_DIRECTORY")] #[override_opts(path = system.data_directory)] - data_directory: Option, + pub data_directory: Option, /// Whether config object storage bucket lifecycle to purge stale data. #[clap(long, env = "RW_DO_NOT_CONFIG_BUCKET_LIFECYCLE")] #[override_opts(path = meta.do_not_config_object_storage_lifecycle)] - do_not_config_object_storage_lifecycle: Option, + pub do_not_config_object_storage_lifecycle: Option, /// Remote storage url for storing snapshots. #[clap(long, env = "RW_BACKUP_STORAGE_URL")] #[override_opts(path = system.backup_storage_url)] - backup_storage_url: Option, + pub backup_storage_url: Option, /// Remote directory for storing snapshots. #[clap(long, env = "RW_BACKUP_STORAGE_DIRECTORY")] #[override_opts(path = system.backup_storage_directory)] - backup_storage_directory: Option, + pub backup_storage_directory: Option, /// Enable heap profile dump when memory usage is high. #[clap(long, env = "RW_HEAP_PROFILING_DIR")] diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index 7b9426ec78559..8afc5ab6c836c 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -298,21 +298,15 @@ impl CatalogController { let inner = self.inner.write().await; let txn = inner.db.begin().await?; - // Add fragments, actors and actor dispatchers. - for (fragment, actors, actor_dispatchers) in fragment_actors { + // Add fragments. + let (fragments, actor_with_dispatchers): (Vec<_>, Vec<_>) = fragment_actors + .into_iter() + .map(|(fragment, actors, actor_dispatchers)| (fragment, (actors, actor_dispatchers))) + .unzip(); + for fragment in fragments { let fragment = fragment.into_active_model(); let fragment = fragment.insert(&txn).await?; - for actor in actors { - let actor = actor.into_active_model(); - actor.insert(&txn).await?; - } - for (_, actor_dispatchers) in actor_dispatchers { - for actor_dispatcher in actor_dispatchers { - let mut actor_dispatcher = actor_dispatcher.into_active_model(); - actor_dispatcher.id = NotSet; - actor_dispatcher.insert(&txn).await?; - } - } + // Update fragment id for all state tables. if !for_replace { for state_table_id in fragment.state_table_ids.into_inner() { @@ -327,6 +321,21 @@ impl CatalogController { } } + // Add actors and actor dispatchers. + for (actors, actor_dispatchers) in actor_with_dispatchers { + for actor in actors { + let actor = actor.into_active_model(); + actor.insert(&txn).await?; + } + for (_, actor_dispatchers) in actor_dispatchers { + for actor_dispatcher in actor_dispatchers { + let mut actor_dispatcher = actor_dispatcher.into_active_model(); + actor_dispatcher.id = NotSet; + actor_dispatcher.insert(&txn).await?; + } + } + } + if !for_replace { // // Update dml fragment id. if let StreamingJob::Table(_, table, ..) = streaming_job { diff --git a/src/meta/src/controller/user.rs b/src/meta/src/controller/user.rs index 1444fa2b0e41a..19afd724a82b0 100644 --- a/src/meta/src/controller/user.rs +++ b/src/meta/src/controller/user.rs @@ -37,7 +37,7 @@ use crate::controller::utils::{ extract_grant_obj_id, get_referring_privileges_cascade, get_user_privilege, list_user_info_by_ids, PartialUserPrivilege, }; -use crate::manager::NotificationVersion; +use crate::manager::{NotificationVersion, IGNORED_NOTIFICATION_VERSION}; use crate::{MetaError, MetaResult}; impl CatalogController { @@ -402,9 +402,8 @@ impl CatalogController { ); } if root_user_privileges.is_empty() { - return Err(MetaError::invalid_parameter( - "no privilege to revoke".to_string(), - )); + tracing::warn!("no privilege to revoke, ignore it"); + return Ok(IGNORED_NOTIFICATION_VERSION); } // check if the user granted any privileges to other users. diff --git a/src/rpc_client/src/error.rs b/src/rpc_client/src/error.rs index 92c9ebe5bf338..e44d3ac8c0b9f 100644 --- a/src/rpc_client/src/error.rs +++ b/src/rpc_client/src/error.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::{ErrorCode, RwError}; use risingwave_common::util::meta_addr::MetaAddressStrategyParseError; use thiserror::Error; @@ -54,12 +53,3 @@ impl From for RpcError { RpcError::GrpcStatus(Box::new(TonicStatusWrapper::new(s))) } } - -impl From for RwError { - fn from(r: RpcError) -> Self { - match r { - RpcError::GrpcStatus(status) => TonicStatusWrapper::into(*status), - _ => ErrorCode::RpcError(r.into()).into(), - } - } -} diff --git a/src/storage/benches/bench_row.rs b/src/storage/benches/bench_row.rs index 6a612e87241cd..49fa52ad5adde 100644 --- a/src/storage/benches/bench_row.rs +++ b/src/storage/benches/bench_row.rs @@ -18,7 +18,6 @@ use std::sync::Arc; use criterion::{criterion_group, criterion_main, Criterion}; use itertools::Itertools; use risingwave_common::catalog::{ColumnDesc, ColumnId}; -use risingwave_common::error::Result; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, Datum, ScalarImpl}; use risingwave_common::util::row_serde::OrderedRowSerde; @@ -104,7 +103,7 @@ fn column_aware_encode(c: &Case) -> Vec> { array } -fn memcmp_decode(c: &Case, bytes: &Vec>) -> Result>> { +fn memcmp_decode(c: &Case, bytes: &Vec>) -> anyhow::Result>> { let serde = OrderedRowSerde::new( c.schema.to_vec(), vec![OrderType::descending(); c.schema.len()], @@ -146,7 +145,7 @@ fn memcmp_decode(c: &Case, bytes: &Vec>) -> Result>> { Ok(res) } -fn basic_decode(c: &Case, bytes: &Vec>) -> Result>> { +fn basic_decode(c: &Case, bytes: &Vec>) -> anyhow::Result>> { let table_columns = c .column_ids .iter() @@ -195,7 +194,7 @@ fn basic_decode(c: &Case, bytes: &Vec>) -> Result>> { Ok(res) } -fn column_aware_decode(c: &Case, bytes: &Vec>) -> Result>> { +fn column_aware_decode(c: &Case, bytes: &Vec>) -> anyhow::Result>> { let table_columns = c .column_ids .iter() diff --git a/src/storage/src/error.rs b/src/storage/src/error.rs index dbdbf50e330fa..c76913ebe6eb9 100644 --- a/src/storage/src/error.rs +++ b/src/storage/src/error.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::{ErrorCode, RwError}; use risingwave_common::util::value_encoding::error::ValueEncodingError; use thiserror::Error; @@ -59,9 +58,3 @@ pub enum ErrorKind { } pub type StorageResult = std::result::Result; - -impl From for RwError { - fn from(s: StorageError) -> Self { - ErrorCode::StorageError(Box::new(s)).into() - } -} diff --git a/src/storage/src/row_serde/mod.rs b/src/storage/src/row_serde/mod.rs index 2668d75bc1b93..4cb6696cce931 100644 --- a/src/storage/src/row_serde/mod.rs +++ b/src/storage/src/row_serde/mod.rs @@ -95,7 +95,7 @@ mod test { type_name: "", generated_or_default_column: None, description: None, - additional_columns: AdditionalColumn { + additional_column: AdditionalColumn { column_type: None, }, version: Pr13707, @@ -108,7 +108,7 @@ mod test { type_name: "", generated_or_default_column: None, description: None, - additional_columns: AdditionalColumn { + additional_column: AdditionalColumn { column_type: None, }, version: Pr13707, @@ -141,7 +141,7 @@ mod test { type_name: "", generated_or_default_column: None, description: None, - additional_columns: AdditionalColumn { + additional_column: AdditionalColumn { column_type: None, }, version: Pr13707, @@ -154,7 +154,7 @@ mod test { type_name: "", generated_or_default_column: None, description: None, - additional_columns: AdditionalColumn { + additional_column: AdditionalColumn { column_type: None, }, version: Pr13707, diff --git a/src/stream/clippy.toml b/src/stream/clippy.toml index dd26a976e1ab1..9d73b2ba754ed 100644 --- a/src/stream/clippy.toml +++ b/src/stream/clippy.toml @@ -7,11 +7,7 @@ disallowed-methods = [ { path = "risingwave_expr::expr::Expression::eval_row", reason = "Please use `NonStrictExpression::eval_row_infallible` instead." }, ] -disallowed-types = [ - { path = "risingwave_common::error::ErrorCode", reason = "Please use per-crate error type instead." }, - { path = "risingwave_common::error::RwError", reason = "Please use per-crate error type instead." }, - { path = "risingwave_common::error::Result", reason = "Please use per-crate error type instead." }, -] +disallowed-types = [] doc-valid-idents = [ "RisingWave", diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index 7df7cc2ea8373..8cdba698bec12 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -71,7 +71,7 @@ pub fn get_split_offset_col_idx( let mut split_idx = None; let mut offset_idx = None; for (idx, column) in column_descs.iter().enumerate() { - match column.additional_column_type { + match column.additional_column { AdditionalColumn { column_type: Some(ColumnType::Partition(_) | ColumnType::Filename(_)), } => { diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index a99be097e881d..28d923ffb69cc 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -96,7 +96,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { // the column is from a legacy version && desc.version == ColumnDescVersion::Unspecified as i32 { - desc.additional_columns = Some(AdditionalColumn { + desc.additional_column = Some(AdditionalColumn { column_type: Some(AdditionalColumnType::Key( AdditionalColumnKey {}, )), @@ -110,7 +110,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { { // compatible code: handle legacy column `_rw_kafka_timestamp` // the column is auto added for all kafka source to empower batch query on source - // solution: rewrite the column `additional_columns` to Timestamp + // solution: rewrite the column `additional_column` to Timestamp let _ = source_columns.iter_mut().map(|c| { let _ = c.column_desc.as_mut().map(|desc| { @@ -125,7 +125,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { // the column is from a legacy version && desc.version == ColumnDescVersion::Unspecified as i32 { - desc.additional_columns = Some(AdditionalColumn { + desc.additional_column = Some(AdditionalColumn { column_type: Some(AdditionalColumnType::Timestamp( AdditionalColumnTimestamp {}, )), 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 e7792d5930e03..91aea55a2dc31 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs @@ -14,8 +14,7 @@ use std::time::Duration; -use anyhow::Result; -use risingwave_common::error::anyhow_error; +use anyhow::{anyhow, Result}; use risingwave_simulation::cluster::{Cluster, Configuration, Session}; use tokio::time::sleep; @@ -49,7 +48,7 @@ async fn cancel_stream_jobs(session: &mut Session) -> Result> { .split('\n') .map(|s| { s.parse::() - .map_err(|_e| anyhow_error!("failed to parse {}", s)) + .map_err(|_e| anyhow!("failed to parse {}", s)) }) .collect::>>()?; Ok(ids)