Skip to content

Commit

Permalink
Merge pull request #10114 from drmingdrmer/50-exception
Browse files Browse the repository at this point in the history
refactor: remove dependency of common-exceptionn from common-meta-types
  • Loading branch information
drmingdrmer authored Feb 18, 2023
2 parents 104e171 + a0ed8a2 commit 5d902c0
Show file tree
Hide file tree
Showing 23 changed files with 66 additions and 167 deletions.
6 changes: 2 additions & 4 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 2 additions & 0 deletions src/common/exception/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,8 @@ test = false

[dependencies] # In alphabetical order
common-arrow = { path = "../arrow" }
common-meta-stoerr = { path = "../../meta/stoerr" }
common-meta-types = { path = "../../meta/types" }

# Github dependencies
# TODO: Use the version from crates.io once
Expand Down
2 changes: 1 addition & 1 deletion src/common/exception/src/exception_code.rs
Original file line number Diff line number Diff line change
Expand Up @@ -158,7 +158,7 @@ build_exceptions! {
TableSchemaMismatch(1303),
}

// Metasvr errors [2001, 3000].
// Meta service errors [2001, 3000].
build_exceptions! {
// Meta service does not work.
MetaServiceError(2001),
Expand Down
22 changes: 22 additions & 0 deletions src/common/exception/src/exception_into.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,10 @@ use std::fmt::Display;
use std::fmt::Formatter;
use std::sync::Arc;

use common_meta_stoerr::MetaStorageError;
use common_meta_types::MetaAPIError;
use common_meta_types::MetaError;

use crate::exception::ErrorCodeBacktrace;
use crate::ErrorCode;

Expand Down Expand Up @@ -275,6 +279,24 @@ impl From<tonic::Status> for ErrorCode {
}
}

impl From<MetaError> for ErrorCode {
fn from(e: MetaError) -> Self {
ErrorCode::MetaServiceError(e.to_string())
}
}

impl From<MetaAPIError> for ErrorCode {
fn from(e: MetaAPIError) -> Self {
ErrorCode::MetaServiceError(e.to_string())
}
}

impl From<MetaStorageError> for ErrorCode {
fn from(e: MetaStorageError) -> Self {
ErrorCode::MetaServiceError(e.to_string())
}
}

