Skip to content

Commit

Permalink
fix: print detailed error (#3146)
Browse files Browse the repository at this point in the history
  • Loading branch information
WenyXu authored Jan 12, 2024
1 parent 430ffe0 commit bf96ce3
Show file tree
Hide file tree
Showing 14 changed files with 67 additions and 59 deletions.
6 changes: 3 additions & 3 deletions src/common/meta/src/ddl/alter_table.rs
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ use table::requests::AlterKind;
use crate::cache_invalidator::Context;
use crate::ddl::utils::handle_operate_region_error;
use crate::ddl::DdlContext;
use crate::error::{self, ConvertAlterTableRequestSnafu, InvalidProtoMsgSnafu, Result};
use crate::error::{self, ConvertAlterTableRequestSnafu, Error, InvalidProtoMsgSnafu, Result};
use crate::key::table_info::TableInfoValue;
use crate::key::table_name::TableNameKey;
use crate::key::DeserializedValueWithBytes;
Expand Down Expand Up @@ -374,8 +374,8 @@ impl Procedure for AlterTableProcedure {
}

async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> {
let error_handler = |e| {
if matches!(e, error::Error::RetryLater { .. }) {
let error_handler = |e: Error| {
if e.is_retry_later() {
ProcedureError::retry_later(e)
} else {
ProcedureError::external(e)
Expand Down
2 changes: 1 addition & 1 deletion src/common/meta/src/ddl/utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ pub fn handle_operate_region_error(datanode: Peer) -> impl FnOnce(crate::error::
}

pub fn handle_retry_error(e: Error) -> ProcedureError {
if matches!(e, error::Error::RetryLater { .. }) {
if e.is_retry_later() {
ProcedureError::retry_later(e)
} else {
ProcedureError::external(e)
Expand Down
9 changes: 9 additions & 0 deletions src/meta-srv/src/error.rs
Original file line number Diff line number Diff line change
Expand Up @@ -540,6 +540,13 @@ pub enum Error {
#[snafu(display("Expected to retry later, reason: {}", reason))]
RetryLater { reason: String, location: Location },

#[snafu(display("Expected to retry later, reason: {}", reason))]
RetryLaterWithSource {
reason: String,
location: Location,
source: BoxedError,
},

#[snafu(display("Failed to update table metadata, err_msg: {}", err_msg))]
UpdateTableMetadata { err_msg: String, location: Location },

Expand Down Expand Up @@ -628,6 +635,7 @@ impl Error {
/// Returns `true` if the error is retryable.
pub fn is_retryable(&self) -> bool {
matches!(self, Error::RetryLater { .. })
|| matches!(self, Error::RetryLaterWithSource { .. })
}
}

Expand Down Expand Up @@ -665,6 +673,7 @@ impl ErrorExt for Error {
| Error::MailboxTimeout { .. }
| Error::MailboxReceiver { .. }
| Error::RetryLater { .. }
| Error::RetryLaterWithSource { .. }
| Error::StartGrpc { .. }
| Error::UpdateTableMetadata { .. }
| Error::NoEnoughAvailableDatanode { .. }
Expand Down
7 changes: 4 additions & 3 deletions src/meta-srv/src/procedure/region_failover.rs
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ use snafu::ResultExt;
use store_api::storage::{RegionId, RegionNumber};
use table::metadata::TableId;

use crate::error::{Error, RegisterProcedureLoaderSnafu, Result, TableMetadataManagerSnafu};
use crate::error::{RegisterProcedureLoaderSnafu, Result, TableMetadataManagerSnafu};
use crate::lock::DistLockRef;
use crate::metasrv::{SelectorContext, SelectorRef};
use crate::service::mailbox::MailboxRef;
Expand Down Expand Up @@ -357,7 +357,7 @@ impl Procedure for RegionFailoverProcedure {
.next(&self.context, &self.node.failed_region)
.await
.map_err(|e| {
if matches!(e, Error::RetryLater { .. }) {
if e.is_retryable() {
ProcedureError::retry_later(e)
} else {
ProcedureError::external(e)
Expand Down Expand Up @@ -780,7 +780,8 @@ mod tests {

let result = procedure.execute(&ctx).await;
assert!(result.is_err());
assert!(result.unwrap_err().is_retry_later());
let err = result.unwrap_err();
assert!(err.is_retry_later(), "err: {:?}", err);
assert_eq!(
r#"{"region_failover_state":"RegionFailoverStart","failover_candidate":null}"#,
serde_json::to_string(&procedure.node.state).unwrap()
Expand Down
13 changes: 7 additions & 6 deletions src/meta-srv/src/procedure/region_failover/failover_start.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,17 +13,17 @@
// limitations under the License.

use async_trait::async_trait;
use common_error::ext::ErrorExt;
use common_error::ext::{BoxedError, ErrorExt};
use common_error::status_code::StatusCode;
use common_meta::peer::Peer;
use common_meta::RegionIdent;
use common_telemetry::info;
use serde::{Deserialize, Serialize};
use snafu::ensure;
use snafu::{ensure, location, Location};

use super::deactivate_region::DeactivateRegion;
use super::{RegionFailoverContext, State};
use crate::error::{RegionFailoverCandidatesNotFoundSnafu, Result, RetryLaterSnafu};
use crate::error::{self, RegionFailoverCandidatesNotFoundSnafu, Result};
use crate::selector::SelectorOptions;

#[derive(Serialize, Deserialize, Debug)]
Expand Down Expand Up @@ -93,10 +93,11 @@ impl State for RegionFailoverStart {
.await
.map_err(|e| {
if e.status_code() == StatusCode::RuntimeResourcesExhausted {
RetryLaterSnafu {
reason: format!("{e}"),
error::Error::RetryLaterWithSource {
reason: format!("Region failover aborted for {failed_region:?}"),
location: location!(),
source: BoxedError::new(e),
}
.build()
} else {
e
}
Expand Down
17 changes: 8 additions & 9 deletions src/meta-srv/src/procedure/region_failover/update_metadata.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
use std::collections::HashMap;

use async_trait::async_trait;
use common_error::ext::BoxedError;
use common_meta::key::datanode_table::RegionInfo;
use common_meta::key::table_route::TableRouteKey;
use common_meta::peer::Peer;
Expand All @@ -27,7 +28,7 @@ use store_api::storage::RegionNumber;

use super::invalidate_cache::InvalidateCache;
use super::{RegionFailoverContext, State};
use crate::error::{self, Result, RetryLaterSnafu, TableRouteNotFoundSnafu};
use crate::error::{self, Result, TableRouteNotFoundSnafu};
use crate::lock::keys::table_metadata_lock_key;
use crate::lock::Opts;

Expand Down Expand Up @@ -172,14 +173,12 @@ impl State for UpdateRegionMetadata {
) -> Result<Box<dyn State>> {
self.update_metadata(ctx, failed_region)
.await
.map_err(|e| {
RetryLaterSnafu {
reason: format!(
"Failed to update metadata for failed region: {}, error: {}",
failed_region, e
),
}
.build()
.map_err(BoxedError::new)
.context(error::RetryLaterWithSourceSnafu {
reason: format!(
"Failed to update metadata for failed region: {}",
failed_region
),
})?;
Ok(Box::new(InvalidateCache))
}
Expand Down
25 changes: 13 additions & 12 deletions src/meta-srv/src/procedure/region_migration.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ use std::fmt::Debug;
use std::time::Duration;

use api::v1::meta::MailboxMessage;
use common_error::ext::BoxedError;
use common_meta::instruction::Instruction;
use common_meta::key::datanode_table::{DatanodeTableKey, DatanodeTableValue};
use common_meta::key::table_info::TableInfoValue;
Expand All @@ -45,12 +46,12 @@ use common_procedure::error::{
use common_procedure::{Context as ProcedureContext, LockKey, Procedure, Status, StringKey};
pub use manager::RegionMigrationProcedureTask;
use serde::{Deserialize, Serialize};
use snafu::{location, Location, OptionExt, ResultExt};
use snafu::{OptionExt, ResultExt};
use store_api::storage::RegionId;
use tokio::time::Instant;

use self::migration_start::RegionMigrationStart;
use crate::error::{self, Error, Result};
use crate::error::{self, Result};
use crate::service::mailbox::{BroadcastChannel, MailboxRef};

/// It's shared in each step and available even after recovering.
Expand Down Expand Up @@ -220,9 +221,9 @@ impl Context {
.get(table_id)
.await
.context(error::TableMetadataManagerSnafu)
.map_err(|e| error::Error::RetryLater {
reason: e.to_string(),
location: location!(),
.map_err(BoxedError::new)
.context(error::RetryLaterWithSourceSnafu {
reason: format!("Failed to get TableRoute: {table_id}"),
})?
.context(error::TableRouteNotFoundSnafu { table_id })?;

Expand Down Expand Up @@ -256,9 +257,9 @@ impl Context {
.get(table_id)
.await
.context(error::TableMetadataManagerSnafu)
.map_err(|e| error::Error::RetryLater {
reason: e.to_string(),
location: location!(),
.map_err(BoxedError::new)
.context(error::RetryLaterWithSourceSnafu {
reason: format!("Failed to get TableInfo: {table_id}"),
})?
.context(error::TableInfoNotFoundSnafu { table_id })?;

Expand Down Expand Up @@ -289,9 +290,9 @@ impl Context {
})
.await
.context(error::TableMetadataManagerSnafu)
.map_err(|e| error::Error::RetryLater {
reason: e.to_string(),
location: location!(),
.map_err(BoxedError::new)
.context(error::RetryLaterWithSourceSnafu {
reason: format!("Failed to get DatanodeTable: ({datanode_id},{table_id})"),
})?
.context(error::DatanodeTableNotFoundSnafu {
table_id,
Expand Down Expand Up @@ -412,7 +413,7 @@ impl Procedure for RegionMigrationProcedure {
let state = &mut self.state;

let (next, status) = state.next(&mut self.context).await.map_err(|e| {
if matches!(e, Error::RetryLater { .. }) {
if e.is_retryable() {
ProcedureError::retry_later(e)
} else {
ProcedureError::external(e)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -374,7 +374,7 @@ mod tests {

assert_matches!(err, Error::RetryLater { .. });
assert!(err.is_retryable());
assert!(err.to_string().contains("test mocked"));
assert!(format!("{err:?}").contains("test mocked"), "err: {err:?}",);
}

#[tokio::test]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -383,7 +383,7 @@ mod tests {

assert_matches!(err, Error::RetryLater { .. });
assert!(err.is_retryable());
assert!(err.to_string().contains("test mocked"));
assert!(format!("{err:?}").contains("test mocked"));
}

#[tokio::test]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

use common_error::ext::BoxedError;
use common_meta::rpc::router::RegionStatus;
use snafu::ResultExt;

Expand Down Expand Up @@ -61,9 +62,11 @@ impl UpdateMetadata {
.context(error::TableMetadataManagerSnafu)
{
ctx.remove_table_route_value();
return error::RetryLaterSnafu {
reason: format!("Failed to update the table route during the downgrading leader region, error: {err}")
}.fail();
return Err(BoxedError::new(err)).context(error::RetryLaterWithSourceSnafu {
reason: format!(
"Failed to update the table route during the downgrading leader region, region_id: {region_id}, from_peer_id: {from_peer_id}"
),
});
}

ctx.remove_table_route_value();
Expand Down Expand Up @@ -163,13 +166,9 @@ mod tests {
ctx.volatile_ctx.table_route = Some(original_table_route);

let err = state.downgrade_leader_region(&mut ctx).await.unwrap_err();

assert!(ctx.volatile_ctx.table_route.is_none());

assert_matches!(err, Error::RetryLater { .. });

assert!(err.is_retryable());
assert!(err.to_string().contains("Failed to update the table route"));
assert!(format!("{err:?}").contains("Failed to update the table route"));
}

#[tokio::test]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

use common_error::ext::BoxedError;
use snafu::ResultExt;

use crate::error::{self, Result};
Expand Down Expand Up @@ -45,9 +46,9 @@ impl UpdateMetadata {
.context(error::TableMetadataManagerSnafu)
{
ctx.remove_table_route_value();
return error::RetryLaterSnafu {
reason: format!("Failed to update the table route during the rollback downgraded leader region, error: {err}")
}.fail();
return Err(BoxedError::new(err)).context(error::RetryLaterWithSourceSnafu {
reason: format!("Failed to update the table route during the rollback downgraded leader region: {region_id}"),
});
}

ctx.remove_table_route_value();
Expand Down Expand Up @@ -157,9 +158,8 @@ mod tests {
.await
.unwrap_err();
assert!(ctx.volatile_ctx.table_route.is_none());
assert_matches!(err, Error::RetryLater { .. });
assert!(err.is_retryable());
assert!(err.to_string().contains("Failed to update the table route"));
assert!(format!("{err:?}").contains("Failed to update the table route"));

state.rollback_downgraded_region(&mut ctx).await.unwrap();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

use common_error::ext::BoxedError;
use common_meta::key::datanode_table::RegionInfo;
use common_meta::rpc::router::{region_distribution, RegionRoute};
use common_telemetry::{info, warn};
Expand Down Expand Up @@ -168,9 +169,9 @@ impl UpdateMetadata {
.context(error::TableMetadataManagerSnafu)
{
ctx.remove_table_route_value();
return error::RetryLaterSnafu {
reason: format!("Failed to update the table route during the upgrading candidate region, error: {err}")
}.fail();
return Err(BoxedError::new(err)).context(error::RetryLaterWithSourceSnafu {
reason: format!("Failed to update the table route during the upgrading candidate region: {region_id}"),
});
};

ctx.remove_table_route_value();
Expand Down Expand Up @@ -354,15 +355,12 @@ mod tests {
.register(2, RegionId::new(table_id, 1))
.unwrap();
ctx.volatile_ctx.opening_region_guard = Some(guard);

let err = state.upgrade_candidate_region(&mut ctx).await.unwrap_err();

assert!(ctx.volatile_ctx.table_route.is_none());
assert!(ctx.volatile_ctx.opening_region_guard.is_some());
assert_matches!(err, Error::RetryLater { .. });

assert!(err.is_retryable());
assert!(err.to_string().contains("Failed to update the table route"));
assert!(format!("{err:?}").contains("Failed to update the table route"));
}

#[tokio::test]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -336,7 +336,7 @@ mod tests {

assert_matches!(err, Error::RetryLater { .. });
assert!(err.is_retryable());
assert!(err.to_string().contains("test mocked"));
assert!(format!("{err:?}").contains("test mocked"));
}

#[tokio::test]
Expand Down Expand Up @@ -398,7 +398,7 @@ mod tests {

assert_matches!(err, Error::RetryLater { .. });
assert!(err.is_retryable());
assert!(err.to_string().contains("still replaying the wal"));
assert!(format!("{err:?}").contains("still replaying the wal"));

// Sets the `require_ready` to false.
state.require_ready = false;
Expand Down
2 changes: 1 addition & 1 deletion src/meta-srv/src/procedure/utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

#[cfg(feature = "mock")]
#[cfg(any(test, feature = "mock"))]
pub mod mock {
use std::io::Error;
use std::sync::Arc;
Expand Down

0 comments on commit bf96ce3

Please sign in to comment.