Skip to content

Commit 3f47a2a

Browse files
authored
refactor: add peer info to network error context (#15139)
1 parent 7d3b367 commit 3f47a2a

File tree

1 file changed

+36
-15
lines changed

1 file changed

+36
-15
lines changed

src/meta/service/src/network.rs

Lines changed: 36 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@ use std::future::Future;
1818
use std::sync::Arc;
1919
use std::time::Duration;
2020

21+
use anyerror::func_name;
2122
use anyerror::AnyError;
2223
use async_trait::async_trait;
2324
use backon::BackoffBuilder;
@@ -36,6 +37,7 @@ use databend_common_meta_types::protobuf::RaftRequest;
3637
use databend_common_meta_types::protobuf::SnapshotChunkRequest;
3738
use databend_common_meta_types::AppendEntriesRequest;
3839
use databend_common_meta_types::AppendEntriesResponse;
40+
use databend_common_meta_types::Endpoint;
3941
use databend_common_meta_types::Fatal;
4042
use databend_common_meta_types::GrpcConfig;
4143
use databend_common_meta_types::GrpcHelper;
@@ -191,29 +193,37 @@ pub struct NetworkConnection {
191193
impl NetworkConnection {
192194
#[logcall::logcall(err = "debug")]
193195
#[minitrace::trace]
194-
pub async fn make_client(&self) -> Result<RaftClient, Unreachable> {
196+
pub async fn make_client(&self) -> Result<(RaftClient, Endpoint), Unreachable> {
195197
let target = self.target;
196198

197199
let endpoint = self
198200
.sto
199201
.get_node_raft_endpoint(&target)
200202
.await
201-
.map_err(|e| Unreachable::new(&e))?;
203+
.map_err(|e| {
204+
let any_err = AnyError::new(&e)
205+
.add_context(|| format!("{} target: {}", func_name!(), self.target));
206+
Unreachable::new(&any_err)
207+
})?;
202208

203209
let addr = format!("http://{}", endpoint);
204210

205211
debug!(id = self.id; "connect: target={}: {}", target, addr);
206212

207213
match self.conn_pool.get(&addr).await {
208214
Ok(channel) => {
209-
let client = RaftClientApi::new(target, endpoint, channel);
215+
let client = RaftClientApi::new(target, endpoint.clone(), channel);
210216
debug!("connected: target={}: {}", target, addr);
211217

212-
Ok(client)
218+
Ok((client, endpoint))
213219
}
214220
Err(err) => {
215221
raft_metrics::network::incr_connect_failure(&target, &endpoint.to_string());
216-
Err(Unreachable::new(&err))
222+
let any_err = AnyError::new(&err).add_context(|| {
223+
format!("{} target: {}, addr: {}", func_name!(), self.target, addr)
224+
});
225+
226+
Err(Unreachable::new(&any_err))
217227
}
218228
}
219229
}
@@ -287,11 +297,22 @@ impl NetworkConnection {
287297
}
288298

289299
/// Convert gRPC status to `RPCError`
290-
fn status_to_unreachable<E>(&self, status: tonic::Status) -> RPCError<RaftError<E>>
291-
where E: std::error::Error {
292-
warn!("target={}, gRPC error: {:?}", self.target, status);
300+
fn status_to_unreachable<E>(
301+
&self,
302+
status: tonic::Status,
303+
endpoint: Endpoint,
304+
) -> RPCError<RaftError<E>>
305+
where
306+
E: std::error::Error,
307+
{
308+
warn!(
309+
"target={}, endpoint={} gRPC error: {:?}",
310+
self.target, endpoint, status
311+
);
293312

294-
RPCError::Unreachable(Unreachable::new(&status))
313+
let any_err = AnyError::new(&status)
314+
.add_context(|| format!("gRPC target={}, endpoint={}", self.target, endpoint));
315+
RPCError::Unreachable(Unreachable::new(&any_err))
295316
}
296317
}
297318

@@ -310,7 +331,7 @@ impl RaftNetwork<TypeConfig> for NetworkConnection {
310331
"send_append_entries",
311332
);
312333

313-
let mut client = self.make_client().await?;
334+
let (mut client, endpoint) = self.make_client().await?;
314335

315336
let raft_req = self.new_append_entries_raft_req(&rpc)?;
316337
let req = GrpcHelper::traced_req(raft_req);
@@ -327,7 +348,7 @@ impl RaftNetwork<TypeConfig> for NetworkConnection {
327348
self.target, grpc_res
328349
);
329350

330-
let resp = grpc_res.map_err(|e| self.status_to_unreachable(e))?;
351+
let resp = grpc_res.map_err(|e| self.status_to_unreachable(e, endpoint))?;
331352

332353
let raft_res = GrpcHelper::parse_raft_reply(resp)
333354
.map_err(|serde_err| new_net_err(&serde_err, || "parse append_entries reply"))?;
@@ -369,7 +390,7 @@ impl RaftNetwork<TypeConfig> for NetworkConnection {
369390

370391
let _g = snapshot_send_inflight(self.target).counter_guard();
371392

372-
let mut client = self.make_client().await?;
393+
let (mut client, endpoint) = self.make_client().await?;
373394

374395
let bytes = rpc.data.len() as u64;
375396
raft_metrics::network::incr_sendto_bytes(&self.target, bytes);
@@ -422,7 +443,7 @@ impl RaftNetwork<TypeConfig> for NetworkConnection {
422443

423444
let resp = grpc_res.map_err(|e| {
424445
self.report_metrics_snapshot(false);
425-
self.status_to_unreachable(e)
446+
self.status_to_unreachable(e, endpoint)
426447
})?;
427448

428449
let raft_res = GrpcHelper::parse_raft_reply(resp)
@@ -442,7 +463,7 @@ impl RaftNetwork<TypeConfig> for NetworkConnection {
442463
) -> Result<VoteResponse, RPCError<RaftError>> {
443464
info!(id = self.id, target = self.target, rpc = rpc.summary(); "send_vote");
444465

445-
let mut client = self.make_client().await?;
466+
let (mut client, endpoint) = self.make_client().await?;
446467

447468
let raft_req = GrpcHelper::encode_raft_request(&rpc).map_err(|e| Unreachable::new(&e))?;
448469

@@ -454,7 +475,7 @@ impl RaftNetwork<TypeConfig> for NetworkConnection {
454475
let grpc_res = client.vote(req).await;
455476
info!("vote: resp from target={} {:?}", self.target, grpc_res);
456477

457-
let resp = grpc_res.map_err(|e| self.status_to_unreachable(e))?;
478+
let resp = grpc_res.map_err(|e| self.status_to_unreachable(e, endpoint))?;
458479

459480
let raft_res = GrpcHelper::parse_raft_reply(resp)
460481
.map_err(|serde_err| new_net_err(&serde_err, || "parse vote reply"))?;

0 commit comments

Comments
 (0)