impl From<ErrorCode> for tonic::Status {
fn from(err: ErrorCode) -> Self {
let rst_json = serde_json::to_vec::<SerializedError>(&SerializedError {
Expand Down
2 changes: 1 addition & 1 deletion src/meta/api/src/kv_app_error.rs
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ impl From<KVAppError> for ErrorCode {
fn from(e: KVAppError) -> Self {
match e {
KVAppError::AppError(app_err) => app_err.into(),
KVAppError::MetaError(meta_err) => meta_err.into(),
KVAppError::MetaError(meta_err) => ErrorCode::MetaServiceError(meta_err.to_string()),
}
}
}
Expand Down
5 changes: 4 additions & 1 deletion src/meta/client/tests/it/grpc_client.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@ use common_meta_app::schema::GetDatabaseReq;
use common_meta_client::MetaGrpcClient;
use common_meta_client::MIN_METASRV_SEMVER;
use common_meta_types::protobuf::meta_service_client::MetaServiceClient;
use common_meta_types::MetaClientError;
use common_meta_types::MetaError;

use crate::grpc_server::start_grpc_server;

Expand Down Expand Up @@ -69,7 +71,8 @@ async fn test_grpc_client_handshake_timeout() {
.await;

let got = res.unwrap_err();
let got = ErrorCode::from(got).message();
let got =
ErrorCode::from(MetaError::ClientError(MetaClientError::HandshakeError(got))).message();
let expect = "failed to handshake with meta-service: when sending handshake rpc, cause: tonic::status::Status: status: Cancelled, message: \"Timeout expired\", details: [], metadata: MetadataMap { headers: {} } source: transport error source: Timeout expired";
assert_eq!(got, expect);
}
Expand Down
6 changes: 2 additions & 4 deletions src/meta/raft-store/src/state_machine/sm.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ use std::time::Instant;
use std::time::SystemTime;
use std::time::UNIX_EPOCH;

use common_exception::WithContext;
use common_meta_sled_store::get_sled_db;
use common_meta_sled_store::openraft;
use common_meta_sled_store::openraft::EffectiveMembership;
Expand Down Expand Up @@ -147,8 +146,7 @@ impl StateMachine {
let db = get_sled_db();

// it blocks and slow
db.drop_tree(tree_name)
.context(|| "drop prev state machine")?;
db.drop_tree(tree_name)?;

Ok(())
}
Expand Down Expand Up @@ -208,7 +206,7 @@ impl StateMachine {

let mut kvs = Vec::new();
for rkv in view {
let (k, v) = rkv.context(|| "taking snapshot")?;
let (k, v) = rkv?;
kvs.push(vec![k.to_vec(), v.to_vec()]);
}
let snap = SerializableSnapshot { kvs };
Expand Down
1 change: 0 additions & 1 deletion src/meta/service/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ io-uring = [
# Workspace dependencies
common-arrow = { path = "../../common/arrow" }
common-base = { path = "../../common/base" }
common-exception = { path = "../../common/exception" }
common-grpc = { path = "../../common/grpc" }
common-http = { path = "../../common/http" }
common-meta-api = { path = "../api" }
Expand Down
5 changes: 2 additions & 3 deletions src/meta/service/src/store/store_bare.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ use std::ops::RangeBounds;
use anyerror::AnyError;
use common_base::base::tokio::sync::RwLock;
use common_base::base::tokio::sync::RwLockWriteGuard;
use common_exception::WithContext;
use common_meta_raft_store::applied_state::AppliedState;
use common_meta_raft_store::config::RaftConfig;
use common_meta_raft_store::log::RaftLog;
Expand Down Expand Up @@ -262,7 +261,7 @@ impl RaftStoreBare {
for x in snap.kvs.into_iter() {
let k = &x[0];
let v = &x[1];
tree.insert(k, v.clone()).context(|| "insert snapshot")?;
tree.insert(k, v.clone())?;
}

info!(
Expand All @@ -271,7 +270,7 @@ impl RaftStoreBare {
new_sm.get_last_applied()?,
);

tree.flush_async().await.context(|| "flush snapshot")?;
tree.flush_async().await?;

info!("flushed tree, no_kvs: {}", nkvs);

Expand Down
1 change: 0 additions & 1 deletion src/meta/sled-store/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ test = false
io-uring = ["sled/io_uring"]

[dependencies]
common-exception = { path = "../../common/exception" }
common-meta-stoerr = { path = "../stoerr" }
common-meta-types = { path = "../types" }

Expand Down
39 changes: 10 additions & 29 deletions src/meta/sled-store/src/sled_tree.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ use std::marker::PhantomData;
use std::ops::Deref;
use std::ops::RangeBounds;

use common_exception::WithContext;
use common_meta_stoerr::MetaStorageError;
use common_meta_types::anyerror::AnyError;
use common_meta_types::Change;
Expand Down Expand Up @@ -117,9 +116,7 @@ impl SledTree {
let x = &x[0..5];
assert_eq!(x, b"test-");
}
let t = db
.open_tree(&tree_name)
.context(|| format!("open tree: {}", tree_name))?;
let t = db.open_tree(&tree_name)?;

debug!("SledTree opened tree: {}", tree_name);

Expand Down Expand Up @@ -208,10 +205,7 @@ impl SledTree {
&self,
key: &KV::K,
) -> Result<Option<KV::V>, MetaStorageError> {
let got = self
.tree
.get(KV::serialize_key(key)?)
.context(|| format!("get: {}:{}", self.name, key))?;
let got = self.tree.get(KV::serialize_key(key)?)?;

let v = match got {
None => None,
Expand All @@ -237,16 +231,12 @@ impl SledTree {
// Convert K range into sled::IVec range
let sled_range = KV::serialize_range(&range)?;

let range_mes = self.range_message::<KV, _>(&range);

for item in self.tree.range(sled_range) {
let (k, _) = item.context(|| format!("range_remove: {}", range_mes,))?;
let (k, _) = item?;
batch.remove(k);
}

self.tree
.apply_batch(batch)
.context(|| format!("batch remove: {}", range_mes,))?;
self.tree.apply_batch(batch)?;

self.flush_async(flush).await?;

Expand All @@ -265,14 +255,12 @@ impl SledTree {
KV: SledKeySpace,
R: RangeBounds<KV::K>,
{
let range_mes = self.range_message::<KV, _>(&range);

// Convert K range into sled::IVec range
let range = KV::serialize_range(&range)?;

let it = self.tree.range(range);
let it = it.map(move |item| {
let (k, v) = item.context(|| format!("range_get: {}", range_mes,))?;
let (k, v) = item?;

let item = SledItem::new(k, v);
Ok(item)
Expand All @@ -291,11 +279,9 @@ impl SledTree {
{
let mut res = vec![];

let mes = || format!("scan_prefix: {}", prefix);

let pref = KV::serialize_key(prefix)?;
for item in self.tree.scan_prefix(pref) {
let (k, v) = item.context(mes)?;
let (k, v) = item?;

let key = KV::deserialize_key(k)?;
let value = KV::deserialize_value(v)?;
Expand Down Expand Up @@ -323,7 +309,7 @@ impl SledTree {
batch.insert(k, v);
}

self.tree.apply_batch(batch).context(|| "batch append")?;
self.tree.apply_batch(batch)?;

self.flush_async(true).await?;

Expand All @@ -343,10 +329,7 @@ impl SledTree {
let k = KV::serialize_key(key)?;
let v = KV::serialize_value(value)?;

let prev = self
.tree
.insert(k, v)
.context(|| format!("insert_value {}", key))?;
let prev = self.tree.insert(k, v)?;

let prev = match prev {
None => None,
Expand All @@ -359,6 +342,7 @@ impl SledTree {
}

/// Build a string describing the range for a range operation.
#[allow(dead_code)]
fn range_message<KV, R>(&self, range: &R) -> String
where
KV: SledKeySpace,
Expand All @@ -376,10 +360,7 @@ impl SledTree {
#[tracing::instrument(level = "debug", skip(self))]
async fn flush_async(&self, flush: bool) -> Result<(), MetaStorageError> {
if flush && self.sync {
self.tree
.flush_async()
.await
.context(|| "flush sled-tree")?;
self.tree.flush_async().await?;
}
Ok(())
}
Expand Down
1 change: 0 additions & 1 deletion src/meta/stoerr/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@ doctest = false
test = false

[dependencies]
common-exception = { path = "../../common/exception" }

anyerror = { workspace = true }
prost = { workspace = true }
Expand Down
17 changes: 0 additions & 17 deletions src/meta/stoerr/src/meta_storage_errors.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,12 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

use std::any;
use std::backtrace::Backtrace;

use anyerror::AnyError;
use common_exception::ErrorCode;
use common_exception::ErrorWithContext;
use serde::Deserialize;
use serde::Serialize;
use sled::transaction::UnabortableTransactionError;
Expand All @@ -43,12 +38,6 @@ pub enum MetaStorageError {
TransactionConflict,
}

impl From<MetaStorageError> for ErrorCode {
fn from(e: MetaStorageError) -> Self {
ErrorCode::MetaStorageError(e.to_string()).set_backtrace(any::request_ref::<Backtrace>(&e))
}
}

impl From<std::string::FromUtf8Error> for MetaStorageError {
fn from(error: std::string::FromUtf8Error) -> Self {
MetaStorageError::BytesError(MetaBytesError::new(&error))
Expand All @@ -67,12 +56,6 @@ impl From<sled::Error> for MetaStorageError {
}
}

impl From<ErrorWithContext<sled::Error>> for MetaStorageError {
fn from(e: ErrorWithContext<sled::Error>) -> MetaStorageError {
MetaStorageError::SledError(AnyError::new(&e.err).add_context(|| e.context))
}
}

impl From<UnabortableTransactionError> for MetaStorageError {
fn from(error: UnabortableTransactionError) -> Self {
match error {
Expand Down
1 change: 0 additions & 1 deletion src/meta/types/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@ doctest = false
test = false

[dependencies]
common-exception = { path = "../../common/exception" }
common-meta-stoerr = { path = "../stoerr" }

openraft = { workspace = true }
Expand Down
4 changes: 2 additions & 2 deletions src/meta/types/src/cluster.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,10 +13,10 @@
// limitations under the License.

use std::fmt;
use std::net::AddrParseError;
use std::net::SocketAddr;
use std::str::FromStr;

use common_exception::exception::Result;
use serde::Deserialize;
use serde::Serialize;

Expand Down Expand Up @@ -97,7 +97,7 @@ impl NodeInfo {
}
}

pub fn ip_port(&self) -> Result<(String, u16)> {
pub fn ip_port(&self) -> Result<(String, u16), AddrParseError> {
let addr = SocketAddr::from_str(&self.flight_address)?;

Ok((addr.ip().to_string(), addr.port()))
Expand Down
Loading

1 comment on commit 5d902c0

@vercel
Copy link

@vercel vercel bot commented on 5d902c0 Feb 18, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Successfully deployed to the following URLs:

databend – ./

databend.vercel.app
databend.rs
databend-git-main-databend.vercel.app
databend-databend.vercel.app

Please sign in to comment.