Skip to content

Commit beb3604

Browse files
authored
refactor: reduce MetaClientError (#17764)
Move variant `ClientRuntimeError` and `ConfigError` from `MetaClientError` to a new error `CreationError`. Because `ClientRuntimeError` and `ConfigError` are only used when creating a meta-client.
1 parent 3271f0b commit beb3604

File tree

18 files changed

+155
-57
lines changed

18 files changed

+155
-57
lines changed

Cargo.lock

Lines changed: 1 addition & 0 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

Cargo.toml

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -94,18 +94,18 @@ members = [
9494
"src/meta/binaries",
9595
"src/meta/client",
9696
"src/meta/control",
97+
"src/meta/ee",
9798
"src/meta/embedded",
9899
"src/meta/kvapi",
99100
"src/meta/process",
101+
"src/meta/proto-conv",
102+
"src/meta/protos",
100103
"src/meta/raft-store",
104+
"src/meta/semaphore",
101105
"src/meta/sled-store",
102106
"src/meta/stoerr",
103107
"src/meta/store",
104108
"src/meta/types",
105-
"src/meta/ee",
106-
"src/meta/proto-conv",
107-
"src/meta/protos",
108-
"src/meta/semaphore",
109109
"src/meta/service",
110110
"tests/sqllogictests",
111111
"src/tests/sqlsmith",

src/meta/binaries/metactl/main.rs

Lines changed: 2 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ use clap::CommandFactory;
2222
use clap::Parser;
2323
use clap::Subcommand;
2424
use databend_common_base::base::tokio;
25+
use databend_common_meta_client::errors::CreationError;
2526
use databend_common_meta_client::ClientHandle;
2627
use databend_common_meta_client::MetaGrpcClient;
2728
use databend_common_meta_control::admin::MetaAdminClient;
@@ -38,7 +39,6 @@ use databend_common_meta_control::export_from_grpc;
3839
use databend_common_meta_control::import;
3940
use databend_common_meta_kvapi::kvapi::KVApi;
4041
use databend_common_meta_types::protobuf::WatchRequest;
41-
use databend_common_meta_types::MetaClientError;
4242
use databend_common_meta_types::UpsertKV;
4343
use databend_common_tracing::init_logging;
4444
use databend_common_tracing::Config as LogConfig;
@@ -206,10 +206,7 @@ impl App {
206206
Ok(())
207207
}
208208

209-
fn new_grpc_client(
210-
&self,
211-
addresses: Vec<String>,
212-
) -> Result<Arc<ClientHandle>, MetaClientError> {
209+
fn new_grpc_client(&self, addresses: Vec<String>) -> Result<Arc<ClientHandle>, CreationError> {
213210
MetaGrpcClient::try_create(
214211
addresses,
215212
"root",

src/meta/client/Cargo.toml

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@ prost = { workspace = true }
3232
semver = { workspace = true }
3333
serde = { workspace = true }
3434
serde_json = { workspace = true }
35+
thiserror = { workspace = true }
3536
tonic = { workspace = true }
3637

3738
[dev-dependencies]
Lines changed: 44 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,44 @@
1+
// Copyright 2021 Datafuse 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+
15+
use anyerror::AnyError;
16+
17+
/// Error occurs when creating a meta client.
18+
#[derive(thiserror::Error, Debug, Clone, PartialEq, Eq)]
19+
pub enum CreationError {
20+
#[error("meta-client dedicated runtime error: {0}")]
21+
RuntimeError(AnyError),
22+
23+
#[error("meta-client config error: {0}")]
24+
ConfigError(AnyError),
25+
}
26+
27+
impl CreationError {
28+
pub fn new_runtime_error(msg: impl ToString) -> Self {
29+
Self::RuntimeError(AnyError::error(msg))
30+
}
31+
32+
pub fn new_config_error(msg: impl ToString) -> Self {
33+
Self::ConfigError(AnyError::error(msg))
34+
}
35+
36+
pub fn context(self, ctx: impl ToString) -> Self {
37+
let ctx = ctx.to_string();
38+
39+
match self {
40+
Self::RuntimeError(e) => Self::RuntimeError(e.add_context(|| ctx)),
41+
Self::ConfigError(e) => Self::ConfigError(e.add_context(|| ctx)),
42+
}
43+
}
44+
}

src/meta/client/src/errors/mod.rs

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,17 @@
1+
// Copyright 2021 Datafuse 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+
15+
mod creation_error;
16+
17+
pub use creation_error::CreationError;

src/meta/client/src/grpc_client.rs

Lines changed: 42 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@ use databend_common_grpc::GrpcConnectionError;
4343
use databend_common_grpc::RpcClientConf;
4444
use databend_common_grpc::RpcClientTlsConfig;
4545
use databend_common_meta_api::reply::reply_to_api_result;
46+
use databend_common_meta_kvapi::kvapi::ListKVReq;
4647
use databend_common_meta_types::anyerror::AnyError;
4748
use databend_common_meta_types::protobuf as pb;
4849
use databend_common_meta_types::protobuf::meta_service_client::MetaServiceClient;
@@ -54,6 +55,7 @@ use databend_common_meta_types::protobuf::HandshakeRequest;
5455
use databend_common_meta_types::protobuf::MemberListReply;
5556
use databend_common_meta_types::protobuf::MemberListRequest;
5657
use databend_common_meta_types::protobuf::RaftRequest;
58+
use databend_common_meta_types::protobuf::StreamItem;
5759
use databend_common_meta_types::protobuf::WatchRequest;
5860
use databend_common_meta_types::protobuf::WatchResponse;
5961
use databend_common_meta_types::ConnectionError;
@@ -91,6 +93,7 @@ use tonic::Request;
9193
use tonic::Status;
9294

9395
use crate::endpoints::Endpoints;
96+
use crate::errors::CreationError;
9497
use crate::established_client::EstablishedClient;
9598
use crate::from_digit_ver;
9699
use crate::grpc_action::RequestFor;
@@ -101,6 +104,7 @@ use crate::to_digit_ver;
101104
use crate::ClientWorkerRequest;
102105
use crate::MetaGrpcReadReq;
103106
use crate::MetaGrpcReq;
107+
use crate::Streamed;
104108
use crate::METACLI_COMMIT_SEMVER;
105109
use crate::MIN_METASRV_SEMVER;
106110

@@ -294,6 +298,16 @@ impl Drop for ClientHandle {
294298
}
295299

296300
impl ClientHandle {
301+
pub async fn list(&self, prefix: &str) -> Result<BoxStream<StreamItem>, MetaError> {
302+
let strm = self
303+
.request(Streamed(ListKVReq {
304+
prefix: prefix.to_string(),
305+
}))
306+
.await?;
307+
308+
Ok(strm)
309+
}
310+
297311
/// Send a request to the internal worker task, which will be running in another runtime.
298312
#[fastrace::trace]
299313
#[async_backtrace::framed]
@@ -305,7 +319,10 @@ impl ClientHandle {
305319
<Result<Req::Reply, E> as TryFrom<Response>>::Error: std::fmt::Display,
306320
E: From<MetaClientError> + Debug,
307321
{
308-
let rx = self.send_request_to_worker(req)?;
322+
let rx = self
323+
.send_request_to_worker(req)
324+
.map_err(MetaClientError::from)?;
325+
309326
UnlimitedFuture::create(async move {
310327
let _g = grpc_metrics::client_request_inflight.counter_guard();
311328
rx.await
@@ -326,7 +343,10 @@ impl ClientHandle {
326343
{
327344
let _g = grpc_metrics::client_request_inflight.counter_guard();
328345

329-
let rx = self.send_request_to_worker(req)?;
346+
let rx = self
347+
.send_request_to_worker(req)
348+
.map_err(MetaClientError::from)?;
349+
330350
let recv_res = rx.blocking_recv();
331351
Self::parse_worker_result(recv_res)
332352
}
@@ -335,7 +355,7 @@ impl ClientHandle {
335355
fn send_request_to_worker<Req>(
336356
&self,
337357
req: Req,
338-
) -> Result<oneshot::Receiver<Response>, MetaClientError>
358+
) -> Result<oneshot::Receiver<Response>, ConnectionError>
339359
where
340360
Req: Into<message::Request>,
341361
{
@@ -365,7 +385,7 @@ impl ClientHandle {
365385
));
366386

367387
error!("{}", err);
368-
MetaClientError::ClientRuntimeError(err)
388+
ConnectionError::new(err, "Meta ClientHandle failed to send request to worker")
369389
})?;
370390

371391
Ok(rx)
@@ -381,13 +401,14 @@ impl ClientHandle {
381401
E: From<MetaClientError> + Debug,
382402
{
383403
let response = res.map_err(|e| {
404+
let err = AnyError::new(&e).add_context(|| "when recv resp from MetaGrpcClient worker");
384405
error!(
385406
error :? =(&e);
386407
"Meta ClientHandle recv response from meta client worker failed"
387408
);
388-
MetaClientError::ClientRuntimeError(
389-
AnyError::new(&e).add_context(|| "when recv resp from MetaGrpcClient worker"),
390-
)
409+
let conn_err =
410+
ConnectionError::new(err, "Meta ClientHandle failed to receive from worker");
411+
MetaClientError::from(conn_err)
391412
})?;
392413

393414
let res: Result<Reply, E> = response
@@ -462,7 +483,7 @@ impl MetaGrpcClient {
462483
///
463484
/// The worker is a singleton and the returned handle is cheap to clone.
464485
/// When all handles are dropped the worker will quit, then the runtime will be destroyed.
465-
pub fn try_new(conf: &RpcClientConf) -> Result<Arc<ClientHandle>, MetaClientError> {
486+
pub fn try_new(conf: &RpcClientConf) -> Result<Arc<ClientHandle>, CreationError> {
466487
Self::try_create(
467488
conf.get_endpoints(),
468489
&conf.username,
@@ -481,7 +502,7 @@ impl MetaGrpcClient {
481502
timeout: Option<Duration>,
482503
auto_sync_interval: Option<Duration>,
483504
tls_config: Option<RpcClientTlsConfig>,
484-
) -> Result<Arc<ClientHandle>, MetaClientError> {
505+
) -> Result<Arc<ClientHandle>, CreationError> {
485506
Self::endpoints_non_empty(&endpoints_str)?;
486507

487508
let endpoints = Arc::new(Mutex::new(Endpoints::new(endpoints_str.clone())));
@@ -494,9 +515,7 @@ impl MetaGrpcClient {
494515
Some(format!("meta-client-rt-{}", endpoints_str.join(","))),
495516
)
496517
.map_err(|e| {
497-
MetaClientError::ClientRuntimeError(
498-
AnyError::new(&e).add_context(|| "when creating meta-client"),
499-
)
518+
CreationError::new_runtime_error(e.to_string()).context("when creating meta-client")
500519
})?;
501520
let rt = Arc::new(rt);
502521

@@ -768,11 +787,9 @@ impl MetaGrpcClient {
768787
))
769788
}
770789

771-
pub fn endpoints_non_empty(endpoints: &[String]) -> Result<(), MetaClientError> {
790+
pub fn endpoints_non_empty(endpoints: &[String]) -> Result<(), CreationError> {
772791
if endpoints.is_empty() {
773-
return Err(MetaClientError::ConfigError(AnyError::error(
774-
"endpoints is empty",
775-
)));
792+
return Err(CreationError::new_config_error("endpoints is empty"));
776793
}
777794
Ok(())
778795
}
@@ -784,8 +801,8 @@ impl MetaGrpcClient {
784801

785802
#[fastrace::trace]
786803
#[async_backtrace::framed]
787-
pub async fn set_endpoints(&self, endpoints: Vec<String>) -> Result<(), MetaError> {
788-
Self::endpoints_non_empty(&endpoints)?;
804+
pub fn set_endpoints(&self, endpoints: Vec<String>) {
805+
debug_assert!(!endpoints.is_empty());
789806

790807
// Older meta nodes may not store endpoint information and need to be filtered out.
791808
let distinct_cnt = endpoints.iter().filter(|n| !(*n).is_empty()).count();
@@ -798,12 +815,11 @@ impl MetaGrpcClient {
798815
endpoints.len(),
799816
endpoints
800817
);
801-
return Ok(());
818+
return;
802819
}
803820

804821
let mut eps = self.endpoints.lock();
805822
eps.replace_nodes(endpoints);
806-
Ok(())
807823
}
808824

809825
#[fastrace::trace]
@@ -833,7 +849,12 @@ impl MetaGrpcClient {
833849
let result: Vec<String> = endpoints?.data;
834850
debug!("received meta endpoints: {:?}", result);
835851

836-
self.set_endpoints(result).await?;
852+
if result.is_empty() {
853+
error!("Can not update local endpoints, the returned result is empty");
854+
} else {
855+
self.set_endpoints(result);
856+
}
857+
837858
Ok(())
838859
}
839860

src/meta/client/src/lib.rs

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
extern crate core;
1818

1919
pub mod endpoints;
20+
pub mod errors;
2021
pub(crate) mod established_client;
2122
mod grpc_action;
2223
mod grpc_client;

src/meta/service/tests/it/grpc/metasrv_connection_error.rs

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,10 +20,10 @@ use std::sync::Arc;
2020
use std::time::Duration;
2121

2222
use databend_common_base::base::Stoppable;
23+
use databend_common_meta_client::errors::CreationError;
2324
use databend_common_meta_client::ClientHandle;
2425
use databend_common_meta_client::MetaGrpcClient;
2526
use databend_common_meta_kvapi::kvapi::KVApi;
26-
use databend_common_meta_types::MetaClientError;
2727
use databend_common_meta_types::UpsertKV;
2828
use log::info;
2929
use test_harness::test;
@@ -144,7 +144,7 @@ async fn test_metasrv_one_client_leader_down() -> anyhow::Result<()> {
144144
Ok(())
145145
}
146146

147-
fn make_client(addresses: Vec<String>) -> Result<Arc<ClientHandle>, MetaClientError> {
147+
fn make_client(addresses: Vec<String>) -> Result<Arc<ClientHandle>, CreationError> {
148148
let client = MetaGrpcClient::try_create(
149149
addresses, // a1() will be shut down
150150
"root",

src/meta/service/tests/it/tests/service.rs

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,12 +23,12 @@ use async_trait::async_trait;
2323
use databend_common_base::base::tokio;
2424
use databend_common_base::base::GlobalSequence;
2525
use databend_common_base::base::Stoppable;
26+
use databend_common_meta_client::errors::CreationError;
2627
use databend_common_meta_client::ClientHandle;
2728
use databend_common_meta_client::MetaGrpcClient;
2829
use databend_common_meta_kvapi::kvapi;
2930
use databend_common_meta_types::protobuf::raft_service_client::RaftServiceClient;
3031
use databend_common_meta_types::raft_types::NodeId;
31-
use databend_common_meta_types::MetaClientError;
3232
use databend_meta::api::GrpcServer;
3333
use databend_meta::configs;
3434
use databend_meta::message::ForwardRequest;
@@ -91,7 +91,7 @@ pub async fn start_metasrv_cluster(node_ids: &[NodeId]) -> anyhow::Result<Vec<Me
9191
Ok(res)
9292
}
9393

94-
pub fn make_grpc_client(addresses: Vec<String>) -> Result<Arc<ClientHandle>, MetaClientError> {
94+
pub fn make_grpc_client(addresses: Vec<String>) -> Result<Arc<ClientHandle>, CreationError> {
9595
let client = MetaGrpcClient::try_create(
9696
addresses,
9797
"root",

src/meta/store/src/lib.rs

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@ use std::task::Context;
1818
use std::task::Poll;
1919

2020
use databend_common_grpc::RpcClientConf;
21+
use databend_common_meta_client::errors::CreationError;
2122
use databend_common_meta_client::ClientHandle;
2223
use databend_common_meta_client::MetaGrpcClient;
2324
use databend_common_meta_embedded::MemMeta;
@@ -119,7 +120,7 @@ impl MetaStoreProvider {
119120
MetaStoreProvider { rpc_conf }
120121
}
121122

122-
pub async fn create_meta_store(&self) -> Result<MetaStore, MetaError> {
123+
pub async fn create_meta_store(&self) -> Result<MetaStore, CreationError> {
123124
if self.rpc_conf.local_mode() {
124125
info!(
125126
conf :? =(&self.rpc_conf);

0 commit comments

Comments
 (0)