Newer
Older
//! This module incapsulates most of the application-specific logics.
//!
//! It's responsible for
//! - handling proposals,
//! - handling configuration changes,
//! - processing raft `Ready` - persisting entries, communicating with other raft nodes.

Georgy Moshkin
committed
use crate::governor;
use crate::kvcell::KVCell;
use crate::loop_start;
use crate::r#loop::FlowControl;
use crate::schema::{Distribution, IndexDef, SpaceDef};
use crate::storage::ddl_meta_drop_space;
use crate::storage::SnapshotData;
use crate::storage::{ddl_abort_on_master, ddl_meta_space_update_operable};
use crate::storage::{local_schema_version, set_local_schema_version};
use crate::storage::{Clusterwide, ClusterwideSpaceId, PropertyName};
use crate::stringify_cfunc;
use crate::traft::error::Error;
use crate::traft::event;
use crate::traft::event::Event;
use crate::traft::notify::{notification, Notifier, Notify};
use crate::traft::op::{Acl, Ddl, Dml, Op, OpResult};
use crate::traft::Address;
use crate::traft::RaftId;
use crate::traft::RaftIndex;
use crate::traft::RaftSpaceAccess;
use crate::util::AnyWithTypeName;
use crate::warn_or_panic;
use ::raft::prelude as raft;
use ::raft::Error as RaftError;
use ::raft::StateRole as RaftStateRole;
use ::raft::StorageError;
use ::raft::INVALID_ID;
use ::tarantool::fiber::mutex::MutexGuard;
use ::tarantool::fiber::r#async::timeout::IntoTimeout as _;
use ::tarantool::fiber::r#async::{oneshot, watch};
use ::tarantool::fiber::Mutex;
use ::tarantool::index::FieldType as IFT;
use ::tarantool::index::Part;
use ::tarantool::space::FieldType as SFT;
use ::tarantool::space::SpaceId;
use ::tarantool::time::Instant;
use ::tarantool::tuple::Decode;
use protobuf::Message as _;
use std::cell::Cell;
use std::collections::{HashMap, HashSet};
use std::convert::TryFrom;
use std::rc::Rc;
use std::time::Duration;
use ApplyEntryResult::*;
type RawNode = raft::RawNode<RaftSpaceAccess>;
::tarantool::define_str_enum! {
pub enum RaftState {
Follower = "Follower",
Candidate = "Candidate",
Leader = "Leader",
PreCandidate = "PreCandidate",
}
}
impl RaftState {
pub fn is_leader(&self) -> bool {
matches!(self, Self::Leader)
}
}
impl From<RaftStateRole> for RaftState {
fn from(role: RaftStateRole) -> Self {
match role {
RaftStateRole::Follower => Self::Follower,
RaftStateRole::Candidate => Self::Candidate,
RaftStateRole::Leader => Self::Leader,
RaftStateRole::PreCandidate => Self::PreCandidate,
}
}
}
#[derive(Copy, Clone, Debug, tlua::Push, tlua::PushInto)]
/// `raft_id` of the current instance
/// `raft_id` of the leader instance
pub leader_id: Option<RaftId>,
/// Current term number
pub term: RaftTerm,
/// Current raft state
pub raft_state: RaftState,
pub fn check_term(&self, requested_term: RaftTerm) -> traft::Result<()> {
if requested_term != self.term {
return Err(Error::TermMismatch {
requested: requested_term,
current: self.term,
});
}
Ok(())
}
}
type StorageWatchers = HashMap<SpaceId, watch::Sender<()>>;
type StorageChanges = HashSet<SpaceId>;
/// The heart of `traft` module - the Node.
/// RaftId of the Node.
//
// It appears twice in the Node: here and in `status.id`.
// This is a concious decision.
// `self.raft_id()` is used in Rust API, and
// `self.status()` is mostly useful in Lua API.

