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("Invalid reference: {0}")]
154 InvalidReference(String),
155 #[error("Item not found: {0}")]
156 ItemNotFound(String),
157 #[error("Duplicate Relation Name: {0}")]
158 DuplicateRelationName(String),
159 #[error("Invalid insert operation: {0}")]
160 InsertViolation(String),
161 #[error("Invalid input syntax: {0}")]
162 InvalidInputSyntax(#[message] String),
163 #[error("Can not compare in memory: {0}")]
164 MemComparableError(#[from] memcomparable::Error),
165 #[error("Error while de/se values: {0}")]
166 ValueEncodingError(
167 #[from]
168 #[backtrace]
169 ValueEncodingError,
170 ),
171 #[error("Invalid value `{config_value}` for `{config_entry}`")]
172 InvalidConfigValue {
173 config_entry: String,
174 config_value: String,
175 },
176 #[error("Invalid Parameter Value: {0}")]
177 InvalidParameterValue(String),
178 #[error("Sink error: {0}")]
179 SinkError(
180 #[source]
181 #[backtrace]
182 BoxedError,
183 ),
184 #[error("Permission denied: {0}")]
185 PermissionDenied(#[message] String),
186 #[error("Failed to get/set session config: {0}")]
187 SessionConfig(
188 #[from]
189 #[backtrace]
190 SessionConfigError,
191 ),
192 #[error("Secret error: {0}")]
193 SecretError(
194 #[from]
195 #[backtrace]
196 SecretError,
197 ),
198 #[error("{0} has been deprecated, please use {1} instead.")]
199 Deprecated(String, String),
200 #[error(transparent)]
201 FeatureNotAvailable(
202 #[from]
203 #[backtrace]
204 FeatureNotAvailable,
205 ),
206}
207
208pub type Result<T> = std::result::Result<T, RwError>;
210
211impl From<TonicStatusWrapper> for RwError {
212 fn from(status: TonicStatusWrapper) -> Self {
213 use tonic::Code;
214
215 let message = status.inner().message();
216
217 match status.inner().code() {
219 Code::InvalidArgument => ErrorCode::InvalidParameterValue(message.to_owned()),
220 Code::NotFound | Code::AlreadyExists => ErrorCode::CatalogError(status.into()),
221 Code::PermissionDenied => ErrorCode::PermissionDenied(message.to_owned()),
222 Code::Cancelled => ErrorCode::SchedulerError(status.into()),
223 _ => ErrorCode::RpcError(status.into()),
224 }
225 .into()
226 }
227}
228
229impl From<RpcError> for RwError {
230 fn from(r: RpcError) -> Self {
231 match r {
232 RpcError::GrpcStatus(status) => TonicStatusWrapper::into(*status),
233 _ => ErrorCode::RpcError(r.into()).into(),
234 }
235 }
236}
237
238impl From<ExprError> for RwError {
239 fn from(s: ExprError) -> Self {
240 ErrorCode::ExprError(Box::new(s)).into()
241 }
242}
243
244impl From<SinkError> for RwError {
245 fn from(e: SinkError) -> Self {
246 ErrorCode::SinkError(Box::new(e)).into()
247 }
248}
249
250impl From<ConnectorError> for RwError {
251 fn from(e: ConnectorError) -> Self {
252 ErrorCode::ConnectorError(e.into()).into()
253 }
254}
255
256impl From<PbFieldNotFound> for RwError {
257 fn from(err: PbFieldNotFound) -> Self {
258 ErrorCode::InternalError(format!(
259 "Failed to decode prost: field not found `{}`",
260 err.0
261 ))
262 .into()
263 }
264}
265
266impl From<BatchError> for RwError {
267 fn from(s: BatchError) -> Self {
268 ErrorCode::BatchError(Box::new(s)).into()
269 }
270}
271
272impl From<JoinError> for RwError {
273 fn from(join_error: JoinError) -> Self {
274 ErrorCode::Uncategorized(join_error.into()).into()
275 }
276}
277
278impl From<BoxedError> for RwError {
280 fn from(e: BoxedError) -> Self {
281 let e = anyhow::__private::kind::BoxedKind::anyhow_kind(&e).new(e);
284 ErrorCode::Uncategorized(e).into()
285 }
286}
287
288impl From<risingwave_sqlparser::parser::ParserError> for ErrorCode {
289 fn from(e: risingwave_sqlparser::parser::ParserError) -> Self {
290 ErrorCode::InvalidInputSyntax(e.to_report_string())
291 }
292}
293
294impl From<RwError> for tonic::Status {
295 fn from(err: RwError) -> Self {
296 err.to_status(tonic::Code::Internal, "RwError")
297 }
298}