void BitmapFreelistManager::enumerate_reset()
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
enumerate_offset = 0;
enumerate_bl_pos = 0;
enumerate_bl.clear();
bool BitmapFreelistManager::enumerate_next(uint64_t *offset, uint64_t *length)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
// initial base case is a bit awkward
if (enumerate_offset == 0 && enumerate_bl_pos == 0) {
void IOContext::aio_wait()
{
- std::unique_lock<std::mutex> l(lock);
+ std::unique_lock l(lock);
// see _aio_thread for waker logic
while (num_running.load() > 0) {
dout(10) << __func__ << " " << this
void BlockDevice::queue_reap_ioc(IOContext *ioc)
{
- std::lock_guard<std::mutex> l(ioc_reap_lock);
+ std::lock_guard l(ioc_reap_lock);
if (ioc_reap_count.load() == 0)
++ioc_reap_count;
ioc_reap_queue.push_back(ioc);
void BlockDevice::reap_ioc()
{
if (ioc_reap_count.load()) {
- std::lock_guard<std::mutex> l(ioc_reap_lock);
+ std::lock_guard l(ioc_reap_lock);
for (auto p : ioc_reap_queue) {
dout(20) << __func__ << " reap ioc " << p << dendl;
delete p;
// as there is no lock protection for aio_submit.
// Hence we might have false conditional trigger.
// aio_wait has to handle that hence do not care here.
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
cond.notify_all();
}
}
void BlueFS::add_block_extent(unsigned id, uint64_t offset, uint64_t length)
{
- std::unique_lock<std::mutex> l(lock);
+ std::unique_lock l(lock);
dout(1) << __func__ << " bdev " << id
<< " 0x" << std::hex << offset << "~" << length << std::dec
<< dendl;
int BlueFS::reclaim_blocks(unsigned id, uint64_t want,
PExtentVector *extents)
{
- std::unique_lock<std::mutex> l(lock);
+ std::unique_lock l(lock);
dout(1) << __func__ << " bdev " << id
<< " want 0x" << std::hex << want << std::dec << dendl;
ceph_assert(id < alloc.size());
uint64_t BlueFS::get_used()
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
uint64_t used = 0;
for (unsigned id = 0; id < MAX_BDEV; ++id) {
if (alloc[id]) {
uint64_t BlueFS::get_total(unsigned id)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
ceph_assert(id < block_all.size());
return block_all[id].size();
}
uint64_t BlueFS::get_free(unsigned id)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
ceph_assert(id < alloc.size());
return alloc[id]->get_free();
}
void BlueFS::get_usage(vector<pair<uint64_t,uint64_t>> *usage)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
usage->resize(bdev.size());
for (unsigned id = 0; id < bdev.size(); ++id) {
if (!bdev[id]) {
int BlueFS::get_block_extents(unsigned id, interval_set<uint64_t> *extents)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
dout(10) << __func__ << " bdev " << id << dendl;
if (id >= block_all.size())
return -EINVAL;
int BlueFS::mkfs(uuid_d osd_uuid)
{
- std::unique_lock<std::mutex> l(lock);
+ std::unique_lock l(lock);
dout(1) << __func__
<< " osd_uuid " << osd_uuid
<< dendl;
int BlueFS::fsck()
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
dout(1) << __func__ << dendl;
// hrm, i think we check everything on mount...
return 0;
void BlueFS::compact_log()
{
- std::unique_lock<std::mutex> l(lock);
+ std::unique_lock l(lock);
if (cct->_conf->bluefs_compact_log_sync) {
_compact_log_sync();
} else {
void BlueFS::flush_log()
{
- std::unique_lock<std::mutex> l(lock);
+ std::unique_lock l(lock);
flush_bdev();
_flush_and_sync_log(l);
}
void BlueFS::sync_metadata()
{
- std::unique_lock<std::mutex> l(lock);
+ std::unique_lock l(lock);
if (log_t.empty()) {
dout(10) << __func__ << " - no pending log events" << dendl;
} else {
FileWriter **h,
bool overwrite)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
dout(10) << __func__ << " " << dirname << "/" << filename << dendl;
map<string,DirRef>::iterator p = dir_map.find(dirname);
DirRef dir;
FileReader **h,
bool random)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
dout(10) << __func__ << " " << dirname << "/" << filename
<< (random ? " (random)":" (sequential)") << dendl;
map<string,DirRef>::iterator p = dir_map.find(dirname);
const string& old_dirname, const string& old_filename,
const string& new_dirname, const string& new_filename)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
dout(10) << __func__ << " " << old_dirname << "/" << old_filename
<< " -> " << new_dirname << "/" << new_filename << dendl;
map<string,DirRef>::iterator p = dir_map.find(old_dirname);
int BlueFS::mkdir(const string& dirname)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
dout(10) << __func__ << " " << dirname << dendl;
map<string,DirRef>::iterator p = dir_map.find(dirname);
if (p != dir_map.end()) {
int BlueFS::rmdir(const string& dirname)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
dout(10) << __func__ << " " << dirname << dendl;
map<string,DirRef>::iterator p = dir_map.find(dirname);
if (p == dir_map.end()) {
bool BlueFS::dir_exists(const string& dirname)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
map<string,DirRef>::iterator p = dir_map.find(dirname);
bool exists = p != dir_map.end();
dout(10) << __func__ << " " << dirname << " = " << (int)exists << dendl;
int BlueFS::stat(const string& dirname, const string& filename,
uint64_t *size, utime_t *mtime)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
dout(10) << __func__ << " " << dirname << "/" << filename << dendl;
map<string,DirRef>::iterator p = dir_map.find(dirname);
if (p == dir_map.end()) {
int BlueFS::lock_file(const string& dirname, const string& filename,
FileLock **plock)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
dout(10) << __func__ << " " << dirname << "/" << filename << dendl;
map<string,DirRef>::iterator p = dir_map.find(dirname);
if (p == dir_map.end()) {
int BlueFS::unlock_file(FileLock *fl)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
dout(10) << __func__ << " " << fl << " on " << fl->file->fnode << dendl;
ceph_assert(fl->file->locked);
fl->file->locked = false;
int BlueFS::readdir(const string& dirname, vector<string> *ls)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
dout(10) << __func__ << " " << dirname << dendl;
if (dirname.empty()) {
// list dirs
int BlueFS::unlink(const string& dirname, const string& filename)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
dout(10) << __func__ << " " << dirname << "/" << filename << dendl;
map<string,DirRef>::iterator p = dir_map.find(dirname);
if (p == dir_map.end()) {
bool random = false);
void close_writer(FileWriter *h) {
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
_close_writer(h);
}
void handle_discard(unsigned dev, interval_set<uint64_t>& to_release);
void flush(FileWriter *h) {
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
_flush(h, false);
}
void flush_range(FileWriter *h, uint64_t offset, uint64_t length) {
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
_flush_range(h, offset, length);
}
int fsync(FileWriter *h) {
- std::unique_lock<std::mutex> l(lock);
+ std::unique_lock l(lock);
return _fsync(h, l);
}
int read(FileReader *h, FileReaderBuffer *buf, uint64_t offset, size_t len,
return _read_random(h, offset, len, out);
}
void invalidate_cache(FileRef f, uint64_t offset, uint64_t len) {
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
_invalidate_cache(f, offset, len);
}
int preallocate(FileRef f, uint64_t offset, uint64_t len) {
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
return _preallocate(f, offset, len);
}
int truncate(FileWriter *h, uint64_t offset) {
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
return _truncate(h, offset);
}
void BlueStore::Cache::trim(uint64_t onode_max, uint64_t buffer_max)
{
- std::lock_guard<std::recursive_mutex> l(lock);
+ std::lock_guard l(lock);
_trim(onode_max, buffer_max);
}
void BlueStore::Cache::trim_all()
{
- std::lock_guard<std::recursive_mutex> l(lock);
+ std::lock_guard l(lock);
_trim(0, 0);
}
uint32_t end = offset + length;
{
- std::lock_guard<std::recursive_mutex> l(cache->lock);
+ std::lock_guard l(cache->lock);
for (auto i = _data_lower_bound(offset);
i != buffer_map.end() && offset < end && i->first < end;
++i) {
void BlueStore::BufferSpace::split(Cache* cache, size_t pos, BlueStore::BufferSpace &r)
{
- std::lock_guard<std::recursive_mutex> lk(cache->lock);
+ std::lock_guard lk(cache->lock);
if (buffer_map.empty())
return;
BlueStore::OnodeRef BlueStore::OnodeSpace::add(const ghobject_t& oid, OnodeRef o)
{
- std::lock_guard<std::recursive_mutex> l(cache->lock);
+ std::lock_guard l(cache->lock);
auto p = onode_map.find(oid);
if (p != onode_map.end()) {
ldout(cache->cct, 30) << __func__ << " " << oid << " " << o
bool hit = false;
{
- std::lock_guard<std::recursive_mutex> l(cache->lock);
+ std::lock_guard l(cache->lock);
ceph::unordered_map<ghobject_t,OnodeRef>::iterator p = onode_map.find(oid);
if (p == onode_map.end()) {
ldout(cache->cct, 30) << __func__ << " " << oid << " miss" << dendl;
void BlueStore::OnodeSpace::clear()
{
- std::lock_guard<std::recursive_mutex> l(cache->lock);
+ std::lock_guard l(cache->lock);
ldout(cache->cct, 10) << __func__ << dendl;
for (auto &p : onode_map) {
cache->_rm_onode(p.second);
bool BlueStore::OnodeSpace::empty()
{
- std::lock_guard<std::recursive_mutex> l(cache->lock);
+ std::lock_guard l(cache->lock);
return onode_map.empty();
}
const ghobject_t& new_oid,
const mempool::bluestore_cache_other::string& new_okey)
{
- std::lock_guard<std::recursive_mutex> l(cache->lock);
+ std::lock_guard l(cache->lock);
ldout(cache->cct, 30) << __func__ << " " << old_oid << " -> " << new_oid
<< dendl;
ceph::unordered_map<ghobject_t,OnodeRef>::iterator po, pn;
bool BlueStore::OnodeSpace::map_any(std::function<bool(OnodeRef)> f)
{
- std::lock_guard<std::recursive_mutex> l(cache->lock);
+ std::lock_guard l(cache->lock);
ldout(cache->cct, 20) << __func__ << dendl;
for (auto& i : onode_map) {
if (f(i.second)) {
again:
auto coll_snap = coll;
if (coll_snap) {
- std::lock_guard<std::recursive_mutex> l(coll_snap->cache->lock);
+ std::lock_guard l(coll_snap->cache->lock);
if (coll_snap != coll) {
goto again;
}
{
while (true) {
Cache *cache = coll->cache;
- std::lock_guard<std::recursive_mutex> l(cache->lock);
+ std::lock_guard l(cache->lock);
if (coll->cache != cache) {
ldout(coll->store->cct, 20) << __func__
<< " raced with sb cache update, was " << cache
template <int LogLevelV = 30>
void BlueStore::SharedBlobSet::dump(CephContext *cct)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
for (auto& i : sb_map) {
ldout(cct, LogLevelV) << i.first << " : " << *i.second << dendl;
}
{
if (flushing_count.load()) {
ldout(c->store->cct, 20) << __func__ << " cnt:" << flushing_count << dendl;
- std::unique_lock<std::mutex> l(flush_lock);
+ std::unique_lock l(flush_lock);
while (flushing_count.load()) {
flush_cond.wait(l);
}
// lock (one or both) cache shards
std::lock(cache->lock, dest->cache->lock);
- std::lock_guard<std::recursive_mutex> l(cache->lock, std::adopt_lock);
- std::lock_guard<std::recursive_mutex> l2(dest->cache->lock, std::adopt_lock);
+ std::lock_guard l(cache->lock, std::adopt_lock);
+ std::lock_guard l2(dest->cache->lock, std::adopt_lock);
int destbits = dest->cnode.bits;
spg_t destpg;
}
{
- std::lock_guard<std::mutex> l(vstatfs_lock);
+ std::lock_guard l(vstatfs_lock);
buf->allocated = vstatfs.allocated();
buf->data_stored = vstatfs.stored();
buf->data_compressed = vstatfs.compressed();
logger->inc(l_bluestore_compressed_original, txc->statfs_delta.compressed_original());
{
- std::lock_guard<std::mutex> l(vstatfs_lock);
+ std::lock_guard l(vstatfs_lock);
vstatfs += txc->statfs_delta;
}
}
}
{
- std::lock_guard<std::mutex> l(kv_lock);
+ std::lock_guard l(kv_lock);
kv_queue.push_back(txc);
kv_cond.notify_one();
if (txc->state != TransContext::STATE_KV_SUBMITTED) {
*/
OpSequencer *osr = txc->osr.get();
- std::lock_guard<std::mutex> l(osr->qlock);
+ std::lock_guard l(osr->qlock);
txc->state = TransContext::STATE_IO_DONE;
txc->ioc.release_running_aios();
OpSequencer::q_list_t::iterator p = osr->q.iterator_to(*txc);
dout(20) << __func__ << " onode " << o << " had " << o->flushing_count
<< dendl;
if (--o->flushing_count == 0) {
- std::lock_guard<std::mutex> l(o->flush_lock);
+ std::lock_guard l(o->flush_lock);
o->flush_cond.notify_all();
}
}
{
dout(20) << __func__ << " txc " << txc << dendl;
{
- std::lock_guard<std::mutex> l(txc->osr->qlock);
+ std::lock_guard l(txc->osr->qlock);
txc->state = TransContext::STATE_KV_DONE;
if (txc->ch->commit_queue) {
txc->ch->commit_queue->queue(txc->oncommits);
bool submit_deferred = false;
OpSequencer::q_list_t releasing_txc;
{
- std::lock_guard<std::mutex> l(osr->qlock);
+ std::lock_guard l(osr->qlock);
txc->state = TransContext::STATE_DONE;
bool notify = false;
while (!osr->q.empty()) {
}
if (empty && osr->zombie) {
- std::lock_guard<std::mutex> l(zombie_osr_lock);
+ std::lock_guard l(zombie_osr_lock);
if (zombie_osr_set.erase(osr->cid)) {
dout(10) << __func__ << " reaping empty zombie osr " << osr << dendl;
} else {
<< " reusing osr " << c->osr << " from existing coll "
<< q->second << dendl;
} else {
- std::lock_guard<std::mutex> l(zombie_osr_lock);
+ std::lock_guard l(zombie_osr_lock);
auto p = zombie_osr_set.find(c->cid);
if (p == zombie_osr_set.end()) {
c->osr = new OpSequencer(this, c->cid);
void BlueStore::_osr_register_zombie(OpSequencer *osr)
{
- std::lock_guard<std::mutex> l(zombie_osr_lock);
+ std::lock_guard l(zombie_osr_lock);
dout(10) << __func__ << " " << osr << " " << osr->cid << dendl;
osr->zombie = true;
auto i = zombie_osr_set.emplace(osr->cid, osr);
}
{
// wake up any previously finished deferred events
- std::lock_guard<std::mutex> l(kv_lock);
+ std::lock_guard l(kv_lock);
kv_cond.notify_one();
}
osr->drain_preceding(txc);
}
{
// wake up any previously finished deferred events
- std::lock_guard<std::mutex> l(kv_lock);
+ std::lock_guard l(kv_lock);
kv_cond.notify_one();
}
osr->drain();
}
}
{
- std::lock_guard<std::mutex> l(zombie_osr_lock);
+ std::lock_guard l(zombie_osr_lock);
for (auto& i : zombie_osr_set) {
s.insert(i.second);
zombies.push_back(i.second);
}
{
// wake up any previously finished deferred events
- std::lock_guard<std::mutex> l(kv_lock);
+ std::lock_guard l(kv_lock);
kv_cond.notify_one();
}
{
- std::lock_guard<std::mutex> l(kv_finalize_lock);
+ std::lock_guard l(kv_finalize_lock);
kv_finalize_cond.notify_one();
}
for (auto osr : s) {
--deferred_aggressive;
{
- std::lock_guard<std::mutex> l(zombie_osr_lock);
+ std::lock_guard l(zombie_osr_lock);
for (auto& osr : zombies) {
if (zombie_osr_set.erase(osr->cid)) {
dout(10) << __func__ << " reaping empty zombie osr " << osr << dendl;
{
dout(10) << __func__ << dendl;
{
- std::unique_lock<std::mutex> l(kv_lock);
+ std::unique_lock l(kv_lock);
while (!kv_sync_started) {
kv_cond.wait(l);
}
kv_cond.notify_all();
}
{
- std::unique_lock<std::mutex> l(kv_finalize_lock);
+ std::unique_lock l(kv_finalize_lock);
while (!kv_finalize_started) {
kv_finalize_cond.wait(l);
}
kv_finalize_thread.join();
ceph_assert(removed_collections.empty());
{
- std::lock_guard<std::mutex> l(kv_lock);
+ std::lock_guard l(kv_lock);
kv_stop = false;
}
{
- std::lock_guard<std::mutex> l(kv_finalize_lock);
+ std::lock_guard l(kv_finalize_lock);
kv_finalize_stop = false;
}
dout(10) << __func__ << " stopping finishers" << dendl;
{
dout(10) << __func__ << " start" << dendl;
deque<DeferredBatch*> deferred_stable_queue; ///< deferred ios done + stable
- std::unique_lock<std::mutex> l(kv_lock);
+ std::unique_lock l(kv_lock);
ceph_assert(!kv_sync_started);
kv_sync_started = true;
kv_cond.notify_all();
--txc->osr->kv_committing_serially;
txc->state = TransContext::STATE_KV_SUBMITTED;
if (txc->osr->kv_submitted_waiters) {
- std::lock_guard<std::mutex> l(txc->osr->qlock);
+ std::lock_guard l(txc->osr->qlock);
txc->osr->qcond.notify_all();
}
ceph_assert(r == 0);
{
- std::unique_lock<std::mutex> m(kv_finalize_lock);
+ std::unique_lock m(kv_finalize_lock);
if (kv_committing_to_finalize.empty()) {
kv_committing_to_finalize.swap(kv_committing);
} else {
deque<TransContext*> kv_committed;
deque<DeferredBatch*> deferred_stable;
dout(10) << __func__ << " start" << dendl;
- std::unique_lock<std::mutex> l(kv_finalize_lock);
+ std::unique_lock l(kv_finalize_lock);
ceph_assert(!kv_finalize_started);
kv_finalize_started = true;
kv_finalize_cond.notify_all();
{
dout(20) << __func__ << " " << deferred_queue.size() << " osrs, "
<< deferred_queue_size << " txcs" << dendl;
- std::lock_guard<std::mutex> l(deferred_lock);
+ std::lock_guard l(deferred_lock);
vector<OpSequencerRef> osrs;
osrs.reserve(deferred_queue.size());
for (auto& osr : deferred_queue) {
DeferredBatch *b = osr->deferred_running;
{
- std::lock_guard<std::mutex> l(deferred_lock);
+ std::lock_guard l(deferred_lock);
ceph_assert(osr->deferred_running == b);
osr->deferred_running = nullptr;
if (!osr->deferred_pending) {
{
uint64_t costs = 0;
{
- std::lock_guard<std::mutex> l2(osr->qlock);
+ std::lock_guard l2(osr->qlock);
for (auto& i : b->txcs) {
TransContext *txc = &i;
txc->log_state_latency(logger, l_bluestore_state_deferred_aio_wait_lat);
}
}
throttle_deferred_bytes.put(costs);
- std::lock_guard<std::mutex> l(kv_lock);
+ std::lock_guard l(kv_lock);
deferred_done_queue.emplace_back(b);
}
// in the normal case, do not bother waking up the kv thread; it will
// catch us on the next commit anyway.
if (deferred_aggressive) {
- std::lock_guard<std::mutex> l(kv_lock);
+ std::lock_guard l(kv_lock);
kv_cond.notify_one();
}
}
deferred_try_submit();
{
// wake up any previously finished deferred events
- std::lock_guard<std::mutex> l(kv_lock);
+ std::lock_guard l(kv_lock);
kv_cond.notify_one();
}
throttle_deferred_bytes.get(txc->cost);
dout(LogLevelV) << __func__ << " csum: " << std::hex << v << std::dec
<< dendl;
}
- std::lock_guard<std::recursive_mutex> l(e.blob->shared_blob->get_cache()->lock);
+ std::lock_guard l(e.blob->shared_blob->get_cache()->lock);
for (auto& i : e.blob->shared_blob->bc.buffer_map) {
dout(LogLevelV) << __func__ << " 0x" << std::hex << i.first
<< "~" << i.second->length << std::dec
// return value is the highest cache_private of a trimmed buffer, or 0.
int discard(Cache* cache, uint32_t offset, uint32_t length) {
- std::lock_guard<std::recursive_mutex> l(cache->lock);
+ std::lock_guard l(cache->lock);
return _discard(cache, offset, length);
}
int _discard(Cache* cache, uint32_t offset, uint32_t length);
void write(Cache* cache, uint64_t seq, uint32_t offset, bufferlist& bl,
unsigned flags) {
- std::lock_guard<std::recursive_mutex> l(cache->lock);
+ std::lock_guard l(cache->lock);
Buffer *b = new Buffer(this, Buffer::STATE_WRITING, seq, offset, bl,
flags);
b->cache_private = _discard(cache, offset, bl.length());
}
void _finish_write(Cache* cache, uint64_t seq);
void did_read(Cache* cache, uint32_t offset, bufferlist& bl) {
- std::lock_guard<std::recursive_mutex> l(cache->lock);
+ std::lock_guard l(cache->lock);
Buffer *b = new Buffer(this, Buffer::STATE_CLEAN, 0, offset, bl);
b->cache_private = _discard(cache, offset, bl.length());
_add_buffer(cache, b, 1, nullptr);
void split(Cache* cache, size_t pos, BufferSpace &r);
void dump(Cache* cache, Formatter *f) const {
- std::lock_guard<std::recursive_mutex> l(cache->lock);
+ std::lock_guard l(cache->lock);
f->open_array_section("buffers");
for (auto& i : buffer_map) {
f->open_object_section("buffer");
mempool::bluestore_cache_other::unordered_map<uint64_t,SharedBlob*> sb_map;
SharedBlobRef lookup(uint64_t sbid) {
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
auto p = sb_map.find(sbid);
if (p == sb_map.end() ||
p->second->nref == 0) {
}
void add(Collection* coll, SharedBlob *sb) {
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
sb_map[sb->get_sbid()] = sb;
sb->coll = coll;
}
void remove(SharedBlob *sb) {
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
ceph_assert(sb->get_parent() == this);
// only remove if it still points to us
auto p = sb_map.find(sb->get_sbid());
}
bool empty() {
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
return sb_map.empty();
}
}
bool can_split() const {
- std::lock_guard<std::recursive_mutex> l(shared_blob->get_cache()->lock);
+ std::lock_guard l(shared_blob->get_cache()->lock);
// splitting a BufferSpace writing list is too hard; don't try.
return shared_blob->bc.writing.empty() &&
used_in_blob.can_split() &&
uint64_t *bytes) = 0;
bool empty() {
- std::lock_guard<std::recursive_mutex> l(lock);
+ std::lock_guard l(lock);
return _get_num_onodes() == 0 && _get_buffer_bytes() == 0;
}
uint64_t *blobs,
uint64_t *buffers,
uint64_t *bytes) override {
- std::lock_guard<std::recursive_mutex> l(lock);
+ std::lock_guard l(lock);
*onodes += onode_lru.size();
*extents += num_extents;
*blobs += num_blobs;
uint64_t *blobs,
uint64_t *buffers,
uint64_t *bytes) override {
- std::lock_guard<std::recursive_mutex> l(lock);
+ std::lock_guard l(lock);
*onodes += onode_lru.size();
*extents += num_extents;
*blobs += num_blobs;
}
void queue_new(TransContext *txc) {
- std::lock_guard<std::mutex> l(qlock);
+ std::lock_guard l(qlock);
txc->seq = ++last_seq;
q.push_back(*txc);
}
void drain() {
- std::unique_lock<std::mutex> l(qlock);
+ std::unique_lock l(qlock);
while (!q.empty())
qcond.wait(l);
}
void drain_preceding(TransContext *txc) {
- std::unique_lock<std::mutex> l(qlock);
+ std::unique_lock l(qlock);
while (!q.empty() && &q.front() != txc)
qcond.wait(l);
}
}
void flush() {
- std::unique_lock<std::mutex> l(qlock);
+ std::unique_lock l(qlock);
while (true) {
// set flag before the check because the condition
// may become true outside qlock, and we need to make
}
void flush_all_but_last() {
- std::unique_lock<std::mutex> l(qlock);
+ std::unique_lock l(qlock);
assert (q.size() >= 1);
while (true) {
// set flag before the check because the condition
}
bool flush_commit(Context *c) {
- std::lock_guard<std::mutex> l(qlock);
+ std::lock_guard l(qlock);
if (q.empty()) {
return true;
}
// aio completion notification will not return before that aio is
// stable on disk: whichever thread sees the flag first will block
// followers until the aio is stable.
- std::lock_guard<std::mutex> l(flush_mutex);
+ std::lock_guard l(flush_mutex);
bool expect = true;
if (!io_since_flush.compare_exchange_strong(expect, false)) {
{
dout(10) << __func__ << dendl;
{
- std::unique_lock<std::mutex> l(discard_lock);
+ std::unique_lock l(discard_lock);
while (!discard_started) {
discard_cond.wait(l);
}
}
discard_thread.join();
{
- std::lock_guard<std::mutex> l(discard_lock);
+ std::lock_guard l(discard_lock);
discard_stop = false;
}
dout(10) << __func__ << " stopped" << dendl;
void KernelDevice::discard_drain()
{
dout(10) << __func__ << dendl;
- std::unique_lock<std::mutex> l(discard_lock);
+ std::unique_lock l(discard_lock);
while (!discard_queued.empty() || discard_running) {
discard_cond.wait(l);
}
IOContext *ioc = static_cast<IOContext*>(aio[i]->priv);
_aio_log_finish(ioc, aio[i]->offset, aio[i]->length);
if (aio[i]->queue_item.is_linked()) {
- std::lock_guard<std::mutex> l(debug_queue_lock);
+ std::lock_guard l(debug_queue_lock);
debug_aio_unlink(*aio[i]);
}
}
if (cct->_conf->bdev_debug_aio) {
utime_t now = ceph_clock_now();
- std::lock_guard<std::mutex> l(debug_queue_lock);
+ std::lock_guard l(debug_queue_lock);
if (debug_oldest) {
if (debug_stall_since == utime_t()) {
debug_stall_since = now;
void KernelDevice::_discard_thread()
{
- std::unique_lock<std::mutex> l(discard_lock);
+ std::unique_lock l(discard_lock);
ceph_assert(!discard_started);
discard_started = true;
discard_cond.notify_all();
if (to_release.empty())
return 0;
- std::lock_guard<std::mutex> l(discard_lock);
+ std::lock_guard l(discard_lock);
discard_queued.insert(to_release);
discard_cond.notify_all();
return 0;
list<aio_t>::iterator p = ioc->running_aios.begin();
while (p != e) {
dout(30) << __func__ << " " << *p << dendl;
- std::lock_guard<std::mutex> l(debug_queue_lock);
+ std::lock_guard l(debug_queue_lock);
debug_aio_link(*p++);
}
}
if (spdk_nvme_retry_count < 0)
spdk_nvme_retry_count = SPDK_NVME_DEFAULT_RETRY_COUNT;
- std::unique_lock<std::mutex> l(probe_queue_lock);
+ std::unique_lock l(probe_queue_lock);
while (true) {
if (!probe_queue.empty()) {
ProbeContext* ctxt = probe_queue.front();
ProbeContext ctx{trid, this, nullptr, false};
{
- std::unique_lock<std::mutex> l(probe_queue_lock);
+ std::unique_lock l(probe_queue_lock);
probe_queue.push_back(&ctx);
while (!ctx.done)
probe_queue_cond.wait(l);
uint64_t want_size, uint64_t alloc_unit, int64_t hint,
uint64_t *offset, uint32_t *length)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
ldout(cct, 10) << __func__ << " want_size 0x" << std::hex << want_size
<< " alloc_unit 0x" << alloc_unit
<< " hint 0x" << hint << std::dec
void StupidAllocator::release(
const interval_set<uint64_t>& release_set)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
for (interval_set<uint64_t>::const_iterator p = release_set.begin();
p != release_set.end();
++p) {
uint64_t StupidAllocator::get_free()
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
return num_free;
}
uint64_t max_intervals = 0;
uint64_t intervals = 0;
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
max_intervals = p2roundup(num_free, alloc_unit) / alloc_unit;
for (unsigned bin = 0; bin < free.size(); ++bin) {
intervals += free[bin].num_intervals();
void StupidAllocator::dump()
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
for (unsigned bin = 0; bin < free.size(); ++bin) {
ldout(cct, 0) << __func__ << " free bin " << bin << ": "
<< free[bin].num_intervals() << " extents" << dendl;
void StupidAllocator::init_add_free(uint64_t offset, uint64_t length)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
ldout(cct, 10) << __func__ << " 0x" << std::hex << offset << "~" << length
<< std::dec << dendl;
_insert_free(offset, length);
void StupidAllocator::init_rm_free(uint64_t offset, uint64_t length)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
ldout(cct, 10) << __func__ << " 0x" << std::hex << offset << "~" << length
<< std::dec << dendl;
interval_set_t rm;
public:
uint64_t debug_get_free(uint64_t pos0 = 0, uint64_t pos1 = 0)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
return l1.debug_get_free(pos0 * l1._children_per_slot() * bits_per_slot,
pos1 * l1._children_per_slot() * bits_per_slot);
}
uint64_t debug_get_allocated(uint64_t pos0 = 0, uint64_t pos1 = 0)
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
return l1.debug_get_allocated(pos0 * l1._children_per_slot() * bits_per_slot,
pos1 * l1._children_per_slot() * bits_per_slot);
}
uint64_t get_available()
{
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
return available;
}
inline uint64_t get_min_alloc_size() const
uint64_t l1_w = slotset_width * l1._children_per_slot();
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
if (available < min_length) {
return;
void _free_l2(const interval_set<uint64_t> & rr)
{
uint64_t released = 0;
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
for (auto r : rr) {
released += l1._free_l1(r.first, r.second);
uint64_t l2_pos = r.first / l2_granularity;
void _free_l2(const T& rr)
{
uint64_t released = 0;
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
for (auto r : rr) {
released += l1._free_l1(r.offset, r.length);
uint64_t l2_pos = r.offset / l2_granularity;
uint64_t l2_pos = o / l2_granularity;
uint64_t l2_pos_end = p2roundup(int64_t(o + len), int64_t(l2_granularity)) / l2_granularity;
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
auto allocated = l1._mark_alloc_l1(o, len);
ceph_assert(available >= allocated);
available -= allocated;
uint64_t l2_pos = o / l2_granularity;
uint64_t l2_pos_end = p2roundup(int64_t(o + len), int64_t(l2_granularity)) / l2_granularity;
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
available += l1._free_l1(o, len);
_mark_l2_free(l2_pos, l2_pos_end);
}
last_pos = 0;
}
double _get_fragmentation() {
- std::lock_guard<std::mutex> l(lock);
+ std::lock_guard l(lock);
return l1.get_fragmentation();
}
};