risingwave_stream/executor/exchange/error.rs
1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7// http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use risingwave_rpc_client::error::TonicStatusWrapper;
16
17use crate::task::ActorId;
18
19/// The error type for the exchange channel closed unexpectedly.
20///
21/// In most cases, this error happens when the upstream or downstream actor
22/// exits or panics on other errors, or the network connection is broken.
23/// Therefore, this error is usually not the root case of the failure in the
24/// streaming graph.
25#[derive(Debug)]
26pub struct ExchangeChannelClosed {
27 message: String,
28
29 /// `Some` if there is a gRPC error from the remote actor.
30 source: Option<TonicStatusWrapper>,
31}
32
33impl std::fmt::Display for ExchangeChannelClosed {
34 fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
35 write!(f, "{}", self.message)
36 }
37}
38
39impl std::error::Error for ExchangeChannelClosed {
40 fn source(&self) -> Option<&(dyn std::error::Error + 'static)> {
41 self.source.as_ref().map(|s| s as _)
42 }
43
44 fn provide<'a>(&'a self, request: &mut std::error::Request<'a>) {
45 use std::backtrace::Backtrace;
46
47 // Always provide a fake disabled backtrace, so that the upper layer will
48 // not capture any other backtraces or include the backtrace in the error
49 // log.
50 //
51 // Otherwise, when an actor exits on a significant error, all connected
52 // actor will then exit with the `ExchangeChannelClosed` error, resulting
53 // in a very noisy log with flood of useless backtraces.
54 static DISABLED_BACKTRACE: Backtrace = Backtrace::disabled();
55 request.provide_ref::<Backtrace>(&DISABLED_BACKTRACE);
56
57 if let Some(source) = &self.source {
58 source.provide(request);
59 }
60 }
61}
62
63impl ExchangeChannelClosed {
64 /// Creates a new error indicating that the exchange channel from the local
65 /// upstream actor is closed unexpectedly.
66 pub fn local_input(upstream: ActorId) -> Self {
67 Self {
68 message: format!(
69 "exchange channel from local upstream actor {upstream} closed unexpectedly"
70 ),
71 source: None,
72 }
73 }
74
75 /// Creates a new error indicating that the exchange channel from the remote
76 /// upstream actor is closed unexpectedly, with an optional gRPC error as the cause.
77 pub fn remote_input(upstream: ActorId, source: Option<tonic::Status>) -> Self {
78 Self {
79 message: format!(
80 "exchange channel from remote upstream actor {upstream} closed unexpectedly"
81 ),
82 source: source.map(Into::into),
83 }
84 }
85
86 /// Creates a new error indicating that the exchange channel to the downstream
87 /// actor is closed unexpectedly.
88 pub fn output(downstream: ActorId) -> Self {
89 Self {
90 message: format!(
91 "exchange channel to downstream actor {downstream} closed unexpectedly"
92 ),
93 source: None,
94 }
95 }
96}