// for this. Shoudl just give objects a string name with a rank
// suffix, like we do for MDSTables. Requires a little refactor
// of Journaler.
+// TODO: if Objecter has any slow requests, take that as a hint and
+// slow down our rate of purging (keep accepting pushes though)
PurgeQueue::PurgeQueue(
CephContext *cct_,
mds_rank_t rank_,
// and passing the PurgeItem straight into _execute_item
}
+#if 0
+uint32_t StrayManager::_calculate_ops_required(CInode *in, bool trunc)
+{
+ uint32_t ops_required = 0;
+ if (in->is_dir()) {
+ // Directory, count dirfrags to be deleted
+ std::list<frag_t> ls;
+ if (!in->dirfragtree.is_leaf(frag_t())) {
+ in->dirfragtree.get_leaves(ls);
+ }
+ // One for the root, plus any leaves
+ ops_required = 1 + ls.size();
+ } else {
+ // File, work out concurrent Filer::purge deletes
+ const uint64_t to = MAX(in->inode.max_size_ever,
+ MAX(in->inode.size, in->inode.get_max_size()));
+
+ const uint64_t num = (to > 0) ? Striper::get_num_objects(in->inode.layout, to) : 1;
+ ops_required = MIN(num, g_conf->filer_max_purge_ops);
+
+ // Account for removing (or zeroing) backtrace
+ ops_required += 1;
+
+ // Account for deletions for old pools
+ if (!trunc) {
+ ops_required += in->get_projected_inode()->old_pools.size();
+ }
+ }
+
+ return ops_required;
+}
+#endif
+
bool PurgeQueue::can_consume()
{
+#if 0
+ // Calculate how much of the ops allowance is available, allowing
+ // for the case where the limit is currently being exceeded.
+ uint32_t ops_avail;
+ if (ops_in_flight <= max_purge_ops) {
+ ops_avail = max_purge_ops - ops_in_flight;
+ } else {
+ ops_avail = 0;
+ }
+
+ dout(10) << __func__ << ": allocating allowance "
+ << ops_required << " to " << ops_in_flight << " in flight" << dendl;
+
+ logger->set(l_mdc_num_purge_ops, ops_in_flight);
+#endif
+
// TODO: enforce limits (currently just allowing one in flight)
- if (in_flight.size() > 0) {
+ if (in_flight.size() > cct->_conf->mds_max_purge_files) {
return false;
} else {
return true;
in_flight.erase(iter);
+#if 0
+ // Release resources
+ dout(10) << __func__ << ": decrementing op allowance "
+ << ops_allowance << " from " << ops_in_flight << " in flight" << dendl;
+ assert(ops_in_flight >= ops_allowance);
+ ops_in_flight -= ops_allowance;
+ logger->set(l_mdc_num_purge_ops, ops_in_flight);
+ files_purging -= 1;
+#endif
+
_consume();
}
+void PurgeQueue::update_op_limit(const MDSMap &mds_map)
+{
+ Mutex::Locker l(lock);
+
+ uint64_t pg_count = 0;
+ objecter->with_osdmap([&](const OSDMap& o) {
+ // Number of PGs across all data pools
+ const std::set<int64_t> &data_pools = mds_map.get_data_pools();
+ for (const auto dp : data_pools) {
+ if (o.get_pg_pool(dp) == NULL) {
+ // It is possible that we have an older OSDMap than MDSMap,
+ // because we don't start watching every OSDMap until after
+ // MDSRank is initialized
+ dout(4) << " data pool " << dp << " not found in OSDMap" << dendl;
+ continue;
+ }
+ pg_count += o.get_pg_num(dp);
+ }
+ });
+
+ // Work out a limit based on n_pgs / n_mdss, multiplied by the user's
+ // preference for how many ops per PG
+ max_purge_ops = uint64_t(((double)pg_count / (double)mds_map.get_max_mds()) *
+ cct->_conf->mds_max_purge_ops_per_pg);
+
+ // User may also specify a hard limit, apply this if so.
+ if (cct->_conf->mds_max_purge_ops) {
+ max_purge_ops = MIN(max_purge_ops, cct->_conf->mds_max_purge_ops);
+ }
+}
+
+void PurgeQueue::handle_conf_change(const struct md_config_t *conf,
+ const std::set <std::string> &changed,
+ const MDSMap &mds_map)
+{
+ if (changed.count("mds_max_purge_ops")
+ || changed.count("mds_max_purge_ops_per_pg")) {
+ update_op_limit(mds_map);
+ }
+}
+
// How many ops_in_flight were allocated to this purge?
uint32_t ops_allowance;
public:
- C_IO_PurgeStrayPurged(StrayManager *sm_, CDentry *d, bool oh, uint32_t ops) :
- StrayManagerIOContext(sm_), dn(d), only_head(oh), ops_allowance(ops) { }
+ C_IO_PurgeStrayPurged(StrayManager *sm_, CDentry *d, bool oh) :
+ StrayManagerIOContext(sm_), dn(d), only_head(oh) { }
void finish(int r) override {
assert(r == 0 || r == -ENOENT);
sm->_purge_stray_purged(dn, ops_allowance, only_head);
}
};
-void StrayManager::purge(CDentry *dn, uint32_t op_allowance)
+
+void StrayManager::purge(CDentry *dn)
{
CDentry::linkage_t *dnl = dn->get_projected_linkage();
CInode *in = dnl->get_inode();
dout(10) << __func__ << " " << *dn << " " << *in << dendl;
assert(!dn->is_replicated());
- num_strays_purging++;
- logger->set(l_mdc_num_strays_purging, num_strays_purging);
-
-
// CHEAT. there's no real need to journal our intent to purge, since
// that is implicit in the dentry's presence and non-use in the stray
// dir. on recovery, we'll need to re-eval all strays anyway.
C_GatherBuilder gather(
g_ceph_context,
new C_OnFinisher(new C_IO_PurgeStrayPurged(
- this, dn, false, op_allowance), mds->finisher));
+ this, dn, false), mds->finisher));
if (in->is_dir()) {
object_locator_t oloc(mds->mdsmap->get_metadata_pool());
CInode *in = dn->get_projected_linkage()->get_inode();
dout(10) << "_purge_stray_purged " << *dn << " " << *in << dendl;
+ logger->inc(l_mdc_strays_enqueued);
+ num_strays_enqueuing--;
+ logger->set(l_mdc_num_strays_enqueuing, num_strays_enqueuing);
+
if (only_head) {
/* This was a ::truncate */
EUpdate *le = new EUpdate(mds->mdlog, "purge_stray truncate");
num_strays--;
logger->set(l_mdc_num_strays, num_strays);
- logger->inc(l_mdc_strays_purged);
}
-
- num_strays_purging--;
- logger->set(l_mdc_num_strays_purging, num_strays_purging);
-
- // Release resources
- dout(10) << __func__ << ": decrementing op allowance "
- << ops_allowance << " from " << ops_in_flight << " in flight" << dendl;
- assert(ops_in_flight >= ops_allowance);
- ops_in_flight -= ops_allowance;
- logger->set(l_mdc_num_purge_ops, ops_in_flight);
- files_purging -= 1;
- _advance();
}
void StrayManager::_purge_stray_logged(CDentry *dn, version_t pdv, LogSegment *ls)
void StrayManager::enqueue(CDentry *dn, bool trunc)
{
- if (aborted) {
- dout(10) << __func__ << ": aborted, skip purging: " << *dn << dendl;
- return;
- }
-
CDentry::linkage_t *dnl = dn->get_projected_linkage();
assert(dnl);
CInode *in = dnl->get_inode();
}
}
- const uint32_t ops_required = _calculate_ops_required(in, trunc);
-
- // Try to purge immediately if there is nothing in the queue, otherwise
- // we will go to the back of the queue (even if there is allowance available
- // to run us immediately) in order to be fair to others.
- bool consumed = false;
- if (ready_for_purge.empty()) {
- consumed = _consume(dn, trunc, ops_required);
- }
-
- if (consumed) {
- dout(10) << __func__ << ": purging this dentry immediately: "
- << *dn << dendl;
- } else {
- dout(10) << __func__ << ": enqueuing this dentry for later purge: "
- << *dn << dendl;
- if (!dn->state_test(CDentry::STATE_PURGINGPINNED) &&
- ready_for_purge.size() < g_conf->mds_max_purge_files) {
- dn->get(CDentry::PIN_PURGING);
- dn->state_set(CDentry::STATE_PURGINGPINNED);
- }
- ready_for_purge.push_back(QueuedStray(dn, trunc, ops_required));
- }
-}
-
-class C_StraysFetched : public StrayManagerContext {
-public:
- C_StraysFetched(StrayManager *sm_) :
- StrayManagerContext(sm_) { }
- void finish(int r) override {
- sm->_advance();
- }
-};
-
-void StrayManager::_advance()
-{
- if (aborted)
- return;
-
- std::map<CDir*, std::set<dentry_key_t> > to_fetch;
-
- for (auto p = ready_for_purge.begin();
- p != ready_for_purge.end();) {
- const QueuedStray &qs = *p;
- auto q = p++;
- CDentry *dn = qs.dir->lookup_exact_snap(qs.name, CEPH_NOSNAP);
- if (!dn) {
- assert(trimmed_strays.count(qs.name) > 0);
- if (fetching_strays.size() >= g_conf->mds_max_purge_files) {
- break;
- }
-
- dout(10) << __func__ << ": fetching stray dentry " << qs.name << dendl;
-
- auto it = fetching_strays.insert(qs);
- assert(it.second);
- to_fetch[qs.dir].insert(dentry_key_t(CEPH_NOSNAP, (it.first)->name.c_str()));
- ready_for_purge.erase(q);
- continue;
- }
-
- const bool consumed = _consume(dn, qs.trunc, qs.ops_required);
- if (!consumed) {
- break;
- }
- ready_for_purge.erase(q);
- }
-
- MDSGatherBuilder gather(g_ceph_context);
- for (auto p = to_fetch.begin(); p != to_fetch.end(); ++p)
- p->first->fetch(gather.new_sub(), p->second);
-
- if (gather.has_subs()) {
- gather.set_finisher(new C_StraysFetched(this));
- gather.activate();
- }
-}
-
-/*
- * Note that there are compromises to how throttling
- * is implemented here, in the interests of simplicity:
- * * If insufficient ops are available to execute
- * the next item on the queue, we block even if
- * there are items further down the queue requiring
- * fewer ops which might be executable
- * * The ops considered "in use" by a purge will be
- * an overestimate over the period of execution, as
- * we count filer_max_purge_ops and ops for old backtraces
- * as in use throughout, even though towards the end
- * of the purge the actual ops in flight will be
- * lower.
- * * The ops limit may be exceeded if the number of ops
- * required by a single inode is greater than the
- * limit, for example directories with very many
- * fragments.
- */
-bool StrayManager::_consume(CDentry *dn, bool trunc, uint32_t ops_required)
-{
- const int files_avail = g_conf->mds_max_purge_files - files_purging;
-
- if (!started) {
- dout(20) << __func__ << ": haven't started purging yet" << dendl;
- return false;
- }
-
- if (files_avail <= 0) {
- dout(20) << __func__ << ": throttling on max files" << dendl;
- return false;
- } else {
- dout(20) << __func__ << ": purging dn: " << *dn << dendl;
- }
-
- // Calculate how much of the ops allowance is available, allowing
- // for the case where the limit is currently being exceeded.
- uint32_t ops_avail;
- if (ops_in_flight <= max_purge_ops) {
- ops_avail = max_purge_ops - ops_in_flight;
- } else {
- ops_avail = 0;
- }
-
- /* The ops_in_flight > 0 condition here handles the case where the
- * ops required by this inode would never fit in the limit: we wait
- * instead until nothing else is running */
- if (ops_in_flight > 0 && ops_avail < ops_required) {
- dout(20) << __func__ << ": throttling on max ops (require "
- << ops_required << ", " << ops_in_flight << " in flight" << dendl;
- return false;
- }
+ dout(20) << __func__ << ": purging dn: " << *dn << dendl;
if (!dn->state_test(CDentry::STATE_PURGINGPINNED)) {
dn->get(CDentry::PIN_PURGING);
dn->state_set(CDentry::STATE_PURGINGPINNED);
}
+ ++num_strays_enqueuing;
+ logger->set(l_mdc_num_strays_enqueuing, num_strays_enqueuing);
+
// Resources are available, acquire them and execute the purge
- files_purging += 1;
- dout(10) << __func__ << ": allocating allowance "
- << ops_required << " to " << ops_in_flight << " in flight" << dendl;
- ops_in_flight += ops_required;
- logger->set(l_mdc_num_purge_ops, ops_in_flight);
-
- _process(dn, trunc, ops_required);
- return true;
+ _enqueue(dn, trunc);
+
+ dout(10) << __func__ << ": purging this dentry immediately: "
+ << *dn << dendl;
}
class C_OpenSnapParents : public StrayManagerContext {
CDentry *dn;
bool trunc;
- uint32_t ops_required;
public:
- C_OpenSnapParents(StrayManager *sm_, CDentry *dn_, bool t, uint32_t ops) :
- StrayManagerContext(sm_), dn(dn_), trunc(t), ops_required(ops) { }
+ C_OpenSnapParents(StrayManager *sm_, CDentry *dn_, bool t) :
+ StrayManagerContext(sm_), dn(dn_), trunc(t) { }
void finish(int r) override {
- sm->_process(dn, trunc, ops_required);
+ sm->_enqueue(dn, trunc);
}
};
-void StrayManager::_process(CDentry *dn, bool trunc, uint32_t ops_required)
+void StrayManager::_enqueue(CDentry *dn, bool trunc)
{
CInode *in = dn->get_linkage()->get_inode();
if (in->snaprealm &&
!in->snaprealm->have_past_parents_open() &&
- !in->snaprealm->open_parents(new C_OpenSnapParents(this, dn, trunc,
- ops_required))) {
+ !in->snaprealm->open_parents(new C_OpenSnapParents(this, dn, trunc))) {
// this can happen if the dentry had been trimmed from cache.
return;
}
- if (trunc) {
- truncate(dn, ops_required);
- } else {
- purge(dn, ops_required);
+ if (!started) {
+ // If the MDS is not yet active, defer executing this purge
+ // in order to avoid the mdlog writes we do on purge completion.
+ mds->wait_for_active(
+ new MDSInternalContextWrapper(mds,
+ new FunctionContext([this, dn, trunc](int r){
+ // It is safe to hold on to this CDentry* pointer
+ // because the dentry is pinned with PIN_PURGING
+ _enqueue(dn, trunc);
+ })
+ )
+ );
+
+ return;
}
-}
-uint32_t StrayManager::_calculate_ops_required(CInode *in, bool trunc)
-{
- uint32_t ops_required = 0;
- if (in->is_dir()) {
- // Directory, count dirfrags to be deleted
- std::list<frag_t> ls;
- if (!in->dirfragtree.is_leaf(frag_t())) {
- in->dirfragtree.get_leaves(ls);
- }
- // One for the root, plus any leaves
- ops_required = 1 + ls.size();
+ if (trunc) {
+ truncate(dn);
} else {
- // File, work out concurrent Filer::purge deletes
- const uint64_t to = MAX(in->inode.max_size_ever,
- MAX(in->inode.size, in->inode.get_max_size()));
-
- const uint64_t num = (to > 0) ? Striper::get_num_objects(in->inode.layout, to) : 1;
- ops_required = MIN(num, g_conf->filer_max_purge_ops);
-
- // Account for removing (or zeroing) backtrace
- ops_required += 1;
-
- // Account for deletions for old pools
- if (!trunc) {
- ops_required += in->get_projected_inode()->old_pools.size();
- }
+ purge(dn);
}
-
- return ops_required;
}
+
void StrayManager::advance_delayed()
{
for (elist<CDentry*>::iterator p = delayed_eval_stray.begin(); !p.end(); ) {
}
};
-bool StrayManager::__eval_stray(CDentry *dn, bool delay)
+bool StrayManager::_eval_stray(CDentry *dn, bool delay)
{
dout(10) << "eval_stray " << *dn << dendl;
CDentry::linkage_t *dnl = dn->get_projected_linkage();
{
dout(10) << __func__ << dendl;
started = true;
- _advance();
}
bool StrayManager::eval_stray(CDentry *dn, bool delay)
return false;
dn->state_set(CDentry::STATE_EVALUATINGSTRAY);
- bool ret = __eval_stray(dn, delay);
+ bool ret = _eval_stray(dn, delay);
dn->state_clear(CDentry::STATE_EVALUATINGSTRAY);
return ret;
}
StrayManager::StrayManager(MDSRank *mds, PurgeQueue &purge_queue_)
: delayed_eval_stray(member_offset(CDentry, item_stray)),
- mds(mds), purge_queue(purge_queue_), logger(NULL), started(false),
- aborted(false),
- ops_in_flight(0), files_purging(0),
- max_purge_ops(0),
- num_strays(0), num_strays_purging(0), num_strays_delayed(0),
- filer(mds->objecter, mds->finisher)
+ mds(mds), logger(NULL), started(false), num_strays(0),
+ num_strays_delayed(0), num_strays_enqueuing(0),
+ filer(mds->objecter, mds->finisher), purge_queue(purge_queue_)
{
assert(mds != NULL);
}
-void StrayManager::abort_queue()
-{
- for (std::list<QueuedStray>::iterator i = ready_for_purge.begin();
- i != ready_for_purge.end(); ++i)
- {
- const QueuedStray &qs = *i;
- CDentry *dn = qs.dir->lookup_exact_snap(qs.name, CEPH_NOSNAP);
- if (!dn)
- continue;
-
- dout(10) << __func__ << ": aborting enqueued purge " << *dn << dendl;
-
- CDentry::linkage_t *dnl = dn->get_projected_linkage();
- assert(dnl);
- CInode *in = dnl->get_inode();
- assert(in);
-
- // Clear flags set in enqueue
- if (dn->state_test(CDentry::STATE_PURGINGPINNED))
- dn->put(CDentry::PIN_PURGING);
- dn->state_clear(CDentry::STATE_PURGING | CDentry::STATE_PURGINGPINNED);
- in->state_clear(CInode::STATE_PURGING);
- }
- ready_for_purge.clear();
-
- trimmed_strays.clear();
- fetching_strays.clear();
-
- aborted = true;
-}
-
-void StrayManager::truncate(CDentry *dn, uint32_t op_allowance)
+void StrayManager::truncate(CDentry *dn)
{
CDentry::linkage_t *dnl = dn->get_projected_linkage();
CInode *in = dnl->get_inode();
dout(10) << __func__ << ": " << *dn << " " << *in << dendl;
assert(!dn->is_replicated());
- num_strays_purging++;
- logger->set(l_mdc_num_strays_purging, num_strays_purging);
-
C_GatherBuilder gather(
g_ceph_context,
- new C_OnFinisher(new C_IO_PurgeStrayPurged(this, dn, true, 0),
+ new C_OnFinisher(new C_IO_PurgeStrayPurged(this, dn, true),
mds->finisher));
SnapRealm *realm = in->find_snaprealm();
eval_stray(dn);
}
-
-void StrayManager::update_op_limit()
-{
- uint64_t pg_count = 0;
- mds->objecter->with_osdmap([&](const OSDMap& o) {
- // Number of PGs across all data pools
- const std::set<int64_t> &data_pools = mds->mdsmap->get_data_pools();
- for (const auto dp : data_pools) {
- if (o.get_pg_pool(dp) == NULL) {
- // It is possible that we have an older OSDMap than MDSMap,
- // because we don't start watching every OSDMap until after
- // MDSRank is initialized
- dout(4) << __func__ << " data pool " << dp
- << " not found in OSDMap" << dendl;
- continue;
- }
- pg_count += o.get_pg_num(dp);
- }
- });
-
- uint64_t mds_count = mds->mdsmap->get_max_mds();
-
- // Work out a limit based on n_pgs / n_mdss, multiplied by the user's
- // preference for how many ops per PG
- max_purge_ops = uint64_t(((double)pg_count / (double)mds_count) *
- g_conf->mds_max_purge_ops_per_pg);
-
- // User may also specify a hard limit, apply this if so.
- if (g_conf->mds_max_purge_ops) {
- max_purge_ops = MIN(max_purge_ops, g_conf->mds_max_purge_ops);
- }
-}
-
-void StrayManager::notify_stray_loaded(CDentry *dn)
-{
- dout(10) << __func__ << ": " << *dn << dendl;
-
- dn->state_set(CDentry::STATE_STRAY);
- CInode *in = dn->get_linkage()->get_inode();
- if (in->inode.nlink == 0)
- in->state_set(CInode::STATE_ORPHAN);
-
- if (aborted)
- return;
-
- auto p = trimmed_strays.find(dn->name);
- if (p != trimmed_strays.end()) {
- dn->state_set(CDentry::STATE_PURGING);
- in->state_set(CInode::STATE_PURGING);
- trimmed_strays.erase(p);
-
- QueuedStray key(dn, false, 0);
- auto q = fetching_strays.find(key);
- if (q != fetching_strays.end()) {
- ready_for_purge.push_front(*q);
- fetching_strays.erase(q);
- }
- }
-}
-
-void StrayManager::notify_stray_trimmed(CDentry *dn)
-{
- dout(10) << __func__ << ": " << *dn << dendl;
-
- if (aborted)
- return;
-
- trimmed_strays.insert(dn->name);
-}
class StrayManager
{
protected:
- class QueuedStray {
- public:
- CDir *dir;
- std::string name;
- bool trunc;
- uint32_t ops_required;
- QueuedStray(CDentry *dn, bool t, uint32_t ops)
- : dir(dn->get_dir()), name(dn->name),
- trunc(t), ops_required(ops) {}
- bool operator<(const QueuedStray& o) const {
- return (name < o.name);
- }
- };
-
// Has passed through eval_stray and still has refs
elist<CDentry*> delayed_eval_stray;
- // No more refs, can purge these
- std::list<QueuedStray> ready_for_purge;
-
// strays that have been trimmed from cache
std::set<std::string> trimmed_strays;
- // strays that are being fetching
- std::set<QueuedStray> fetching_strays;
// Global references for doing I/O
MDSRank *mds;
PerfCounters *logger;
bool started;
- bool aborted;
-
- // Throttled allowances
- uint64_t ops_in_flight;
- uint64_t files_purging;
-
- // Dynamic op limit per MDS based on PG count
- uint64_t max_purge_ops;
- // Statistics
+ // Stray dentries for this rank (including those not in cache)
uint64_t num_strays;
- uint64_t num_strays_purging;
+
+ // Stray dentries
uint64_t num_strays_delayed;
+ // Entries that have entered enqueue() but not been persistently
+ // recorded by PurgeQueue yet
+ uint64_t num_strays_enqueuing;
+
Filer filer;
PurgeQueue &purge_queue;
- void truncate(CDentry *dn, uint32_t op_allowance);
+ void truncate(CDentry *dn);
/**
* Purge a dentry from a stray directory. This function
* throttling is also satisfied. There is no going back
* at this stage!
*/
- void purge(CDentry *dn, uint32_t op_allowance);
+ void purge(CDentry *dn);
/**
* Completion handler for a Filer::purge on a stray inode.
friend class C_TruncateStrayLogged;
friend class C_IO_PurgeStrayPurged;
- /**
- * Enqueue a purge operation on a dentry that has passed the tests
- * in eval_stray. This may start the operation inline if the throttle
- * allowances are already available.
- *
- * @param trunc false to purge dentry (normal), true to just truncate
- * inode (snapshots)
- */
- void enqueue(CDentry *dn, bool trunc);
-
- /**
- * Iteratively call _consume on items from the ready_for_purge
- * list until it returns false (throttle limit reached)
- */
- void _advance();
-
- /**
- * Attempt to purge an inode, if throttling permits
- * its.
- *
- * Return true if we successfully consumed resource,
- * false if insufficient resource was available.
- */
- bool _consume(CDentry *dn, bool trunc, uint32_t ops_required);
- void _process(CDentry *dn, bool trunc, uint32_t ops_required);
+ // Call this on a dentry that has been identified as
+ // elegible for purging. It will be passed on to PurgeQueue.
+ void enqueue(CDentry *dn, bool trunc);
+ // Final part of enqueue() which we may have to retry
+ // after opening snap parents.
+ void _enqueue(CDentry *dn, bool trunc);
- /**
- * Return the maximum number of concurrent RADOS ops that
- * may be executed while purging this inode.
- *
- * @param trunc true if it's a truncate, false if it's a purge
- */
- uint32_t _calculate_ops_required(CInode *in, bool trunc);
/**
* When hard links exist to an inode whose primary dentry
* @returns true if the dentry will be purged (caller should never
* take more refs after this happens), else false.
*/
- bool __eval_stray(CDentry *dn, bool delay=false);
+ bool _eval_stray(CDentry *dn, bool delay=false);
// My public interface is for consumption by MDCache
public:
* this MDS to another MDS.
*/
void notify_stray_removed();
-
- /**
- * For any strays that are enqueued for purge, but
- * currently blocked on throttling, clear their
- * purging status. Used during MDS rank shutdown
- * so that it can migrate these strays instead
- * of waiting for them to trickle through the
- * queue.
- */
- void abort_queue();
-
- /*
- * Calculate our local RADOS op throttle limit based on
- * (mds_max_purge_ops_per_pg / number_of_mds) * number_of_pg
- *
- * Call this whenever one of those operands changes.
- */
- void update_op_limit();
-
- /*
- * track stray dentries that have been trimmed from cache
- */
- void notify_stray_trimmed(CDentry *dn);
- /*
- * restore stray dentry's previous stats
- */
- void notify_stray_loaded(CDentry *dn);
};
#endif // STRAY_MANAGER_H