image_watcher(NULL),
journal(NULL),
owner_lock(util::unique_lock_name("librbd::ImageCtx::owner_lock", this)),
- md_lock(util::unique_lock_name("librbd::ImageCtx::md_lock", this)),
image_lock(util::unique_lock_name("librbd::ImageCtx::image_lock", this)),
parent_lock(util::unique_lock_name("librbd::ImageCtx::parent_lock", this)),
object_map_lock(util::unique_lock_name("librbd::ImageCtx::object_map_lock", this)),
#include "include/int_types.h"
+#include <atomic>
#include <list>
#include <map>
#include <set>
/**
* Lock ordering:
*
- * owner_lock, md_lock, image_lock, parent_lock,
+ * owner_lock, image_lock, parent_lock,
* object_map_lock, async_op_lock, timestamp_lock
*/
RWLock owner_lock; // protects exclusive lock leadership updates
- RWLock md_lock; // protects access to the mutable image metadata that
- // isn't guarded by other locks below, and blocks writes
- // when held exclusively, so snapshots can be consistent.
- // Fields guarded include:
- // total_bytes_read
- // exclusive_locked
- // lock_tag
- // lockers
RWLock image_lock; // protects snapshot-related member variables,
// features (and associated helper classes), and flags
+ // protects access to the mutable image metadata that
+ // isn't guarded by other locks below, and blocks writes
+ // when held exclusively, so snapshots can be consistent.
+ // Fields guarded include:
+ // total_bytes_read
+ // exclusive_locked
+ // lock_tag
+ // lockers
RWLock parent_lock; // protects parent_md and parent
RWLock object_map_lock; // protects object map updates and object_map itself
cache::ImageCache *image_cache = nullptr;
Readahead readahead;
- uint64_t total_bytes_read;
+ std::atomic<uint64_t> total_bytes_read = {0};
std::map<uint64_t, io::CopyupRequest<ImageCtx>*> copyup_list;
uint64_t from_size = 0;
uint64_t end_size;
{
- RWLock::RLocker md_locker(m_image_ctx.md_lock);
RWLock::RLocker image_locker(m_image_ctx.image_lock);
head_ctx.dup(m_image_ctx.data_ctx);
if (m_from_snap_name) {
ldout(cct, 20) << this << " " << __func__ << dendl;
RWLock::WLocker owner_locker(m_image_ctx.owner_lock);
- RWLock::WLocker md_locker(m_image_ctx.md_lock);
+ RWLock::WLocker image_locker(m_image_ctx.image_lock);
+ RWLock::WLocker parent_locker(m_image_ctx.parent_lock);
- {
- RWLock::WLocker image_locker(m_image_ctx.image_lock);
- RWLock::WLocker parent_locker(m_image_ctx.parent_lock);
-
- m_image_ctx.size = m_size;
- m_image_ctx.lockers = m_lockers;
- m_image_ctx.lock_tag = m_lock_tag;
- m_image_ctx.exclusive_locked = m_exclusive_locked;
+ m_image_ctx.size = m_size;
+ m_image_ctx.lockers = m_lockers;
+ m_image_ctx.lock_tag = m_lock_tag;
+ m_image_ctx.exclusive_locked = m_exclusive_locked;
- std::map<uint64_t, uint64_t> migration_reverse_snap_seq;
+ std::map<uint64_t, uint64_t> migration_reverse_snap_seq;
- if (m_image_ctx.old_format) {
- m_image_ctx.order = m_order;
- m_image_ctx.features = 0;
- m_image_ctx.flags = 0;
- m_image_ctx.op_features = 0;
- m_image_ctx.operations_disabled = false;
- m_image_ctx.object_prefix = std::move(m_object_prefix);
- m_image_ctx.init_layout();
- } else {
- // HEAD revision doesn't have a defined overlap so it's only
- // applicable to snapshots
- if (!m_head_parent_overlap) {
- m_parent_md = {};
- }
+ if (m_image_ctx.old_format) {
+ m_image_ctx.order = m_order;
+ m_image_ctx.features = 0;
+ m_image_ctx.flags = 0;
+ m_image_ctx.op_features = 0;
+ m_image_ctx.operations_disabled = false;
+ m_image_ctx.object_prefix = std::move(m_object_prefix);
+ m_image_ctx.init_layout();
+ } else {
+ // HEAD revision doesn't have a defined overlap so it's only
+ // applicable to snapshots
+ if (!m_head_parent_overlap) {
+ m_parent_md = {};
+ }
- m_image_ctx.features = m_features;
- m_image_ctx.flags = m_flags;
- m_image_ctx.op_features = m_op_features;
- m_image_ctx.operations_disabled = (
- (m_op_features & ~RBD_OPERATION_FEATURES_ALL) != 0ULL);
- m_image_ctx.group_spec = m_group_spec;
- if (get_migration_info(&m_image_ctx.parent_md,
- &m_image_ctx.migration_info)) {
- for (auto it : m_image_ctx.migration_info.snap_map) {
- migration_reverse_snap_seq[it.second.front()] = it.first;
- }
- } else {
- m_image_ctx.parent_md = m_parent_md;
- m_image_ctx.migration_info = {};
+ m_image_ctx.features = m_features;
+ m_image_ctx.flags = m_flags;
+ m_image_ctx.op_features = m_op_features;
+ m_image_ctx.operations_disabled = (
+ (m_op_features & ~RBD_OPERATION_FEATURES_ALL) != 0ULL);
+ m_image_ctx.group_spec = m_group_spec;
+ if (get_migration_info(&m_image_ctx.parent_md,
+ &m_image_ctx.migration_info)) {
+ for (auto it : m_image_ctx.migration_info.snap_map) {
+ migration_reverse_snap_seq[it.second.front()] = it.first;
}
+ } else {
+ m_image_ctx.parent_md = m_parent_md;
+ m_image_ctx.migration_info = {};
}
+ }
- for (size_t i = 0; i < m_snapc.snaps.size(); ++i) {
- std::vector<librados::snap_t>::const_iterator it = std::find(
- m_image_ctx.snaps.begin(), m_image_ctx.snaps.end(),
- m_snapc.snaps[i].val);
- if (it == m_image_ctx.snaps.end()) {
- m_flush_aio = true;
- ldout(cct, 20) << "new snapshot id=" << m_snapc.snaps[i].val
- << " name=" << m_snap_infos[i].name
- << " size=" << m_snap_infos[i].image_size
- << dendl;
- }
+ for (size_t i = 0; i < m_snapc.snaps.size(); ++i) {
+ std::vector<librados::snap_t>::const_iterator it = std::find(
+ m_image_ctx.snaps.begin(), m_image_ctx.snaps.end(),
+ m_snapc.snaps[i].val);
+ if (it == m_image_ctx.snaps.end()) {
+ m_flush_aio = true;
+ ldout(cct, 20) << "new snapshot id=" << m_snapc.snaps[i].val
+ << " name=" << m_snap_infos[i].name
+ << " size=" << m_snap_infos[i].image_size
+ << dendl;
}
+ }
- m_image_ctx.snaps.clear();
- m_image_ctx.snap_info.clear();
- m_image_ctx.snap_ids.clear();
- auto overlap = m_image_ctx.parent_md.overlap;
- for (size_t i = 0; i < m_snapc.snaps.size(); ++i) {
- uint64_t flags = m_image_ctx.old_format ? 0 : m_snap_flags[i];
- uint8_t protection_status = m_image_ctx.old_format ?
- static_cast<uint8_t>(RBD_PROTECTION_STATUS_UNPROTECTED) :
- m_snap_protection[i];
- ParentImageInfo parent;
- if (!m_image_ctx.old_format) {
- if (!m_image_ctx.migration_info.empty()) {
- parent = m_image_ctx.parent_md;
- auto it = migration_reverse_snap_seq.find(m_snapc.snaps[i].val);
- if (it != migration_reverse_snap_seq.end()) {
- parent.spec.snap_id = it->second;
- parent.overlap = m_snap_infos[i].image_size;
- } else {
- overlap = std::min(overlap, m_snap_infos[i].image_size);
- parent.overlap = overlap;
- }
+ m_image_ctx.snaps.clear();
+ m_image_ctx.snap_info.clear();
+ m_image_ctx.snap_ids.clear();
+ auto overlap = m_image_ctx.parent_md.overlap;
+ for (size_t i = 0; i < m_snapc.snaps.size(); ++i) {
+ uint64_t flags = m_image_ctx.old_format ? 0 : m_snap_flags[i];
+ uint8_t protection_status = m_image_ctx.old_format ?
+ static_cast<uint8_t>(RBD_PROTECTION_STATUS_UNPROTECTED) :
+ m_snap_protection[i];
+ ParentImageInfo parent;
+ if (!m_image_ctx.old_format) {
+ if (!m_image_ctx.migration_info.empty()) {
+ parent = m_image_ctx.parent_md;
+ auto it = migration_reverse_snap_seq.find(m_snapc.snaps[i].val);
+ if (it != migration_reverse_snap_seq.end()) {
+ parent.spec.snap_id = it->second;
+ parent.overlap = m_snap_infos[i].image_size;
} else {
- parent = m_snap_parents[i];
+ overlap = std::min(overlap, m_snap_infos[i].image_size);
+ parent.overlap = overlap;
}
+ } else {
+ parent = m_snap_parents[i];
}
- m_image_ctx.add_snap(m_snap_infos[i].snapshot_namespace,
- m_snap_infos[i].name, m_snapc.snaps[i].val,
- m_snap_infos[i].image_size, parent,
- protection_status, flags,
- m_snap_infos[i].timestamp);
- }
- m_image_ctx.parent_md.overlap = std::min(overlap, m_image_ctx.size);
- m_image_ctx.snapc = m_snapc;
-
- if (m_image_ctx.snap_id != CEPH_NOSNAP &&
- m_image_ctx.get_snap_id(m_image_ctx.snap_namespace,
- m_image_ctx.snap_name) != m_image_ctx.snap_id) {
- lderr(cct) << "tried to read from a snapshot that no longer exists: "
- << m_image_ctx.snap_name << dendl;
- m_image_ctx.snap_exists = false;
}
+ m_image_ctx.add_snap(m_snap_infos[i].snapshot_namespace,
+ m_snap_infos[i].name, m_snapc.snaps[i].val,
+ m_snap_infos[i].image_size, parent,
+ protection_status, flags,
+ m_snap_infos[i].timestamp);
+ }
+ m_image_ctx.parent_md.overlap = std::min(overlap, m_image_ctx.size);
+ m_image_ctx.snapc = m_snapc;
- if (m_refresh_parent != nullptr) {
- m_refresh_parent->apply();
- }
- m_image_ctx.data_ctx.selfmanaged_snap_set_write_ctx(m_image_ctx.snapc.seq,
- m_image_ctx.snaps);
+ if (m_image_ctx.snap_id != CEPH_NOSNAP &&
+ m_image_ctx.get_snap_id(m_image_ctx.snap_namespace,
+ m_image_ctx.snap_name) != m_image_ctx.snap_id) {
+ lderr(cct) << "tried to read from a snapshot that no longer exists: "
+ << m_image_ctx.snap_name << dendl;
+ m_image_ctx.snap_exists = false;
+ }
- // handle dynamically enabled / disabled features
- if (m_image_ctx.exclusive_lock != nullptr &&
- !m_image_ctx.test_features(RBD_FEATURE_EXCLUSIVE_LOCK,
+ if (m_refresh_parent != nullptr) {
+ m_refresh_parent->apply();
+ }
+ m_image_ctx.data_ctx.selfmanaged_snap_set_write_ctx(m_image_ctx.snapc.seq,
+ m_image_ctx.snaps);
+
+ // handle dynamically enabled / disabled features
+ if (m_image_ctx.exclusive_lock != nullptr &&
+ !m_image_ctx.test_features(RBD_FEATURE_EXCLUSIVE_LOCK,
+ m_image_ctx.image_lock)) {
+ // disabling exclusive lock will automatically handle closing
+ // object map and journaling
+ ceph_assert(m_exclusive_lock == nullptr);
+ m_exclusive_lock = m_image_ctx.exclusive_lock;
+ } else {
+ if (m_exclusive_lock != nullptr) {
+ ceph_assert(m_image_ctx.exclusive_lock == nullptr);
+ std::swap(m_exclusive_lock, m_image_ctx.exclusive_lock);
+ }
+ if (!m_image_ctx.test_features(RBD_FEATURE_JOURNALING,
m_image_ctx.image_lock)) {
- // disabling exclusive lock will automatically handle closing
- // object map and journaling
- ceph_assert(m_exclusive_lock == nullptr);
- m_exclusive_lock = m_image_ctx.exclusive_lock;
- } else {
- if (m_exclusive_lock != nullptr) {
- ceph_assert(m_image_ctx.exclusive_lock == nullptr);
- std::swap(m_exclusive_lock, m_image_ctx.exclusive_lock);
- }
- if (!m_image_ctx.test_features(RBD_FEATURE_JOURNALING,
- m_image_ctx.image_lock)) {
- if (!m_image_ctx.clone_copy_on_read && m_image_ctx.journal != nullptr) {
- m_image_ctx.io_work_queue->set_require_lock(io::DIRECTION_READ,
- false);
- }
- std::swap(m_journal, m_image_ctx.journal);
- } else if (m_journal != nullptr) {
- std::swap(m_journal, m_image_ctx.journal);
- }
- if (!m_image_ctx.test_features(RBD_FEATURE_OBJECT_MAP,
- m_image_ctx.image_lock) ||
- m_object_map != nullptr) {
- std::swap(m_object_map, m_image_ctx.object_map);
+ if (!m_image_ctx.clone_copy_on_read && m_image_ctx.journal != nullptr) {
+ m_image_ctx.io_work_queue->set_require_lock(io::DIRECTION_READ,
+ false);
}
+ std::swap(m_journal, m_image_ctx.journal);
+ } else if (m_journal != nullptr) {
+ std::swap(m_journal, m_image_ctx.journal);
+ }
+ if (!m_image_ctx.test_features(RBD_FEATURE_OBJECT_MAP,
+ m_image_ctx.image_lock) ||
+ m_object_map != nullptr) {
+ std::swap(m_object_map, m_image_ctx.object_map);
}
}
}
#include "librbd/io/ImageRequest.h"
#include "librbd/io/ImageRequestWQ.h"
#include "librbd/io/ObjectDispatcher.h"
-#include "librbd/io/ObjectDispatchSpec.h"
#include "librbd/io/ObjectRequest.h"
#include "librbd/io/ReadResult.h"
#include "librbd/journal/Types.h"
if (r < 0)
return r;
- RWLock::RLocker locker(ictx->md_lock);
+ RWLock::RLocker locker(ictx->image_lock);
if (exclusive)
*exclusive = ictx->exclusive_locked;
if (tag)
* duplicate that code.
*/
{
- RWLock::RLocker locker(ictx->md_lock);
+ RWLock::RLocker locker(ictx->image_lock);
r = rados::cls::lock::lock(&ictx->md_ctx, ictx->header_oid, RBD_LOCK_NAME,
exclusive ? LOCK_EXCLUSIVE : LOCK_SHARED,
cookie, tag, "", utime_t(), 0);
return r;
{
- RWLock::RLocker locker(ictx->md_lock);
+ RWLock::RLocker locker(ictx->image_lock);
r = rados::cls::lock::unlock(&ictx->md_ctx, ictx->header_oid,
RBD_LOCK_NAME, cookie);
if (r < 0) {
return -ENOENT;
}
- RWLock::RLocker locker(ictx->md_lock);
librados::Rados rados(ictx->md_ctx);
r = rados.blacklist_add(
client_address,
return cls_client::metadata_list(&ictx->md_ctx, ictx->header_oid, start, max, pairs);
}
- struct C_RBD_Readahead : public Context {
- ImageCtx *ictx;
- object_t oid;
- uint64_t offset;
- uint64_t length;
-
- bufferlist read_data;
- io::ExtentMap extent_map;
-
- C_RBD_Readahead(ImageCtx *ictx, object_t oid, uint64_t offset, uint64_t length)
- : ictx(ictx), oid(oid), offset(offset), length(length) {
- ictx->readahead.inc_pending();
- }
-
- void finish(int r) override {
- ldout(ictx->cct, 20) << "C_RBD_Readahead on " << oid << ": "
- << offset << "~" << length << dendl;
- ictx->readahead.dec_pending();
- }
- };
-
- void readahead(ImageCtx *ictx,
- const vector<pair<uint64_t,uint64_t> >& image_extents)
- {
- uint64_t total_bytes = 0;
- for (vector<pair<uint64_t,uint64_t> >::const_iterator p = image_extents.begin();
- p != image_extents.end();
- ++p) {
- total_bytes += p->second;
- }
-
- ictx->md_lock.get_write();
- bool abort = ictx->readahead_disable_after_bytes != 0 &&
- ictx->total_bytes_read > ictx->readahead_disable_after_bytes;
- if (abort) {
- ictx->md_lock.put_write();
- return;
- }
- ictx->total_bytes_read += total_bytes;
- ictx->image_lock.get_read();
- uint64_t image_size = ictx->get_image_size(ictx->snap_id);
- auto snap_id = ictx->snap_id;
- ictx->image_lock.put_read();
- ictx->md_lock.put_write();
-
- pair<uint64_t, uint64_t> readahead_extent = ictx->readahead.update(image_extents, image_size);
- uint64_t readahead_offset = readahead_extent.first;
- uint64_t readahead_length = readahead_extent.second;
-
- if (readahead_length > 0) {
- ldout(ictx->cct, 20) << "(readahead logical) " << readahead_offset << "~" << readahead_length << dendl;
- map<object_t,vector<ObjectExtent> > readahead_object_extents;
- Striper::file_to_extents(ictx->cct, ictx->format_string, &ictx->layout,
- readahead_offset, readahead_length, 0, readahead_object_extents);
- for (map<object_t,vector<ObjectExtent> >::iterator p = readahead_object_extents.begin(); p != readahead_object_extents.end(); ++p) {
- for (vector<ObjectExtent>::iterator q = p->second.begin(); q != p->second.end(); ++q) {
- ldout(ictx->cct, 20) << "(readahead) oid " << q->oid << " " << q->offset << "~" << q->length << dendl;
-
- auto req_comp = new C_RBD_Readahead(ictx, q->oid, q->offset,
- q->length);
- auto req = io::ObjectDispatchSpec::create_read(
- ictx, io::OBJECT_DISPATCH_LAYER_NONE, q->oid.name, q->objectno,
- q->offset, q->length, snap_id, 0, {}, &req_comp->read_data,
- &req_comp->extent_map, req_comp);
- req->send();
- }
- }
- ictx->perfcounter->inc(l_librbd_readahead);
- ictx->perfcounter->inc(l_librbd_readahead_bytes, readahead_length);
- }
- }
-
int list_watchers(ImageCtx *ictx,
std::list<librbd::image_watcher_t> &watchers)
{
int64_t read_iterate(ImageCtx *ictx, uint64_t off, uint64_t len,
int (*cb)(uint64_t, size_t, const char *, void *),
void *arg);
- void readahead(ImageCtx *ictx,
- const vector<pair<uint64_t,uint64_t> >& image_extents);
int invalidate_cache(ImageCtx *ictx);
int poll_io_events(ImageCtx *ictx, io::AioCompletion **comps, int numcomp);
#define dout_subsys ceph_subsys_rbd
#undef dout_prefix
-#define dout_prefix *_dout << "librbd::io::ImageRequest: " << this \
- << " " << __func__ << ": "
+#define dout_prefix *_dout << "librbd::io::ImageRequest: " << __func__ << ": "
namespace librbd {
namespace io {
namespace {
+template <typename I>
+struct C_RBD_Readahead : public Context {
+ I *ictx;
+ object_t oid;
+ uint64_t offset;
+ uint64_t length;
+
+ bufferlist read_data;
+ io::ExtentMap extent_map;
+
+ C_RBD_Readahead(I *ictx, object_t oid, uint64_t offset, uint64_t length)
+ : ictx(ictx), oid(oid), offset(offset), length(length) {
+ ictx->readahead.inc_pending();
+ }
+
+ void finish(int r) override {
+ ldout(ictx->cct, 20) << "C_RBD_Readahead on " << oid << ": "
+ << offset << "~" << length << dendl;
+ ictx->readahead.dec_pending();
+ }
+};
+
+template <typename I>
+void readahead(I *ictx, const Extents& image_extents) {
+ uint64_t total_bytes = 0;
+ for (auto& image_extent : image_extents) {
+ total_bytes += image_extent.second;
+ }
+
+ ictx->image_lock.get_read();
+ auto total_bytes_read = ictx->total_bytes_read.fetch_add(total_bytes);
+ bool abort = (
+ ictx->readahead_disable_after_bytes != 0 &&
+ total_bytes_read > ictx->readahead_disable_after_bytes);
+ if (abort) {
+ ictx->image_lock.put_read();
+ return;
+ }
+
+ uint64_t image_size = ictx->get_image_size(ictx->snap_id);
+ auto snap_id = ictx->snap_id;
+ ictx->image_lock.put_read();
+
+ auto readahead_extent = ictx->readahead.update(image_extents, image_size);
+ uint64_t readahead_offset = readahead_extent.first;
+ uint64_t readahead_length = readahead_extent.second;
+
+ if (readahead_length > 0) {
+ ldout(ictx->cct, 20) << "(readahead logical) " << readahead_offset << "~"
+ << readahead_length << dendl;
+ std::map<object_t, std::vector<ObjectExtent> > readahead_object_extents;
+ Striper::file_to_extents(ictx->cct, ictx->format_string, &ictx->layout,
+ readahead_offset, readahead_length, 0,
+ readahead_object_extents);
+ for (auto& readahead_object_extent : readahead_object_extents) {
+ for (auto& object_extent : readahead_object_extent.second) {
+ ldout(ictx->cct, 20) << "(readahead) oid " << object_extent.oid << " "
+ << object_extent.offset << "~"
+ << object_extent.length << dendl;
+
+ auto req_comp = new C_RBD_Readahead<I>(ictx, object_extent.oid,
+ object_extent.offset,
+ object_extent.length);
+ auto req = io::ObjectDispatchSpec::create_read(
+ ictx, io::OBJECT_DISPATCH_LAYER_NONE, object_extent.oid.name,
+ object_extent.objectno, object_extent.offset, object_extent.length,
+ snap_id, 0, {}, &req_comp->read_data, &req_comp->extent_map,
+ req_comp);
+ req->send();
+ }
+ }
+
+ ictx->perfcounter->inc(l_librbd_readahead);
+ ictx->perfcounter->inc(l_librbd_readahead_bytes, readahead_length);
+ }
+}
+
template <typename I>
struct C_UpdateTimestamp : public Context {
public:
} // anonymous namespace
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::io::ImageRequest: " << this \
+ << " " << __func__ << ": "
+
template <typename I>
void ImageRequest<I>::aio_read(I *ictx, AioCompletion *c,
Extents &&image_extents,
I &image_ctx = this->m_image_ctx;
CephContext *cct = image_ctx.cct;
- RWLock::RLocker md_locker(image_ctx.md_lock);
-
bool journaling = false;
AioCompletion *aio_comp = this->m_aio_comp;
ldout(cct, 5) << this << " " << __func__ << dendl;
{
- RWLock::RLocker md_locker(image_ctx.md_lock);
RWLock::RLocker image_locker(image_ctx.image_lock);
librados::ObjectWriteOperation op;
template <typename I>
int SnapshotProtectRequest<I>::verify_and_send_protect_snap() {
I &image_ctx = this->m_image_ctx;
- RWLock::RLocker md_locker(image_ctx.md_lock);
RWLock::RLocker image_locker(image_ctx.image_lock);
CephContext *cct = image_ctx.cct;
void SnapshotRenameRequest<I>::send_rename_snap() {
I &image_ctx = this->m_image_ctx;
ceph_assert(image_ctx.owner_lock.is_locked());
- RWLock::RLocker md_locker(image_ctx.md_lock);
RWLock::RLocker image_locker(image_ctx.image_lock);
CephContext *cct = image_ctx.cct;
template <typename I>
int SnapshotUnprotectRequest<I>::verify_and_send_unprotect_snap_start() {
I &image_ctx = this->m_image_ctx;
- RWLock::RLocker md_locker(image_ctx.md_lock);
RWLock::RLocker image_locker(image_ctx.image_lock);
CephContext *cct = image_ctx.cct;
exclusive_locked(image_ctx.exclusive_locked),
lock_tag(image_ctx.lock_tag),
owner_lock(image_ctx.owner_lock),
- md_lock(image_ctx.md_lock),
image_lock(image_ctx.image_lock),
timestamp_lock(image_ctx.timestamp_lock),
parent_lock(image_ctx.parent_lock),
librados::IoCtx data_ctx;
RWLock &owner_lock;
- RWLock &md_lock;
RWLock &image_lock;
RWLock ×tamp_lock;
RWLock &parent_lock;