Skip to content
Snippets Groups Projects
Commit 5bf9d8a5 authored by Georgy Moshkin's avatar Georgy Moshkin :speech_balloon:
Browse files

perf: introduce TopologyCache

TopologyCache is a collection of deserialized structures with
information about cluster topology. This currently includes data from
_pico_instance, _pico_replicaset, _pico_tier & _pico_service_route_table.
The info is automatically kept up to date with the corresponding system
tables. The TopologyCache also caches the immutable info related to the
current instance, like instance name, replicaset uuid, etc.

From now on we should be reading this data from TopologyCache whenever
possible instead of going directly to system tables as we were doing
previosly.

At the moment only the plugin RPC module has transitioned to using
TopologyCache but other modules should be refactored as well.
Especially governor.
parent f4d66cab
No related branches found
No related tags found
1 merge request!1491perf: introduce TopologyCache
Pipeline #58603 canceled
......@@ -77,6 +77,7 @@ pub mod tarantool;
pub mod tier;
pub mod tlog;
pub mod to_rmpv_named;
pub mod topology_cache;
pub mod traft;
pub mod util;
pub mod vshard;
......
......@@ -2,11 +2,11 @@ use crate::error_code::ErrorCode;
use crate::has_states;
use crate::instance::InstanceName;
use crate::plugin::{rpc, PluginIdentifier};
use crate::replicaset::Replicaset;
use crate::replicaset::ReplicasetState;
use crate::schema::ServiceRouteItem;
use crate::schema::ServiceRouteKey;
use crate::tlog;
use crate::topology_cache::ServiceRouteCheck;
use crate::topology_cache::TopologyCache;
use crate::topology_cache::TopologyCacheRef;
use crate::traft::error::Error;
use crate::traft::network::ConnectionPool;
use crate::traft::node::Node;
......@@ -17,10 +17,8 @@ use picodata_plugin::util::copy_to_region;
use std::time::Duration;
use tarantool::error::BoxError;
use tarantool::error::Error as TntError;
use tarantool::error::IntoBoxError;
use tarantool::error::TarantoolErrorCode;
use tarantool::fiber;
use tarantool::tuple::Tuple;
use tarantool::tuple::TupleBuffer;
use tarantool::tuple::{RawByteBuf, RawBytes};
use tarantool::uuid::Uuid;
......@@ -56,16 +54,13 @@ pub(crate) fn send_rpc_request(
) -> Result<&'static [u8], Error> {
let node = crate::traft::node::global()?;
let pool = &node.plugin_manager.pool;
let topology = &node.topology_cache;
let timeout = Duration::from_secs_f64(timeout);
let instance_name = resolve_rpc_target(plugin_identity, service, target, node)?;
let my_instance_name = node
.raft_storage
.instance_name()?
.expect("should be persisted at this point");
let instance_name = resolve_rpc_target(plugin_identity, service, target, node, topology)?;
let my_instance_name = topology.my_instance_name();
if path.starts_with('.') {
return call_builtin_stored_proc(pool, path, input, &instance_name, timeout);
}
......@@ -194,24 +189,24 @@ fn resolve_rpc_target(
service: &str,
target: &FfiSafeRpcTargetSpecifier,
node: &Node,
topology: &TopologyCache,
) -> Result<InstanceName, Error> {
use FfiSafeRpcTargetSpecifier as Target;
let mut to_master_chosen = false;
let mut tier_and_bucket_id = None;
let mut by_replicaset_name = None;
let mut tier_name_owned;
match target {
Target::InstanceName(name) => {
//
// Request to a specific instance, single candidate
//
// SAFETY: it's required that argument pointers are valid for the lifetime of this function's call
let instance_name = InstanceName::from(unsafe { name.as_str() });
let instance_name = unsafe { name.as_str() };
// A single instance was chosen
check_route_to_instance(node, plugin, service, &instance_name)?;
return Ok(instance_name);
check_route_to_instance(topology, plugin, service, instance_name)?;
return Ok(instance_name.into());
}
&Target::Replicaset {
......@@ -227,11 +222,7 @@ fn resolve_rpc_target(
bucket_id,
to_master,
} => {
tier_name_owned = node
.raft_storage
.tier()?
.expect("storage for instance should exists");
let tier = &*tier_name_owned;
let tier = topology.my_tier_name();
to_master_chosen = to_master;
tier_and_bucket_id = Some((tier, bucket_id));
}
......@@ -250,83 +241,57 @@ fn resolve_rpc_target(
let mut by_bucket_id = None;
let mut tier_and_replicaset_uuid = None;
let mut replicaset_tuple = None;
let mut replicaset_uuid_owned;
let replicaset_uuid_owned;
if let Some((tier, bucket_id)) = tier_and_bucket_id {
// This call must be done here, because this function may yield
// but later we take a volatile reference to TopologyCache which can't be held across yields
replicaset_uuid_owned = vshard::get_replicaset_uuid_by_bucket_id(tier, bucket_id)?;
let uuid = &*replicaset_uuid_owned;
by_bucket_id = Some((tier, bucket_id, uuid));
tier_and_replicaset_uuid = Some((tier, uuid));
}
let topology_ref = topology.get();
//
// Request to replicaset master, single candidate
//
if to_master_chosen {
let mut target_master_name = None;
if let Some(replicaset_name) = by_replicaset_name {
let tuple = node.storage.replicasets.get_raw(replicaset_name)?;
replicaset_tuple = Some(tuple);
let replicaset = topology_ref.replicaset_by_name(replicaset_name)?;
target_master_name = Some(&replicaset.target_master_name);
}
if let Some((_, _, replicaset_uuid)) = by_bucket_id {
let tuple = node.storage.replicasets.by_uuid_raw(replicaset_uuid)?;
replicaset_tuple = Some(tuple);
let replicaset = topology_ref.replicaset_by_uuid(replicaset_uuid)?;
target_master_name = Some(&replicaset.target_master_name);
}
let tuple = replicaset_tuple.expect("set in one of ifs above");
let Some(master_name) = tuple
.field(Replicaset::FIELD_TARGET_MASTER_NAME)
.map_err(IntoBoxError::into_box_error)?
else {
#[rustfmt::skip]
return Err(BoxError::new(ErrorCode::StorageCorrupted, "couldn't find 'target_master_name' field in _pico_replicaset tuple").into());
};
let instance_name = target_master_name.expect("set in one of ifs above");
// A single instance was chosen
check_route_to_instance(node, plugin, service, &master_name)?;
return Ok(master_name);
check_route_to_instance(topology, plugin, service, instance_name)?;
return Ok(instance_name.clone());
}
if let Some(replicaset_name) = by_replicaset_name {
let tuple = node.storage.replicasets.get_raw(replicaset_name)?;
let Some(found_replicaset_uuid) = tuple
.field(Replicaset::FIELD_REPLICASET_UUID)
.map_err(IntoBoxError::into_box_error)?
else {
#[rustfmt::skip]
return Err(BoxError::new(ErrorCode::StorageCorrupted, "couldn't find 'replicaset_uuid' field in _pico_replicaset tuple").into());
};
replicaset_uuid_owned = found_replicaset_uuid;
let Some(found_tier) = tuple
.field::<'_, String>(Replicaset::FIELD_TIER)
.map_err(IntoBoxError::into_box_error)?
else {
#[rustfmt::skip]
return Err(BoxError::new(ErrorCode::StorageCorrupted, "couldn't find 'tier' field in _pico_replicaset tuple").into());
};
tier_name_owned = found_tier;
replicaset_tuple = Some(tuple);
tier_and_replicaset_uuid = Some((&tier_name_owned, &replicaset_uuid_owned));
let replicaset = topology_ref.replicaset_by_name(replicaset_name)?;
tier_and_replicaset_uuid = Some((&replicaset.tier, &replicaset.uuid));
}
let my_instance_name = node
.raft_storage
.instance_name()?
.expect("should be persisted at this point");
// Get list of all possible targets
let mut all_instances_with_service = node
.storage
.service_route_table
.get_available_instances(plugin, service)?;
let mut all_instances_with_service: Vec<_> = topology_ref
.instances_running_service(&plugin.name, &plugin.version, service)
.collect();
// Get list of all possible targets
filter_instances_by_state(node, &mut all_instances_with_service)?;
// Remove non-online instances
filter_instances_by_state(&topology_ref, &mut all_instances_with_service)?;
#[rustfmt::skip]
if all_instances_with_service.is_empty() {
// TODO: put service definitions into TopologyCache as well
if node.storage.services.get(plugin, service)?.is_none() {
return Err(BoxError::new(ErrorCode::NoSuchService, "service '{plugin}.{service}' not found").into());
} else {
......@@ -334,6 +299,8 @@ fn resolve_rpc_target(
}
};
let my_instance_name = topology.my_instance_name();
//
// Request to any replica in replicaset, multiple candidates
//
......@@ -347,22 +314,26 @@ fn resolve_rpc_target(
// XXX: this shouldn't be a problem if replicasets aren't too big,
// but if they are we might want to construct a HashSet from candidates
for instance_name in replicas {
if my_instance_name == instance_name {
if my_instance_name == &*instance_name {
// Prefer someone else instead of self
skipped_self = true;
continue;
}
if all_instances_with_service.contains(&instance_name) {
if all_instances_with_service.contains(&&*instance_name) {
return Ok(instance_name);
}
}
// In case there's no other suitable candidates, fallback to calling self
if skipped_self && all_instances_with_service.contains(&my_instance_name) {
return Ok(my_instance_name);
return Ok(my_instance_name.into());
}
check_replicaset_is_not_expelled(node, replicaset_uuid, replicaset_tuple)?;
let replicaset = topology_ref.replicaset_by_uuid(replicaset_uuid)?;
if replicaset.state == ReplicasetState::Expelled {
#[rustfmt::skip]
return Err(BoxError::new(ErrorCode::ReplicasetExpelled, format!("replicaset with id {replicaset_uuid} was expelled")).into());
}
#[rustfmt::skip]
return Err(BoxError::new(ErrorCode::ServiceNotAvailable, format!("no {replicaset_uuid} replicas are available for service {plugin}.{service}")).into());
......@@ -381,17 +352,17 @@ fn resolve_rpc_target(
let index = (random_index + 1) % candidates.len();
instance_name = std::mem::take(&mut candidates[index]);
}
return Ok(instance_name);
return Ok(instance_name.into());
}
fn filter_instances_by_state(
node: &Node,
instance_names: &mut Vec<InstanceName>,
topology_ref: &TopologyCacheRef,
instance_names: &mut Vec<&str>,
) -> Result<(), Error> {
let mut index = 0;
while index < instance_names.len() {
let name = &instance_names[index];
let instance = node.storage.instances.get(name)?;
let instance = topology_ref.instance_by_name(name)?;
if has_states!(instance, Expelled -> *) || !instance.may_respond() {
instance_names.swap_remove(index);
} else {
......@@ -403,63 +374,35 @@ fn filter_instances_by_state(
}
fn check_route_to_instance(
node: &Node,
topology: &TopologyCache,
plugin: &PluginIdentifier,
service: &str,
instance_name: &InstanceName,
instance_name: &str,
) -> Result<(), Error> {
let instance = node.storage.instances.get(instance_name)?;
let topology_ref = topology.get();
let instance = topology_ref.instance_by_name(instance_name)?;
if has_states!(instance, Expelled -> *) {
#[rustfmt::skip]
return Err(BoxError::new(ErrorCode::InstanceExpelled, format!("instance named '{instance_name}' was expelled")).into());
}
if !instance.may_respond() {
#[rustfmt::skip]
return Err(BoxError::new(ErrorCode::InstanceUnavaliable, format!("instance with instance_name \"{instance_name}\" can't respond due it's state"),).into());
}
let res = node.storage.service_route_table.get_raw(&ServiceRouteKey {
instance_name,
plugin_name: &plugin.name,
plugin_version: &plugin.version,
service_name: service,
})?;
#[rustfmt::skip]
let Some(tuple) = res else {
return Err(BoxError::new(ErrorCode::ServiceNotStarted, format!("service '{plugin}.{service}' is not running on {instance_name}")).into());
};
let res = tuple
.field(ServiceRouteItem::FIELD_POISON)
.map_err(IntoBoxError::into_box_error)?;
let Some(is_poisoned) = res else {
#[rustfmt::skip]
return Err(BoxError::new(ErrorCode::StorageCorrupted, "invalid contents has _pico_service_route").into());
};
if is_poisoned {
#[rustfmt::skip]
return Err(BoxError::new(ErrorCode::ServicePoisoned, format!("service '{plugin}.{service}' is poisoned on {instance_name}")).into());
}
Ok(())
}
fn check_replicaset_is_not_expelled(
node: &Node,
uuid: &str,
maybe_tuple: Option<Tuple>,
) -> Result<(), Error> {
let tuple;
if let Some(t) = maybe_tuple {
tuple = t;
} else {
tuple = node.storage.replicasets.by_uuid_raw(uuid)?;
}
let check =
topology_ref.check_service_route(&plugin.name, &plugin.version, service, instance_name);
let state = tuple.field(Replicaset::FIELD_STATE)?;
let state: ReplicasetState = state.expect("replicaset should always have a state column");
if state == ReplicasetState::Expelled {
#[rustfmt::skip]
return Err(BoxError::new(ErrorCode::ReplicasetExpelled, format!("replicaset with id {uuid} was expelled")).into());
match check {
ServiceRouteCheck::Ok => {}
ServiceRouteCheck::RoutePoisoned => {
#[rustfmt::skip]
return Err(BoxError::new(ErrorCode::ServicePoisoned, format!("service '{plugin}.{service}' is poisoned on {instance_name}")).into());
}
ServiceRouteCheck::ServiceNotEnabled => {
#[rustfmt::skip]
return Err(BoxError::new(ErrorCode::ServiceNotStarted, format!("service '{plugin}.{service}' is not running on {instance_name}")).into());
}
}
Ok(())
......
......@@ -665,9 +665,12 @@ pub struct ServiceRouteKey<'a> {
/// Instance name.
pub instance_name: &'a InstanceName,
}
impl<'a> Encode for ServiceRouteKey<'a> {}
////////////////////////////////////////////////////////////////////////////////
// PluginMigrationRecord
////////////////////////////////////////////////////////////////////////////////
/// Single record in _pico_plugin_migration system table.
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
pub struct PluginMigrationRecord {
......@@ -708,6 +711,10 @@ impl PluginMigrationRecord {
}
}
////////////////////////////////////////////////////////////////////////////////
// PluginConfigRecord
////////////////////////////////////////////////////////////////////////////////
/// Single record in _pico_plugin_config system table.
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)]
pub struct PluginConfigRecord {
......
......@@ -2768,6 +2768,15 @@ impl ServiceRouteTable {
}
}
impl ToEntryIter<MP_SERDE> for ServiceRouteTable {
type Entry = ServiceRouteItem;
#[inline(always)]
fn index_iter(&self) -> tarantool::Result<IndexIterator> {
self.space.select(IteratorType::All, &())
}
}
////////////////////////////////////////////////////////////////////////////////
// PluginMigration
////////////////////////////////////////////////////////////////////////////////
......
This diff is collapsed.
......@@ -36,6 +36,7 @@ use crate::storage::{local_schema_version, set_local_schema_version};
use crate::storage::{Clusterwide, ClusterwideTable, PropertyName};
use crate::system_parameter_name;
use crate::tlog;
use crate::topology_cache::TopologyCache;
use crate::traft;
use crate::traft::error::Error;
use crate::traft::network::WorkerOptions;
......@@ -56,7 +57,6 @@ use ::raft::prelude as raft;
use ::raft::storage::Storage as _;
use ::raft::Error as RaftError;
use ::raft::StateRole as RaftStateRole;
use ::raft::StorageError;
use ::raft::INVALID_ID;
use ::tarantool::error::BoxError;
use ::tarantool::error::TarantoolErrorCode;
......@@ -153,6 +153,7 @@ pub struct Node {
node_impl: Rc<Mutex<NodeImpl>>,
pub(crate) storage: Clusterwide,
pub(crate) topology_cache: Rc<TopologyCache>,
pub(crate) raft_storage: RaftSpaceAccess,
pub(crate) main_loop: MainLoop,
pub(crate) governor_loop: governor::Loop,
......@@ -211,6 +212,7 @@ impl Node {
raft_storage.clone(),
plugin_manager.clone(),
)?;
let topology_cache = node_impl.topology_cache.clone();
let raft_id = node_impl.raft_id();
let status = node_impl.status.subscribe();
......@@ -240,6 +242,7 @@ impl Node {
),
pool,
node_impl,
topology_cache,
storage,
raft_storage,
status,
......@@ -447,6 +450,7 @@ pub(crate) struct NodeImpl {
pub read_state_wakers: HashMap<LogicalClock, oneshot::Sender<RaftIndex>>,
joint_state_latch: KVCell<RaftIndex, oneshot::Sender<Result<(), RaftError>>>,
storage: Clusterwide,
topology_cache: Rc<TopologyCache>,
raft_storage: RaftSpaceAccess,
pool: Rc<ConnectionPool>,
lc: LogicalClock,
......@@ -462,15 +466,12 @@ impl NodeImpl {
storage: Clusterwide,
raft_storage: RaftSpaceAccess,
plugin_manager: Rc<PluginManager>,
) -> Result<Self, RaftError> {
let box_err = |e| StorageError::Other(Box::new(e));
) -> traft::Result<Self> {
let raft_id: RaftId = raft_storage
.raft_id()
.map_err(box_err)?
.raft_id()?
.expect("raft_id should be set by the time the node is being initialized");
let applied: RaftIndex = raft_storage.applied().map_err(box_err)?;
let term: RaftTerm = raft_storage.term().map_err(box_err)?;
let applied: RaftIndex = raft_storage.applied()?;
let term: RaftTerm = raft_storage.term()?;
let lc = {
let gen = raft_storage.gen().unwrap() + 1;
raft_storage.persist_gen(gen).unwrap();
......@@ -498,11 +499,14 @@ impl NodeImpl {
});
let (applied, _) = watch::channel(applied);
let topology_cache = Rc::new(TopologyCache::load(&storage, raft_id)?);
Ok(Self {
raw_node,
read_state_wakers: Default::default(),
joint_state_latch: KVCell::new(),
storage,
topology_cache,
raft_storage,
instance_reachability: pool.instance_reachability.clone(),
pool,
......@@ -789,7 +793,13 @@ impl NodeImpl {
//
// TODO: merge this into `do_dml` once `box_tuple_extract_key` is fixed.
let old = match space {
Ok(s @ (ClusterwideTable::Property | ClusterwideTable::Instance)) => {
Ok(
s @ (ClusterwideTable::Property
| ClusterwideTable::Instance
| ClusterwideTable::Replicaset
| ClusterwideTable::Tier
| ClusterwideTable::ServiceRouteTable),
) => {
let s = space_by_id(s.id()).expect("system space must exist");
match &op {
// There may be no previous version for inserts.
......@@ -828,31 +838,68 @@ impl NodeImpl {
return true;
};
let initiator = op.initiator();
let initiator_def = user_by_id(initiator).expect("user must exist");
// FIXME: all of this should be done only after the transaction is committed
// See <https://git.picodata.io/core/picodata/-/issues/1149>
if let Some(new) = &new {
// Dml::Delete mandates that new tuple is None.
assert!(!matches!(op, Dml::Delete { .. }));
match space {
ClusterwideTable::Instance => {
let old = old
.as_ref()
.map(|x| x.decode().expect("schema upgrade not supported yet"));
let new = new
.as_ref()
.map(|x| x.decode().expect("format was already verified"));
// Handle insert, replace, update in _pico_instance
if space == ClusterwideTable::Instance {
let old: Option<Instance> =
old.as_ref().map(|x| x.decode().expect("must be Instance"));
// Handle insert, replace, update in _pico_instance
if let Some(new) = &new {
// Dml::Delete mandates that new tuple is None.
assert!(!matches!(op, Dml::Delete { .. }));
let new: Instance = new
.decode()
.expect("tuple already passed format verification");
let initiator = op.initiator();
let initiator_def = user_by_id(initiator).expect("user must exist");
do_audit_logging_for_instance_update(old.as_ref(), &new, &initiator_def);
do_audit_logging_for_instance_update(old.as_ref(), new, &initiator_def);
if has_states!(new, Expelled -> *) && new.raft_id == self.raft_id() {
// cannot exit during a transaction
*expelled = true;
if has_states!(new, Expelled -> *) && new.raft_id == self.raft_id() {
// cannot exit during a transaction
*expelled = true;
}
}
self.topology_cache.update_instance(old, new);
}
ClusterwideTable::Replicaset => {
let old = old
.as_ref()
.map(|x| x.decode().expect("schema upgrade not supported yet"));
let new = new
.as_ref()
.map(|x| x.decode().expect("format was already verified"));
self.topology_cache.update_replicaset(old, new);
}
ClusterwideTable::Tier => {
let old = old
.as_ref()
.map(|x| x.decode().expect("schema upgrade not supported yet"));
let new = new
.as_ref()
.map(|x| x.decode().expect("format was already verified"));
self.topology_cache.update_tier(old, new);
}
ClusterwideTable::ServiceRouteTable => {
let old = old
.as_ref()
.map(|x| x.decode().expect("schema upgrade not supported yet"));
let new = new
.as_ref()
.map(|x| x.decode().expect("format was already verified"));
self.topology_cache.update_service_route(old, new);
}
_ => {}
}
true
......@@ -2129,6 +2176,7 @@ impl NodeImpl {
if hard_state.is_some() || !entries_to_persist.is_empty() || snapshot_data.is_some() {
let mut new_term = None;
let mut new_applied = None;
let mut received_snapshot = false;
if let Err(e) = transaction(|| -> Result<(), Error> {
self.main_loop_status("persisting hard state, entries and/or snapshot");
......@@ -2170,6 +2218,7 @@ impl NodeImpl {
self.storage
.apply_snapshot_data(&snapshot_data, is_master)?;
new_applied = Some(meta.index);
received_snapshot = true;
}
// TODO: As long as the snapshot was sent to us in response to
......@@ -2201,6 +2250,19 @@ impl NodeImpl {
.expect("applied shouldn't ever be borrowed across yields");
}
if received_snapshot {
// Need to reload the whole topology cache. We could be more
// clever about it and only update the records which changed,
// but at the moment this would require doing a full scan and
// comparing each record, which is no better than full reload.
// A better solution would be to store a raft index in each
// tuple of each global table, then we would only need to check
// if the index changed, but we don't have that..
self.topology_cache
.full_reload(&self.storage)
.expect("schema upgrade not supported yet");
}
if hard_state.is_some() {
crate::error_injection!(exit "EXIT_AFTER_RAFT_PERSISTS_HARD_STATE");
}
......
......@@ -17,10 +17,14 @@ use std::collections::HashMap;
use tarantool::space::SpaceId;
use tarantool::tlua;
/// This function **never yields**
pub fn get_replicaset_priority_list(
tier: &str,
replicaset_uuid: &str,
) -> Result<Vec<InstanceName>, Error> {
#[cfg(debug_assertions)]
let _guard = tarantool::fiber::safety::NoYieldsGuard::new();
let lua = tarantool::lua_state();
let pico: tlua::LuaTable<_> = lua
.get("pico")
......@@ -29,13 +33,20 @@ pub fn get_replicaset_priority_list(
let func: tlua::LuaFunction<_> = pico.try_get("_replicaset_priority_list")?;
let res = func.call_with_args((tier, replicaset_uuid));
if res.is_err() {
check_tier_exists(tier)?;
// Check if tier exists, return corresponding error in that case
node::global()
.expect("initilized by this point")
.topology_cache
.get()
.tier_by_name(tier)?;
}
res.map_err(|e| tlua::LuaError::from(e).into())
}
/// Returns the replicaset uuid and an array of replicas in descending priority
/// order.
///
/// This function **may yield** if vshard needs to update it's bucket mapping.
pub fn get_replicaset_uuid_by_bucket_id(tier: &str, bucket_id: u64) -> Result<String, Error> {
let max_bucket_id = PicodataConfig::total_bucket_count();
if bucket_id < 1 || bucket_id > max_bucket_id {
......@@ -51,27 +62,16 @@ pub fn get_replicaset_uuid_by_bucket_id(tier: &str, bucket_id: u64) -> Result<St
let func: tlua::LuaFunction<_> = pico.try_get("_replicaset_uuid_by_bucket_id")?;
let res = func.call_with_args((tier, bucket_id));
if res.is_err() {
check_tier_exists(tier)?;
// Check if tier exists, return corresponding error in that case
node::global()
.expect("initilized by this point")
.topology_cache
.get()
.tier_by_name(tier)?;
}
res.map_err(|e| tlua::LuaError::from(e).into())
}
#[inline(always)]
fn check_tier_exists(tier: &str) -> Result<()> {
// Check if tier exists, return corresponding error in that case
if node::global()
.expect("initilized by this point")
.storage
.tiers
.by_name(tier)?
.is_none()
{
return Err(Error::NoSuchTier(tier.into()));
}
Ok(())
}
#[rustfmt::skip]
#[derive(serde::Serialize, serde::Deserialize)]
#[derive(Default, Clone, Debug, PartialEq, tlua::PushInto, tlua::Push, tlua::LuaRead)]
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment