type: bool
level: advanced
desc: allow recursive quiesce across auth boundaries
- default: false
+ default: true
services:
- mds
flags:
in->snaprealm->adjust_parent();
else if (in->is_any_caps())
in->move_to_realm(inode->find_snaprealm());
+
+ bool is_quiesced = inode->is_quiesced();
+ if (is_quiesced) {
+ mdcache->add_quiesce(inode, in);
+ }
}
void CDir::unlink_inode(CDentry *dn, bool adjust_lru)
out << " " << in.filelock;
if (!in.xattrlock.is_sync_and_unlocked())
out << " " << in.xattrlock;
- if (!in.versionlock.is_sync_and_unlocked())
+ if (in.versionlock.is_locked())
out << " " << in.versionlock;
- if (!in.quiescelock.is_sync_and_unlocked())
+ if (in.quiescelock.is_locked())
out << " " << in.quiescelock;
// hack: spit out crap on which clients have caps
const static int lock_types[] = {
CEPH_LOCK_IVERSION, CEPH_LOCK_IFILE, CEPH_LOCK_IAUTH, CEPH_LOCK_ILINK, CEPH_LOCK_IDFT,
CEPH_LOCK_IXATTR, CEPH_LOCK_ISNAP, CEPH_LOCK_INEST, CEPH_LOCK_IFLOCK, CEPH_LOCK_IPOLICY, 0
+ //TODO: add iquiesce here?
};
for (int i = 0; lock_types[i]; ++i) {
auto lock = get_lock(lock_types[i]);
}
}
+int CInode::get_caps_quiesce_mask() const
+{
+ if (is_quiesced()) {
+ // what we allow to our clients for a quiesced node
+ return CEPH_CAP_ANY_RD | CEPH_CAP_FILE_CACHE | CEPH_CAP_FILE_BUFFER | CEPH_CAP_PIN;
+ } else {
+ return CEPH_CAP_ANY;
+ }
+}
+
// caps allowed
int CInode::get_caps_liked() const
{
if (is_dir())
- return CEPH_CAP_PIN | CEPH_CAP_ANY_EXCL | CEPH_CAP_ANY_SHARED; // but not, say, FILE_RD|WR|WRBUFFER
+ return get_caps_quiesce_mask() & (CEPH_CAP_PIN | CEPH_CAP_ANY_EXCL | CEPH_CAP_ANY_SHARED); // but not, say, FILE_RD|WR|WRBUFFER
else
- return CEPH_CAP_ANY & ~CEPH_CAP_FILE_LAZYIO;
+ return get_caps_quiesce_mask() & (CEPH_CAP_ANY & ~CEPH_CAP_FILE_LAZYIO);
}
int CInode::get_caps_allowed_ever() const
int CInode::get_caps_allowed_by_type(int type) const
{
- return
+ return get_caps_quiesce_mask() & (
CEPH_CAP_PIN |
(filelock.gcaps_allowed(type) << filelock.get_cap_shift()) |
(authlock.gcaps_allowed(type) << authlock.get_cap_shift()) |
(xattrlock.gcaps_allowed(type) << xattrlock.get_cap_shift()) |
- (linklock.gcaps_allowed(type) << linklock.get_cap_shift());
+ (linklock.gcaps_allowed(type) << linklock.get_cap_shift())
+ );
}
int CInode::get_caps_careful() const
{
- return
+ return get_caps_quiesce_mask() & (
(filelock.gcaps_careful() << filelock.get_cap_shift()) |
(authlock.gcaps_careful() << authlock.get_cap_shift()) |
(xattrlock.gcaps_careful() << xattrlock.get_cap_shift()) |
- (linklock.gcaps_careful() << linklock.get_cap_shift());
+ (linklock.gcaps_careful() << linklock.get_cap_shift())
+ );
}
int CInode::get_xlocker_mask(client_t client) const
{
- return
+ return get_caps_quiesce_mask() & (
(filelock.gcaps_xlocker_mask(client) << filelock.get_cap_shift()) |
(authlock.gcaps_xlocker_mask(client) << authlock.get_cap_shift()) |
(xattrlock.gcaps_xlocker_mask(client) << xattrlock.get_cap_shift()) |
- (linklock.gcaps_xlocker_mask(client) << linklock.get_cap_shift());
+ (linklock.gcaps_xlocker_mask(client) << linklock.get_cap_shift())
+ );
}
int CInode::get_caps_allowed_for_client(Session *session, Capability *cap,
other |= p.second;
//cout << " get_caps_wanted mds " << it->first << " " << cap_string(it->second) << endl;
}
+
+ // we adjust wanted caps to prevent unnecessary lock transitions
+ // don't worry, when the quiesce lock is dropped
+ // the whole thing will get evaluated again, with a fixed mask
+ loner &= get_caps_quiesce_mask();
+ other &= get_caps_quiesce_mask();
+ w &= get_caps_quiesce_mask();
+
if (ploner) *ploner = (loner >> shift) & mask;
if (pother) *pother = (other >> shift) & mask;
return (w >> shift) & mask;
snaplock.mark_need_recover();
flocklock.mark_need_recover();
policylock.mark_need_recover();
- quiescelock.mark_need_recover();
}
DECODE_FINISH(p);
}
bool is_file() const { return get_inode()->is_file(); }
bool is_symlink() const { return get_inode()->is_symlink(); }
bool is_dir() const { return get_inode()->is_dir(); }
+ bool is_quiesced() const { return quiescelock.is_xlocked(); }
bool is_head() const { return last == CEPH_NOSNAP; }
int count_nonstale_caps();
bool multiple_nonstale_caps();
+ int get_caps_quiesce_mask() const;
+
bool is_any_caps() { return !client_caps.empty(); }
bool is_any_nonstale_caps() { return count_nonstale_caps(); }
* quiescelock.
*/
- SimpleLock quiescelock; // FIXME not part of mempool
+ LocalLockC quiescelock; // FIXME not part of mempool
LocalLockC versionlock; // FIXME not part of mempool
SimpleLock authlock; // FIXME not part of mempool
SimpleLock linklock; // FIXME not part of mempool
case CEPH_LOCK_IQUIESCE:
break;
default:
- CInode *in = static_cast<CInode*>(object);
- if (need_quiescelock) {
+ CInode* in = static_cast<CInode*>(object);
+ if (need_quiescelock && (lock->get_cap_shift() > 0)) {
+ dout(20) << "need shared quiesce lock for " << p << " on " << SimpleLock::get_lock_type_name(t) << " of " << in << dendl;
need_quiescelock = false;
- lov.add_rdlock(&in->quiescelock, i + 1);
+ CInode *in = static_cast<CInode*>(object);
+ lov.add_wrlock(&in->quiescelock, i + 1);
}
if (!in->is_auth())
continue;
<< " in case we need to request a scatter" << dendl;
mustpin.insert(object);
}
- if (need_quiescelock && is_inode_lock(t) && t != CEPH_LOCK_IQUIESCE) {
- CInode *in = static_cast<CInode*>(object);
- lov.add_rdlock(&in->quiescelock, i + 1);
+ if (need_quiescelock && (lock->get_cap_shift() > 0)) {
+ dout(20) << "need shared quiesce lock for " << p << " on " << SimpleLock::get_lock_type_name(t) << " of " << object << dendl;
need_quiescelock = false;
+ CInode *in = static_cast<CInode*>(object);
+ lov.add_wrlock(&in->quiescelock, i + 1);
}
} else if (p.is_remote_wrlock()) {
dout(20) << " must remote_wrlock on mds." << p.wrlock_target << " "
<< *lock << " " << *object << dendl;
mustpin.insert(object);
- if (need_quiescelock && is_inode_lock(t) && t != CEPH_LOCK_IQUIESCE) {
- CInode *in = static_cast<CInode*>(object);
- lov.add_rdlock(&in->quiescelock, i + 1);
+ if (need_quiescelock && (lock->get_cap_shift() > 0)) {
+ dout(20) << "need shared quiesce lock for " << p << " on " << SimpleLock::get_lock_type_name(t) << " of " << object << dendl;
need_quiescelock = false;
+ CInode *in = static_cast<CInode*>(object);
+ lov.add_wrlock(&in->quiescelock, i + 1);
}
} else if (p.is_rdlock()) {
dout(20) << " must rdlock " << *lock << " " << *object << dendl;
<< " in case we need to request a rdlock" << dendl;
mustpin.insert(object);
}
-
- /* We treat rdlocks differently when adding the quiescelock. If the lock
- * can be acquired immediately for reading without waiting
- * (SimpleLock::can_rdlock), then skip adding the quiescelock. This is to
- * allow some rdonly operations (like lookup) to proceed without blocking
- * on the exclusively locked quiescelock. This is safe from deadlock (due
- * to lock ordering) when Locker::acquire_locks is called more than once
- * with different LockOpVectors for a given inode (already a dangerous
- * thing to do) where there may be a wrlock/xlock in one set but not the
- * other. The reason is simple: if Locker::acquire_locks ever adds the
- * quiescelock, it is always the first lock to be acquired, and if it is
- * xlocked, then all locks are dropped (s.f.
- * Locker::handle_quiesce_failure). So adding the quiescelock can never
- * contribute to deadlock.
- */
-
- if (need_quiescelock && !mdr->is_rdlocked(lock)) {
- /* Can we get the lock without waiting? */
- if (!lock->can_rdlock(client)) {
- /* To prevent deadlock where an op holds a parent snaplock
- * (Locker::try_rdlock_snap_layout), add quiescelock.
- */
- CInode *in = static_cast<CInode*>(object);
- lov.add_rdlock(&in->quiescelock, i + 1);
- need_quiescelock = false;
- }
- }
} else {
ceph_assert(0 == "locker unknown lock operation");
}
if (t == CEPH_LOCK_IQUIESCE) {
handle_quiesce_failure(mdr, marker.message);
} else {
- marker.message = "failed to xlock, waiting";
+ marker.message = "failed to xlock, waiting";
}
goto out;
}
cancel_locking(mdr.get(), &issue_set);
marker.message = "waiting for remote wrlocks";
remote_wrlock_start(lock, p.wrlock_target, mdr);
+ marker.message = "failed to wrlock, dropping remote wrlock and waiting";
goto out;
}
}
client_t _client = p.is_state_pin() ? lock->get_excl_client() : client;
if (p.is_remote_wrlock()) {
// nowait if we have already gotten remote wrlock
+ ceph_assert(t != CEPH_LOCK_IQUIESCE);
if (!wrlock_try(lock, mdr, _client)) {
// can't take the wrlock because the scatter lock is gathering. need to
// release the remote wrlock, so that the gathering process can finish.
ceph_assert(it != mdr->locks.end());
remote_wrlock_finish(it, mdr.get());
remote_wrlock_start(lock, p.wrlock_target, mdr);
- if (t == CEPH_LOCK_IQUIESCE) {
- handle_quiesce_failure(mdr, marker.message);
- } else {
- marker.message = "failed to wrlock, dropping remote wrlock and waiting";
- }
goto out;
}
} else {
}
if (!rdlock_start(lock, mdr)) {
- if (t == CEPH_LOCK_IQUIESCE) {
- handle_quiesce_failure(mdr, marker.message);
- } else {
- marker.message = "failed to rdlock, waiting";
- }
+ ceph_assert(t != CEPH_LOCK_IQUIESCE); // rdlock is undefined for LocalLock
+ marker.message = "failed to rdlock, waiting";
goto out;
}
dout(10) << " got rdlock on " << *lock << " " << *lock->get_parent() << dendl;
issue_caps_set(need_issue);
}
-void Locker::drop_rdlock(MutationImpl* mut, SimpleLock* what)
+void Locker::drop_lock(MutationImpl* mut, SimpleLock* what)
{
dout(20) << __func__ << ": " << *what << dendl;
auto* lock = it->lock;
if (lock == what) {
dout(20) << __func__ << ": found lock " << *lock << dendl;
- ceph_assert(it->is_rdlock());
bool ni = false;
- rdlock_finish(it, mut, &ni);
+ if (it->is_xlock()) {
+ xlock_finish(it, mut, &ni);
+ } else if (it->is_wrlock()) {
+ wrlock_finish(it, mut, &ni);
+ } else if (it->is_rdlock()) {
+ rdlock_finish(it, mut, &ni);
+ }
if (ni) {
set<CInode*> need_issue;
need_issue.insert(static_cast<CInode*>(lock->get_parent()));
eval_any(&in->flocklock, &need_issue, &finishers, caps_imported);
if (mask & CEPH_LOCK_IPOLICY)
eval_any(&in->policylock, &need_issue, &finishers, caps_imported);
- if (mask & CEPH_LOCK_IQUIESCE)
- eval_any(&in->quiescelock, &need_issue, &finishers, caps_imported);
+ // LocalLocks should not be eval'd
// drop loner?
if (in->is_auth() && in->is_head() && in->get_wanted_loner() != in->get_loner()) {
switch (lock->get_type()) {
case CEPH_LOCK_DVERSION:
case CEPH_LOCK_IVERSION:
+ case CEPH_LOCK_IQUIESCE:
return local_wrlock_grab(static_cast<LocalLockC*>(lock), mut);
default:
break;
switch (lock->get_type()) {
case CEPH_LOCK_DVERSION:
case CEPH_LOCK_IVERSION:
+ case CEPH_LOCK_IQUIESCE:
return local_wrlock_start(static_cast<LocalLockC*>(lock), mut);
default:
break;
switch (lock->get_type()) {
case CEPH_LOCK_DVERSION:
case CEPH_LOCK_IVERSION:
+ case CEPH_LOCK_IQUIESCE:
return local_wrlock_finish(it, mut);
default:
break;
switch (lock->get_type()) {
case CEPH_LOCK_DVERSION:
case CEPH_LOCK_IVERSION:
+ case CEPH_LOCK_IQUIESCE:
return local_xlock_start(static_cast<LocalLockC*>(lock), mut);
default:
break;
switch (lock->get_type()) {
case CEPH_LOCK_DVERSION:
case CEPH_LOCK_IVERSION:
+ case CEPH_LOCK_IQUIESCE:
return local_xlock_finish(it, mut);
default:
break;
if (in->is_file()) {
bool forced_change_max = false;
dout(20) << "inode is file" << dendl;
- if (cap && ((cap->issued() | cap->wanted()) & CEPH_CAP_ANY_FILE_WR)) {
+ if (cap && ((cap->issued() | cap->wanted()) & CEPH_CAP_ANY_FILE_WR & in->get_caps_quiesce_mask())) {
dout(20) << "client has write caps; m->get_max_size="
<< m->get_max_size() << "; old_max=" << old_max << dendl;
if (m->get_max_size() > new_max) {
dout(7) << "local_wrlock_grab on " << *lock
<< " on " << *lock->get_parent() << dendl;
- ceph_assert(lock->get_parent()->is_auth());
ceph_assert(lock->can_wrlock());
lock->get_wrlock(mut->get_client());
dout(7) << "local_wrlock_start on " << *lock
<< " on " << *lock->get_parent() << dendl;
- ceph_assert(lock->get_parent()->is_auth());
if (lock->can_wrlock()) {
lock->get_wrlock(mut->get_client());
auto it = mut->emplace_lock(lock, MutationImpl::LockOp::WRLOCK);
dout(7) << "local_xlock_start on " << *lock
<< " on " << *lock->get_parent() << dendl;
- ceph_assert(lock->get_parent()->is_auth());
+ if (lock->is_cached()) {
+ invalidate_lock_caches(lock);
+ }
if (!lock->can_xlock_local()) {
lock->add_waiter(SimpleLock::WAIT_WR|SimpleLock::WAIT_STABLE, new C_MDS_RetryRequest(mdcache, mut));
return false;
lock->put_xlock();
mut->locks.erase(it);
- lock->finish_waiters(SimpleLock::WAIT_STABLE |
- SimpleLock::WAIT_WR |
+ lock->finish_waiters(SimpleLock::WAIT_STABLE |
+ SimpleLock::WAIT_WR |
SimpleLock::WAIT_RD);
+
+ /* Evaluate locks only after finishing waiters so blocked requests have a
+ * chance to acquire ifile/ixattr/etc. before issuing back caps.
+ */
+ if (lock->get_type() == CEPH_LOCK_IQUIESCE) {
+ auto in = static_cast<CInode*>(lock->get_parent());
+ // reevaluate everything related related to caps
+ eval(in, CEPH_LOCK_IFILE | CEPH_LOCK_IAUTH | CEPH_LOCK_ILINK | CEPH_LOCK_IXATTR, true);
+ }
}
void set_xlocks_done(MutationImpl *mut, bool skip_dentry=false);
void drop_non_rdlocks(MutationImpl *mut, std::set<CInode*> *pneed_issue=0);
void drop_rdlocks_for_early_reply(MutationImpl *mut);
- void drop_rdlock(MutationImpl* mut, SimpleLock* what);
+ void drop_lock(MutationImpl* mut, SimpleLock* what);
void drop_locks_for_fragment_unfreeze(MutationImpl *mut);
int get_cap_bit_for_lock_cache(int op);
#include <string>
#include <string_view>
#include <map>
+#include <memory>
#include "MDCache.h"
#include "MDSRank.h"
explicit MDCacheLogContext(MDCache *mdc_) : mdcache(mdc_) {}
};
+struct QuiesceInodeState {
+ MDRequestRef qrmdr;
+ std::shared_ptr<MDCache::QuiesceStatistics> qs;
+ std::chrono::milliseconds delay = 0ms;
+ bool splitauth = false;
+};
+using QuiesceInodeStateRef = std::shared_ptr<QuiesceInodeState>;
+
MDCache::MDCache(MDSRank *m, PurgeQueue &purge_queue_) :
mds(m),
open_file_table(m),
void MDCache::dispatch_request(const MDRequestRef& mdr)
{
+ if (mdr->dead) {
+ dout(20) << __func__ << ": dead " << *mdr << dendl;
+ return;
+ }
if (mdr->client_request) {
mds->server->dispatch_client_request(mdr);
} else if (mdr->peer_request) {
mdr->clear_ambiguous_auth();
if (!mdr->more()->waiting_for_finish.empty())
mds->queue_waiters(mdr->more()->waiting_for_finish);
+ uint64_t count = 0;
for (auto& [in, reqid] : mdr->more()->quiesce_ops) {
if (auto it = active_requests.find(reqid); it != active_requests.end()) {
auto qimdr = it->second;
dout(20) << "killing quiesce op " << *qimdr << dendl;
request_kill(qimdr);
+ if (!(++count % mds->heartbeat_reset_grace())) {
+ mds->heartbeat_reset();
+ }
}
}
}
- if (mdr->internal_op == CEPH_MDS_OP_QUIESCE_PATH) {
- /* This construction is obviously not performant but it's rarely done and only for subvolumes */
- for (auto it = quiesced_subvolumes.begin(); it != quiesced_subvolumes.end();) {
- if (it->second == mdr) {
- it = quiesced_subvolumes.erase(it);
- } else {
- ++it;
- }
+ switch(mdr->internal_op) {
+ case CEPH_MDS_OP_QUIESCE_INODE: {
+ auto* qisp = static_cast<QuiesceInodeStateRef*>(mdr->internal_op_private);
+ delete qisp;
+ mdr->internal_op_private = nullptr;
+ break;
}
+ default:
+ break;
}
request_drop_locks(mdr);
void MDCache::dispatch_quiesce_inode(const MDRequestRef& mdr)
{
- if (mdr->killed) {
- dout(20) << __func__ << " " << *mdr << " not dispatching killed " << *mdr << dendl;
- return;
- } else if (mdr->internal_op_finish == nullptr) {
+ if (mdr->internal_op_finish == nullptr) {
dout(20) << __func__ << " " << *mdr << " already finished quiesce" << dendl;
return;
}
- auto* qfinisher = static_cast<C_MDS_QuiescePath*>(mdr->internal_op_private);
- auto delay = qfinisher->delay;
- auto splitauth = qfinisher->splitauth;
- auto& qs = *qfinisher->qs;
- auto qrmdr = qfinisher->mdr;
+ auto& qis = *static_cast<QuiesceInodeStateRef*>(mdr->internal_op_private);
+ auto& qrmdr = qis->qrmdr;
+ auto& qops = qrmdr->more()->quiesce_ops;
+ auto& qs = *qis->qs;
+ auto& delay = qis->delay;
+ auto& splitauth = qis->splitauth;
CInode *in = get_inode(mdr->get_filepath().get_ino());
if (in == nullptr) {
- qs.add_failed(mdr, -CEPHFS_ENOENT);
- mds->server->respond_to_request(mdr, -CEPHFS_ENOENT);
+ /* It has been trimmed from cache before we could acquire locks/pins, complete quietly. */
+ qops.erase(in); // allow a future try if it comes back into cache
+ qs.inc_inodes_dropped();
+ mds->server->respond_to_request(mdr, 0);
return;
}
- const bool is_root = (mdr->get_filepath().get_ino() == mdr->get_filepath2().get_ino());
+
+ [[maybe_unused]] const bool is_root = (mdr->get_filepath().get_ino() == mdr->get_filepath2().get_ino());
dout(20) << __func__ << " " << *mdr << " quiescing " << *in << dendl;
}
quiesce_counter.hit();
- {
- /* Acquire authpins on `in` to prevent migrations after this rank considers
- * it (and its children) quiesced.
- */
-
- MutationImpl::LockOpVec lov;
- if (!mds->locker->acquire_locks(mdr, lov, nullptr, {in}, false, true)) {
- return;
- }
- }
-
- /* TODO: Consider:
+ /* Acquire cap-related locks. This pushes the lock state in the right
+ * direction.
+ *
+ * The quiescelock, once held exclusively, prevents future operations from
+ * acquiring the cap-related locks. It also prevents issuance of caps to
+ * clients.
*
- * rank0 is auth for /foo
- * rank1 quiesces /foo with no dirents in cache (and stops)
- * rank0 begins quiescing /foo
- * rank0 exports a dirfrag of /foo/bar to rank1 (/foo/bar is not authpinned by rank1 nor by rank0 (yet))
- * rank1 discovers relevant paths in /foo/bar
- * rank1 now has /foo/bar in cache and may issue caps / execute operations
+ * Because we're dropping the cap-related locks, other MDS can still complete
+ * operations requiring those locks or issue caps. This protocol necessarily
+ * requires that all ranks cooperatively quiesce the same tree. This is
+ * effected through the QuiesceDB.
*
- * The solution is probably to have rank1 mark /foo has STATE_QUIESCED and reject export ops from rank0.
+ * The xlock on the quiescelock is important to prevent future requests from
+ * blocking on other inode locks while holding path traversal locks. See dev
+ * doc doc/dev/mds_internals/quiesce.rst for more details.
*/
- if (in->is_auth()) {
- /* Acquire rdlocks on anything which prevents writing.
- *
- * Because files are treated specially allowing multiple reader/writers, we
- * need an xlock here to recall all write caps. This unfortunately means
- * there can be no readers.
- *
- * The xlock on the quiescelock is important to prevent future requests
- * from blocking on other inode locks while holding path traversal locks.
- * See dev doc doc/dev/mds_internals/quiesce.rst for more details.
- */
+ ceph_assert(in->is_auth() || splitauth);
+
+ if (in->is_quiesced()) {
+ auto qimdr = get_quiesce_inode_op(in);
+ if (qimdr != mdr) {
+ dout(5) << __func__ << ": already quiesced by " << *qimdr << dendl;
+ qs.add_failed(mdr, -CEPHFS_EINPROGRESS);
+ mds->server->respond_to_request(mdr, -CEPHFS_EINPROGRESS);
+ return;
+ }
+ }
+ if (!(mdr->locking_state & MutationImpl::ALL_LOCKED)) {
MutationImpl::LockOpVec lov;
lov.add_rdlock(&in->authlock);
- lov.add_rdlock(&in->dirfragtreelock);
lov.add_rdlock(&in->filelock);
lov.add_rdlock(&in->linklock);
- lov.add_rdlock(&in->nestlock);
- lov.add_rdlock(&in->policylock);
- // N.B.: NO xlock/wrlock on quiescelock; we need to allow access to mksnap/lookup
- // This is an unfortunate inconsistency. It may be possible to circumvent
- // this issue by having those ops acquire the quiscelock only if necessary.
- if (is_root) {
- lov.add_rdlock(&in->quiescelock);
- } else {
- lov.add_xlock(&in->quiescelock); /* !! */
- }
- lov.add_rdlock(&in->snaplock);
+ lov.add_xlock(&in->quiescelock); /* !! */
lov.add_rdlock(&in->xattrlock);
if (!mds->locker->acquire_locks(mdr, lov, nullptr, {in}, false, true)) {
return;
}
- } else if (!splitauth) {
- dout(5) << "auth is split and splitauth is false: " << *in << dendl;
- qs.add_failed(mdr, -CEPHFS_EPERM);
- mds->server->respond_to_request(mdr, -CEPHFS_EPERM);
- return;
- }
+ mdr->locking_state |= MutationImpl::ALL_LOCKED;
+
+ if (in->get_projected_inode()->get_quiesce_block()) {
+ dout(10) << __func__ << " quiesce is blocked for this inode; dropping locks!" << dendl;
+ mdr->mark_event("quiesce blocked");
+ mds->locker->drop_locks(mdr.get());
+ /* keep authpins! */
+ qs.inc_inodes_blocked();
+ auto* c = mdr->internal_op_finish;
+ mdr->internal_op_finish = nullptr; // prevent ::request_kill recursion
+ c->complete(0);
+ return;
+ }
- if (in->get_projected_inode()->get_quiesce_block()) {
- dout(10) << __func__ << " quiesce is blocked for this inode; dropping locks!" << dendl;
- mdr->mark_event("quiesce blocked");
- mds->locker->drop_locks(mdr.get());
- /* keep authpins! */
- qs.inc_inodes_blocked();
- mdr->internal_op_finish->complete(0);
- mdr->internal_op_finish = nullptr;
- return;
+ if (splitauth) {
+ /* Once we have the queiscelock, we no longer need these locks. However,
+ * if splitauth==false, the replicas do not try quiescing so we must keep
+ * them locked.
+ */
+ mds->locker->drop_lock(mdr.get(), &in->authlock);
+ mds->locker->drop_lock(mdr.get(), &in->filelock);
+ mds->locker->drop_lock(mdr.get(), &in->linklock);
+ mds->locker->drop_lock(mdr.get(), &in->xattrlock);
+ }
}
if (in->is_dir()) {
}
}
MDSGatherBuilder gather(g_ceph_context, new C_MDS_RetryRequest(this, mdr));
- auto& qops = qrmdr->more()->quiesce_ops;
for (auto& dir : in->get_dirfrags()) {
for (auto& [dnk, dn] : *dir) {
auto* in = dn->get_projected_inode();
if (auto it = qops.find(in); it != qops.end()) {
dout(25) << __func__ << ": existing quiesce metareqid: " << it->second << dendl;
- if (auto reqit = active_requests.find(it->second); reqit != active_requests.end()) {
- auto& qimdr = reqit->second;
- dout(25) << __func__ << ": found in-progress " << qimdr << dendl;
- continue;
- }
+ continue;
}
dout(10) << __func__ << ": scheduling op to quiesce " << *in << dendl;
MDRequestRef qimdr = request_start_internal(CEPH_MDS_OP_QUIESCE_INODE);
qimdr->set_filepath(filepath(in->ino()));
qimdr->internal_op_finish = gather.new_sub();
- qimdr->internal_op_private = qfinisher;
+ qimdr->internal_op_private = new QuiesceInodeStateRef(qis);
qops[in] = qimdr->reqid;
qs.inc_inodes();
if (delay > 0ms) {
}
qs.inc_inodes_quiesced();
- mdr->internal_op_finish->complete(0);
- mdr->internal_op_finish = nullptr;
+ auto* c = mdr->internal_op_finish;
+ mdr->internal_op_finish = nullptr; // prevent ::request_kill recursion
+ c->complete(0);
/* do not respond/complete so locks are not lost, parent request will complete */
}
-void MDCache::dispatch_quiesce_path(const MDRequestRef& mdr)
+void MDCache::add_quiesce(CInode* parent, CInode* in)
{
- if (mdr->killed) {
- dout(20) << __func__ << " not dispatching killed " << *mdr << dendl;
+ ceph_assert(parent->is_quiesced());
+ auto mdr = get_quiesce_inode_op(parent);
+
+ auto& qis = *static_cast<QuiesceInodeStateRef*>(mdr->internal_op_private);
+ auto& qrmdr = qis->qrmdr;
+ auto& qs = *qis->qs;
+ auto& qops = qrmdr->more()->quiesce_ops;
+
+ if (auto it = qops.find(in); it != qops.end()) {
+ dout(25) << __func__ << ": existing quiesce metareqid: " << it->second << dendl;
return;
}
+ dout(10) << __func__ << ": scheduling op to quiesce " << *in << dendl;
+ MDRequestRef qimdr = request_start_internal(CEPH_MDS_OP_QUIESCE_INODE);
+ qimdr->set_filepath(filepath(in->ino()));
+ qimdr->internal_op_finish = new LambdaContext([](int r) {});
+ qimdr->internal_op_private = new QuiesceInodeStateRef(qis);
+ qops[in] = qimdr->reqid;
+ qs.inc_inodes();
+ dispatch_request(qimdr);
+ if (!(qs.inc_heartbeat_count() % mds->heartbeat_reset_grace())) {
+ mds->heartbeat_reset();
+ }
+}
+
+void MDCache::dispatch_quiesce_path(const MDRequestRef& mdr)
+{
if (!mds->is_active()) {
dout(20) << __func__ << " is not active!" << dendl;
mds->server->respond_to_request(mdr, -CEPHFS_EAGAIN);
C_MDS_QuiescePath* qfinisher = static_cast<C_MDS_QuiescePath*>(mdr->internal_op_finish);
auto& qs = *qfinisher->qs;
- auto delay = qfinisher->delay = g_conf().get_val<std::chrono::milliseconds>("mds_cache_quiesce_delay");
- auto splitauth = qfinisher->splitauth = g_conf().get_val<bool>("mds_cache_quiesce_splitauth");
+ auto delay = g_conf().get_val<std::chrono::milliseconds>("mds_cache_quiesce_delay");
+ auto splitauth = g_conf().get_val<bool>("mds_cache_quiesce_splitauth");
+
+ QuiesceInodeStateRef qis = std::make_shared<QuiesceInodeState>();
+ *qis = {mdr, qfinisher->qs, delay, splitauth};
CInode* diri = nullptr;
CF_MDS_RetryRequestFactory cf(this, mdr, true);
return;
}
- if (auto [it, inserted] = quiesced_subvolumes.try_emplace(diri->ino(), mdr); !inserted) {
- if (!it->second) {
- it->second = mdr;
- } else if (it->second != mdr) {
- dout(5) << __func__ << ": quiesce operation already in flight: " << it->second << dendl;
- mds->server->respond_to_request(mdr, -CEPHFS_EINPROGRESS);
- return;
- }
- }
-
qfinisher->mdr = mdr;
- for (auto& [qimdr, rc] : qs.get_failed()) {
- dout(5) << __func__ << ": op " << *qimdr << " failed with " << rc << "!" << dendl;
- mds->server->respond_to_request(mdr, rc);
- return;
+ {
+ int myrc = 0;
+ for (auto& [qimdr, rc] : qs.get_failed()) {
+ dout(5) << __func__ << ": op " << *qimdr << " failed with " << rc << "!" << dendl;
+ myrc = rc;
+ }
+ if (myrc) {
+ mds->server->respond_to_request(mdr, myrc);
+ return;
+ }
}
if (!diri->is_auth() && !splitauth) {
qimdr->set_filepath(filepath(diri->ino()));
qimdr->set_filepath2(filepath(diri->ino())); /* is_root! */
qimdr->internal_op_finish = new C_MDS_RetryRequest(this, mdr);
- qimdr->internal_op_private = qfinisher;
+ qimdr->internal_op_private = new QuiesceInodeStateRef(qis);
qops[diri] = qimdr->reqid;
qs.inc_inodes();
if (delay > 0ms) {
}
if (qfinisher) {
- qfinisher->complete(0);
- mdr->internal_op_finish = nullptr;
+ auto* c = mdr->internal_op_finish;
+ mdr->internal_op_finish = nullptr; // prevent ::request_kill recursion
+ c->complete(0);
}
mdr->result = 0;
void inc_inodes_blocked() {
inodes_blocked++;
}
+ void inc_inodes_dropped() {
+ inodes_dropped++;
+ }
uint64_t get_inodes() const {
return inodes;
}
f->dump_unsigned("inodes", inodes);
f->dump_unsigned("inodes_quiesced", inodes_quiesced);
f->dump_unsigned("inodes_blocked", inodes_blocked);
+ f->dump_unsigned("inodes_dropped", inodes_dropped);
f->open_array_section("failed");
for (auto& [mdr, rc] : failed) {
f->open_object_section("failure");
uint64_t inodes = 0;
uint64_t inodes_quiesced = 0;
uint64_t inodes_blocked = 0;
+ uint64_t inodes_dropped = 0;
std::map<MDRequestRef, int> failed;
};
class C_MDS_QuiescePath : public MDSInternalContext {
}
}
std::shared_ptr<QuiesceStatistics> qs = std::make_shared<QuiesceStatistics>();
- std::chrono::milliseconds delay = 0ms;
- bool splitauth = false;
MDCache *cache;
MDRequestRef mdr;
Context* finisher = nullptr;
};
MDRequestRef quiesce_path(filepath p, C_MDS_QuiescePath* c, Formatter *f = nullptr, std::chrono::milliseconds delay = 0ms);
+ MDRequestRef get_quiesce_inode_op(CInode* in) {
+ if (in->is_quiesced()) {
+ auto mut = in->quiescelock.get_xlock_by();
+ ceph_assert(mut); /* that would be weird */
+ auto* mdr = dynamic_cast<MDRequestImpl*>(mut.get());
+ ceph_assert(mdr); /* also would be weird */
+ ceph_assert(mdr->internal_op == CEPH_MDS_OP_QUIESCE_INODE);
+ return MDRequestRef(mdr);
+ } else {
+ return MDRequestRef();
+ }
+ }
+ void add_quiesce(CInode* parent, CInode* in);
void clean_open_file_lists();
void dump_openfiles(Formatter *f);
uint64_t kill_shutdown_at = 0;
- std::map<inodeno_t, MDRequestRef> quiesced_subvolumes;
DecayCounter quiesce_counter;
uint64_t quiesce_threshold;
std::chrono::milliseconds quiesce_sleep;
/* always at the front */
seq = 0;
}
- waiting.insert(std::pair<waiter_seq_t, waiter>(seq, waiter(mask, c)));
+ waiting.insert(std::pair<waiter_seq_t, waiter>(seq, waiter{mask, c}));
}
virtual void take_waiting(uint64_t mask, MDSContext::vec& ls) {
take_waiting(waitmask_t(mask), ls);
DECODE_FINISH(blp);
+ // add inode?
+ if (added) {
+ mdcache->add_inode(in);
+ dout(10) << "added " << *in << dendl;
+ } else {
+ dout(10) << " had " << *in << dendl;
+ }
+
// link before state -- or not! -sage
if (dn->get_linkage()->get_inode() != in) {
ceph_assert(!dn->get_linkage()->get_inode());
if (in->is_dir())
dn->dir->pop_lru_subdirs.push_back(&in->item_pop_lru);
- // add inode?
- if (added) {
- mdcache->add_inode(in);
- dout(10) << "added " << *in << dendl;
- } else {
- dout(10) << " had " << *in << dendl;
- }
-
if (in->get_inode()->is_dirty_rstat())
in->mark_dirty_rstat();
break;
}
- // don't add quiescelock, let the peer acquire that rdlock themselves
+ // don't add quiescelock, let the peer acquire that lock themselves
if (!mds->locker->acquire_locks(mdr, lov, nullptr, {}, false, true))
return;
}
/* readdir can add dentries to cache: acquire the quiescelock */
- lov.add_rdlock(&diri->quiescelock);
lov.add_rdlock(&diri->filelock);
lov.add_rdlock(&diri->dirfragtreelock);
explicit LockType(int t) : type(t) {
switch (type) {
case CEPH_LOCK_DN:
- case CEPH_LOCK_IQUIESCE:
case CEPH_LOCK_IAUTH:
case CEPH_LOCK_ILINK:
case CEPH_LOCK_IXATTR:
break;
case CEPH_LOCK_DVERSION:
case CEPH_LOCK_IVERSION:
+ case CEPH_LOCK_IQUIESCE:
sm = &sm_locallock;
break;
default: