diff --git a/Cargo.lock b/Cargo.lock index b1ac42becd271..0e398b72d6896 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", @@ -9288,6 +9289,7 @@ dependencies = [ "madsim-tonic", "maplit", "md5", + "memcomparable", "num-integer", "parking_lot 0.12.1", "parse-display", 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/common/src/error.rs b/src/common/src/error.rs index f8265c68a81a4..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 { @@ -103,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. @@ -463,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() { @@ -479,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() ); } @@ -517,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); @@ -526,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/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/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 4eb01e33370fd..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::{ErrorCode, RwError}; use risingwave_common::metrics::{ LabelGuardedHistogram, LabelGuardedIntCounter, LabelGuardedIntGauge, }; @@ -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/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 7cd9eab4c1bec..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; 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 965105e1e710e..387c78f300282 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -20,7 +20,6 @@ use risingwave_common::catalog::{ ColumnCatalog, ConflictBehavior, TableDesc, TableId, TableVersionId, }; use risingwave_common::constants::hummock::TABLE_OPTION_DUMMY_RETENTION_SECOND; -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}; @@ -29,6 +28,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; 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 6a072ee624cb9..df7b49f556f35 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 c09c36b9b9802..98362b00f8a7f 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 abe046f08fce1..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, diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index b69d2e5cebbd3..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, 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 c0b58701f4140..80c675009d134 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 27c2e34cd60d3..64154c23ea206 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 77dfa8d5f758d..4de12b9a82ce6 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -288,7 +288,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 { @@ -325,12 +325,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 cb76304afe6f2..2ed0876a2dc9f 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/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/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/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)