ExtentCache.cc
mClockOpClassQueue.cc
mClockClientQueue.cc
- PGQueueable.cc
+ OpQueueItem.cc
${CMAKE_SOURCE_DIR}/src/common/TrackedOp.cc
${osd_cyg_functions_src}
${osdc_osd_srcs})
<< " in e" << m->get_map_epoch() << "/" << osdmap->get_epoch();
}
-void OSDService::enqueue_back(spg_t pgid, PGQueueable qi)
+void OSDService::enqueue_back(spg_t pgid, OpQueueItem qi)
{
osd->op_shardedwq.queue(make_pair(pgid, qi));
}
-void OSDService::enqueue_front(spg_t pgid, PGQueueable qi)
+void OSDService::enqueue_front(spg_t pgid, OpQueueItem qi)
{
osd->op_shardedwq.queue_front(make_pair(pgid, qi));
}
osd->op_shardedwq.queue(
make_pair(
pg->pg_id,
- PGQueueable(
+ OpQueueItem(
PGSnapTrim(pg->get_osdmap()->get_epoch()),
cct->_conf->osd_snap_trim_cost,
cct->_conf->osd_snap_trim_priority,
op->osd_trace.keyval("cost", op->get_req()->get_cost());
op->mark_queued_for_pg();
logger->tinc(l_osd_op_before_queue_op_lat, latency);
- op_shardedwq.queue(make_pair(pg, PGQueueable(op, epoch)));
+ op_shardedwq.queue(make_pair(pg, OpQueueItem(op, epoch)));
}
return;
}
}
- pair<spg_t, PGQueueable> item = sdata->pqueue->dequeue();
+ pair<spg_t, OpQueueItem> item = sdata->pqueue->dequeue();
if (osd->is_stopping()) {
sdata->sdata_op_ordering_lock.Unlock();
return; // OSD shutdown, discard.
osd->service.maybe_inject_dispatch_delay();
- boost::optional<PGQueueable> qi;
+ boost::optional<OpQueueItem> qi;
// we don't use a Mutex::Locker here because of the
// osd->service.release_reserved_pushes() call below
pg->unlock();
}
-void OSD::ShardedOpWQ::_enqueue(pair<spg_t, PGQueueable> item) {
+void OSD::ShardedOpWQ::_enqueue(pair<spg_t, OpQueueItem> item) {
uint32_t shard_index =
item.first.hash_to_shard(shard_list.size());
}
-void OSD::ShardedOpWQ::_enqueue_front(pair<spg_t, PGQueueable> item)
+void OSD::ShardedOpWQ::_enqueue_front(pair<spg_t, OpQueueItem> item)
{
uint32_t shard_index = item.first.hash_to_shard(shard_list.size());
ShardData* sdata = shard_list[shard_index];
#include "msg/Dispatcher.h"
+#include "common/backport14.h"
#include "common/Mutex.h"
#include "common/RWLock.h"
#include "common/Timer.h"
#include "OpRequest.h"
#include "Session.h"
-#include "osd/PGQueueable.h"
+#include "osd/OpQueueItem.h"
#include <atomic>
#include <map>
GenContextWQ recovery_gen_wq;
ClassHandler *&class_handler;
- void enqueue_back(spg_t pgid, PGQueueable qi);
- void enqueue_front(spg_t pgid, PGQueueable qi);
+ void enqueue_back(spg_t pgid, OpQueueItem qi);
+ void enqueue_front(spg_t pgid, OpQueueItem qi);
void maybe_inject_dispatch_delay() {
if (g_conf->osd_debug_inject_dispatch_delay_probability > 0) {
}
enqueue_back(
pg->get_pgid(),
- PGQueueable(
+ OpQueueItem(
PGScrub(pg->get_osdmap()->get_epoch()),
cct->_conf->osd_scrub_cost,
scrub_queue_priority,
assert(recovery_lock.is_locked_by_me());
enqueue_back(
p.second->get_pgid(),
- PGQueueable(
+ OpQueueItem(
PGRecovery(p.first, reserved_pushes),
cct->_conf->osd_recovery_cost,
cct->_conf->osd_recovery_priority,
* wake_pg_waiters, and (2) when wake_pg_waiters just ran, waiting_for_pg
* and already requeued the items.
*/
- friend class PGQueueable;
+ friend class OpQueueItem;
class ShardedOpWQ
- : public ShardedThreadPool::ShardedWQ<pair<spg_t,PGQueueable>>
+ : public ShardedThreadPool::ShardedWQ<pair<spg_t,OpQueueItem>>
{
struct ShardData {
Mutex sdata_lock;
OSDMapRef waiting_for_pg_osdmap;
struct pg_slot {
PGRef pg; ///< cached pg reference [optional]
- deque<PGQueueable> to_process; ///< order items for this slot
+ deque<OpQueueItem> to_process; ///< order items for this slot
int num_running = 0; ///< _process threads doing pg lookup/lock
/// true if pg does/did not exist. if so all new items go directly to
unordered_map<spg_t,pg_slot> pg_slots;
/// priority queue
- std::unique_ptr<OpQueue< pair<spg_t, PGQueueable>, uint64_t>> pqueue;
+ std::unique_ptr<OpQueue< pair<spg_t, OpQueueItem>, uint64_t>> pqueue;
- void _enqueue_front(pair<spg_t, PGQueueable> item, unsigned cutoff) {
+ void _enqueue_front(pair<spg_t, OpQueueItem> item, unsigned cutoff) {
unsigned priority = item.second.get_priority();
unsigned cost = item.second.get_cost();
if (priority >= cutoff)
false, cct) {
if (opqueue == io_queue::weightedpriority) {
pqueue = std::unique_ptr
- <WeightedPriorityQueue<pair<spg_t,PGQueueable>,uint64_t>>(
- new WeightedPriorityQueue<pair<spg_t,PGQueueable>,uint64_t>(
+ <WeightedPriorityQueue<pair<spg_t,OpQueueItem>,uint64_t>>(
+ new WeightedPriorityQueue<pair<spg_t,OpQueueItem>,uint64_t>(
max_tok_per_prio, min_cost));
} else if (opqueue == io_queue::prioritized) {
pqueue = std::unique_ptr
- <PrioritizedQueue<pair<spg_t,PGQueueable>,uint64_t>>(
- new PrioritizedQueue<pair<spg_t,PGQueueable>,uint64_t>(
+ <PrioritizedQueue<pair<spg_t,OpQueueItem>,uint64_t>>(
+ new PrioritizedQueue<pair<spg_t,OpQueueItem>,uint64_t>(
max_tok_per_prio, min_cost));
} else if (opqueue == io_queue::mclock_opclass) {
- pqueue = std::unique_ptr
- <ceph::mClockOpClassQueue>(new ceph::mClockOpClassQueue(cct));
+ pqueue = ceph::make_unique<ceph::mClockOpClassQueue>(cct);
} else if (opqueue == io_queue::mclock_client) {
- pqueue = std::unique_ptr
- <ceph::mClockClientQueue>(new ceph::mClockClientQueue(cct));
+ pqueue = ceph::make_unique<ceph::mClockClientQueue>(cct);
}
}
}; // struct ShardData
time_t ti,
time_t si,
ShardedThreadPool* tp)
- : ShardedThreadPool::ShardedWQ<pair<spg_t,PGQueueable>>(ti, si, tp),
+ : ShardedThreadPool::ShardedWQ<pair<spg_t,OpQueueItem>>(ti, si, tp),
osd(o),
num_shards(pnum_shards) {
for (uint32_t i = 0; i < num_shards; i++) {
void _process(uint32_t thread_index, heartbeat_handle_d *hb) override;
/// enqueue a new item
- void _enqueue(pair <spg_t, PGQueueable> item) override;
+ void _enqueue(pair <spg_t, OpQueueItem> item) override;
/// requeue an old item (at the front of the line)
- void _enqueue_front(pair <spg_t, PGQueueable> item) override;
+ void _enqueue_front(pair <spg_t, OpQueueItem> item) override;
void return_waiting_threads() override {
for(uint32_t i = 0; i < num_shards; i++) {
uint64_t reserved_pushes_to_free;
Pred(spg_t pg, list<OpRequestRef> *out_ops = 0)
: pgid(pg), out_ops(out_ops), reserved_pushes_to_free(0) {}
- void accumulate(const PGQueueable &op) {
+ void accumulate(const OpQueueItem &op) {
reserved_pushes_to_free += op.get_reserved_pushes();
if (out_ops) {
boost::optional<OpRequestRef> mop = op.maybe_get_op();
out_ops->push_front(*mop);
}
}
- bool operator()(const pair<spg_t, PGQueueable> &op) {
+ bool operator()(const pair<spg_t, OpQueueItem> &op) {
if (op.first == pgid) {
accumulate(op.second);
return true;
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2016 Red Hat Inc.
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+
+
+#include "PG.h"
+#include "OpQueueItem.h"
+#include "OSD.h"
+
+
+void OpQueueItem::RunVis::operator()(const OpRequestRef &op) {
+ osd->dequeue_op(pg, op, handle);
+}
+
+void OpQueueItem::RunVis::operator()(const PGSnapTrim &op) {
+ pg->snap_trimmer(op.epoch_queued);
+}
+
+void OpQueueItem::RunVis::operator()(const PGScrub &op) {
+ pg->scrub(op.epoch_queued, handle);
+}
+
+void OpQueueItem::RunVis::operator()(const PGRecovery &op) {
+ osd->do_recovery(pg.get(), op.epoch_queued, op.reserved_pushes, handle);
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2016 Red Hat Inc.
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+
+
+#pragma once
+
+#include <ostream>
+
+#include "include/types.h"
+#include "include/utime.h"
+#include "osd/OpRequest.h"
+#include "osd/PG.h"
+
+
+class OSD;
+
+
+struct PGScrub {
+ epoch_t epoch_queued;
+ explicit PGScrub(epoch_t e) : epoch_queued(e) {}
+ ostream &operator<<(ostream &rhs) {
+ return rhs << "PGScrub";
+ }
+};
+
+struct PGSnapTrim {
+ epoch_t epoch_queued;
+ explicit PGSnapTrim(epoch_t e) : epoch_queued(e) {}
+ ostream &operator<<(ostream &rhs) {
+ return rhs << "PGSnapTrim";
+ }
+};
+
+struct PGRecovery {
+ epoch_t epoch_queued;
+ uint64_t reserved_pushes;
+ PGRecovery(epoch_t e, uint64_t reserved_pushes)
+ : epoch_queued(e), reserved_pushes(reserved_pushes) {}
+ ostream &operator<<(ostream &rhs) {
+ return rhs << "PGRecovery(epoch=" << epoch_queued
+ << ", reserved_pushes: " << reserved_pushes << ")";
+ }
+};
+
+
+class OpQueueItem {
+ typedef boost::variant<
+ OpRequestRef,
+ PGSnapTrim,
+ PGScrub,
+ PGRecovery
+ > QVariant;
+ QVariant qvariant;
+ int cost;
+ unsigned priority;
+ utime_t start_time;
+ uint64_t owner; ///< global id (e.g., client.XXX)
+ epoch_t map_epoch; ///< an epoch we expect the PG to exist in
+
+ struct RunVis : public boost::static_visitor<> {
+ OSD *osd;
+ PGRef &pg;
+ ThreadPool::TPHandle &handle;
+ RunVis(OSD *osd, PGRef &pg, ThreadPool::TPHandle &handle)
+ : osd(osd), pg(pg), handle(handle) {}
+ void operator()(const OpRequestRef &op);
+ void operator()(const PGSnapTrim &op);
+ void operator()(const PGScrub &op);
+ void operator()(const PGRecovery &op);
+ }; // struct RunVis
+
+ struct StringifyVis : public boost::static_visitor<std::string> {
+ std::string operator()(const OpRequestRef &op) {
+ return stringify(op);
+ }
+ std::string operator()(const PGSnapTrim &op) {
+ return "PGSnapTrim";
+ }
+ std::string operator()(const PGScrub &op) {
+ return "PGScrub";
+ }
+ std::string operator()(const PGRecovery &op) {
+ return "PGRecovery";
+ }
+ };
+
+ friend ostream& operator<<(ostream& out, const OpQueueItem& q) {
+ StringifyVis v;
+ return out << "OpQueueItem(" << boost::apply_visitor(v, q.qvariant)
+ << " prio " << q.priority << " cost " << q.cost
+ << " e" << q.map_epoch << ")";
+ }
+
+public:
+
+ OpQueueItem(OpRequestRef op, epoch_t e)
+ : qvariant(op), cost(op->get_req()->get_cost()),
+ priority(op->get_req()->get_priority()),
+ start_time(op->get_req()->get_recv_stamp()),
+ owner(op->get_req()->get_source().num()),
+ map_epoch(e)
+ {}
+ OpQueueItem(
+ const PGSnapTrim &op, int cost, unsigned priority, utime_t start_time,
+ uint64_t owner, epoch_t e)
+ : qvariant(op), cost(cost), priority(priority), start_time(start_time),
+ owner(owner), map_epoch(e) {}
+ OpQueueItem(
+ const PGScrub &op, int cost, unsigned priority, utime_t start_time,
+ uint64_t owner, epoch_t e)
+ : qvariant(op), cost(cost), priority(priority), start_time(start_time),
+ owner(owner), map_epoch(e) {}
+ OpQueueItem(
+ const PGRecovery &op, int cost, unsigned priority, utime_t start_time,
+ uint64_t owner, epoch_t e)
+ : qvariant(op), cost(cost), priority(priority), start_time(start_time),
+ owner(owner), map_epoch(e) {}
+
+ const boost::optional<OpRequestRef> maybe_get_op() const {
+ const OpRequestRef *op = boost::get<OpRequestRef>(&qvariant);
+ return op ? OpRequestRef(*op) : boost::optional<OpRequestRef>();
+ }
+ uint64_t get_reserved_pushes() const {
+ const PGRecovery *op = boost::get<PGRecovery>(&qvariant);
+ return op ? op->reserved_pushes : 0;
+ }
+ void run(OSD *osd, PGRef &pg, ThreadPool::TPHandle &handle) {
+ RunVis v(osd, pg, handle);
+ boost::apply_visitor(v, qvariant);
+ }
+ unsigned get_priority() const { return priority; }
+ int get_cost() const { return cost; }
+ utime_t get_start_time() const { return start_time; }
+ uint64_t get_owner() const { return owner; }
+ epoch_t get_map_epoch() const { return map_epoch; }
+ const QVariant& get_variant() const { return qvariant; }
+}; // struct OpQueueItem
p->second.push_front(op);
} else {
dout(20) << __func__ << " " << op << dendl;
- osd->enqueue_front(info.pgid, PGQueueable(op, get_osdmap()->get_epoch()));
+ osd->enqueue_front(info.pgid, OpQueueItem(op, get_osdmap()->get_epoch()));
}
}
p->second.push_front(*i);
} else {
dout(20) << __func__ << " " << *i << dendl;
- osd->enqueue_front(info.pgid, PGQueueable(*i, get_osdmap()->get_epoch()));
+ osd->enqueue_front(info.pgid, OpQueueItem(*i, get_osdmap()->get_epoch()));
}
}
ls.clear();
} else {
dout(20) << __func__ << " " << p->first << " " << p->second << dendl;
for (auto q = p->second.rbegin(); q != p->second.rend(); ++q) {
- osd->enqueue_front(info.pgid, PGQueueable(*q, epoch));
+ osd->enqueue_front(info.pgid, OpQueueItem(*q, epoch));
}
p = waiting_for_map.erase(p);
}
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2016 Red Hat Inc.
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#include "PG.h"
-#include "PGQueueable.h"
-#include "OSD.h"
-
-
-void PGQueueable::RunVis::operator()(const OpRequestRef &op) {
- osd->dequeue_op(pg, op, handle);
-}
-
-void PGQueueable::RunVis::operator()(const PGSnapTrim &op) {
- pg->snap_trimmer(op.epoch_queued);
-}
-
-void PGQueueable::RunVis::operator()(const PGScrub &op) {
- pg->scrub(op.epoch_queued, handle);
-}
-
-void PGQueueable::RunVis::operator()(const PGRecovery &op) {
- osd->do_recovery(pg.get(), op.epoch_queued, op.reserved_pushes, handle);
-}
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2016 Red Hat Inc.
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#pragma once
-
-#include <ostream>
-
-#include "include/types.h"
-#include "include/utime.h"
-#include "osd/OpRequest.h"
-#include "osd/PG.h"
-
-
-class OSD;
-
-
-struct PGScrub {
- epoch_t epoch_queued;
- explicit PGScrub(epoch_t e) : epoch_queued(e) {}
- ostream &operator<<(ostream &rhs) {
- return rhs << "PGScrub";
- }
-};
-
-struct PGSnapTrim {
- epoch_t epoch_queued;
- explicit PGSnapTrim(epoch_t e) : epoch_queued(e) {}
- ostream &operator<<(ostream &rhs) {
- return rhs << "PGSnapTrim";
- }
-};
-
-struct PGRecovery {
- epoch_t epoch_queued;
- uint64_t reserved_pushes;
- PGRecovery(epoch_t e, uint64_t reserved_pushes)
- : epoch_queued(e), reserved_pushes(reserved_pushes) {}
- ostream &operator<<(ostream &rhs) {
- return rhs << "PGRecovery(epoch=" << epoch_queued
- << ", reserved_pushes: " << reserved_pushes << ")";
- }
-};
-
-
-class PGQueueable {
- typedef boost::variant<
- OpRequestRef,
- PGSnapTrim,
- PGScrub,
- PGRecovery
- > QVariant;
- QVariant qvariant;
- int cost;
- unsigned priority;
- utime_t start_time;
- uint64_t owner; ///< global id (e.g., client.XXX)
- epoch_t map_epoch; ///< an epoch we expect the PG to exist in
-
- struct RunVis : public boost::static_visitor<> {
- OSD *osd;
- PGRef &pg;
- ThreadPool::TPHandle &handle;
- RunVis(OSD *osd, PGRef &pg, ThreadPool::TPHandle &handle)
- : osd(osd), pg(pg), handle(handle) {}
- void operator()(const OpRequestRef &op);
- void operator()(const PGSnapTrim &op);
- void operator()(const PGScrub &op);
- void operator()(const PGRecovery &op);
- }; // struct RunVis
-
- struct StringifyVis : public boost::static_visitor<std::string> {
- std::string operator()(const OpRequestRef &op) {
- return stringify(op);
- }
- std::string operator()(const PGSnapTrim &op) {
- return "PGSnapTrim";
- }
- std::string operator()(const PGScrub &op) {
- return "PGScrub";
- }
- std::string operator()(const PGRecovery &op) {
- return "PGRecovery";
- }
- };
-
- friend ostream& operator<<(ostream& out, const PGQueueable& q) {
- StringifyVis v;
- return out << "PGQueueable(" << boost::apply_visitor(v, q.qvariant)
- << " prio " << q.priority << " cost " << q.cost
- << " e" << q.map_epoch << ")";
- }
-
-public:
-
- PGQueueable(OpRequestRef op, epoch_t e)
- : qvariant(op), cost(op->get_req()->get_cost()),
- priority(op->get_req()->get_priority()),
- start_time(op->get_req()->get_recv_stamp()),
- owner(op->get_req()->get_source().num()),
- map_epoch(e)
- {}
- PGQueueable(
- const PGSnapTrim &op, int cost, unsigned priority, utime_t start_time,
- uint64_t owner, epoch_t e)
- : qvariant(op), cost(cost), priority(priority), start_time(start_time),
- owner(owner), map_epoch(e) {}
- PGQueueable(
- const PGScrub &op, int cost, unsigned priority, utime_t start_time,
- uint64_t owner, epoch_t e)
- : qvariant(op), cost(cost), priority(priority), start_time(start_time),
- owner(owner), map_epoch(e) {}
- PGQueueable(
- const PGRecovery &op, int cost, unsigned priority, utime_t start_time,
- uint64_t owner, epoch_t e)
- : qvariant(op), cost(cost), priority(priority), start_time(start_time),
- owner(owner), map_epoch(e) {}
-
- const boost::optional<OpRequestRef> maybe_get_op() const {
- const OpRequestRef *op = boost::get<OpRequestRef>(&qvariant);
- return op ? OpRequestRef(*op) : boost::optional<OpRequestRef>();
- }
- uint64_t get_reserved_pushes() const {
- const PGRecovery *op = boost::get<PGRecovery>(&qvariant);
- return op ? op->reserved_pushes : 0;
- }
- void run(OSD *osd, PGRef &pg, ThreadPool::TPHandle &handle) {
- RunVis v(osd, pg, handle);
- boost::apply_visitor(v, qvariant);
- }
- unsigned get_priority() const { return priority; }
- int get_cost() const { return cost; }
- utime_t get_start_time() const { return start_time; }
- uint64_t get_owner() const { return owner; }
- epoch_t get_map_epoch() const { return map_epoch; }
- const QVariant& get_variant() const { return qvariant; }
-}; // struct PGQueueable
scrubber.active_rep_scrub->get_req())->scrub_to) {
osd->enqueue_back(
info.pgid,
- PGQueueable(scrubber.active_rep_scrub, get_osdmap()->get_epoch()));
+ OpQueueItem(scrubber.active_rep_scrub, get_osdmap()->get_epoch()));
scrubber.active_rep_scrub = OpRequestRef();
}
}
scrubber.active_rep_scrub->get_req())->chunky) {
osd->enqueue_back(
info.pgid,
- PGQueueable(scrubber.active_rep_scrub, get_osdmap()->get_epoch()));
+ OpQueueItem(scrubber.active_rep_scrub, get_osdmap()->get_epoch()));
scrubber.active_rep_scrub = OpRequestRef();
}
unlock();
#include "common/config.h"
#include "common/ceph_context.h"
-#include "osd/PGQueueable.h"
+#include "osd/OpQueueItem.h"
#include "common/mClockPriorityQueue.h"
namespace ceph {
- using Request = std::pair<spg_t, PGQueueable>;
+ using Request = std::pair<spg_t, OpQueueItem>;
using Client = uint64_t;
-
// This class exists to bridge the ceph code, which treats the class
// as the client, and the queue, where the class is
// osd_op_type_t. So this adapter class will transform calls
#include "common/config.h"
#include "common/ceph_context.h"
-#include "osd/PGQueueable.h"
+#include "osd/OpQueueItem.h"
#include "common/mClockPriorityQueue.h"
namespace ceph {
- using Request = std::pair<spg_t, PGQueueable>;
+ using Request = std::pair<spg_t, OpQueueItem>;
using Client = uint64_t;
-
// This class exists to bridge the ceph code, which treats the class
// as the client, and the queue, where the class is
// osd_op_type_t. So this adapter class will transform calls
#if 0 // more work needed here
Request create_client_op(epoch_t e, uint64_t owner) {
- return Request(spg_t(), PGQueueable(OpRequestRef(), e));
+ return Request(spg_t(), OpQueueItem(OpRequestRef(), e));
}
#endif
Request create_snaptrim(epoch_t e, uint64_t owner) {
return Request(spg_t(),
- PGQueueable(PGSnapTrim(e),
+ OpQueueItem(PGSnapTrim(e),
12, 12,
utime_t(), owner, e));
}
Request create_scrub(epoch_t e, uint64_t owner) {
return Request(spg_t(),
- PGQueueable(PGScrub(e),
+ OpQueueItem(PGScrub(e),
12, 12,
utime_t(), owner, e));
}
Request create_recovery(epoch_t e, uint64_t owner) {
return Request(spg_t(),
- PGQueueable(PGRecovery(e, 64),
+ OpQueueItem(PGRecovery(e, 64),
12, 12,
utime_t(), owner, e));
}
#if 0 // more work needed here
Request create_client_op(epoch_t e, uint64_t owner) {
- return Request(spg_t(), PGQueueable(OpRequestRef(), e));
+ return Request(spg_t(), OpQueueItem(OpRequestRef(), e));
}
#endif
Request create_snaptrim(epoch_t e, uint64_t owner) {
return Request(spg_t(),
- PGQueueable(PGSnapTrim(e),
+ OpQueueItem(PGSnapTrim(e),
12, 12,
utime_t(), owner, e));
}
Request create_scrub(epoch_t e, uint64_t owner) {
return Request(spg_t(),
- PGQueueable(PGScrub(e),
+ OpQueueItem(PGScrub(e),
12, 12,
utime_t(), owner, e));
}
Request create_recovery(epoch_t e, uint64_t owner) {
return Request(spg_t(),
- PGQueueable(PGRecovery(e, 64),
+ OpQueueItem(PGRecovery(e, 64),
12, 12,
utime_t(), owner, e));
}