Georgy Moshkin
committed
pub(crate) raft_id: RaftId,
pub(crate) storage: Clusterwide,
pub(crate) raft_storage: RaftSpaceAccess,
pub(crate) main_loop: MainLoop,
pub(crate) governor_loop: governor::Loop,
status: watch::Receiver<Status>,
watchers: Rc<Mutex<StorageWatchers>>,
topology: Rc<RefCell<Topology>>,
impl std::fmt::Debug for Node {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.debug_struct("Node")
.field("raft_id", &self.raft_id)
.finish_non_exhaustive()
}
/// Initialize the raft node.
/// **This function yields**
pub fn new(storage: Clusterwide, raft_storage: RaftSpaceAccess) -> Result<Self, RaftError> {
let topology = Rc::new(RefCell::new(Topology::from(storage.clone())));
let node_impl = NodeImpl::new(storage.clone(), raft_storage.clone(), topology.clone())?;
let raft_id = node_impl.raft_id();
let status = node_impl.status.subscribe();
let node_impl = Rc::new(Mutex::new(node_impl));
let watchers = Rc::new(Mutex::new(HashMap::new()));
main_loop: MainLoop::start(node_impl.clone(), watchers.clone()), // yields
governor_loop: governor::Loop::start(

Georgy Moshkin
committed
status.clone(),
storage.clone(),
raft_storage.clone(),
),
storage,
raft_storage,
watchers,
};
// Wait for the node to enter the main loop
node.tick_and_yield(0);
pub fn raft_id(&self) -> RaftId {
pub(crate) fn node_impl(&self) -> MutexGuard<NodeImpl> {
self.node_impl.lock()
}
/// Wait for the status to be changed.
/// **This function yields**
fiber::block_on(self.status.clone().changed()).unwrap();
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
/// Returns current applied [`RaftIndex`].
pub fn get_index(&self) -> RaftIndex {
self.raft_storage
.applied()
.expect("reading from memtx should never fail")
}
/// Performs the quorum read operation.
///
/// If works the following way:
///
/// 1. The instance forwards a request (`MsgReadIndex`) to a raft
/// leader. In case there's no leader at the moment, the function
/// returns `Err(ProposalDropped)`.
/// 2. Raft leader tracks its `commit_index` and broadcasts a
/// heartbeat to followers to make certain that it's still a
/// leader.
/// 3. As soon as the heartbeat is acknowlenged by the quorum, the
/// function returns that index.
/// 4. The instance awaits when the index is applied. If timeout
/// expires beforehand, the function returns `Err(Timeout)`.
///
/// Returns current applied [`RaftIndex`].
///
/// **This function yields**
pub fn read_index(&self, timeout: Duration) -> traft::Result<RaftIndex> {
let deadline = fiber::clock().saturating_add(timeout);
let notify = self.raw_operation(|node_impl| node_impl.read_index_async())?;
let index: RaftIndex = fiber::block_on(notify.recv_timeout(timeout))?;
self.wait_index(index, deadline.duration_since(fiber::clock()))
}
/// Waits for [`RaftIndex`] to be applied to the storage locally.
///
/// Returns current applied [`RaftIndex`]. It can be equal to or
/// greater than the target one. If timeout expires beforehand, the
/// function returns `Err(Timeout)`.
///
/// **This function yields**
#[inline]
pub fn wait_index(&self, target: RaftIndex, timeout: Duration) -> traft::Result<RaftIndex> {
let deadline = fiber::clock().saturating_add(timeout);
loop {
let current = self.get_index();
if current >= target {
return Ok(current);
}
if event::wait_deadline(event::Event::EntryApplied, deadline)?.is_timeout() {
return Err(Error::Timeout);
}
}
/// Propose an operation and wait for it's result.
/// **This function yields**
pub fn propose_and_wait<T: OpResult + Into<Op>>(
&self,
op: T,
timeout: Duration,
) -> traft::Result<T::Result> {
let notify = self.raw_operation(|node_impl| node_impl.propose_async(op))?;
fiber::block_on(notify.recv_timeout::<T::Result>(timeout))
/// Become a candidate and wait for a main loop round so that there's a
/// chance we become the leader.
/// **This function yields**
pub fn campaign_and_yield(&self) -> traft::Result<()> {
self.raw_operation(|node_impl| node_impl.campaign())?;
// Even though we don't expect a response, we still should let the
// main_loop do an iteration. Without rescheduling, the Ready state
// wouldn't be processed, the Status wouldn't be updated, and some
// assertions may fail (e.g. in `postjoin()` in `main.rs`).
fiber::reschedule();
Ok(())
/// **This function yields**
pub fn step_and_yield(&self, msg: raft::Message) {
self.raw_operation(|node_impl| node_impl.step(msg))
.map_err(|e| tlog!(Error, "{e}"))
.ok();
// even though we don't expect a response, we still should let the
// main_loop do an iteration
fiber::reschedule();
/// **This function yields**
pub fn tick_and_yield(&self, n_times: u32) {
self.raw_operation(|node_impl| node_impl.tick(n_times));
// even though we don't expect a response, we still should let the
// main_loop do an iteration
fiber::reschedule();
/// **This function yields**
let raft_id = self.raft_id();
self.step_and_yield(raft::Message {
to: raft_id,
from: raft_id,
msg_type: raft::MessageType::MsgTimeoutNow,
..Default::default()
})
/// Processes the [`rpc::join::Request`] and appends necessary
/// entries to the raft log (if successful).
/// Returns the resulting [`Instance`] when the entry is committed.
// TODO: to make this function async and have an outer timeout,
// wait_* fns also need to be async.
pub fn handle_join_request_and_wait(
&self,
) -> traft::Result<(Box<Instance>, HashSet<Address>)> {
let deadline = fiber::clock().saturating_add(timeout);
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
loop {
let instance = self
.topology
.borrow()
.build_instance(
req.instance_id.as_ref(),
req.replicaset_id.as_ref(),
&req.failure_domain,
)
.map_err(RaftError::ConfChangeError)?;
let mut replication_addresses = self.storage.peer_addresses.addresses_by_ids(
self.topology
.borrow()
.get_replication_ids(&instance.replicaset_id),
)?;
replication_addresses.insert(req.advertise_address.clone());
let peer_address = traft::PeerAddress {
raft_id: instance.raft_id,
address: req.advertise_address.clone(),
};
let op_addr = Dml::replace(ClusterwideSpaceId::Address, &peer_address)
.expect("encoding should not fail");
let op_instance = Dml::replace(ClusterwideSpaceId::Instance, &instance)
.expect("encoding should not fail");
let ranges = vec![
cas::Range::new(ClusterwideSpaceId::Instance),
cas::Range::new(ClusterwideSpaceId::Address),
cas::Range::new(ClusterwideSpaceId::Property)
.eq((PropertyName::ReplicationFactor,)),
];
macro_rules! handle_result {
($res:expr) => {
match $res {
Ok((index, term)) => {
self.wait_index(index, deadline.duration_since(fiber::clock()))?;
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
if term != raft::Storage::term(&self.raft_storage, index)? {
// leader switched - retry
self.wait_status();
continue;
}
}
Err(err) => {
if err.is_cas_err() | err.is_term_mismatch_err() {
// cas error - retry
fiber::sleep(Duration::from_millis(500));
continue;
} else {
return Err(err);
}
}
}
};
}
// Only in this order - so that when instance exists - address will always be there.
handle_result!(cas::compare_and_swap(
Op::Dml(op_addr),
cas::Predicate {
index: self.raft_storage.applied()?,
term: self.raft_storage.term()?,
ranges: ranges.clone(),
},
deadline.duration_since(fiber::clock()),
));
handle_result!(cas::compare_and_swap(
Op::Dml(op_instance),
cas::Predicate {
index: self.raft_storage.applied()?,
term: self.raft_storage.term()?,
ranges,
},
deadline.duration_since(fiber::clock()),
));
self.main_loop.wakeup();
return Ok((instance.into(), replication_addresses));
}
/// Processes the [`rpc::update_instance::Request`] and appends
/// the corresponding [`Op::Dml`] entry to the raft log (if successful).
///
/// Returns `Ok(())` when the entry is committed.
///
/// **This function yields**
// TODO: for this function to be async and have an outer timeout wait_* fns need to be async
pub fn handle_update_instance_request_and_wait(
&self,
req: rpc::update_instance::Request,
) -> traft::Result<()> {
let deadline = fiber::clock().saturating_add(timeout);
loop {
let instance = self
.topology
.borrow()
.build_updated_instance(&req)
.map_err(RaftError::ConfChangeError)?;
let dml = Dml::replace(ClusterwideSpaceId::Instance, &instance)
.expect("encoding should not fail");
let ranges = vec![
cas::Range::new(ClusterwideSpaceId::Instance),
cas::Range::new(ClusterwideSpaceId::Address),
cas::Range::new(ClusterwideSpaceId::Property)
.eq((PropertyName::ReplicationFactor,)),
];
let res = cas::compare_and_swap(
Op::Dml(dml),
cas::Predicate {
index: self.raft_storage.applied()?,
term: self.raft_storage.term()?,
ranges,
},
deadline.duration_since(fiber::clock()),
);
match res {
Ok((index, term)) => {
self.wait_index(index, deadline.duration_since(fiber::clock()))?;
if term != raft::Storage::term(&self.raft_storage, index)? {
// leader switched - retry
self.wait_status();
continue;
}
}
Err(err) => {
if err.is_cas_err() | err.is_term_mismatch_err() {
// cas error - retry
fiber::sleep(Duration::from_millis(500));
continue;
} else {
return Err(err);
}
}
}
self.main_loop.wakeup();
return Ok(());
}
/// Only the conf_change_loop on a leader is eligible to call this function.
///
/// **This function yields**

Georgy Moshkin
committed
pub(crate) fn propose_conf_change_and_wait(
&self,
term: RaftTerm,
conf_change: raft::ConfChangeV2,
) -> traft::Result<()> {
let notify =
self.raw_operation(|node_impl| node_impl.propose_conf_change_async(term, conf_change))?;
fiber::block_on(notify).unwrap()?;
Ok(())
/// Attempt to transfer leadership to a given node and yield.
///
/// **This function yields**
pub fn transfer_leadership_and_yield(&self, new_leader_id: RaftId) {
self.raw_operation(|node_impl| node_impl.raw_node.transfer_leader(new_leader_id));
fiber::reschedule();
}
/// This function **may yield** if `self.node_impl` mutex is acquired.
#[inline]
fn raw_operation<R>(&self, f: impl FnOnce(&mut NodeImpl) -> R) -> R {
let mut node_impl = self.node_impl.lock();
res
#[inline]
pub fn all_traft_entries(&self) -> ::tarantool::Result<Vec<traft::Entry>> {
self.raft_storage.all_traft_entries()
}
/// Returns a watch which will be notified when a clusterwide space is
/// modified via the specified `index`.
///
/// You can also pass a [ClusterwideSpace](crate::storage::ClusterwideSpace) in which case the space's
/// primary index will be used.
#[inline(always)]
pub fn storage_watcher(&self, space: impl Into<SpaceId>) -> watch::Receiver<()> {
use std::collections::hash_map::Entry;
let mut watchers = self.watchers.lock();
match watchers.entry(space.into()) {
Entry::Vacant(entry) => {
let (tx, rx) = watch::channel(());
entry.insert(tx);
rx
}
Entry::Occupied(entry) => entry.get().subscribe(),
}
}
pub raw_node: RawNode,
pub notifications: HashMap<LogicalClock, Notifier>,
topology: Rc<RefCell<Topology>>,
joint_state_latch: KVCell<RaftIndex, oneshot::Sender<Result<(), RaftError>>>,
storage: Clusterwide,
raft_storage: RaftSpaceAccess,
status: watch::Sender<Status>,
fn new(
storage: Clusterwide,
raft_storage: RaftSpaceAccess,
topology: Rc<RefCell<Topology>>,
) -> Result<Self, RaftError> {
let box_err = |e| StorageError::Other(Box::new(e));
let raft_id: RaftId = raft_storage
.raft_id()
.map_err(box_err)?
.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 gen = raft_storage.gen().unwrap() + 1;
raft_storage.persist_gen(gen).unwrap();
LogicalClock::new(raft_id, gen)
};
let pool = ConnectionPool::builder(storage.clone())
.handler_name(stringify_cfunc!(proc_raft_interact))
.call_timeout(MainLoop::TICK.saturating_mul(4))
let cfg = raft::Config {
id: raft_id,
applied,
pre_vote: true,
..Default::default()
};
let raw_node = RawNode::new(&cfg, raft_storage.clone(), &tlog::root())?;
let (status, _) = watch::channel(Status {
id: raft_id,
leader_id: None,
term: traft::INIT_RAFT_TERM,
raft_state: RaftState::Follower,
});
Ok(Self {
raw_node,
notifications: Default::default(),
joint_state_latch: KVCell::new(),
raft_storage,
})
}
fn raft_id(&self) -> RaftId {
self.raw_node.raft.id
}
pub fn read_index_async(&mut self) -> Result<Notify, RaftError> {
// In some states `raft-rs` ignores the ReadIndex request.
// Check it preliminary, don't wait for the timeout.
//
// See for details:
// - <https://github.com/tikv/raft-rs/blob/v0.6.0/src/raft.rs#L2058>
// - <https://github.com/tikv/raft-rs/blob/v0.6.0/src/raft.rs#L2323>
let leader_doesnt_exist = self.raw_node.raft.leader_id == INVALID_ID;
let term_just_started = // ...
self.raw_node.raft.state == RaftStateRole::Leader
&& !self.raw_node.raft.commit_to_current_term();
if leader_doesnt_exist || term_just_started {
return Err(RaftError::ProposalDropped);
}
let (lc, notify) = self.schedule_notification();
// read_index puts this context into an Entry,
// so we've got to compose full EntryContext,
// despite single LogicalClock would be enough
let ctx = traft::EntryContextNormal::new(lc, Op::Nop);
self.raw_node.read_index(ctx.to_bytes());
Ok(notify)
}
/// **Doesn't yield**
#[inline]
pub fn propose_async<T>(&mut self, op: T) -> Result<Notify, RaftError>
where
{
let (lc, notify) = self.schedule_notification();
let ctx = traft::EntryContextNormal::new(lc, op.into());
self.raw_node.propose(ctx.to_bytes(), vec![])?;
Ok(notify)
}
/// Proposes a raft entry to be appended to the log and returns raft index
/// at which it is expected to be committed unless it gets rejected.
///
/// **Doesn't yield**
pub fn propose(&mut self, op: Op) -> Result<RaftIndex, RaftError> {
self.lc.inc();
let ctx = traft::EntryContextNormal::new(self.lc, op);
self.raw_node.propose(ctx.to_bytes(), vec![])?;
let index = self.raw_node.raft.raft_log.last_index();
Ok(index)
}
pub fn campaign(&mut self) -> Result<(), RaftError> {
self.raw_node.campaign()
}
pub fn step(&mut self, msg: raft::Message) -> Result<(), RaftError> {
if msg.to != self.raft_id() {
return Ok(());
}
// TODO check it's not a MsgPropose with op::Dml for updating _pico_instance.
// TODO check it's not a MsgPropose with ConfChange.
self.raw_node.step(msg)
}
pub fn tick(&mut self, n_times: u32) {
for _ in 0..n_times {
self.raw_node.tick();
}
}
fn propose_conf_change_async(
&mut self,
term: RaftTerm,
conf_change: raft::ConfChangeV2,
) -> Result<oneshot::Receiver<Result<(), RaftError>>, RaftError> {
// In some states proposing a ConfChange is impossible.
// Check if there's a reason to reject it.
// Checking leadership is only needed for the
// correct latch management. It doesn't affect
// raft correctness. Checking the instance is a
// leader makes sure the proposed `ConfChange`
// is appended to the raft log immediately
// instead of sending `MsgPropose` over the
// network.
if self.raw_node.raft.state != RaftStateRole::Leader {
return Err(RaftError::ConfChangeError("not a leader".into()));
}
if term != self.raw_node.raft.term {
return Err(RaftError::ConfChangeError("raft term mismatch".into()));
}
// Without this check the node would silently ignore the conf change.
// See https://github.com/tikv/raft-rs/blob/v0.6.0/src/raft.rs#L2014-L2026
if self.raw_node.raft.has_pending_conf() {
return Err(RaftError::ConfChangeError(
"already has pending confchange".into(),
));
}
let prev_index = self.raw_node.raft.raft_log.last_index();
self.raw_node.propose_conf_change(vec![], conf_change)?;
// Ensure the ConfChange was actually appended to the log.
// Otherwise it's a problem: current instance isn't actually a
// leader (which is impossible in theory, but we're not sure in
// practice) and sent the message to the raft network. It may
// lead to an inconsistency.
let last_index = self.raw_node.raft.raft_log.last_index();
assert_eq!(last_index, prev_index + 1);
if !self.joint_state_latch.is_empty() {
warn_or_panic!("joint state latch is locked");
}
let (tx, rx) = oneshot::channel();
self.joint_state_latch.insert(last_index, tx);
event::broadcast(Event::JointStateEnter);
Ok(rx)
}
/// Is called during a transaction
fn handle_committed_entries(
&mut self,
entries: &[raft::Entry],
wake_governor: &mut bool,
expelled: &mut bool,
storage_changes: &mut StorageChanges,
) -> traft::Result<()> {
let mut entries = entries.iter().peekable();
while let Some(&entry) = entries.peek() {
let entry = match traft::Entry::try_from(entry) {
Ok(v) => v,
Err(e) => {
tlog!(Error, "abnormal entry: {e}"; "entry" => ?entry);
continue;
}
};
let mut apply_entry_result = EntryApplied;
let entry_index = entry.index;
match entry.entry_type {
raft::EntryType::EntryNormal => {
apply_entry_result = self.handle_committed_normal_entry(
entry,
wake_governor,
expelled,
storage_changes,
);
if apply_entry_result != EntryApplied {
return Ok(());
}
}
raft::EntryType::EntryConfChange | raft::EntryType::EntryConfChangeV2 => {
self.handle_committed_conf_change(entry)
}
let res = self.raft_storage.persist_applied(entry_index);
event::broadcast(Event::EntryApplied);
if let Err(e) = res {
tlog!(
Error,
"error persisting applied index: {e}";
"index" => entry_index
);
}
Ok(())
})?;
match apply_entry_result {
SleepAndRetry => {
let timeout = MainLoop::TICK * 4;
fiber::sleep(timeout);
continue;
}
EntryApplied => {
// Actually advance the iterator.
let _ = entries.next();
Ok(())
}
/// Is called during a transaction
fn handle_committed_normal_entry(
&mut self,
entry: traft::Entry,
wake_governor: &mut bool,
expelled: &mut bool,
storage_changes: &mut StorageChanges,
) -> ApplyEntryResult {
assert_eq!(entry.entry_type, raft::EntryType::EntryNormal);
let lc = entry.lc();
let index = entry.index;
let op = entry.into_op().unwrap_or(Op::Nop);
tlog!(Debug, "applying entry: {op}"; "index" => index);
let mut instance_update = None;
let mut old_instance = None;
Op::Dml(op) => {
if space == ClusterwideSpaceId::Property as SpaceId
|| space == ClusterwideSpaceId::Replicaset as SpaceId
*wake_governor = true;
} else if space == ClusterwideSpaceId::Instance as SpaceId {
*wake_governor = true;
let instance = match op {
Dml::Insert { tuple, .. } => Some(tuple),
Dml::Replace { tuple, .. } => Some(tuple),
Dml::Update { .. } => None,
Dml::Delete { .. } => None,
};
if let Some(instance) = instance {
let instance: Instance = rmp_serde::from_slice(instance.as_ref())
.expect("should be a valid instance tuple");
if has_grades!(instance, Expelled -> *)
&& instance.raft_id == self.raft_id()
{
// cannot exit during a transaction
*expelled = true;
}
if self
.storage
.instances
.contains(&instance.instance_id)
.expect("storage should not fail")
{
old_instance = Some(
self.storage
.instances
.get(&instance.instance_id)
.expect("storage should not fail"),
);
}
instance_update = Some(instance);
}
storage_changes.insert(space);
Op::DdlPrepare { .. } => {
*wake_governor = true;
}
let storage_properties = &self.storage.properties;
// apply the operation
let mut result = Box::new(()) as Box<dyn AnyWithTypeName>;
match op {
Op::Nop => {}
Op::Dml(op) => {
Dml::Insert { space, tuple } => self.storage.insert(*space, tuple).map(Some),
Dml::Replace { space, tuple } => self.storage.replace(*space, tuple).map(Some),
Dml::Update { space, key, ops } => self.storage.update(*space, key, ops),
Dml::Delete { space, key } => self.storage.delete(*space, key),
};
result = Box::new(res) as _;
}
Op::DdlPrepare {
ddl,
schema_version,
} => {
self.apply_op_ddl_prepare(ddl, schema_version)
.expect("storage error");
}
Op::DdlCommit => {
let v_local = local_schema_version().expect("storage error");
let v_pending = storage_properties
.pending_schema_version()
.expect("storage error")
.expect("granted we don't mess up log compaction, this should not be None");
let ddl = storage_properties
.pending_schema_change()
.expect("storage error")
.expect("granted we don't mess up log compaction, this should not be None");
// This instance is catching up to the cluster.
if v_local < v_pending {
// Master applies schema change at this point.
let resp = rpc::ddl_apply::apply_schema_change(
&self.storage,
&ddl,
)
.expect("storage error");
match resp {
rpc::ddl_apply::Response::Abort { reason } => {
tlog!(Warning, "failed applying committed ddl operation: {reason}";
"ddl" => ?ddl,
);
return SleepAndRetry;
}
rpc::ddl_apply::Response::Ok => {}
}
match ddl {
Ddl::CreateSpace { id, .. } => {
ddl_meta_space_update_operable(&self.storage, id, true)
.expect("storage shouldn't fail");
Ddl::DropSpace { id } => {
ddl_meta_drop_space(&self.storage, id).expect("storage shouldn't fail");
_ => {
todo!()
}
}
storage_properties
.delete(PropertyName::PendingSchemaChange)
.expect("storage error");
storage_properties
.delete(PropertyName::PendingSchemaVersion)
.expect("storage error");
storage_properties
.put(PropertyName::GlobalSchemaVersion, &v_pending)
.expect("storage error");
}
Op::DdlAbort => {
let v_local = local_schema_version().expect("storage error");
let v_pending: u64 = storage_properties
.pending_schema_version()
.expect("storage error")
.expect("granted we don't mess up log compaction, this should not be None");
let ddl = storage_properties
.pending_schema_change()
.expect("storage error")
.expect("granted we don't mess up log compaction, this should not be None");
// This condition means, schema versions must always increase
// even after an DdlAbort
if v_local == v_pending {
} else {
let v_global = storage_properties
.global_schema_version()
.expect("storage error");
ddl_abort_on_master(&ddl, v_global).expect("storage error");
match ddl {
Ddl::CreateSpace { id, .. } => {
ddl_meta_drop_space(&self.storage, id).expect("storage shouldn't fail");
Ddl::DropSpace { id } => {
ddl_meta_space_update_operable(&self.storage, id, true)
.expect("storage shouldn't fail");
_ => {
todo!()
}
}
storage_properties
.delete(PropertyName::PendingSchemaChange)
.expect("storage error");
storage_properties
.delete(PropertyName::PendingSchemaVersion)
.expect("storage error");
}
let v_local = local_schema_version().expect("storage error");
let v_pending = acl.schema_version();
if v_local < v_pending {
if self.is_readonly() {
// Wait for tarantool replication with master to progress.
return SleepAndRetry;
} else {
match &acl {
Acl::CreateUser { user_def } => {
acl::on_master_create_user(user_def)
.expect("creating user shouldn't fail");
}
acl::on_master_change_user_auth(*user_id, auth)
.expect("changing user auth shouldn't fail");
}