risingwave_frontend/
error.rs1use risingwave_batch::error::BatchError;
16use risingwave_common::array::ArrayError;
17use risingwave_common::error::{BoxedError, NoFunction, NotImplemented};
18use risingwave_common::license::FeatureNotAvailable;
19use risingwave_common::secret::SecretError;
20use risingwave_common::session_config::SessionConfigError;
21use risingwave_common::util::value_encoding::error::ValueEncodingError;
22use risingwave_connector::error::ConnectorError;
23use risingwave_connector::sink::SinkError;
24use risingwave_expr::ExprError;
25use risingwave_pb::PbFieldNotFound;
26use risingwave_rpc_client::error::{RpcError, ToTonicStatus, TonicStatusWrapper};
27use thiserror::Error;
28use thiserror_ext::AsReport;
29use tokio::task::JoinError;
30
31use crate::expr::CastError;
32
33#[derive(Error, thiserror_ext::ReportDebug, thiserror_ext::Box, thiserror_ext::Macro)]
41#[thiserror_ext(newtype(name = RwError, backtrace), macro(path = "crate::error"))]
42pub enum ErrorCode {
43 #[error("internal error: {0}")]
44 InternalError(String),
45 #[error(transparent)]
47 Uncategorized(
48 #[from]
49 #[backtrace]
50 anyhow::Error,
51 ),
52 #[error("connector error: {0}")]
53 ConnectorError(
54 #[source]
55 #[backtrace]
56 BoxedError,
57 ),
58 #[error(transparent)]
59 NotImplemented(#[from] NotImplemented),
60 #[error("Not supported: {0}\nHINT: {1}")]
62 NotSupported(String, String),
63 #[error(transparent)]
64 NoFunction(#[from] NoFunction),
65 #[error(transparent)]
66 IoError(#[from] std::io::Error),
67 #[error("Storage error: {0}")]
68 StorageError(
69 #[backtrace]
70 #[source]
71 BoxedError,
72 ),
73 #[error("Expr error: {0}")]
74 ExprError(
75 #[source]
76 #[backtrace]
77 BoxedError,
78 ),
79 #[error("{0}")]
82 BatchError(
83 #[source]
84 #[backtrace]
85 BoxedError,
87 ),
88 #[error("Array error: {0}")]
89 ArrayError(
90 #[from]
91 #[backtrace]
92 ArrayError,
93 ),
94 #[cfg(feature = "datafusion")]
95 #[error("DataFusion error: {0}")]
96 DataFusionError(
97 #[from]
98 #[backtrace]
99 datafusion_common::DataFusionError,
100 ),
101 #[error("Stream error: {0}")]
102 StreamError(
103 #[backtrace]
104 #[source]
105 BoxedError,
106 ),
107 #[error("{0}")]
110 RpcError(
111 #[source]
112 #[backtrace]
113 BoxedError,
115 ),
116 #[error("Bind error: {0}")]
119 BindError(#[message] String),
120 #[error("Failed to bind expression: {expr}: {error}")]
122 BindErrorRoot {
123 expr: String,
124 #[source]
125 #[backtrace]
126 error: BoxedError,
127 },
128 #[error(transparent)]
129 CastError(
130 #[from]
131 #[backtrace]
132 CastError,
133 ),
134 #[error("Catalog error: {0}")]
135 CatalogError(
136 #[source]
137 #[backtrace]
138 #[message]
139 BoxedError,
140 ),
141 #[error("Protocol error: {0}")]
142 ProtocolError(#[message] String),
143 #[error("Scheduler error: {0}")]
144 SchedulerError(
145 #[source]
146 #[backtrace]
147 BoxedError,
148 ),
149 #[error("Task not found")]
150 TaskNotFound,
151 #[error("Session not found")]
152 SessionNotFound,
153 #[error("Item not found: {0}")]
154 ItemNotFound(String),
155 #[error("Duplicate Relation Name: {0}")]
156 DuplicateRelationName(String),
157 #[error("Invalid input syntax: {0}")]
158 InvalidInputSyntax(#[message] String),
159 #[error("Can not compare in memory: {0}")]
160 MemComparableError(#[from] memcomparable::Error),
161 #[error("Error while de/se values: {0}")]
162 ValueEncodingError(
163 #[from]
164 #[backtrace]
165 ValueEncodingError,
166 ),
167 #[error("Invalid value `{config_value}` for `{config_entry}`")]
168 InvalidConfigValue {
169 config_entry: String,
170 config_value: String,
171 },
172 #[error("Invalid Parameter Value: {0}")]
173 InvalidParameterValue(String),
174 #[error("Sink error: {0}")]
175 SinkError(
176 #[source]
177 #[backtrace]
178 BoxedError,
179 ),
180 #[error("Permission denied: {0}")]
181 PermissionDenied(#[message] String),
182 #[error("Failed to get/set session config: {0}")]
183 SessionConfig(
184 #[from]
185 #[backtrace]
186 SessionConfigError,
187 ),
188 #[error("Secret error: {0}")]
189 SecretError(
190 #[from]
191 #[backtrace]
192 SecretError,
193 ),
194 #[error("{0} has been deprecated, please use {1} instead.")]
195 Deprecated(String, String),
196 #[error(transparent)]
197 FeatureNotAvailable(
198 #[from]
199 #[backtrace]
200 FeatureNotAvailable,
201 ),
202}
203
204pub type Result<T> = std::result::Result<T, RwError>;
206
207impl From<TonicStatusWrapper> for RwError {
208 fn from(status: TonicStatusWrapper) -> Self {
209 use tonic::Code;
210
211 let message = status.inner().message();
212
213 match status.inner().code() {
215 Code::InvalidArgument => ErrorCode::InvalidParameterValue(message.to_owned()),
216 Code::NotFound | Code::AlreadyExists => ErrorCode::CatalogError(status.into()),
217 Code::PermissionDenied => ErrorCode::PermissionDenied(message.to_owned()),
218 Code::Cancelled => ErrorCode::SchedulerError(status.into()),
219 _ => ErrorCode::RpcError(status.into()),
220 }
221 .into()
222 }
223}
224
225impl From<RpcError> for RwError {
226 fn from(r: RpcError) -> Self {
227 match r {
228 RpcError::GrpcStatus(status) => TonicStatusWrapper::into(*status),
229 _ => ErrorCode::RpcError(r.into()).into(),
230 }
231 }
232}
233
234impl From<ExprError> for RwError {
235 fn from(s: ExprError) -> Self {
236 ErrorCode::ExprError(Box::new(s)).into()
237 }
238}
239
240impl From<SinkError> for RwError {
241 fn from(e: SinkError) -> Self {
242 ErrorCode::SinkError(Box::new(e)).into()
243 }
244}
245
246impl From<ConnectorError> for RwError {
247 fn from(e: ConnectorError) -> Self {
248 ErrorCode::ConnectorError(e.into()).into()
249 }
250}
251
252impl From<PbFieldNotFound> for RwError {
253 fn from(err: PbFieldNotFound) -> Self {
254 ErrorCode::InternalError(format!(
255 "Failed to decode prost: field not found `{}`",
256 err.0
257 ))
258 .into()
259 }
260}
261
262impl From<BatchError> for RwError {
263 fn from(s: BatchError) -> Self {
264 ErrorCode::BatchError(Box::new(s)).into()
265 }
266}
267
268impl From<JoinError> for RwError {
269 fn from(join_error: JoinError) -> Self {
270 ErrorCode::Uncategorized(join_error.into()).into()
271 }
272}
273
274impl From<BoxedError> for RwError {
276 fn from(e: BoxedError) -> Self {
277 let e = anyhow::__private::kind::BoxedKind::anyhow_kind(&e).new(e);
280 ErrorCode::Uncategorized(e).into()
281 }
282}
283
284impl From<risingwave_sqlparser::parser::ParserError> for ErrorCode {
285 fn from(e: risingwave_sqlparser::parser::ParserError) -> Self {
286 ErrorCode::InvalidInputSyntax(e.to_report_string())
287 }
288}
289
290impl From<RwError> for tonic::Status {
291 fn from(err: RwError) -> Self {
292 err.to_status(tonic::Code::Internal, "RwError")
293 }
294}