friend bool operator==(const hobject_t&, const hobject_t&);
friend bool operator!=(const hobject_t&, const hobject_t&);
friend struct ghobject_t;
-
- struct NibblewiseComparator {
- bool operator()(const hobject_t& l, const hobject_t& r) const {
- return cmp(l, r) < 0;
- }
- };
-
- struct BitwiseComparator {
- bool operator()(const hobject_t& l, const hobject_t& r) const {
- return cmp(l, r) < 0;
- }
- };
-
- struct Comparator {
- bool bitwise;
- explicit Comparator(bool b) : bitwise(b) {}
- bool operator()(const hobject_t& l, const hobject_t& r) const {
- return cmp(l, r) < 0;
- }
- };
- struct ComparatorWithDefault {
- bool bitwise;
- explicit ComparatorWithDefault(bool b=true) : bitwise(b) {}
- bool operator()(const hobject_t& l, const hobject_t& r) const {
- return cmp(l, r) < 0;
- }
- };
- template <typename T>
- using bitwisemap = std::map<hobject_t, T, BitwiseComparator>;
-
- using bitwiseset = std::set<hobject_t, BitwiseComparator>;
};
WRITE_CLASS_ENCODER(hobject_t)
friend bool operator==(const ghobject_t&, const ghobject_t&);
friend bool operator!=(const ghobject_t&, const ghobject_t&);
- struct NibblewiseComparator {
- bool operator()(const ghobject_t& l, const ghobject_t& r) const {
- return cmp(l, r) < 0;
- }
- };
-
- struct BitwiseComparator {
- bool operator()(const ghobject_t& l, const ghobject_t& r) const {
- return cmp(l, r) < 0;
- }
- };
-
- struct Comparator {
- bool bitwise;
- explicit Comparator(bool b) : bitwise(b) {}
- bool operator()(const ghobject_t& l, const ghobject_t& r) const {
- return cmp(l, r) < 0;
- }
- };
};
WRITE_CLASS_ENCODER(ghobject_t)
map<string,bufferlist> attrset;
interval_set<uint64_t> data_subset;
- map<hobject_t, interval_set<uint64_t>, hobject_t::BitwiseComparator> clone_subsets;
+ map<hobject_t, interval_set<uint64_t>> clone_subsets;
bool first, complete;
void *osr {nullptr}; // NULL on replay
map<coll_t, __le32> coll_index;
- map<ghobject_t, __le32, ghobject_t::BitwiseComparator> object_index;
+ map<ghobject_t, __le32> object_index;
__le32 coll_id {0};
__le32 object_id {0};
}
vector<__le32> om(other.object_index.size());
- map<ghobject_t, __le32, ghobject_t::BitwiseComparator>::iterator object_index_p;
+ map<ghobject_t, __le32>::iterator object_index_p;
for (object_index_p = other.object_index.begin();
object_index_p != other.object_index.end();
++object_index_p) {
colls[coll_index_p->second] = coll_index_p->first;
}
- map<ghobject_t, __le32, ghobject_t::BitwiseComparator>::iterator object_index_p;
+ map<ghobject_t, __le32>::iterator object_index_p;
for (object_index_p = t->object_index.begin();
object_index_p != t->object_index.end();
++object_index_p) {
return index_id;
}
__le32 _get_object_id(const ghobject_t& oid) {
- map<ghobject_t, __le32, ghobject_t::BitwiseComparator>::iterator o = object_index.find(oid);
+ map<ghobject_t, __le32>::iterator o = object_index.find(oid);
if (o != object_index.end())
return o->second;
list<CollectionRef> removed_collections;
RWLock debug_read_error_lock;
- set<ghobject_t, ghobject_t::BitwiseComparator> debug_data_error_objects;
- set<ghobject_t, ghobject_t::BitwiseComparator> debug_mdata_error_objects;
+ set<ghobject_t> debug_data_error_objects;
+ set<ghobject_t> debug_mdata_error_objects;
std::atomic<int> csum_type;
* Set of headers currently in use
*/
set<uint64_t> in_use;
- set<ghobject_t, ghobject_t::BitwiseComparator> map_header_in_use;
+ set<ghobject_t> map_header_in_use;
/**
* Takes the map_header_in_use entry in constructor, releases in
/// Implicit lock on Header->seq
typedef ceph::shared_ptr<_Header> Header;
Mutex cache_lock;
- SimpleLRU<ghobject_t, _Header, ghobject_t::BitwiseComparator> caches;
+ SimpleLRU<ghobject_t, _Header> caches;
string map_header_key(const ghobject_t &oid);
string header_key(uint64_t seq);
private:
CephContext *cct;
const int registry_shards;
- SharedLRU<ghobject_t, FD, ghobject_t::BitwiseComparator> *registry;
+ SharedLRU<ghobject_t, FD> *registry;
public:
explicit FDCache(CephContext *cct) : cct(cct),
registry_shards(MAX(cct->_conf->filestore_fd_cache_shards, 1)) {
assert(cct);
cct->_conf->add_observer(this);
- registry = new SharedLRU<ghobject_t, FD, ghobject_t::BitwiseComparator>[registry_shards];
+ registry = new SharedLRU<ghobject_t, FD>[registry_shards];
for (int i = 0; i < registry_shards; ++i) {
registry[i].set_cct(cct);
registry[i].set_size(
// DEBUG read error injection, an object is removed from both on delete()
Mutex read_error_lock;
- set<ghobject_t, ghobject_t::BitwiseComparator> data_error_set; // read() will return -EIO
- set<ghobject_t, ghobject_t::BitwiseComparator> mdata_error_set; // getattr(),stat() will return -EIO
+ set<ghobject_t> data_error_set; // read() will return -EIO
+ set<ghobject_t> mdata_error_set; // getattr(),stat() will return -EIO
void inject_data_error(const ghobject_t &oid);
void inject_mdata_error(const ghobject_t &oid);
void debug_obj_on_delete(const ghobject_t &oid);
f->close_section();
f->open_array_section("objects");
- for (map<ghobject_t,ObjectRef,ghobject_t::BitwiseComparator>::iterator q = p->second->object_map.begin();
+ for (map<ghobject_t,ObjectRef>::iterator q = p->second->object_map.begin();
q != p->second->object_map.end();
++q) {
f->open_object_section("object");
dout(10) << __func__ << " cid " << cid << " start " << start
<< " end " << end << dendl;
- map<ghobject_t,ObjectRef,ghobject_t::BitwiseComparator>::iterator p = c->object_map.lower_bound(start);
+ map<ghobject_t,ObjectRef>::iterator p = c->object_map.lower_bound(start);
while (p != c->object_map.end() &&
ls->size() < (unsigned)max &&
p->first < end) {
RWLock::WLocker l1(MIN(&(*sc), &(*dc))->lock);
RWLock::WLocker l2(MAX(&(*sc), &(*dc))->lock);
- map<ghobject_t,ObjectRef,ghobject_t::BitwiseComparator>::iterator p = sc->object_map.begin();
+ map<ghobject_t,ObjectRef>::iterator p = sc->object_map.begin();
while (p != sc->object_map.end()) {
if (p->first.match(bits, match)) {
dout(20) << " moving " << p->first << dendl;
CephContext *cct;
bool use_page_set;
ceph::unordered_map<ghobject_t, ObjectRef> object_hash; ///< for lookup
- map<ghobject_t, ObjectRef,ghobject_t::BitwiseComparator> object_map; ///< for iteration
+ map<ghobject_t, ObjectRef> object_map; ///< for iteration
map<string,bufferptr> xattr;
RWLock lock; ///< for object_{map,hash}
bool exists;
::encode(use_page_set, bl);
uint32_t s = object_map.size();
::encode(s, bl);
- for (map<ghobject_t, ObjectRef,ghobject_t::BitwiseComparator>::const_iterator p = object_map.begin();
+ for (map<ghobject_t, ObjectRef>::const_iterator p = object_map.begin();
p != object_map.end();
++p) {
::encode(p->first, bl);
uint64_t used_bytes() const {
uint64_t result = 0;
- for (map<ghobject_t, ObjectRef,ghobject_t::BitwiseComparator>::const_iterator p = object_map.begin();
+ for (map<ghobject_t, ObjectRef>::const_iterator p = object_map.begin();
p != object_map.end();
++p) {
result += p->second->get_size();
struct RecoveryMessages {
map<hobject_t,
- ECBackend::read_request_t, hobject_t::BitwiseComparator> reads;
+ ECBackend::read_request_t> reads;
void read(
ECBackend *ec,
const hobject_t &hoid, uint64_t off, uint64_t len,
add_temp_objs(op.temp_added);
}
if (op.backfill) {
- for (set<hobject_t, hobject_t::BitwiseComparator>::iterator i = op.temp_removed.begin();
+ for (set<hobject_t>::iterator i = op.temp_removed.begin();
i != op.temp_removed.end();
++i) {
dout(10) << __func__ << ": removing object " << *i
ECSubReadReply *reply)
{
shard_id_t shard = get_parent()->whoami_shard().shard;
- for(map<hobject_t, list<boost::tuple<uint64_t, uint64_t, uint32_t> >, hobject_t::BitwiseComparator>::iterator i =
+ for(map<hobject_t, list<boost::tuple<uint64_t, uint64_t, uint32_t> >>::iterator i =
op.to_read.begin();
i != op.to_read.end();
++i) {
reply->buffers_read.erase(i->first);
reply->errors[i->first] = r;
}
- for (set<hobject_t, hobject_t::BitwiseComparator>::iterator i = op.attrs_to_read.begin();
+ for (set<hobject_t>::iterator i = op.attrs_to_read.begin();
i != op.attrs_to_read.end();
++i) {
dout(10) << __func__ << ": fulfilling attr request on "
return;
}
ReadOp &rop = iter->second;
- for (map<hobject_t, list<pair<uint64_t, bufferlist> >, hobject_t::BitwiseComparator>::iterator i =
+ for (map<hobject_t, list<pair<uint64_t, bufferlist> >>::iterator i =
op.buffers_read.begin();
i != op.buffers_read.end();
++i) {
riter->get<2>()[from].claim(j->second);
}
}
- for (map<hobject_t, map<string, bufferlist>, hobject_t::BitwiseComparator>::iterator i = op.attrs_read.begin();
+ for (map<hobject_t, map<string, bufferlist>>::iterator i = op.attrs_read.begin();
i != op.attrs_read.end();
++i) {
assert(!op.errors.count(i->first)); // if read error better not have sent an attribute
rop.complete[i->first].attrs = map<string, bufferlist>();
(*(rop.complete[i->first].attrs)).swap(i->second);
}
- for (map<hobject_t, int, hobject_t::BitwiseComparator>::iterator i = op.errors.begin();
+ for (map<hobject_t, int>::iterator i = op.errors.begin();
i != op.errors.end();
++i) {
rop.complete[i->first].errors.insert(
void ECBackend::complete_read_op(ReadOp &rop, RecoveryMessages *m)
{
- map<hobject_t, read_request_t, hobject_t::BitwiseComparator>::iterator reqiter =
+ map<hobject_t, read_request_t>::iterator reqiter =
rop.to_read.begin();
- map<hobject_t, read_result_t, hobject_t::BitwiseComparator>::iterator resiter =
+ map<hobject_t, read_result_t>::iterator resiter =
rop.complete.begin();
assert(rop.to_read.size() == rop.complete.size());
for (; reqiter != rop.to_read.end(); ++reqiter, ++resiter) {
const OSDMapRef& osdmap,
ReadOp &op)
{
- set<hobject_t, hobject_t::BitwiseComparator> to_cancel;
- for (map<pg_shard_t, set<hobject_t, hobject_t::BitwiseComparator> >::iterator i = op.source_to_obj.begin();
+ set<hobject_t> to_cancel;
+ for (map<pg_shard_t, set<hobject_t> >::iterator i = op.source_to_obj.begin();
i != op.source_to_obj.end();
++i) {
if (osdmap->is_down(i->first.osd)) {
if (to_cancel.empty())
return;
- for (map<pg_shard_t, set<hobject_t, hobject_t::BitwiseComparator> >::iterator i = op.source_to_obj.begin();
+ for (map<pg_shard_t, set<hobject_t> >::iterator i = op.source_to_obj.begin();
i != op.source_to_obj.end();
) {
- for (set<hobject_t, hobject_t::BitwiseComparator>::iterator j = i->second.begin();
+ for (set<hobject_t>::iterator j = i->second.begin();
j != i->second.end();
) {
if (to_cancel.count(*j))
}
}
- for (set<hobject_t, hobject_t::BitwiseComparator>::iterator i = to_cancel.begin();
+ for (set<hobject_t>::iterator i = to_cancel.begin();
i != to_cancel.end();
++i) {
get_parent()->cancel_pull(*i);
i != tid_to_read_map.end();
++i) {
dout(10) << __func__ << ": cancelling " << i->second << dendl;
- for (map<hobject_t, read_request_t, hobject_t::BitwiseComparator>::iterator j =
+ for (map<hobject_t, read_request_t>::iterator j =
i->second.to_read.begin();
j != i->second.to_read.end();
++j) {
void ECBackend::dump_recovery_info(Formatter *f) const
{
f->open_array_section("recovery_ops");
- for (map<hobject_t, RecoveryOp, hobject_t::BitwiseComparator>::const_iterator i = recovery_ops.begin();
+ for (map<hobject_t, RecoveryOp>::const_iterator i = recovery_ops.begin();
i != recovery_ops.end();
++i) {
f->open_object_section("op");
}
}
- map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator>::const_iterator miter =
+ map<hobject_t, set<pg_shard_t>>::const_iterator miter =
get_parent()->get_missing_loc_shards().find(hoid);
if (miter != get_parent()->get_missing_loc_shards().end()) {
for (set<pg_shard_t>::iterator i = miter->second.begin();
void ECBackend::start_read_op(
int priority,
- map<hobject_t, read_request_t, hobject_t::BitwiseComparator> &to_read,
+ map<hobject_t, read_request_t> &to_read,
OpRequestRef _op,
bool do_redundant_reads,
bool for_recovery)
dout(10) << __func__ << ": starting read " << op << dendl;
map<pg_shard_t, ECSubRead> messages;
- for (map<hobject_t, read_request_t,
- hobject_t::BitwiseComparator>::iterator i = op.to_read.begin();
+ for (map<hobject_t, read_request_t>::iterator i = op.to_read.begin();
i != op.to_read.end();
++i) {
bool need_attrs = i->second.want_attrs;
assert(get_parent()->get_pool().is_hacky_ecoverwrites());
objects_read_async_no_cache(
op->remote_read,
- [this, op](hobject_t::bitwisemap<pair<int, extent_map> > &&results) {
+ [this, op](map<hobject_t,pair<int, extent_map> > &&results) {
for (auto &&i: results) {
op->remote_read_result.emplace(i.first, i.second.second);
}
trans[i->shard];
}
- hobject_t::bitwisemap<extent_map> written;
+ map<hobject_t,extent_map> written;
if (op->plan.t) {
ECTransaction::generate_transactions(
op->plan,
}
}
- hobject_t::bitwisemap<extent_set> written_set;
+ map<hobject_t,extent_set> written_set;
for (auto &&i: written) {
written_set[i.first] = i.second.get_interval_set();
}
Context *on_complete,
bool fast_read)
{
- hobject_t::bitwisemap<std::list<boost::tuple<uint64_t, uint64_t, uint32_t> > >
+ map<hobject_t,std::list<boost::tuple<uint64_t, uint64_t, uint32_t> > >
reads;
uint32_t flags = 0;
hoid(hoid),
to_read(to_read),
on_complete(on_complete) {}
- void operator()(hobject_t::bitwisemap<pair<int, extent_map> > &&results) {
+ void operator()(map<hobject_t,pair<int, extent_map> > &&results) {
auto dpp = ec->get_parent()->get_dpp();
ldpp_dout(dpp, 20) << "objects_read_async_cb: got: " << results
<< dendl;
reads,
fast_read,
make_gen_lambda_context<
- hobject_t::bitwisemap<pair<int, extent_map> > &&, cb>(
+ map<hobject_t,pair<int, extent_map> > &&, cb>(
cb(this,
hoid,
to_read,
};
void ECBackend::objects_read_and_reconstruct(
- const hobject_t::bitwisemap<
+ const map<hobject_t,
std::list<boost::tuple<uint64_t, uint64_t, uint32_t> >
> &reads,
bool fast_read,
- GenContextURef<hobject_t::bitwisemap<pair<int, extent_map> > &&> &&func)
+ GenContextURef<map<hobject_t,pair<int, extent_map> > &&> &&func)
{
in_progress_client_reads.emplace_back(
reads.size(), std::move(func));
set<int> want_to_read;
get_want_to_read_shards(&want_to_read);
- map<hobject_t, read_request_t, hobject_t::BitwiseComparator> for_read_op;
+ map<hobject_t, read_request_t> for_read_op;
for (auto &&to_read: reads) {
set<pg_shard_t> shards;
int r = get_min_avail_to_read_shards(
GenContext<pair<RecoveryMessages *, read_result_t& > &> *c =
rop.to_read.find(hoid)->second.cb;
- map<hobject_t, read_request_t, hobject_t::BitwiseComparator> for_read_op;
+ map<hobject_t, read_request_t> for_read_op;
for_read_op.insert(
make_pair(
hoid,
* check_recovery_sources.
*/
void objects_read_and_reconstruct(
- const hobject_t::bitwisemap<
- std::list<boost::tuple<uint64_t, uint64_t, uint32_t> >
+ const map<hobject_t, std::list<boost::tuple<uint64_t, uint64_t, uint32_t> >
> &reads,
bool fast_read,
- GenContextURef<hobject_t::bitwisemap<pair<int, extent_map> > &&> &&func);
+ GenContextURef<map<hobject_t,pair<int, extent_map> > &&> &&func);
friend struct CallClientContexts;
struct ClientAsyncReadStatus {
unsigned objects_to_read;
- GenContextURef<hobject_t::bitwisemap<pair<int, extent_map> > &&> func;
- hobject_t::bitwisemap<pair<int, extent_map> > results;
+ GenContextURef<map<hobject_t,pair<int, extent_map> > &&> func;
+ map<hobject_t,pair<int, extent_map> > results;
explicit ClientAsyncReadStatus(
unsigned objects_to_read,
- GenContextURef<hobject_t::bitwisemap<pair<int, extent_map> > &&> &&func)
+ GenContextURef<map<hobject_t,pair<int, extent_map> > &&> &&func)
: objects_to_read(objects_to_read), func(std::move(func)) {}
void complete_object(
const hobject_t &hoid,
template <typename Func>
void objects_read_async_no_cache(
- const hobject_t::bitwisemap<extent_set> &to_read,
+ const map<hobject_t,extent_set> &to_read,
Func &&on_complete) {
- hobject_t::bitwisemap<
- std::list<boost::tuple<uint64_t, uint64_t, uint32_t> > > _to_read;
+ map<hobject_t,std::list<boost::tuple<uint64_t, uint64_t, uint32_t> > > _to_read;
for (auto &&hpair: to_read) {
auto &l = _to_read[hpair.first];
for (auto extent: hpair.second) {
_to_read,
false,
make_gen_lambda_context<
- hobject_t::bitwisemap<pair<int, extent_map> > &&, Func>(
+ map<hobject_t,pair<int, extent_map> > &&, Func>(
std::forward<Func>(on_complete)));
}
void kick_reads() {
RecoveryOp() : state(IDLE) {}
};
friend ostream &operator<<(ostream &lhs, const RecoveryOp &rhs);
- map<hobject_t, RecoveryOp, hobject_t::BitwiseComparator> recovery_ops;
+ map<hobject_t, RecoveryOp> recovery_ops;
void continue_recovery_op(
RecoveryOp &op,
// of the available shards.
bool for_recovery;
- map<hobject_t, read_request_t, hobject_t::BitwiseComparator> to_read;
- map<hobject_t, read_result_t, hobject_t::BitwiseComparator> complete;
+ map<hobject_t, read_request_t> to_read;
+ map<hobject_t, read_result_t> complete;
- map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator> obj_to_source;
- map<pg_shard_t, set<hobject_t, hobject_t::BitwiseComparator> > source_to_obj;
+ map<hobject_t, set<pg_shard_t>> obj_to_source;
+ map<pg_shard_t, set<hobject_t> > source_to_obj;
void dump(Formatter *f) const;
bool do_redundant_reads,
bool for_recovery,
OpRequestRef op,
- map<hobject_t, read_request_t, hobject_t::BitwiseComparator> &&_to_read)
+ map<hobject_t, read_request_t> &&_to_read)
: priority(priority), tid(tid), op(op), do_redundant_reads(do_redundant_reads),
for_recovery(for_recovery), to_read(std::move(_to_read)) {
for (auto &&hpair: to_read) {
map<pg_shard_t, set<ceph_tid_t> > shard_to_read_map;
void start_read_op(
int priority,
- map<hobject_t, read_request_t, hobject_t::BitwiseComparator> &to_read,
+ map<hobject_t, read_request_t> &to_read,
OpRequestRef op,
bool do_redundant_reads, bool for_recovery);
eversion_t roll_forward_to; /// Soon to be generated internally
/// Ancillary also provided from submit_transaction caller
- map<hobject_t, ObjectContextRef, hobject_t::BitwiseComparator> obc_map;
+ map<hobject_t, ObjectContextRef> obc_map;
/// see call_write_ordered
std::list<std::function<void(void)> > on_write;
/// Generated internally
- set<hobject_t, hobject_t::BitwiseComparator> temp_added;
- set<hobject_t, hobject_t::BitwiseComparator> temp_cleared;
+ set<hobject_t> temp_added;
+ set<hobject_t> temp_cleared;
ECTransaction::WritePlan plan;
bool requires_rmw() const { return !plan.to_read.empty(); }
bool using_cache = false;
/// In progress read state;
- hobject_t::bitwisemap<extent_set> pending_read; // subset already being read
- hobject_t::bitwisemap<extent_set> remote_read; // subset we must read
- hobject_t::bitwisemap<extent_map> remote_read_result;
+ map<hobject_t,extent_set> pending_read; // subset already being read
+ map<hobject_t,extent_set> remote_read; // subset we must read
+ map<hobject_t,extent_map> remote_read_result;
bool read_in_progress() const {
return !remote_read.empty() && remote_read_result.empty();
}
const ECUtil::stripe_info_t sinfo;
/// If modified, ensure that the ref is held until the update is applied
- SharedPtrRegistry<hobject_t, ECUtil::HashInfo, hobject_t::BitwiseComparator> unstable_hashinfo_registry;
+ SharedPtrRegistry<hobject_t, ECUtil::HashInfo> unstable_hashinfo_registry;
ECUtil::HashInfoRef get_hash_info(const hobject_t &hoid, bool checks = true,
const map<string,bufferptr> *attr = NULL);
ENCODE_START(1, 1, bl);
::encode(from, bl);
::encode(tid, bl);
- map<hobject_t, list<pair<uint64_t, uint64_t> >, hobject_t::BitwiseComparator> tmp;
- for (map<hobject_t, list<boost::tuple<uint64_t, uint64_t, uint32_t> >, hobject_t::BitwiseComparator>::const_iterator m = to_read.begin();
+ map<hobject_t, list<pair<uint64_t, uint64_t> >> tmp;
+ for (map<hobject_t, list<boost::tuple<uint64_t, uint64_t, uint32_t> >>::const_iterator m = to_read.begin();
m != to_read.end(); ++m) {
list<pair<uint64_t, uint64_t> > tlist;
for (list<boost::tuple<uint64_t, uint64_t, uint32_t> >::const_iterator l = m->second.begin();
::decode(from, bl);
::decode(tid, bl);
if (struct_v == 1) {
- map<hobject_t, list<pair<uint64_t, uint64_t> >, hobject_t::BitwiseComparator>tmp;
+ map<hobject_t, list<pair<uint64_t, uint64_t> >>tmp;
::decode(tmp, bl);
- for (map<hobject_t, list<pair<uint64_t, uint64_t> >, hobject_t::BitwiseComparator>::const_iterator m = tmp.begin();
+ for (map<hobject_t, list<pair<uint64_t, uint64_t> >>::const_iterator m = tmp.begin();
m != tmp.end(); ++m) {
list<boost::tuple<uint64_t, uint64_t, uint32_t> > tlist;
for (list<pair<uint64_t, uint64_t> > ::const_iterator l = m->second.begin();
f->dump_stream("from") << from;
f->dump_unsigned("tid", tid);
f->open_array_section("objects");
- for (map<hobject_t, list<boost::tuple<uint64_t, uint64_t, uint32_t> >, hobject_t::BitwiseComparator>::const_iterator i =
+ for (map<hobject_t, list<boost::tuple<uint64_t, uint64_t, uint32_t> >>::const_iterator i =
to_read.begin();
i != to_read.end();
++i) {
f->close_section();
f->open_array_section("object_attrs_requested");
- for (set<hobject_t,hobject_t::BitwiseComparator>::const_iterator i = attrs_to_read.begin();
+ for (set<hobject_t>::const_iterator i = attrs_to_read.begin();
i != attrs_to_read.end();
++i) {
f->open_object_section("object");
f->dump_stream("from") << from;
f->dump_unsigned("tid", tid);
f->open_array_section("buffers_read");
- for (map<hobject_t, list<pair<uint64_t, bufferlist> >, hobject_t::BitwiseComparator>::const_iterator i =
+ for (map<hobject_t, list<pair<uint64_t, bufferlist> >>::const_iterator i =
buffers_read.begin();
i != buffers_read.end();
++i) {
f->close_section();
f->open_array_section("attrs_returned");
- for (map<hobject_t, map<string, bufferlist>, hobject_t::BitwiseComparator>::const_iterator i =
+ for (map<hobject_t, map<string, bufferlist>>::const_iterator i =
attrs_read.begin();
i != attrs_read.end();
++i) {
f->close_section();
f->open_array_section("errors");
- for (map<hobject_t, int, hobject_t::BitwiseComparator>::const_iterator i = errors.begin();
+ for (map<hobject_t, int>::const_iterator i = errors.begin();
i != errors.end();
++i) {
f->open_object_section("error_pair");
eversion_t trim_to;
eversion_t roll_forward_to;
vector<pg_log_entry_t> log_entries;
- set<hobject_t, hobject_t::BitwiseComparator> temp_added;
- set<hobject_t, hobject_t::BitwiseComparator> temp_removed;
+ set<hobject_t> temp_added;
+ set<hobject_t> temp_removed;
boost::optional<pg_hit_set_history_t> updated_hit_set_history;
bool backfill = false;
ECSubWrite() : tid(0) {}
eversion_t roll_forward_to,
vector<pg_log_entry_t> log_entries,
boost::optional<pg_hit_set_history_t> updated_hit_set_history,
- const set<hobject_t, hobject_t::BitwiseComparator> &temp_added,
- const set<hobject_t, hobject_t::BitwiseComparator> &temp_removed,
+ const set<hobject_t> &temp_added,
+ const set<hobject_t> &temp_removed,
bool backfill)
: from(from), tid(tid), reqid(reqid),
soid(soid), stats(stats), t(t),
struct ECSubRead {
pg_shard_t from;
ceph_tid_t tid;
- map<hobject_t, list<boost::tuple<uint64_t, uint64_t, uint32_t> >, hobject_t::BitwiseComparator> to_read;
- set<hobject_t, hobject_t::BitwiseComparator> attrs_to_read;
+ map<hobject_t, list<boost::tuple<uint64_t, uint64_t, uint32_t> >> to_read;
+ set<hobject_t> attrs_to_read;
void encode(bufferlist &bl, uint64_t features) const;
void decode(bufferlist::iterator &bl);
void dump(Formatter *f) const;
struct ECSubReadReply {
pg_shard_t from;
ceph_tid_t tid;
- map<hobject_t, list<pair<uint64_t, bufferlist> >, hobject_t::BitwiseComparator> buffers_read;
- map<hobject_t, map<string, bufferlist>, hobject_t::BitwiseComparator> attrs_read;
- map<hobject_t, int, hobject_t::BitwiseComparator> errors;
+ map<hobject_t, list<pair<uint64_t, bufferlist> >> buffers_read;
+ map<hobject_t, map<string, bufferlist>> attrs_read;
+ map<hobject_t, int> errors;
void encode(bufferlist &bl) const;
void decode(bufferlist::iterator &bl);
void dump(Formatter *f) const;
pg_t pgid,
bool legacy_log_entries,
const ECUtil::stripe_info_t &sinfo,
- const hobject_t::bitwisemap<extent_map> &partial_extents,
+ const map<hobject_t,extent_map> &partial_extents,
vector<pg_log_entry_t> &entries,
- hobject_t::bitwisemap<extent_map> *written_map,
+ map<hobject_t,extent_map> *written_map,
map<shard_id_t, ObjectStore::Transaction> *transactions,
- set<hobject_t, hobject_t::BitwiseComparator> *temp_added,
- set<hobject_t, hobject_t::BitwiseComparator> *temp_removed,
+ set<hobject_t> *temp_added,
+ set<hobject_t> *temp_removed,
DoutPrefixProvider *dpp)
{
assert(written_map);
assert(temp_added);
assert(temp_removed);
- map<hobject_t, pg_log_entry_t*, hobject_t::BitwiseComparator> obj_to_log;
+ map<hobject_t, pg_log_entry_t*> obj_to_log;
for (auto &&i: entries) {
obj_to_log.insert(make_pair(i.soid, &i));
}
struct WritePlan {
PGTransactionUPtr t;
bool invalidates_cache = false; // Yes, both are possible
- hobject_t::bitwisemap<extent_set> to_read;
- hobject_t::bitwisemap<extent_set> will_write; // superset of to_read
+ map<hobject_t,extent_set> to_read;
+ map<hobject_t,extent_set> will_write; // superset of to_read
- hobject_t::bitwisemap<ECUtil::HashInfoRef> hash_infos;
+ map<hobject_t,ECUtil::HashInfoRef> hash_infos;
};
bool requires_overwrite(
pg_t pgid,
bool legacy_log_entries,
const ECUtil::stripe_info_t &sinfo,
- const hobject_t::bitwisemap<extent_map> &partial_extents,
+ const map<hobject_t,extent_map> &partial_extents,
vector<pg_log_entry_t> &entries,
- hobject_t::bitwisemap<extent_map> *written,
+ map<hobject_t,extent_map> *written,
map<shard_id_t, ObjectStore::Transaction> *transactions,
- set<hobject_t, hobject_t::BitwiseComparator> *temp_added,
- set<hobject_t, hobject_t::BitwiseComparator> *temp_removed,
+ set<hobject_t> *temp_added,
+ set<hobject_t> *temp_removed,
DoutPrefixProvider *dpp);
};
pg->lock();
fout << *pg << std::endl;
- std::map<hobject_t, pg_missing_item, hobject_t::BitwiseComparator>::const_iterator mend =
+ std::map<hobject_t, pg_missing_item>::const_iterator mend =
pg->pg_log.get_missing().get_items().end();
- std::map<hobject_t, pg_missing_item, hobject_t::BitwiseComparator>::const_iterator mi =
+ std::map<hobject_t, pg_missing_item>::const_iterator mi =
pg->pg_log.get_missing().get_items().begin();
for (; mi != mend; ++mi) {
fout << mi->first << " -> " << mi->second << std::endl;
bool agent_valid_iterator;
int agent_ops;
int flush_mode_high_count; //once have one pg with FLUSH_MODE_HIGH then flush objects with high speed
- set<hobject_t, hobject_t::BitwiseComparator> agent_oids;
+ set<hobject_t> agent_oids;
bool agent_active;
struct AgentThread : public Thread {
OSDService *osd;
uint64_t recovery_ops_reserved;
bool recovery_paused;
#ifdef DEBUG_RECOVERY_OIDS
- map<spg_t, set<hobject_t, hobject_t::BitwiseComparator> > recovery_oids;
+ map<spg_t, set<hobject_t> > recovery_oids;
#endif
bool _recover_now(uint64_t *available_pushes);
void _maybe_queue_recovery();
dout(10) << " peer osd." << from << " now " << oinfo << " " << omissing << dendl;
might_have_unfound.insert(from);
- for (map<hobject_t, pg_missing_item, hobject_t::ComparatorWithDefault>::const_iterator i =
+ for (map<hobject_t, pg_missing_item>::const_iterator i =
omissing.get_items().begin();
i != omissing.get_items().end();
++i) {
ldout(pg->cct, 10) << __func__ << ": adding sources in batch "
<< sources.size() << dendl;
unsigned loop = 0;
- for (map<hobject_t, pg_missing_item, hobject_t::ComparatorWithDefault>::const_iterator i = needs_recovery_map.begin();
+ for (map<hobject_t, pg_missing_item>::const_iterator i = needs_recovery_map.begin();
i != needs_recovery_map.end();
++i) {
if (handle && ++loop >= pg->cct->_conf->osd_loop_before_reset_tphandle) {
bool found_missing = false;
unsigned loop = 0;
// found items?
- for (map<hobject_t,pg_missing_item, hobject_t::ComparatorWithDefault>::const_iterator p = needs_recovery_map.begin();
+ for (map<hobject_t,pg_missing_item>::const_iterator p = needs_recovery_map.begin();
p != needs_recovery_map.end();
++p) {
const hobject_t &soid(p->first);
void PG::clear_backoffs()
{
dout(10) << __func__ << " " << dendl;
- map<hobject_t,set<BackoffRef>,hobject_t::BitwiseComparator> ls;
+ map<hobject_t,set<BackoffRef>> ls;
{
Mutex::Locker l(backoff_lock);
ls.swap(backoffs);
}
}
-void PG::requeue_object_waiters(map<hobject_t, list<OpRequestRef>, hobject_t::BitwiseComparator>& m)
+void PG::requeue_object_waiters(map<hobject_t, list<OpRequestRef>>& m)
{
- for (map<hobject_t, list<OpRequestRef>, hobject_t::BitwiseComparator>::iterator it = m.begin();
+ for (map<hobject_t, list<OpRequestRef>>::iterator it = m.begin();
it != m.end();
++it)
requeue_ops(it->second);
void PG::_scan_snaps(ScrubMap &smap)
{
- for (map<hobject_t, ScrubMap::object, hobject_t::BitwiseComparator>::iterator i = smap.objects.begin();
+ for (map<hobject_t, ScrubMap::object>::iterator i = smap.objects.begin();
i != smap.objects.end();
++i) {
const hobject_t &hoid = i->first;
// construct authoritative scrub map for type specific scrubbing
scrubber.cleaned_meta_map.insert(scrubber.primary_scrubmap);
- map<hobject_t, pair<uint32_t, uint32_t>, hobject_t::BitwiseComparator> missing_digest;
+ map<hobject_t, pair<uint32_t, uint32_t>> missing_digest;
if (acting.size() > 1) {
dout(10) << __func__ << " comparing replica scrub maps" << dendl;
stringstream ss;
// Map from object with errors to good peer
- map<hobject_t, list<pg_shard_t>, hobject_t::BitwiseComparator> authoritative;
+ map<hobject_t, list<pg_shard_t>> authoritative;
map<pg_shard_t, ScrubMap *> maps;
dout(2) << __func__ << " osd." << acting[0] << " has "
osd->clog->error(ss);
}
- for (map<hobject_t, list<pg_shard_t>, hobject_t::BitwiseComparator>::iterator i = authoritative.begin();
+ for (map<hobject_t, list<pg_shard_t>>::iterator i = authoritative.begin();
i != authoritative.end();
++i) {
list<pair<ScrubMap::object, pg_shard_t> > good_peers;
good_peers));
}
- for (map<hobject_t, list<pg_shard_t>, hobject_t::BitwiseComparator>::iterator i = authoritative.begin();
+ for (map<hobject_t, list<pg_shard_t>>::iterator i = authoritative.begin();
i != authoritative.end();
++i) {
scrubber.cleaned_meta_map.objects.erase(i->first);
osd->clog->error(ss);
if (repair) {
state_clear(PG_STATE_CLEAN);
- for (map<hobject_t, list<pair<ScrubMap::object, pg_shard_t> >, hobject_t::BitwiseComparator>::iterator i =
+ for (map<hobject_t, list<pair<ScrubMap::object, pg_shard_t> >>::iterator i =
scrubber.authoritative.begin();
i != scrubber.authoritative.end();
++i) {
ghobject_t pgmeta_oid;
class MissingLoc {
- map<hobject_t, pg_missing_item, hobject_t::BitwiseComparator> needs_recovery_map;
- map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator > missing_loc;
+ map<hobject_t, pg_missing_item> needs_recovery_map;
+ map<hobject_t, set<pg_shard_t> > missing_loc;
set<pg_shard_t> missing_loc_sources;
PG *pg;
set<pg_shard_t> empty_set;
bool needs_recovery(
const hobject_t &hoid,
eversion_t *v = 0) const {
- map<hobject_t, pg_missing_item, hobject_t::BitwiseComparator>::const_iterator i =
+ map<hobject_t, pg_missing_item>::const_iterator i =
needs_recovery_map.find(hoid);
if (i == needs_recovery_map.end())
return false;
const set<pg_shard_t> &acting) const;
uint64_t num_unfound() const {
uint64_t ret = 0;
- for (map<hobject_t, pg_missing_item, hobject_t::BitwiseComparator>::const_iterator i =
+ for (map<hobject_t, pg_missing_item>::const_iterator i =
needs_recovery_map.begin();
i != needs_recovery_map.end();
++i) {
}
bool have_unfound() const {
- for (map<hobject_t, pg_missing_item, hobject_t::BitwiseComparator>::const_iterator i =
+ for (map<hobject_t, pg_missing_item>::const_iterator i =
needs_recovery_map.begin();
i != needs_recovery_map.end();
++i) {
missing_loc[hoid].erase(location);
}
void add_active_missing(const pg_missing_t &missing) {
- for (map<hobject_t, pg_missing_item, hobject_t::BitwiseComparator>::const_iterator i =
+ for (map<hobject_t, pg_missing_item>::const_iterator i =
missing.get_items().begin();
i != missing.get_items().end();
++i) {
- map<hobject_t, pg_missing_item, hobject_t::BitwiseComparator>::const_iterator j =
+ map<hobject_t, pg_missing_item>::const_iterator j =
needs_recovery_map.find(i->first);
if (j == needs_recovery_map.end()) {
needs_recovery_map.insert(*i);
return missing_loc.count(hoid) ?
missing_loc.find(hoid)->second : empty_set;
}
- const map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator> &get_missing_locs() const {
+ const map<hobject_t, set<pg_shard_t>> &get_missing_locs() const {
return missing_loc;
}
- const map<hobject_t, pg_missing_item, hobject_t::BitwiseComparator> &get_needs_recovery() const {
+ const map<hobject_t, pg_missing_item> &get_needs_recovery() const {
return needs_recovery_map;
}
} missing_loc;
int recovery_ops_active;
set<pg_shard_t> waiting_on_backfill;
#ifdef DEBUG_RECOVERY_OIDS
- set<hobject_t, hobject_t::BitwiseComparator> recovering_oids;
+ set<hobject_t> recovering_oids;
#endif
protected:
struct BackfillInterval {
// info about a backfill interval on a peer
eversion_t version; /// version at which the scan occurred
- map<hobject_t,eversion_t,hobject_t::Comparator> objects;
+ map<hobject_t,eversion_t> objects;
bool sort_bitwise;
hobject_t begin;
hobject_t end;
explicit BackfillInterval(bool bitwise=true)
- : objects(hobject_t::Comparator(bitwise)),
- sort_bitwise(bitwise)
+ : sort_bitwise(bitwise)
{}
/// clear content
/// clear objects list only
void clear_objects() {
- // make sure we preserve the allocator and ordering!
- objects = map<hobject_t,eversion_t,hobject_t::Comparator>(
- hobject_t::Comparator(sort_bitwise));
+ objects.clear();
}
/// reinstantiate with a new start+end position and sort order
f->dump_stream("begin") << begin;
f->dump_stream("end") << end;
f->open_array_section("objects");
- for (map<hobject_t, eversion_t, hobject_t::Comparator>::const_iterator i =
+ for (map<hobject_t, eversion_t>::const_iterator i =
objects.begin();
i != objects.end();
++i) {
list<OpRequestRef> waiting_for_cache_not_full;
list<OpRequestRef> waiting_for_all_missing;
- map<hobject_t, list<OpRequestRef>, hobject_t::BitwiseComparator> waiting_for_unreadable_object,
+ map<hobject_t, list<OpRequestRef>> waiting_for_unreadable_object,
waiting_for_degraded_object,
waiting_for_blocked_object;
- set<
- hobject_t,
- hobject_t::BitwiseComparator> objects_blocked_on_cache_full;
- map<
- hobject_t,
- snapid_t,
- hobject_t::BitwiseComparator> objects_blocked_on_degraded_snap;
- map<
- hobject_t,
- ObjectContextRef,
- hobject_t::BitwiseComparator> objects_blocked_on_snap_promotion;
+ set<hobject_t> objects_blocked_on_cache_full;
+ map<hobject_t,snapid_t> objects_blocked_on_degraded_snap;
+ map<hobject_t,ObjectContextRef> objects_blocked_on_snap_promotion;
// Callbacks should assume pg (and nothing else) is locked
- map<hobject_t, list<Context*>, hobject_t::BitwiseComparator> callbacks_for_degraded_object;
+ map<hobject_t, list<Context*>> callbacks_for_degraded_object;
map<eversion_t,
list<pair<OpRequestRef, version_t> > > waiting_for_ondisk;
void split_ops(PG *child, unsigned split_bits);
- void requeue_object_waiters(map<hobject_t, list<OpRequestRef>, hobject_t::BitwiseComparator>& m);
+ void requeue_object_waiters(map<hobject_t, list<OpRequestRef>>& m);
void requeue_op(OpRequestRef op);
void requeue_ops(list<OpRequestRef> &l);
// -- backoff --
Mutex backoff_lock; // orders inside Backoff::lock
- map<hobject_t,set<BackoffRef>,hobject_t::BitwiseComparator> backoffs;
+ map<hobject_t,set<BackoffRef>> backoffs;
void add_backoff(SessionRef s, const hobject_t& begin, const hobject_t& end);
void release_backoffs(const hobject_t& begin, const hobject_t& end);
bool auto_repair;
// Maps from objects with errors to missing/inconsistent peers
- map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator> missing;
- map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator> inconsistent;
+ map<hobject_t, set<pg_shard_t>> missing;
+ map<hobject_t, set<pg_shard_t>> inconsistent;
// Map from object with errors to good peers
- map<hobject_t, list<pair<ScrubMap::object, pg_shard_t> >, hobject_t::BitwiseComparator> authoritative;
+ map<hobject_t, list<pair<ScrubMap::object, pg_shard_t> >> authoritative;
// Cleaned map pending snap metadata scrub
ScrubMap cleaned_meta_map;
const hobject_t &begin, const hobject_t &end) = 0;
virtual void scrub_snapshot_metadata(
ScrubMap &map,
- const std::map<hobject_t, pair<uint32_t, uint32_t>, hobject_t::BitwiseComparator> &missing_digest) { }
+ const std::map<hobject_t, pair<uint32_t, uint32_t>> &missing_digest) { }
virtual void _scrub_clear_state() { }
virtual void _scrub_finish() { }
virtual void split_colls(
{
dout(10) << __func__ << dendl;
// clear temp
- for (set<hobject_t, hobject_t::BitwiseComparator>::iterator i = temp_contents.begin();
+ for (set<hobject_t>::iterator i = temp_contents.begin();
i != temp_contents.end();
++i) {
dout(10) << __func__ << ": Removing oid "
for (map<pg_shard_t, ScrubMap *>::const_iterator j = maps.begin();
j != maps.end();
++j) {
- map<hobject_t, ScrubMap::object, hobject_t::BitwiseComparator>::iterator i =
+ map<hobject_t, ScrubMap::object>::iterator i =
j->second->objects.find(obj);
if (i == j->second->objects.end()) {
continue;
void PGBackend::be_compare_scrubmaps(
const map<pg_shard_t,ScrubMap*> &maps,
bool repair,
- map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator> &missing,
- map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator> &inconsistent,
- map<hobject_t, list<pg_shard_t>, hobject_t::BitwiseComparator> &authoritative,
- map<hobject_t, pair<uint32_t,uint32_t>, hobject_t::BitwiseComparator> &missing_digest,
+ map<hobject_t, set<pg_shard_t>> &missing,
+ map<hobject_t, set<pg_shard_t>> &inconsistent,
+ map<hobject_t, list<pg_shard_t>> &authoritative,
+ map<hobject_t, pair<uint32_t,uint32_t>> &missing_digest,
int &shallow_errors, int &deep_errors,
Scrub::Store *store,
const spg_t& pgid,
const vector<int> &acting,
ostream &errorstream)
{
- map<hobject_t,ScrubMap::object, hobject_t::BitwiseComparator>::const_iterator i;
- map<pg_shard_t, ScrubMap *, hobject_t::BitwiseComparator>::const_iterator j;
- set<hobject_t, hobject_t::BitwiseComparator> master_set;
+ map<hobject_t,ScrubMap::object>::const_iterator i;
+ map<pg_shard_t, ScrubMap *>::const_iterator j;
+ set<hobject_t> master_set;
utime_t now = ceph_clock_now();
// Construct master set
}
// Check maps against master set and each other
- for (set<hobject_t, hobject_t::BitwiseComparator>::const_iterator k = master_set.begin();
+ for (set<hobject_t>::const_iterator k = master_set.begin();
k != master_set.end();
++k) {
object_info_t auth_oi;
virtual std::string gen_dbg_prefix() const = 0;
- virtual const map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator> &get_missing_loc_shards()
+ virtual const map<hobject_t, set<pg_shard_t>> &get_missing_loc_shards()
const = 0;
virtual const pg_missing_tracker_t &get_local_missing() const = 0;
virtual void dump_recovery_info(Formatter *f) const = 0;
private:
- set<hobject_t, hobject_t::BitwiseComparator> temp_contents;
+ set<hobject_t> temp_contents;
public:
// Track contents of temp collection, clear on reset
void add_temp_obj(const hobject_t &oid) {
temp_contents.insert(oid);
}
- void add_temp_objs(const set<hobject_t, hobject_t::BitwiseComparator> &oids) {
+ void add_temp_objs(const set<hobject_t> &oids) {
temp_contents.insert(oids.begin(), oids.end());
}
void clear_temp_obj(const hobject_t &oid) {
temp_contents.erase(oid);
}
- void clear_temp_objs(const set<hobject_t, hobject_t::BitwiseComparator> &oids) {
- for (set<hobject_t, hobject_t::BitwiseComparator>::const_iterator i = oids.begin();
+ void clear_temp_objs(const set<hobject_t> &oids) {
+ for (set<hobject_t>::const_iterator i = oids.begin();
i != oids.end();
++i) {
temp_contents.erase(*i);
void be_compare_scrubmaps(
const map<pg_shard_t,ScrubMap*> &maps,
bool repair,
- map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator> &missing,
- map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator> &inconsistent,
- map<hobject_t, list<pg_shard_t>, hobject_t::BitwiseComparator> &authoritative,
- map<hobject_t, pair<uint32_t,uint32_t>, hobject_t::BitwiseComparator> &missing_digest,
+ map<hobject_t, set<pg_shard_t>> &missing,
+ map<hobject_t, set<pg_shard_t>> &inconsistent,
+ map<hobject_t, list<pg_shard_t>> &authoritative,
+ map<hobject_t, pair<uint32_t,uint32_t>> &missing_digest,
int &shallow_errors, int &deep_errors,
Scrub::Store *store,
const spg_t& pgid,
we will send the peer enough log to arrive at the same state.
*/
- for (map<hobject_t, pg_missing_item, hobject_t::BitwiseComparator>::const_iterator i = omissing.get_items().begin();
+ for (map<hobject_t, pg_missing_item>::const_iterator i = omissing.get_items().begin();
i != omissing.get_items().end();
++i) {
dout(20) << " before missing " << i->first << " need " << i->second.need
// The logs must overlap.
assert(log.head >= olog.tail && olog.head >= log.tail);
- for (map<hobject_t, pg_missing_item, hobject_t::BitwiseComparator>::const_iterator i = missing.get_items().begin();
+ for (map<hobject_t, pg_missing_item>::const_iterator i = missing.get_items().begin();
i != missing.get_items().end();
++i) {
dout(20) << "pg_missing_t sobject: " << i->first << dendl;
//////////////////// get or set missing ////////////////////
const pg_missing_tracker_t& get_missing() const { return missing; }
- void resort_missing(bool sort_bitwise) {
- missing.resort(sort_bitwise);
- }
-
void revise_have(hobject_t oid, eversion_t have) {
missing.revise_have(oid, have);
}
protected:
static void split_by_object(
mempool::osd::list<pg_log_entry_t> &entries,
- map<hobject_t, mempool::osd::list<pg_log_entry_t>, hobject_t::BitwiseComparator> *out_entries) {
+ map<hobject_t, mempool::osd::list<pg_log_entry_t>> *out_entries) {
while (!entries.empty()) {
mempool::osd::list<pg_log_entry_t> &out_list = (*out_entries)[entries.front().soid];
out_list.splice(out_list.end(), entries, entries.begin());
LogEntryHandler *rollbacker, ///< [in] optional rollbacker object
const DoutPrefixProvider *dpp ///< [in] logging provider
) {
- map<hobject_t, mempool::osd::list<pg_log_entry_t>, hobject_t::BitwiseComparator > split;
+ map<hobject_t, mempool::osd::list<pg_log_entry_t> > split;
split_by_object(entries, &split);
- for (map<hobject_t, mempool::osd::list<pg_log_entry_t>, hobject_t::BitwiseComparator>::iterator i = split.begin();
+ for (map<hobject_t, mempool::osd::list<pg_log_entry_t>>::iterator i = split.begin();
i != split.end();
++i) {
_merge_object_divergent_entries(
<< info.last_complete
<< "," << info.last_update << "]" << dendl;
- set<hobject_t, hobject_t::BitwiseComparator> did;
- set<hobject_t, hobject_t::BitwiseComparator> checked;
- set<hobject_t, hobject_t::BitwiseComparator> skipped;
+ set<hobject_t> did;
+ set<hobject_t> checked;
+ set<hobject_t> skipped;
for (list<pg_log_entry_t>::reverse_iterator i = log.log.rbegin();
i != log.log.rend();
++i) {
*/
class PGTransaction {
public:
- map<hobject_t, ObjectContextRef, hobject_t::BitwiseComparator> obc_map;
+ map<hobject_t, ObjectContextRef> obc_map;
class ObjectOperation {
public:
friend class PGTransaction;
};
- map<hobject_t, ObjectOperation, hobject_t::BitwiseComparator> op_map;
+ map<hobject_t, ObjectOperation> op_map;
private:
ObjectOperation &get_object_op_for_modify(const hobject_t &hoid) {
auto &op = op_map[hoid];
*/
template <typename T>
void safe_create_traverse(T &&t) {
- map<hobject_t, list<hobject_t>, hobject_t::BitwiseComparator> dgraph;
+ map<hobject_t, list<hobject_t>> dgraph;
list<hobject_t> stack;
// Populate stack with roots, dgraph with edges
missing_loc.recovered(soid);
publish_stats_to_osd();
dout(10) << "pushed " << soid << " to all replicas" << dendl;
- map<hobject_t, ObjectContextRef, hobject_t::BitwiseComparator>::iterator i = recovering.find(soid);
+ map<hobject_t, ObjectContextRef>::iterator i = recovering.find(soid);
assert(i != recovering.end());
// recover missing won't have had an obc, but it gets filled in
if (!missing_loc.needs_recovery(soid, &v))
return;
- map<hobject_t, ObjectContextRef, hobject_t::BitwiseComparator>::const_iterator p = recovering.find(soid);
+ map<hobject_t, ObjectContextRef>::const_iterator p = recovering.find(soid);
if (p != recovering.end()) {
dout(7) << "object " << soid << " v " << v << ", already recovering." << dendl;
} else if (missing_loc.is_unfound(soid)) {
}
f->dump_int("num_missing", missing.num_missing());
f->dump_int("num_unfound", get_num_unfound());
- const map<hobject_t, pg_missing_item, hobject_t::BitwiseComparator> &needs_recovery_map =
+ const map<hobject_t, pg_missing_item> &needs_recovery_map =
missing_loc.get_needs_recovery();
- map<hobject_t, pg_missing_item, hobject_t::BitwiseComparator>::const_iterator p =
+ map<hobject_t, pg_missing_item>::const_iterator p =
needs_recovery_map.upper_bound(offset);
{
f->open_array_section("objects");
assert(snapid == CEPH_NOSNAP || pg_log.get_missing().get_items().empty());
- // ensure sort order is correct
- pg_log.resort_missing(get_sort_bitwise());
-
- map<hobject_t, pg_missing_item, hobject_t::ComparatorWithDefault>::const_iterator missing_iter =
+ map<hobject_t, pg_missing_item>::const_iterator missing_iter =
pg_log.get_missing().get_items().lower_bound(current);
vector<hobject_t>::iterator ls_iter = sentries.begin();
hobject_t _max = hobject_t::get_max();
assert(snapid == CEPH_NOSNAP || pg_log.get_missing().get_items().empty());
- // ensure sort order is correct
- pg_log.resort_missing(get_sort_bitwise());
-
- map<hobject_t, pg_missing_item, hobject_t::ComparatorWithDefault>::const_iterator missing_iter =
+ map<hobject_t, pg_missing_item>::const_iterator missing_iter =
pg_log.get_missing().get_items().lower_bound(current);
vector<hobject_t>::iterator ls_iter = sentries.begin();
hobject_t _max = hobject_t::get_max();
_pool.info, curmap, this, coll_t(p), ch, o->store, cct)),
object_contexts(o->cct, o->cct->_conf->osd_pg_object_context_cache_count),
snapset_contexts_lock("PrimaryLogPG::snapset_contexts_lock"),
- backfills_in_flight(hobject_t::Comparator(true)),
- pending_backfill_updates(hobject_t::Comparator(true)),
new_backfill(false),
temp_seq(0),
snap_trimmer_machine(this)
// verify there is in fact a flush in progress
// FIXME: we could make this a stronger test.
- map<hobject_t,FlushOpRef, hobject_t::BitwiseComparator>::iterator p = flush_ops.find(obc->obs.oi.soid);
+ map<hobject_t,FlushOpRef>::iterator p = flush_ops.find(obc->obs.oi.soid);
if (p == flush_ops.end()) {
dout(10) << __func__ << " no flush in progress, aborting" << dendl;
reply_ctx(ctx, -EINVAL);
}
proxyread_ops.erase(tid);
- map<hobject_t, list<OpRequestRef>, hobject_t::BitwiseComparator>::iterator q = in_progress_proxy_ops.find(oid);
+ map<hobject_t, list<OpRequestRef>>::iterator q = in_progress_proxy_ops.find(oid);
if (q == in_progress_proxy_ops.end()) {
dout(10) << __func__ << " no in_progress_proxy_ops found" << dendl;
return;
void PrimaryLogPG::kick_proxy_ops_blocked(hobject_t& soid)
{
- map<hobject_t, list<OpRequestRef>, hobject_t::BitwiseComparator>::iterator p = in_progress_proxy_ops.find(soid);
+ map<hobject_t, list<OpRequestRef>>::iterator p = in_progress_proxy_ops.find(soid);
if (p == in_progress_proxy_ops.end())
return;
}
if (requeue) {
- map<hobject_t, list<OpRequestRef>, hobject_t::BitwiseComparator>::iterator p =
+ map<hobject_t, list<OpRequestRef>>::iterator p =
in_progress_proxy_ops.begin();
while (p != in_progress_proxy_ops.end()) {
list<OpRequestRef>& ls = p->second;
* for this case we don't use DONTNEED.
*/
unsigned src_fadvise_flags = LIBRADOS_OP_FLAG_FADVISE_SEQUENTIAL;
- map<hobject_t, list<OpRequestRef>, hobject_t::BitwiseComparator>::iterator q = in_progress_proxy_ops.find(obc->obs.oi.soid);
+ map<hobject_t, list<OpRequestRef>>::iterator q = in_progress_proxy_ops.find(obc->obs.oi.soid);
if (q == in_progress_proxy_ops.end()) {
src_fadvise_flags |= LIBRADOS_OP_FLAG_FADVISE_DONTNEED;
}
{
dout(10) << __func__ << " " << oid << " tid " << tid
<< " " << cpp_strerror(r) << dendl;
- map<hobject_t,CopyOpRef, hobject_t::BitwiseComparator>::iterator p = copy_ops.find(oid);
+ map<hobject_t,CopyOpRef>::iterator p = copy_ops.find(oid);
if (p == copy_ops.end()) {
dout(10) << __func__ << " no copy_op found" << dendl;
return;
// pass error to everyone blocked on this object
// FIXME: this is pretty sloppy, but at this point we got
// something unexpected and don't have many other options.
- map<hobject_t,list<OpRequestRef>, hobject_t::BitwiseComparator>::iterator blocked_iter =
+ map<hobject_t,list<OpRequestRef>>::iterator blocked_iter =
waiting_for_blocked_object.find(soid);
if (blocked_iter != waiting_for_blocked_object.end()) {
while (!blocked_iter->second.empty()) {
void PrimaryLogPG::cancel_copy_ops(bool requeue)
{
dout(10) << __func__ << dendl;
- map<hobject_t,CopyOpRef, hobject_t::BitwiseComparator>::iterator p = copy_ops.begin();
+ map<hobject_t,CopyOpRef>::iterator p = copy_ops.begin();
while (p != copy_ops.end()) {
// requeue this op? can I queue up all of them?
cancel_copy((p++)->second, requeue);
if (blocking)
obc->start_block();
- map<hobject_t,FlushOpRef, hobject_t::BitwiseComparator>::iterator p = flush_ops.find(soid);
+ map<hobject_t,FlushOpRef>::iterator p = flush_ops.find(soid);
if (p != flush_ops.end()) {
FlushOpRef fop = p->second;
if (fop->op == op) {
{
dout(10) << __func__ << " " << oid << " tid " << tid
<< " " << cpp_strerror(r) << dendl;
- map<hobject_t,FlushOpRef, hobject_t::BitwiseComparator>::iterator p = flush_ops.find(oid);
+ map<hobject_t,FlushOpRef>::iterator p = flush_ops.find(oid);
if (p == flush_ops.end()) {
dout(10) << __func__ << " no flush_op found" << dendl;
return;
void PrimaryLogPG::cancel_flush_ops(bool requeue)
{
dout(10) << __func__ << dendl;
- map<hobject_t,FlushOpRef, hobject_t::BitwiseComparator>::iterator p = flush_ops.begin();
+ map<hobject_t,FlushOpRef>::iterator p = flush_ops.begin();
while (p != flush_ops.end()) {
cancel_flush((p++)->second, requeue);
}
return;
}
- map<hobject_t, list<OpRequestRef>, hobject_t::BitwiseComparator>::iterator p = waiting_for_blocked_object.find(soid);
+ map<hobject_t, list<OpRequestRef>>::iterator p = waiting_for_blocked_object.find(soid);
if (p != waiting_for_blocked_object.end()) {
list<OpRequestRef>& ls = p->second;
dout(10) << __func__ << " " << soid << " requeuing " << ls.size() << " requests" << dendl;
{
Mutex::Locker l(snapset_contexts_lock);
SnapSetContext *ssc;
- map<hobject_t, SnapSetContext*, hobject_t::BitwiseComparator>::iterator p = snapset_contexts.find(
+ map<hobject_t, SnapSetContext*>::iterator p = snapset_contexts.find(
oid.get_snapdir());
if (p != snapset_contexts.end()) {
if (can_create || p->second->exists) {
mempool::osd::list<pg_log_entry_t> log_entries;
utime_t mtime = ceph_clock_now();
- map<hobject_t, pg_missing_item, hobject_t::ComparatorWithDefault>::const_iterator m =
+ map<hobject_t, pg_missing_item>::const_iterator m =
missing_loc.get_needs_recovery().begin();
- map<hobject_t, pg_missing_item, hobject_t::ComparatorWithDefault>::const_iterator mend =
+ map<hobject_t, pg_missing_item>::const_iterator mend =
missing_loc.get_needs_recovery().end();
ObcLockManager manager;
void PrimaryLogPG::_on_new_interval()
{
- // re-sort obc map?
- if (object_contexts.get_comparator().bitwise != get_sort_bitwise()) {
- dout(20) << __func__ << " resorting object_contexts" << dendl;
- object_contexts.reset_comparator(
- hobject_t::ComparatorWithDefault(get_sort_bitwise()));
- }
}
void PrimaryLogPG::on_change(ObjectStore::Transaction *t)
} else {
waiting_for_unreadable_object.clear();
}
- for (map<hobject_t,list<OpRequestRef>, hobject_t::BitwiseComparator>::iterator p = waiting_for_degraded_object.begin();
+ for (map<hobject_t,list<OpRequestRef>>::iterator p = waiting_for_degraded_object.begin();
p != waiting_for_degraded_object.end();
waiting_for_degraded_object.erase(p++)) {
release_backoffs(p->first);
// requeues waiting_for_scrub
scrub_clear_state();
- for (map<hobject_t,list<OpRequestRef>, hobject_t::BitwiseComparator>::iterator p = waiting_for_blocked_object.begin();
+ for (auto p = waiting_for_blocked_object.begin();
p != waiting_for_blocked_object.end();
waiting_for_blocked_object.erase(p++)) {
if (is_primary())
else
p->second.clear();
}
- for (map<hobject_t, list<Context*>, hobject_t::BitwiseComparator>::iterator i =
- callbacks_for_degraded_object.begin();
+ for (auto i = callbacks_for_degraded_object.begin();
i != callbacks_for_degraded_object.end();
) {
finish_degraded_object((i++)->first);
recovering_oids.clear();
#endif
last_backfill_started = hobject_t();
- set<hobject_t, hobject_t::Comparator>::iterator i = backfills_in_flight.begin();
+ set<hobject_t>::iterator i = backfills_in_flight.begin();
while (i != backfills_in_flight.end()) {
assert(recovering.count(*i));
backfills_in_flight.erase(i++);
}
list<OpRequestRef> blocked_ops;
- for (map<hobject_t, ObjectContextRef, hobject_t::BitwiseComparator>::iterator i = recovering.begin();
+ for (map<hobject_t, ObjectContextRef>::iterator i = recovering.begin();
i != recovering.end();
recovering.erase(i++)) {
if (i->second) {
<< missing_loc_sources << dendl;
// filter missing_loc
- map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator>::iterator p = missing_loc.begin();
+ map<hobject_t, set<pg_shard_t>>::iterator p = missing_loc.begin();
while (p != missing_loc.end()) {
set<pg_shard_t>::iterator q = p->second.begin();
while (q != p->second.end())
}
dout(10) << __func__ << ": recover_object_replicas(" << soid << ")" << dendl;
- map<hobject_t,pg_missing_item, hobject_t::ComparatorWithDefault>::const_iterator r = m.get_items().find(soid);
+ map<hobject_t,pg_missing_item>::const_iterator r = m.get_items().find(soid);
started += prep_object_replica_pushes(soid, r->second.need,
h);
}
backfill_info.reset(last_backfill_started,
get_sort_bitwise());
- // initialize comparators
- backfills_in_flight = set<hobject_t, hobject_t::Comparator>(
- hobject_t::Comparator(get_sort_bitwise()));
- pending_backfill_updates = map<hobject_t, pg_stat_t, hobject_t::Comparator>(
- hobject_t::Comparator(get_sort_bitwise()));
+ backfills_in_flight.clear();
+ pending_backfill_updates.clear();
}
// sanity check sort orders
i != peer_backfill_info.end();
++i) {
assert(i->second.sort_bitwise == get_sort_bitwise());
- assert(i->second.objects.key_comp().bitwise == get_sort_bitwise());
}
- assert(backfills_in_flight.key_comp().bitwise == get_sort_bitwise());
- assert(pending_backfill_updates.key_comp().bitwise == get_sort_bitwise());
for (set<pg_shard_t>::iterator i = backfill_targets.begin();
i != backfill_targets.end();
vector<boost::tuple<hobject_t, eversion_t,
ObjectContextRef, vector<pg_shard_t> > > to_push;
vector<boost::tuple<hobject_t, eversion_t, pg_shard_t> > to_remove;
- set<hobject_t, hobject_t::BitwiseComparator> add_to_stat;
+ set<hobject_t> add_to_stat;
for (set<pg_shard_t>::iterator i = backfill_targets.begin();
i != backfill_targets.end();
hobject_t backfill_pos =
std::min(backfill_info.begin, earliest_peer_backfill());
- for (set<hobject_t, hobject_t::BitwiseComparator>::iterator i = add_to_stat.begin();
+ for (set<hobject_t>::iterator i = add_to_stat.begin();
i != add_to_stat.end();
++i) {
ObjectContextRef obc = get_object_context(*i, false);
pgbackend->run_recovery_op(h, get_recovery_op_priority());
dout(5) << "backfill_pos is " << backfill_pos << dendl;
- for (set<hobject_t, hobject_t::Comparator>::iterator i = backfills_in_flight.begin();
+ for (set<hobject_t>::iterator i = backfills_in_flight.begin();
i != backfills_in_flight.end();
++i) {
dout(20) << *i << " is still in flight" << dendl;
backfill_pos : *(backfills_in_flight.begin());
hobject_t new_last_backfill = earliest_backfill();
dout(10) << "starting new_last_backfill at " << new_last_backfill << dendl;
- for (map<hobject_t, pg_stat_t, hobject_t::Comparator>::iterator i =
+ for (map<hobject_t, pg_stat_t>::iterator i =
pending_backfill_updates.begin();
i != pending_backfill_updates.end() &&
i->first < next_backfill_to_complete;
return;
// just scan the log.
- set<hobject_t, hobject_t::BitwiseComparator> did;
+ set<hobject_t> did;
for (list<pg_log_entry_t>::const_reverse_iterator p = pg_log.get_log().log.rbegin();
p != pg_log.get_log().log.rend();
++p) {
*/
void PrimaryLogPG::scrub_snapshot_metadata(
ScrubMap &scrubmap,
- const map<hobject_t, pair<uint32_t, uint32_t>, hobject_t::BitwiseComparator> &missing_digest)
+ const map<hobject_t, pair<uint32_t, uint32_t>> &missing_digest)
{
dout(10) << __func__ << dendl;
bufferlist last_data;
- for (map<hobject_t,ScrubMap::object, hobject_t::BitwiseComparator>::reverse_iterator
+ for (map<hobject_t,ScrubMap::object>::reverse_iterator
p = scrubmap.objects.rbegin(); p != scrubmap.objects.rend(); ++p) {
const hobject_t& soid = p->first;
soid_error = inconsistent_snapset_wrapper{soid};
if (head && head_error.errors)
scrubber.store->add_snap_error(pool.id, head_error);
- for (map<hobject_t,pair<uint32_t,uint32_t>, hobject_t::BitwiseComparator>::const_iterator p =
+ for (map<hobject_t,pair<uint32_t,uint32_t>>::const_iterator p =
missing_digest.begin();
p != missing_digest.end();
++p) {
std::string gen_dbg_prefix() const override { return gen_prefix(); }
- const map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator>
+ const map<hobject_t, set<pg_shard_t>>
&get_missing_loc_shards() const override {
return missing_loc.get_missing_locs();
}
bool already_ack(eversion_t v);
// projected object info
- SharedLRU<hobject_t, ObjectContext, hobject_t::ComparatorWithDefault> object_contexts;
+ SharedLRU<hobject_t, ObjectContext> object_contexts;
// map from oid.snapdir() to SnapSetContext *
- map<hobject_t, SnapSetContext*, hobject_t::BitwiseComparator> snapset_contexts;
+ map<hobject_t, SnapSetContext*> snapset_contexts;
Mutex snapset_contexts_lock;
// debug order that client ops are applied
- map<hobject_t, map<client_t, ceph_tid_t>, hobject_t::BitwiseComparator> debug_op_order;
+ map<hobject_t, map<client_t, ceph_tid_t>> debug_op_order;
void populate_obc_watchers(ObjectContextRef obc);
void check_blacklisted_obc_watchers(ObjectContextRef obc);
}
void put_snapset_context(SnapSetContext *ssc);
- map<hobject_t, ObjectContextRef, hobject_t::BitwiseComparator> recovering;
+ map<hobject_t, ObjectContextRef> recovering;
/*
* Backfill
* - are not included in pg stats (yet)
* - have their stats in pending_backfill_updates on the primary
*/
- set<hobject_t, hobject_t::Comparator> backfills_in_flight;
- map<hobject_t, pg_stat_t, hobject_t::Comparator> pending_backfill_updates;
+ set<hobject_t> backfills_in_flight;
+ map<hobject_t, pg_stat_t> pending_backfill_updates;
void dump_recovery_info(Formatter *f) const override {
f->open_array_section("backfill_targets");
}
{
f->open_array_section("backfills_in_flight");
- for (set<hobject_t, hobject_t::BitwiseComparator>::const_iterator i = backfills_in_flight.begin();
+ for (set<hobject_t>::const_iterator i = backfills_in_flight.begin();
i != backfills_in_flight.end();
++i) {
f->dump_stream("object") << *i;
}
{
f->open_array_section("recovering");
- for (map<hobject_t, ObjectContextRef, hobject_t::BitwiseComparator>::const_iterator i = recovering.begin();
+ for (map<hobject_t, ObjectContextRef>::const_iterator i = recovering.begin();
i != recovering.end();
++i) {
f->dump_stream("object") << i->first;
void recover_got(hobject_t oid, eversion_t v);
// -- copyfrom --
- map<hobject_t, CopyOpRef, hobject_t::BitwiseComparator> copy_ops;
+ map<hobject_t, CopyOpRef> copy_ops;
int fill_in_copy_get(
OpContext *ctx,
friend struct C_Copyfrom;
// -- flush --
- map<hobject_t, FlushOpRef, hobject_t::BitwiseComparator> flush_ops;
+ map<hobject_t, FlushOpRef> flush_ops;
/// start_flush takes ownership of on_flush iff ret == -EINPROGRESS
int start_flush(
const hobject_t &begin, const hobject_t &end) override;
virtual void scrub_snapshot_metadata(
ScrubMap &map,
- const std::map<hobject_t, pair<uint32_t, uint32_t>,
- hobject_t::BitwiseComparator> &missing_digest) override;
+ const std::map<hobject_t, pair<uint32_t, uint32_t>> &missing_digest) override;
virtual void _scrub_clear_state() override;
virtual void _scrub_finish() override;
object_stat_collection_t scrub_cstat;
bool pgls_filter(PGLSFilter *filter, hobject_t& sobj, bufferlist& outdata);
int get_pgls_filter(bufferlist::iterator& iter, PGLSFilter **pfilter);
- map<hobject_t, list<OpRequestRef>, hobject_t::BitwiseComparator> in_progress_proxy_ops;
+ map<hobject_t, list<OpRequestRef>> in_progress_proxy_ops;
void kick_proxy_ops_blocked(hobject_t& soid);
void cancel_proxy_ops(bool requeue);
};
struct SnapTrimmer : public boost::statechart::state_machine< SnapTrimmer, NotTrimming > {
PrimaryLogPG *pg;
- set<hobject_t, hobject_t::BitwiseComparator> in_flight;
+ set<hobject_t> in_flight;
snapid_t snap_to_trim;
explicit SnapTrimmer(PrimaryLogPG *pg) : pg(pg) {}
~SnapTrimmer();
void ReplicatedBackend::check_recovery_sources(const OSDMapRef& osdmap)
{
- for(map<pg_shard_t, set<hobject_t, hobject_t::BitwiseComparator> >::iterator i = pull_from_peer.begin();
+ for(map<pg_shard_t, set<hobject_t> >::iterator i = pull_from_peer.begin();
i != pull_from_peer.end();
) {
if (osdmap->is_down(i->first.osd)) {
dout(10) << "check_recovery_sources resetting pulls from osd." << i->first
<< ", osdmap has it marked down" << dendl;
- for (set<hobject_t, hobject_t::BitwiseComparator>::iterator j = i->second.begin();
+ for (set<hobject_t>::iterator j = i->second.begin();
j != i->second.end();
++j) {
get_parent()->cancel_pull(*j);
bool legacy_log_entries,
vector<pg_log_entry_t> &log_entries,
ObjectStore::Transaction *t,
- set<hobject_t, hobject_t::BitwiseComparator> *added,
- set<hobject_t, hobject_t::BitwiseComparator> *removed)
+ set<hobject_t> *added,
+ set<hobject_t> *removed)
{
assert(t);
assert(added);
vector<pg_log_entry_t> log_entries(_log_entries);
ObjectStore::Transaction op_t;
PGTransactionUPtr t(std::move(_t));
- set<hobject_t, hobject_t::BitwiseComparator> added, removed;
+ set<hobject_t> added, removed;
generate_transaction(
t,
coll,
const pg_missing_t& missing,
const hobject_t &last_backfill,
interval_set<uint64_t>& data_subset,
- map<hobject_t, interval_set<uint64_t>, hobject_t::BitwiseComparator>& clone_subsets,
+ map<hobject_t, interval_set<uint64_t>>& clone_subsets,
ObcLockManager &manager)
{
dout(10) << "calc_head_subsets " << head
const pg_missing_t& missing,
const hobject_t &last_backfill,
interval_set<uint64_t>& data_subset,
- map<hobject_t, interval_set<uint64_t>, hobject_t::BitwiseComparator>& clone_subsets,
+ map<hobject_t, interval_set<uint64_t>>& clone_subsets,
ObcLockManager &manager)
{
dout(10) << "calc_clone_subsets " << soid
eversion_t _v = get_parent()->get_local_missing().get_items().find(
soid)->second.need;
assert(_v == v);
- const map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator> &missing_loc(
+ const map<hobject_t, set<pg_shard_t>> &missing_loc(
get_parent()->get_missing_loc_shards());
const map<pg_shard_t, pg_missing_t > &peer_missing(
get_parent()->get_shard_missing());
- map<hobject_t, set<pg_shard_t>, hobject_t::BitwiseComparator>::const_iterator q = missing_loc.find(soid);
+ map<hobject_t, set<pg_shard_t>>::const_iterator q = missing_loc.find(soid);
assert(q != missing_loc.end());
assert(!q->second.empty());
dout(10) << __func__ << ": " << soid << " v" << oi.version
<< " size " << size << " to osd." << peer << dendl;
- map<hobject_t, interval_set<uint64_t>, hobject_t::BitwiseComparator> clone_subsets;
+ map<hobject_t, interval_set<uint64_t>> clone_subsets;
interval_set<uint64_t> data_subset;
ObcLockManager lock_manager;
interval_set<uint64_t> data_subset;
if (obc->obs.oi.size)
data_subset.insert(0, obc->obs.oi.size);
- map<hobject_t, interval_set<uint64_t>, hobject_t::BitwiseComparator> clone_subsets;
+ map<hobject_t, interval_set<uint64_t>> clone_subsets;
prep_push(obc, soid, peer,
obc->obs.oi.version, data_subset, clone_subsets,
const hobject_t& soid, pg_shard_t peer,
eversion_t version,
interval_set<uint64_t> &data_subset,
- map<hobject_t, interval_set<uint64_t>, hobject_t::BitwiseComparator>& clone_subsets,
+ map<hobject_t, interval_set<uint64_t>>& clone_subsets,
PushOp *pop,
bool cache_dont_need,
ObcLockManager &&lock_manager)
void ReplicatedBackend::submit_push_complete(ObjectRecoveryInfo &recovery_info,
ObjectStore::Transaction *t)
{
- for (map<hobject_t, interval_set<uint64_t>, hobject_t::BitwiseComparator>::const_iterator p =
+ for (map<hobject_t, interval_set<uint64_t>>::const_iterator p =
recovery_info.clone_subset.begin();
p != recovery_info.clone_subset.end();
++p) {
}
void ReplicatedBackend::clear_pull(
- map<hobject_t, PullInfo, hobject_t::BitwiseComparator>::iterator piter,
+ map<hobject_t, PullInfo>::iterator piter,
bool clear_pull_from_peer)
{
auto from = piter->second.from;
virtual void dump_recovery_info(Formatter *f) const {
{
f->open_array_section("pull_from_peer");
- for (map<pg_shard_t, set<hobject_t, hobject_t::BitwiseComparator> >::const_iterator i = pull_from_peer.begin();
+ for (map<pg_shard_t, set<hobject_t> >::const_iterator i = pull_from_peer.begin();
i != pull_from_peer.end();
++i) {
f->open_object_section("pulling_from");
f->dump_stream("pull_from") << i->first;
{
f->open_array_section("pulls");
- for (set<hobject_t, hobject_t::BitwiseComparator>::const_iterator j = i->second.begin();
+ for (set<hobject_t>::const_iterator j = i->second.begin();
j != i->second.end();
++j) {
f->open_object_section("pull_info");
}
{
f->open_array_section("pushing");
- for (map<hobject_t, map<pg_shard_t, PushInfo>, hobject_t::BitwiseComparator>::const_iterator i =
+ for (map<hobject_t, map<pg_shard_t, PushInfo>>::const_iterator i =
pushing.begin();
i != pushing.end();
++i) {
}
}
};
- map<hobject_t, map<pg_shard_t, PushInfo>, hobject_t::BitwiseComparator> pushing;
+ map<hobject_t, map<pg_shard_t, PushInfo>> pushing;
// pull
struct PullInfo {
}
};
- map<hobject_t, PullInfo, hobject_t::BitwiseComparator> pulling;
+ map<hobject_t, PullInfo> pulling;
// Reverse mapping from osd peer to objects beging pulled from that peer
- map<pg_shard_t, set<hobject_t, hobject_t::BitwiseComparator> > pull_from_peer;
+ map<pg_shard_t, set<hobject_t> > pull_from_peer;
void clear_pull(
- map<hobject_t, PullInfo, hobject_t::BitwiseComparator>::iterator piter,
+ map<hobject_t, PullInfo>::iterator piter,
bool clear_pull_from_peer = true);
void sub_op_push(OpRequestRef op);
SnapSet& snapset, const hobject_t& poid, const pg_missing_t& missing,
const hobject_t &last_backfill,
interval_set<uint64_t>& data_subset,
- map<hobject_t, interval_set<uint64_t>, hobject_t::BitwiseComparator>& clone_subsets,
+ map<hobject_t, interval_set<uint64_t>>& clone_subsets,
ObcLockManager &lock_manager);
void prepare_pull(
eversion_t v,
const hobject_t& soid, pg_shard_t peer,
eversion_t version,
interval_set<uint64_t> &data_subset,
- map<hobject_t, interval_set<uint64_t>, hobject_t::BitwiseComparator>& clone_subsets,
+ map<hobject_t, interval_set<uint64_t>>& clone_subsets,
PushOp *op,
bool cache,
ObcLockManager &&lock_manager);
const pg_missing_t& missing,
const hobject_t &last_backfill,
interval_set<uint64_t>& data_subset,
- map<hobject_t, interval_set<uint64_t>, hobject_t::BitwiseComparator>& clone_subsets,
+ map<hobject_t, interval_set<uint64_t>>& clone_subsets,
ObcLockManager &lock_manager);
ObjectRecoveryInfo recalc_subsets(
const ObjectRecoveryInfo& recovery_info,
void Session::clear_backoffs()
{
- map<hobject_t,set<BackoffRef>,hobject_t::BitwiseComparator> ls;
+ map<hobject_t,set<BackoffRef>> ls;
{
Mutex::Locker l(backoff_lock);
ls.swap(backoffs);
/// protects backoffs; orders inside Backoff::lock *and* PG::backoff_lock
Mutex backoff_lock;
std::atomic_int backoff_count= {0}; ///< simple count of backoffs
- map<hobject_t,set<BackoffRef>, hobject_t::BitwiseComparator> backoffs;
+ map<hobject_t,set<BackoffRef>> backoffs;
std::atomic<uint64_t> backoff_seq = {0};
if (struct_v < 2) {
if (!soid.is_max() && soid.pool == -1)
soid.pool = pool;
- map<hobject_t, interval_set<uint64_t>, hobject_t::BitwiseComparator> tmp;
+ map<hobject_t, interval_set<uint64_t>> tmp;
tmp.swap(clone_subset);
- for (map<hobject_t, interval_set<uint64_t>, hobject_t::BitwiseComparator>::iterator i = tmp.begin();
+ for (map<hobject_t, interval_set<uint64_t>>::iterator i = tmp.begin();
i != tmp.end();
++i) {
hobject_t first(i->first);
assert(valid_through == l.incr_since);
valid_through = l.valid_through;
- for (map<hobject_t,object, hobject_t::BitwiseComparator>::const_iterator p = l.objects.begin();
+ for (map<hobject_t,object>::const_iterator p = l.objects.begin();
p != l.objects.end();
++p){
if (p->second.negative) {
- map<hobject_t,object, hobject_t::BitwiseComparator>::iterator q = objects.find(p->first);
+ map<hobject_t,object>::iterator q = objects.find(p->first);
if (q != objects.end()) {
objects.erase(q);
}
// handle hobject_t upgrade
if (struct_v < 3) {
- map<hobject_t, object, hobject_t::ComparatorWithDefault> tmp;
+ map<hobject_t, object> tmp;
tmp.swap(objects);
- for (map<hobject_t, object, hobject_t::ComparatorWithDefault>::iterator i = tmp.begin();
+ for (map<hobject_t, object>::iterator i = tmp.begin();
i != tmp.end();
++i) {
hobject_t first(i->first);
f->dump_stream("valid_through") << valid_through;
f->dump_stream("incremental_since") << incr_since;
f->open_array_section("objects");
- for (map<hobject_t,object, hobject_t::ComparatorWithDefault>::const_iterator p = objects.begin(); p != objects.end(); ++p) {
+ for (map<hobject_t,object>::const_iterator p = objects.begin(); p != objects.end(); ++p) {
f->open_object_section("object");
f->dump_string("name", p->first.oid.name);
f->dump_unsigned("hash", p->first.get_hash());
class pg_missing_const_i {
public:
- virtual const map<hobject_t, pg_missing_item, hobject_t::ComparatorWithDefault> &
+ virtual const map<hobject_t, pg_missing_item> &
get_items() const = 0;
virtual const map<version_t, hobject_t> &get_rmissing() const = 0;
virtual unsigned int num_missing() const = 0;
};
template <>
class ChangeTracker<true> {
- set<hobject_t, hobject_t::BitwiseComparator> _changed;
+ set<hobject_t> _changed;
public:
void changed(const hobject_t &obj) {
_changed.insert(obj);
template <bool TrackChanges>
class pg_missing_set : public pg_missing_const_i {
using item = pg_missing_item;
- map<hobject_t, item, hobject_t::ComparatorWithDefault> missing; // oid -> (need v, have v)
+ map<hobject_t, item> missing; // oid -> (need v, have v)
map<version_t, hobject_t> rmissing; // v -> oid
ChangeTracker<TrackChanges> tracker;
tracker.changed(i.first);
}
- const map<hobject_t, item, hobject_t::ComparatorWithDefault> &get_items() const override {
+ const map<hobject_t, item> &get_items() const override {
return missing;
}
const map<version_t, hobject_t> &get_rmissing() const override {
return true;
}
bool is_missing(const hobject_t& oid, eversion_t v) const override {
- map<hobject_t, item, hobject_t::ComparatorWithDefault>::const_iterator m =
+ map<hobject_t, item>::const_iterator m =
missing.find(oid);
if (m == missing.end())
return false;
return true;
}
eversion_t have_old(const hobject_t& oid) const override {
- map<hobject_t, item, hobject_t::ComparatorWithDefault>::const_iterator m =
+ map<hobject_t, item>::const_iterator m =
missing.find(oid);
if (m == missing.end())
return eversion_t();
*/
void add_next_event(const pg_log_entry_t& e) {
if (e.is_update()) {
- map<hobject_t, item, hobject_t::ComparatorWithDefault>::iterator missing_it;
+ map<hobject_t, item>::iterator missing_it;
missing_it = missing.find(e.soid);
bool is_missing_divergent_item = missing_it != missing.end();
if (e.prior_version == eversion_t() || e.is_clone()) {
}
void rm(const hobject_t& oid, eversion_t v) {
- std::map<hobject_t, item, hobject_t::ComparatorWithDefault>::iterator p = missing.find(oid);
+ std::map<hobject_t, item>::iterator p = missing.find(oid);
if (p != missing.end() && p->second.need <= v)
rm(p);
}
- void rm(std::map<hobject_t, item, hobject_t::ComparatorWithDefault>::const_iterator m) {
+ void rm(std::map<hobject_t, item>::const_iterator m) {
tracker.changed(m->first);
rmissing.erase(m->second.need.version);
missing.erase(m);
}
void got(const hobject_t& oid, eversion_t v) {
- std::map<hobject_t, item, hobject_t::ComparatorWithDefault>::iterator p = missing.find(oid);
+ std::map<hobject_t, item>::iterator p = missing.find(oid);
assert(p != missing.end());
assert(p->second.need <= v);
got(p);
}
- void got(std::map<hobject_t, item, hobject_t::ComparatorWithDefault>::const_iterator m) {
+ void got(std::map<hobject_t, item>::const_iterator m) {
tracker.changed(m->first);
rmissing.erase(m->second.need.version);
missing.erase(m);
unsigned split_bits,
pg_missing_set *omissing) {
unsigned mask = ~((~0)<<split_bits);
- for (map<hobject_t, item, hobject_t::ComparatorWithDefault>::iterator i = missing.begin();
+ for (map<hobject_t, item>::iterator i = missing.begin();
i != missing.end();
) {
if ((i->first.get_hash() & mask) == child_pgid.m_seed) {
rmissing.clear();
}
- void resort(bool sort_bitwise) {
- if (missing.key_comp().bitwise != sort_bitwise) {
- map<hobject_t, item, hobject_t::ComparatorWithDefault> tmp;
- tmp.swap(missing);
- missing = map<hobject_t, item, hobject_t::ComparatorWithDefault>(
- hobject_t::ComparatorWithDefault(sort_bitwise));
- missing.insert(tmp.begin(), tmp.end());
- }
- }
-
void encode(bufferlist &bl) const {
ENCODE_START(3, 2, bl);
::encode(missing, bl);
if (struct_v < 3) {
// Handle hobject_t upgrade
- map<hobject_t, item, hobject_t::ComparatorWithDefault> tmp;
- for (map<hobject_t, item, hobject_t::ComparatorWithDefault>::iterator i =
+ map<hobject_t, item> tmp;
+ for (map<hobject_t, item>::iterator i =
missing.begin();
i != missing.end();
) {
missing.insert(tmp.begin(), tmp.end());
}
- for (map<hobject_t,item, hobject_t::ComparatorWithDefault>::iterator it =
+ for (map<hobject_t,item>::iterator it =
missing.begin();
it != missing.end();
++it)
}
void dump(Formatter *f) const {
f->open_array_section("missing");
- for (map<hobject_t,item, hobject_t::ComparatorWithDefault>::const_iterator p =
+ for (map<hobject_t,item>::const_iterator p =
missing.begin(); p != missing.end(); ++p) {
f->open_object_section("item");
f->dump_stream("object") << p->first;
}
if (oss && !ok) {
*oss << "check_missing: " << check_missing << "\n";
- set<hobject_t, hobject_t::BitwiseComparator> changed;
+ set<hobject_t> changed;
tracker.get_changed([&](const hobject_t &hoid) { changed.insert(hoid); });
*oss << "changed: " << changed << "\n";
}
ObjectContext::RWState::State type)
: obc(obc), type(type) {}
};
- map<hobject_t, ObjectLockState, hobject_t::BitwiseComparator> locks;
+ map<hobject_t, ObjectLockState> locks;
public:
ObcLockManager() = default;
ObcLockManager(ObcLockManager &&) = default;
object_info_t oi;
SnapSet ss;
interval_set<uint64_t> copy_subset;
- map<hobject_t, interval_set<uint64_t>, hobject_t::BitwiseComparator> clone_subset;
+ map<hobject_t, interval_set<uint64_t>> clone_subset;
ObjectRecoveryInfo() : size(0) { }
WRITE_CLASS_ENCODER(object)
bool bitwise; // ephemeral, not encoded
- map<hobject_t,object, hobject_t::ComparatorWithDefault> objects;
+ map<hobject_t,object> objects;
eversion_t valid_through;
eversion_t incr_since;
ScrubMap() : bitwise(true) {}
ScrubMap(bool bitwise)
- : bitwise(bitwise), objects(hobject_t::ComparatorWithDefault(bitwise)) {}
+ : bitwise(bitwise) {}
void merge_incr(const ScrubMap &l);
void insert(const ScrubMap &r) {
void reset_bitwise(bool new_bitwise) {
if (bitwise == new_bitwise)
return;
- map<hobject_t, object, hobject_t::ComparatorWithDefault> new_objects(
+ map<hobject_t, object> new_objects(
objects.begin(),
- objects.end(),
- hobject_t::ComparatorWithDefault(new_bitwise));
+ objects.end());
::swap(new_objects, objects);
bitwise = new_bitwise;
}
map<ceph_tid_t,CommandOp*> command_ops;
// backoffs
- map<hobject_t,OSDBackoff,hobject_t::BitwiseComparator> backoffs;
+ map<hobject_t,OSDBackoff> backoffs;
multimap<uint64_t,OSDBackoff*> backoffs_by_id;
int osd;
r = apply_transaction(store, &osr, std::move(t));
ASSERT_EQ(r, 0);
}
- set<ghobject_t, ghobject_t::BitwiseComparator> all;
+ set<ghobject_t> all;
{
ObjectStore::Transaction t;
for (int i=0; i<200; ++i) {
ASSERT_EQ(r, 0);
}
{
- set<ghobject_t, ghobject_t::BitwiseComparator> saw;
+ set<ghobject_t> saw;
vector<ghobject_t> objects;
ghobject_t next, current;
while (!next.is_max()) {
}
{
ObjectStore::Transaction t;
- for (set<ghobject_t, ghobject_t::BitwiseComparator>::iterator p = all.begin(); p != all.end(); ++p)
+ for (set<ghobject_t>::iterator p = all.begin(); p != all.end(); ++p)
t.remove(cid, *p);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
r = apply_transaction(store, &osr, std::move(t));
ASSERT_EQ(r, 0);
}
- set<ghobject_t, ghobject_t::BitwiseComparator> all;
+ set<ghobject_t> all;
{
ObjectStore::Transaction t;
for (int i=0; i<200; ++i) {
}
{
ObjectStore::Transaction t;
- for (set<ghobject_t, ghobject_t::BitwiseComparator>::iterator p = all.begin(); p != all.end(); ++p)
+ for (set<ghobject_t>::iterator p = all.begin(); p != all.end(); ++p)
t.remove(cid, *p);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
r = apply_transaction(store, &osr, std::move(t));
ASSERT_EQ(r, 0);
}
- set<ghobject_t, ghobject_t::BitwiseComparator> all, saw;
+ set<ghobject_t> all, saw;
{
ObjectStore::Transaction t;
for (int i=0; i<200; ++i) {
}
{
ObjectStore::Transaction t;
- for (set<ghobject_t, ghobject_t::BitwiseComparator>::iterator p = all.begin(); p != all.end(); ++p)
+ for (set<ghobject_t>::iterator p = all.begin(); p != all.end(); ++p)
t.remove(cid, *p);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
coll_t cid;
string base = "";
for (int i = 0; i < 100; ++i) base.append("aaaaa");
- set<ghobject_t, ghobject_t::BitwiseComparator> created;
+ set<ghobject_t> created;
{
ObjectStore::Transaction t;
t.create_collection(cid, 0);
ASSERT_EQ(r, 0);
}
- for (set<ghobject_t, ghobject_t::BitwiseComparator>::iterator i = created.begin();
+ for (set<ghobject_t>::iterator i = created.begin();
i != created.end();
++i) {
struct stat buf;
ASSERT_TRUE(!store->stat(cid, *i, &buf));
}
- set<ghobject_t, ghobject_t::BitwiseComparator> listed, listed2;
+ set<ghobject_t> listed, listed2;
vector<ghobject_t> objects;
r = store->collection_list(cid, ghobject_t(), ghobject_t::get_max(), INT_MAX, &objects, 0);
ASSERT_EQ(r, 0);
ASSERT_TRUE(listed.size() == created.size());
if (listed2.size())
ASSERT_EQ(listed.size(), listed2.size());
- for (set<ghobject_t, ghobject_t::BitwiseComparator>::iterator i = listed.begin();
+ for (set<ghobject_t>::iterator i = listed.begin();
i != listed.end();
++i) {
ASSERT_TRUE(created.count(*i));
}
- for (set<ghobject_t, ghobject_t::BitwiseComparator>::iterator i = created.begin();
+ for (set<ghobject_t>::iterator i = created.begin();
i != created.end();
++i) {
ObjectStore::Transaction t;
unsigned write_alignment;
unsigned max_object_len, max_write_len;
unsigned in_flight;
- map<ghobject_t, Object, ghobject_t::BitwiseComparator> contents;
- set<ghobject_t, ghobject_t::BitwiseComparator> available_objects;
- set<ghobject_t, ghobject_t::BitwiseComparator> in_flight_objects;
+ map<ghobject_t, Object> contents;
+ set<ghobject_t> available_objects;
+ set<ghobject_t> in_flight_objects;
ObjectGenerator *object_gen;
gen_type *rng;
ObjectStore *store;
cond.Wait(lock);
boost::uniform_int<> choose(0, available_objects.size() - 1);
int index = choose(*rng);
- set<ghobject_t, ghobject_t::BitwiseComparator>::iterator i = available_objects.begin();
+ set<ghobject_t>::iterator i = available_objects.begin();
for ( ; index > 0; --index, ++i) ;
ghobject_t ret = *i;
return ret;
while (in_flight)
cond.Wait(lock);
vector<ghobject_t> objects;
- set<ghobject_t, ghobject_t::BitwiseComparator> objects_set, objects_set2;
+ set<ghobject_t> objects_set, objects_set2;
ghobject_t next, current;
while (1) {
//cerr << "scanning..." << std::endl;
}
ASSERT_EQ(objects_set.size(), available_objects.size());
- for (set<ghobject_t, ghobject_t::BitwiseComparator>::iterator i = objects_set.begin();
+ for (set<ghobject_t>::iterator i = objects_set.begin();
i != objects_set.end();
++i) {
ASSERT_GT(available_objects.count(*i), (unsigned)0);
ASSERT_EQ(r, 0);
objects_set2.insert(objects.begin(), objects.end());
ASSERT_EQ(objects_set2.size(), available_objects.size());
- for (set<ghobject_t, ghobject_t::BitwiseComparator>::iterator i = objects_set2.begin();
+ for (set<ghobject_t>::iterator i = objects_set2.begin();
i != objects_set2.end();
++i) {
ASSERT_GT(available_objects.count(*i), (unsigned)0);
}
string base = "";
for (int i = 0; i < 100; ++i) base.append("aaaaa");
- set<ghobject_t, ghobject_t::BitwiseComparator> created;
+ set<ghobject_t> created;
for (int n = 0; n < 10; ++n) {
char nbuf[100];
sprintf(nbuf, "n%d", n);
vector<ghobject_t> objects;
r = store->collection_list(cid, ghobject_t(), ghobject_t::get_max(), INT_MAX, &objects, 0);
ASSERT_EQ(r, 0);
- set<ghobject_t, ghobject_t::BitwiseComparator> listed(objects.begin(), objects.end());
+ set<ghobject_t> listed(objects.begin(), objects.end());
cerr << "listed.size() is " << listed.size() << " and created.size() is " << created.size() << std::endl;
ASSERT_TRUE(listed.size() == created.size());
objects.clear();
}
cerr << "listed.size() is " << listed.size() << std::endl;
ASSERT_TRUE(listed.size() == created.size());
- for (set<ghobject_t, ghobject_t::BitwiseComparator>::iterator i = listed.begin();
+ for (set<ghobject_t>::iterator i = listed.begin();
i != listed.end();
++i) {
ASSERT_TRUE(created.count(*i));
}
- for (set<ghobject_t, ghobject_t::BitwiseComparator>::iterator i = created.begin();
+ for (set<ghobject_t>::iterator i = created.begin();
i != created.end();
++i) {
ObjectStore::Transaction t;
ASSERT_EQ(r, 0);
}
string base = "aaaaa";
- set<ghobject_t, ghobject_t::BitwiseComparator> created;
+ set<ghobject_t> created;
for (int i = 0; i < 1000; ++i) {
char buf[100];
sprintf(buf, "%d", i);
r = store->collection_list(cid, ghobject_t(), ghobject_t::get_max(),
INT_MAX, &objects, 0);
ASSERT_EQ(r, 0);
- set<ghobject_t, ghobject_t::BitwiseComparator> listed(objects.begin(), objects.end());
+ set<ghobject_t> listed(objects.begin(), objects.end());
cerr << "listed.size() is " << listed.size() << " and created.size() is " << created.size() << std::endl;
ASSERT_TRUE(listed.size() == created.size());
objects.clear();
}
cerr << "listed.size() is " << listed.size() << std::endl;
ASSERT_TRUE(listed.size() == created.size());
- for (set<ghobject_t, ghobject_t::BitwiseComparator>::iterator i = listed.begin();
+ for (set<ghobject_t>::iterator i = listed.begin();
i != listed.end();
++i) {
ASSERT_TRUE(created.count(*i));
}
- for (set<ghobject_t, ghobject_t::BitwiseComparator>::iterator i = created.begin();
+ for (set<ghobject_t>::iterator i = created.begin();
i != created.end();
++i) {
ObjectStore::Transaction t;
pg_missing_t init;
pg_missing_t final;
- set<hobject_t, hobject_t::BitwiseComparator> toremove;
+ set<hobject_t> toremove;
list<pg_log_entry_t> torollback;
private:
};
struct LogHandler : public PGLog::LogEntryHandler {
- set<hobject_t, hobject_t::BitwiseComparator> removed;
+ set<hobject_t> removed;
list<pg_log_entry_t> rolledback;
void rollback(
}
{
- set<hobject_t, hobject_t::BitwiseComparator>::const_iterator titer = tcase.toremove.begin();
- set<hobject_t, hobject_t::BitwiseComparator>::const_iterator hiter = handler.removed.begin();
+ set<hobject_t>::const_iterator titer = tcase.toremove.begin();
+ set<hobject_t>::const_iterator hiter = handler.removed.begin();
for (; titer != tcase.toremove.end(); ++titer, ++hiter) {
EXPECT_EQ(*titer, *hiter);
}
class MapperVerifier {
PausyAsyncMap *driver;
boost::scoped_ptr< SnapMapper > mapper;
- map<snapid_t, set<hobject_t, hobject_t::BitwiseComparator> > snap_to_hobject;
- map<hobject_t, set<snapid_t>, hobject_t::BitwiseComparator> hobject_to_snap;
+ map<snapid_t, set<hobject_t> > snap_to_hobject;
+ map<hobject_t, set<snapid_t>> hobject_to_snap;
snapid_t next;
uint32_t mask;
uint32_t bits;
for (set<snapid_t>::iterator i = snaps.begin();
i != snaps.end();
++i) {
- map<snapid_t, set<hobject_t, hobject_t::BitwiseComparator> >::iterator j = snap_to_hobject.find(*i);
+ map<snapid_t, set<hobject_t> >::iterator j = snap_to_hobject.find(*i);
assert(j != snap_to_hobject.end());
j->second.insert(obj);
}
Mutex::Locker l(lock);
if (snap_to_hobject.empty())
return;
- map<snapid_t, set<hobject_t, hobject_t::BitwiseComparator> >::iterator snap =
+ map<snapid_t, set<hobject_t> >::iterator snap =
rand_choose(snap_to_hobject);
- set<hobject_t, hobject_t::BitwiseComparator> hobjects = snap->second;
+ set<hobject_t> hobjects = snap->second;
vector<hobject_t> hoids;
while (mapper->get_next_objects_to_trim(
assert(hobjects.count(hoid));
hobjects.erase(hoid);
- map<hobject_t, set<snapid_t>, hobject_t::BitwiseComparator>::iterator j =
+ map<hobject_t, set<snapid_t>>::iterator j =
hobject_to_snap.find(hoid);
assert(j->second.count(snap->first));
set<snapid_t> old_snaps(j->second);
Mutex::Locker l(lock);
if (hobject_to_snap.empty())
return;
- map<hobject_t, set<snapid_t>, hobject_t::BitwiseComparator>::iterator obj =
+ map<hobject_t, set<snapid_t>>::iterator obj =
rand_choose(hobject_to_snap);
for (set<snapid_t>::iterator i = obj->second.begin();
i != obj->second.end();
++i) {
- map<snapid_t, set<hobject_t, hobject_t::BitwiseComparator> >::iterator j =
+ map<snapid_t, set<hobject_t> >::iterator j =
snap_to_hobject.find(*i);
assert(j->second.count(obj->first));
j->second.erase(obj->first);
Mutex::Locker l(lock);
if (hobject_to_snap.empty())
return;
- map<hobject_t, set<snapid_t>, hobject_t::BitwiseComparator>::iterator obj =
+ map<hobject_t, set<snapid_t>>::iterator obj =
rand_choose(hobject_to_snap);
set<snapid_t> snaps;
int r = mapper->get_snaps(obj->first, &snaps);