librbd/DiffIterate.cc
librbd/ExclusiveLock.cc
librbd/ImageCtx.cc
+ librbd/ImageRefresh.cc
librbd/ImageWatcher.cc
librbd/internal.cc
librbd/Journal.cc
librbd/Utils.cc
librbd/exclusive_lock/AcquireRequest.cc
librbd/exclusive_lock/ReleaseRequest.cc
+ librbd/image/CloseRequest.cc
+ librbd/image/OpenRequest.cc
+ librbd/image/RefreshParentRequest.cc
+ librbd/image/RefreshRequest.cc
+ librbd/image/SetSnapRequest.cc
librbd/object_map/InvalidateRequest.cc
librbd/object_map/LockRequest.cc
librbd/object_map/Request.cc
m_readahead_trigger_pos(0),
m_readahead_size(0),
m_pending(0),
- m_pending_lock("Readahead::m_pending_lock"),
- m_pending_cond() {
+ m_pending_lock("Readahead::m_pending_lock") {
}
Readahead::~Readahead() {
assert(m_pending >= count);
m_pending -= count;
if (m_pending == 0) {
- m_pending_cond.Signal();
+ std::list<Context *> pending_waiting(std::move(m_pending_waiting));
+ m_pending_lock.Unlock();
+
+ for (auto ctx : pending_waiting) {
+ ctx->complete(0);
+ }
+ } else {
+ m_pending_lock.Unlock();
}
- m_pending_lock.Unlock();
}
void Readahead::wait_for_pending() {
+ C_SaferCond ctx;
+ wait_for_pending(&ctx);
+ ctx.wait();
+}
+
+void Readahead::wait_for_pending(Context *ctx) {
m_pending_lock.Lock();
- while (m_pending > 0) {
- m_pending_cond.Wait(m_pending_lock);
+ if (m_pending > 0) {
+ m_pending_lock.Unlock();
+ m_pending_waiting.push_back(ctx);
+ return;
}
m_pending_lock.Unlock();
-}
+ ctx->complete(0);
+}
void Readahead::set_trigger_requests(int trigger_requests) {
m_lock.Lock();
m_trigger_requests = trigger_requests;
#include "Mutex.h"
#include "Cond.h"
+#include <list>
/**
This class provides common state and logic for code that needs to perform readahead
Waits until the pending count reaches 0.
*/
void wait_for_pending();
+ void wait_for_pending(Context *ctx);
/**
Sets the number of sequential requests necessary to trigger readahead.
/// Lock for m_pending
Mutex m_pending_lock;
- /// Signalled when m_pending reaches 0
- Cond m_pending_cond;
+ /// Waiters for pending readahead
+ std::list<Context *> m_pending_waiting;
};
#endif
public:
static const std::string WATCHER_LOCK_TAG;
+ static ExclusiveLock *create(ImageCtxT &image_ctx) {
+ return new ExclusiveLock<ImageCtxT>(image_ctx);
+ }
+
ExclusiveLock(ImageCtxT &image_ctx);
~ExclusiveLock();
}
};
+struct C_ShutDownCache : public Context {
+ ImageCtx *image_ctx;
+ Context *on_finish;
+
+ C_ShutDownCache(ImageCtx *_image_ctx, Context *_on_finish)
+ : image_ctx(_image_ctx), on_finish(_on_finish) {
+ }
+ virtual void finish(int r) {
+ image_ctx->object_cacher->stop();
+ on_finish->complete(r);
+ }
+};
+
struct C_InvalidateCache : public Context {
ImageCtx *image_ctx;
bool purge_on_error;
object_cacher(NULL), writeback_handler(NULL), object_set(NULL),
readahead(),
total_bytes_read(0), copyup_finisher(NULL),
+ exclusive_lock(nullptr),
object_map(*this), object_map_ptr(nullptr), aio_work_queue(NULL), op_work_queue(NULL),
refresh_in_progress(false), asok_hook(new LibrbdAdminSocketHook(this))
{
delete asok_hook;
}
- int ImageCtx::init() {
+ int ImageCtx::init_legacy() {
int r;
if (id.length()) {
return 0;
}
+ void ImageCtx::init() {
+ assert(!header_oid.empty());
+ assert(old_format || !id.empty());
+ if (!old_format) {
+ init_layout();
+ }
+ apply_metadata_confs();
+
+ string pname = string("librbd-") + id + string("-") +
+ data_ctx.get_pool_name() + string("-") + name;
+ if (!snap_name.empty()) {
+ pname += "-";
+ pname += snap_name;
+ }
+
+ perf_start(pname);
+
+ if (cache) {
+ Mutex::Locker l(cache_lock);
+ ldout(cct, 20) << "enabling caching..." << dendl;
+ writeback_handler = new LibrbdWriteback(this, cache_lock);
+
+ uint64_t init_max_dirty = cache_max_dirty;
+ if (cache_writethrough_until_flush)
+ init_max_dirty = 0;
+ ldout(cct, 20) << "Initial cache settings:"
+ << " size=" << cache_size
+ << " num_objects=" << 10
+ << " max_dirty=" << init_max_dirty
+ << " target_dirty=" << cache_target_dirty
+ << " max_dirty_age="
+ << cache_max_dirty_age << dendl;
+
+ object_cacher = new ObjectCacher(cct, pname, *writeback_handler, cache_lock,
+ NULL, NULL,
+ cache_size,
+ 10, /* reset this in init */
+ init_max_dirty,
+ cache_target_dirty,
+ cache_max_dirty_age,
+ cache_block_writes_upfront);
+
+ // size object cache appropriately
+ uint64_t obj = cache_max_dirty_object;
+ if (!obj) {
+ obj = MIN(2000, MAX(10, cache_size / 100 / sizeof(ObjectCacher::Object)));
+ }
+ ldout(cct, 10) << " cache bytes " << cache_size
+ << " -> about " << obj << " objects" << dendl;
+ object_cacher->set_max_objects(obj);
+
+ object_set = new ObjectCacher::ObjectSet(NULL, data_ctx.get_id(), 0);
+ object_set->return_enoent = true;
+ object_cacher->start();
+ }
+
+ if (clone_copy_on_read) {
+ copyup_finisher = new Finisher(cct);
+ copyup_finisher->start();
+ }
+
+ readahead.set_trigger_requests(readahead_trigger_requests);
+ readahead.set_max_readahead_size(readahead_max_bytes);
+ }
+
void ImageCtx::init_layout()
{
if (stripe_unit == 0 || stripe_count == 0) {
return r;
}
+ void ImageCtx::shut_down_cache(Context *on_finish) {
+ if (object_cacher == NULL) {
+ on_finish->complete(0);
+ return;
+ }
+
+ RWLock::RLocker owner_locker(owner_lock);
+ cache_lock.Lock();
+ object_cacher->release_set(object_set);
+ cache_lock.Unlock();
+
+ C_ShutDownCache *shut_down = new C_ShutDownCache(this, on_finish);
+ flush_cache(new C_InvalidateCache(this, true, false, shut_down));
+ }
+
int ImageCtx::invalidate_cache(bool purge_on_error) {
flush_async_operations();
if (object_cacher == NULL) {
on_finish->complete(0);
}
+ void ImageCtx::flush_copyup(Context *on_finish) {
+ if (copyup_finisher == nullptr) {
+ on_finish->complete(0);
+ return;
+ }
+
+ copyup_finisher->queue(on_finish);
+ }
+
void ImageCtx::clear_pending_completions() {
Mutex::Locker l(completed_reqs_lock);
ldout(cct, 10) << "clear pending AioCompletion: count="
namespace librbd {
struct ImageCtx;
+ class AioCompletion;
class AioImageRequestWQ;
class AsyncOperation;
class CopyupRequest;
- class LibrbdAdminSocketHook;
+ template <typename> class ExclusiveLock;
class ImageWatcher;
class Journal;
- class AioCompletion;
+ class LibrbdAdminSocketHook;
namespace operation {
template <typename> class ResizeRequest;
xlist<AsyncRequest<>*> async_requests;
std::list<Context*> async_requests_waiters;
+ ExclusiveLock<ImageCtx> *exclusive_lock;
+
ObjectMap object_map; // TODO
ObjectMap *object_map_ptr;
ImageCtx(const std::string &image_name, const std::string &image_id,
const char *snap, IoCtx& p, bool read_only);
~ImageCtx();
- int init();
+ int init_legacy(); // TODO
+ void init();
void init_layout();
void perf_start(std::string name);
void perf_stop();
void user_flushed();
int flush_cache();
void flush_cache(Context *onfinish);
- int shutdown_cache();
+ int shutdown_cache(); // TODO
+ void shut_down_cache(Context *on_finish);
int invalidate_cache(bool purge_on_error=false);
void invalidate_cache(Context *on_finish);
void clear_nonexistence_cache();
void cancel_async_requests();
void cancel_async_requests(Context *on_finish);
+
+ void flush_copyup(Context *on_finish);
+
void apply_metadata_confs();
ObjectMap *create_object_map();
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/ImageRefresh.h"
+#include "common/dout.h"
+#include "common/errno.h"
+#include "librbd/image/RefreshRequest.h"
+
+#define dout_subsys ceph_subsys_rbd
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::ImageRefresh: "
+
+namespace librbd {
+
+template <typename I>
+ImageRefresh<I>::ImageRefresh(I &image_ctx) : m_image_ctx(image_ctx) {
+}
+
+template <typename I>
+bool ImageRefresh<I>::is_refresh_required() const {
+ // TODO future entry point for AIO ops -- to replace ictx_check call
+ return false;
+}
+
+template <typename I>
+void ImageRefresh<I>::refresh(Context *on_finish) {
+ // TODO simple state machine to restrict to a single in-progress refresh / snap set
+ image::RefreshRequest<I> *req = image::RefreshRequest<I>::create(
+ m_image_ctx, on_finish);
+ req->send();
+}
+
+} // namespace librbd
+
+template class librbd::ImageRefresh<librbd::ImageCtx>;
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_IMAGE_REFRESH_H
+#define CEPH_LIBRBD_IMAGE_REFRESH_H
+
+#include "include/int_types.h"
+
+class Context;
+
+namespace librbd {
+
+class ImageCtx;
+
+template <typename ImageCtxT = ImageCtx>
+class ImageRefresh {
+public:
+ ImageRefresh(ImageCtxT &image_ctx);
+
+ bool is_refresh_required() const;
+
+ void refresh(Context *on_finish);
+
+private:
+ ImageCtxT &m_image_ctx;
+
+};
+
+} // namespace librbd
+
+extern template class librbd::ImageRefresh<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_IMAGE_REFRESH_H
librbd/DiffIterate.cc \
librbd/ExclusiveLock.cc \
librbd/ImageCtx.cc \
+ librbd/ImageRefresh.cc \
librbd/ImageWatcher.cc \
librbd/internal.cc \
librbd/Journal.cc \
librbd/Utils.cc \
librbd/exclusive_lock/AcquireRequest.cc \
librbd/exclusive_lock/ReleaseRequest.cc \
+ librbd/image/CloseRequest.cc \
+ librbd/image/OpenRequest.cc \
+ librbd/image/RefreshParentRequest.cc \
+ librbd/image/RefreshRequest.cc \
+ librbd/image/SetSnapRequest.cc \
librbd/object_map/InvalidateRequest.cc \
librbd/object_map/LockRequest.cc \
librbd/object_map/Request.cc \
librbd/DiffIterate.h \
librbd/ExclusiveLock.h \
librbd/ImageCtx.h \
+ librbd/ImageRefresh.h \
librbd/ImageWatcher.h \
librbd/internal.h \
librbd/Journal.h \
librbd/WatchNotifyTypes.h \
librbd/exclusive_lock/AcquireRequest.h \
librbd/exclusive_lock/ReleaseRequest.h \
+ librbd/image/CloseRequest.h \
+ librbd/image/OpenRequest.h \
+ librbd/image/RefreshParentRequest.h \
+ librbd/image/RefreshRequest.h \
+ librbd/image/SetSnapRequest.h \
librbd/object_map/InvalidateRequest.h \
librbd/object_map/LockRequest.h \
librbd/object_map/Request.h \
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/image/CloseRequest.h"
+#include "common/dout.h"
+#include "common/errno.h"
+#include "common/WorkQueue.h"
+#include "librbd/AioImageRequestWQ.h"
+#include "librbd/ExclusiveLock.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/ImageState.h"
+#include "librbd/ImageWatcher.h"
+#include "librbd/ObjectMap.h"
+#include "librbd/Utils.h"
+
+#define dout_subsys ceph_subsys_rbd
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::image::CloseRequest: "
+
+namespace librbd {
+namespace image {
+
+
+using util::create_async_context_callback;
+using util::create_context_callback;
+
+template <typename I>
+CloseRequest<I>::CloseRequest(I *image_ctx, Context *on_finish)
+ : m_image_ctx(image_ctx), m_on_finish(on_finish), m_error_result(0),
+ m_exclusive_lock(nullptr) {
+ assert(image_ctx != nullptr);
+}
+
+template <typename I>
+void CloseRequest<I>::send() {
+ // TODO
+ send_shut_down_aio_queue();
+ //send_unregister_image_watcher();
+}
+
+template <typename I>
+void CloseRequest<I>::send_unregister_image_watcher() {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ // prevent incoming requests from our peers
+
+}
+
+template <typename I>
+void CloseRequest<I>::handle_unregister_image_watcher(int r) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << r << dendl;
+
+ save_result(r);
+ if (r < 0) {
+ lderr(cct) << "failed to unregister image watcher: " << cpp_strerror(r)
+ << dendl;
+ }
+
+ send_shut_down_aio_queue();
+}
+
+template <typename I>
+void CloseRequest<I>::send_shut_down_aio_queue() {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ RWLock::RLocker owner_locker(m_image_ctx->owner_lock);
+ m_image_ctx->aio_work_queue->shut_down(create_context_callback<
+ CloseRequest<I>, &CloseRequest<I>::handle_shut_down_aio_queue>(this));
+}
+
+template <typename I>
+void CloseRequest<I>::handle_shut_down_aio_queue(int r) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << r << dendl;
+
+ send_shut_down_exclusive_lock();
+}
+
+template <typename I>
+void CloseRequest<I>::send_shut_down_exclusive_lock() {
+ {
+ RWLock::WLocker owner_locker(m_image_ctx->owner_lock);
+ RWLock::WLocker snap_locker(m_image_ctx->snap_lock);
+ std::swap(m_exclusive_lock, m_image_ctx->exclusive_lock);
+
+ if (m_exclusive_lock == nullptr) {
+ delete m_image_ctx->object_map;
+ m_image_ctx->object_map = nullptr;
+ }
+ }
+
+ if (m_exclusive_lock == nullptr) {
+ send_flush();
+ return;
+ }
+
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ m_exclusive_lock->shut_down(create_context_callback<
+ CloseRequest<I>, &CloseRequest<I>::handle_shut_down_exclusive_lock>(this));
+}
+
+template <typename I>
+void CloseRequest<I>::handle_shut_down_exclusive_lock(int r) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << r << dendl;
+
+ // object map and journal closed during exclusive lock shutdown
+ assert(m_image_ctx->journal == nullptr);
+ assert(m_image_ctx->object_map == nullptr);
+ delete m_exclusive_lock;
+
+ save_result(r);
+ if (r < 0) {
+ lderr(cct) << "failed to shut down exclusive lock: " << cpp_strerror(r)
+ << dendl;
+ }
+ send_flush_readahead();
+}
+
+template <typename I>
+void CloseRequest<I>::send_flush() {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ RWLock::RLocker owner_locker(m_image_ctx->owner_lock);
+ m_image_ctx->flush(create_async_context_callback(
+ *m_image_ctx, create_context_callback<
+ CloseRequest<I>, &CloseRequest<I>::handle_flush>(this)));
+}
+
+template <typename I>
+void CloseRequest<I>::handle_flush(int r) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << r << dendl;
+
+ if (r < 0) {
+ lderr(cct) << "failed to flush IO: " << cpp_strerror(r) << dendl;
+ }
+ send_flush_readahead();
+}
+
+template <typename I>
+void CloseRequest<I>::send_flush_readahead() {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ m_image_ctx->readahead.wait_for_pending(create_context_callback<
+ CloseRequest<I>, &CloseRequest<I>::handle_flush_readahead>(this));
+}
+
+template <typename I>
+void CloseRequest<I>::handle_flush_readahead(int r) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << r << dendl;
+
+ send_shut_down_cache();
+}
+
+template <typename I>
+void CloseRequest<I>::send_shut_down_cache() {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ m_image_ctx->shut_down_cache(create_context_callback<
+ CloseRequest<I>, &CloseRequest<I>::handle_shut_down_cache>(this));
+}
+
+template <typename I>
+void CloseRequest<I>::handle_shut_down_cache(int r) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << r << dendl;
+
+ save_result(r);
+ if (r < 0) {
+ lderr(cct) << "failed to shut down cache: " << cpp_strerror(r) << dendl;
+ }
+ send_flush_copyup();
+}
+
+template <typename I>
+void CloseRequest<I>::send_flush_copyup() {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ m_image_ctx->flush_copyup(create_context_callback<
+ CloseRequest<I>, &CloseRequest<I>::handle_flush_copyup>(this));
+}
+
+template <typename I>
+void CloseRequest<I>::handle_flush_copyup(int r) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << r << dendl;
+ send_flush_op_work_queue();
+}
+
+template <typename I>
+void CloseRequest<I>::send_flush_op_work_queue() {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ m_image_ctx->op_work_queue->queue(create_context_callback<
+ CloseRequest<I>, &CloseRequest<I>::handle_flush_op_work_queue>(this), 0);
+}
+
+template <typename I>
+void CloseRequest<I>::handle_flush_op_work_queue(int r) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << r << dendl;
+ send_close_parent();
+}
+
+template <typename I>
+void CloseRequest<I>::send_close_parent() {
+ if (m_image_ctx->parent == nullptr) {
+ finish();
+ return;
+ }
+
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ m_image_ctx->parent->state->close(create_async_context_callback(
+ *m_image_ctx, create_context_callback<
+ CloseRequest<I>, &CloseRequest<I>::handle_close_parent>(this)));
+}
+
+template <typename I>
+void CloseRequest<I>::handle_close_parent(int r) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << r << dendl;
+
+ delete m_image_ctx->parent;
+ save_result(r);
+ if (r < 0) {
+ lderr(cct) << "error closing parent image: " << cpp_strerror(r) << dendl;
+ }
+ finish();
+}
+
+template <typename I>
+void CloseRequest<I>::finish() {
+ if (m_image_ctx->image_watcher) {
+ m_image_ctx->unregister_watch();
+ }
+
+ m_on_finish->complete(m_error_result);
+ delete this;
+}
+
+} // namespace image
+} // namespace librbd
+
+template class librbd::image::CloseRequest<librbd::ImageCtx>;
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_IMAGE_CLOSE_REQUEST_H
+#define CEPH_LIBRBD_IMAGE_CLOSE_REQUEST_H
+
+#include "include/int_types.h"
+#include "librbd/ImageCtx.h"
+
+class Context;
+
+namespace librbd {
+
+class ImageCtx;
+
+namespace image {
+
+template <typename ImageCtxT = ImageCtx>
+class CloseRequest {
+public:
+ static CloseRequest *create(ImageCtxT *image_ctx, Context *on_finish) {
+ return new CloseRequest(image_ctx, on_finish);
+ }
+
+ void send();
+
+private:
+ /**
+ * @verbatim
+ *
+ * <start>
+ * |
+ * v
+ * UNREGISTER_IMAGE_WATCHER
+ * |
+ * v
+ * SHUT_DOWN_AIO_WORK_QUEUE . . .
+ * | .
+ * v .
+ * SHUT_DOWN_EXCLUSIVE_LOCK . (exclusive lock
+ * | . disabled)
+ * v v
+ * FLUSH < . . . . . . . . . . .
+ * |
+ * v
+ * FLUSH_READAHEAD
+ * |
+ * v
+ * SHUTDOWN_CACHE
+ * |
+ * v
+ * FLUSH_COPYUP
+ * |
+ * v
+ * FLUSH_OP_WORK_QUEUE . . . . .
+ * | .
+ * v .
+ * CLOSE_PARENT . (no parent)
+ * | .
+ * v .
+ * <finish> < . . . . . . . . . .
+ *
+ * @endverbatim
+ */
+
+ CloseRequest(ImageCtxT *image_ctx, Context *on_finish);
+
+ ImageCtxT *m_image_ctx;
+ Context *m_on_finish;
+
+ int m_error_result;
+
+ decltype(m_image_ctx->exclusive_lock) m_exclusive_lock;
+
+ void send_unregister_image_watcher();
+ void handle_unregister_image_watcher(int r);
+
+ void send_shut_down_aio_queue();
+ void handle_shut_down_aio_queue(int r);
+
+ void send_shut_down_exclusive_lock();
+ void handle_shut_down_exclusive_lock(int r);
+
+ void send_flush();
+ void handle_flush(int r);
+
+ void send_flush_readahead();
+ void handle_flush_readahead(int r);
+
+ void send_shut_down_cache();
+ void handle_shut_down_cache(int r);
+
+ void send_flush_copyup();
+ void handle_flush_copyup(int r);
+
+ void send_flush_op_work_queue();
+ void handle_flush_op_work_queue(int r);
+
+ void send_close_parent();
+ void handle_close_parent(int r);
+
+ void finish();
+
+ void save_result(int result) {
+ if (m_error_result == 0 && result < 0) {
+ m_error_result = result;
+ }
+ }
+};
+
+} // namespace image
+} // namespace librbd
+
+extern template class librbd::image::CloseRequest<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_IMAGE_CLOSE_REQUEST_H
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/image/OpenRequest.h"
+#include "common/dout.h"
+#include "common/errno.h"
+#include "common/WorkQueue.h"
+#include "cls/rbd/cls_rbd_client.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/Utils.h"
+#include "librbd/image/CloseRequest.h"
+#include "librbd/image/RefreshRequest.h"
+#include "librbd/image/SetSnapRequest.h"
+
+#define dout_subsys ceph_subsys_rbd
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::image::OpenRequest: "
+
+namespace librbd {
+namespace image {
+
+namespace {
+
+template <typename I>
+class C_RegisterWatch : public Context {
+public:
+ I &image_ctx;
+ Context *on_finish;
+
+ C_RegisterWatch(I &image_ctx, Context *on_finish)
+ : image_ctx(image_ctx), on_finish(on_finish) {
+ }
+
+ virtual void finish(int r) {
+ assert(r == 0);
+ on_finish->complete(image_ctx.register_watch());
+ }
+};
+
+} // anonymous namespace
+
+using util::create_context_callback;
+using util::create_rados_ack_callback;
+
+template <typename I>
+OpenRequest<I>::OpenRequest(I *image_ctx, Context *on_finish)
+ : m_image_ctx(image_ctx), m_on_finish(on_finish), m_error_result(0) {
+}
+
+template <typename I>
+void OpenRequest<I>::send() {
+ send_v2_detect_header();
+}
+
+template <typename I>
+void OpenRequest<I>::send_v1_detect_header() {
+ librados::ObjectReadOperation op;
+ op.stat(NULL, NULL, NULL);
+
+ using klass = OpenRequest<I>;
+ librados::AioCompletion *comp =
+ create_rados_ack_callback<klass, &klass::handle_v1_detect_header>(this);
+ m_out_bl.clear();
+ m_image_ctx->md_ctx.aio_operate(util::old_header_name(m_image_ctx->name),
+ comp, &op, &m_out_bl);
+ comp->release();
+}
+
+template <typename I>
+Context *OpenRequest<I>::handle_v1_detect_header(int *result) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to stat image header: " << cpp_strerror(*result)
+ << dendl;
+ send_close_image(*result);
+ } else {
+ m_image_ctx->old_format = true;
+ m_image_ctx->header_oid = util::old_header_name(m_image_ctx->name);
+ send_register_watch();
+ }
+ return nullptr;
+}
+
+template <typename I>
+void OpenRequest<I>::send_v2_detect_header() {
+ if (m_image_ctx->id.empty()) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ librados::ObjectReadOperation op;
+ op.stat(NULL, NULL, NULL);
+
+ using klass = OpenRequest<I>;
+ librados::AioCompletion *comp =
+ create_rados_ack_callback<klass, &klass::handle_v2_detect_header>(this);
+ m_out_bl.clear();
+ m_image_ctx->md_ctx.aio_operate(util::id_obj_name(m_image_ctx->name),
+ comp, &op, &m_out_bl);
+ comp->release();
+ } else {
+ send_v2_get_immutable_metadata();
+ }
+}
+
+template <typename I>
+Context *OpenRequest<I>::handle_v2_detect_header(int *result) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << __func__ << ": r=" << *result << dendl;
+
+ if (*result == -ENOENT) {
+ send_v1_detect_header();
+ } else if (*result < 0) {
+ lderr(cct) << "failed to stat v2 image header: " << cpp_strerror(*result)
+ << dendl;
+ send_close_image(*result);
+ } else {
+ m_image_ctx->old_format = false;
+ send_v2_get_id();
+ }
+ return nullptr;
+}
+
+template <typename I>
+void OpenRequest<I>::send_v2_get_id() {
+ if (m_image_ctx->id.empty()) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ librados::ObjectReadOperation op;
+ cls_client::get_id_start(&op);
+
+ using klass = OpenRequest<I>;
+ librados::AioCompletion *comp =
+ create_rados_ack_callback<klass, &klass::handle_v2_get_id>(this);
+ m_out_bl.clear();
+ m_image_ctx->md_ctx.aio_operate(util::id_obj_name(m_image_ctx->name),
+ comp, &op, &m_out_bl);
+ comp->release();
+ } else {
+ send_v2_get_immutable_metadata();
+ }
+}
+
+template <typename I>
+Context *OpenRequest<I>::handle_v2_get_id(int *result) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << __func__ << ": r=" << *result << dendl;
+
+ if (*result == 0) {
+ bufferlist::iterator it = m_out_bl.begin();
+ *result = cls_client::get_id_finish(&it, &m_image_ctx->id);
+ }
+ if (*result < 0) {
+ lderr(cct) << "failed to retrieve image id: " << cpp_strerror(*result)
+ << dendl;
+ send_close_image(*result);
+ } else {
+ send_v2_get_immutable_metadata();
+ }
+ return nullptr;
+}
+
+template <typename I>
+void OpenRequest<I>::send_v2_get_immutable_metadata() {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ m_image_ctx->old_format = false;
+ m_image_ctx->header_oid = util::header_name(m_image_ctx->id);
+
+ librados::ObjectReadOperation op;
+ cls_client::get_immutable_metadata_start(&op);
+
+ using klass = OpenRequest<I>;
+ librados::AioCompletion *comp = create_rados_ack_callback<
+ klass, &klass::handle_v2_get_immutable_metadata>(this);
+ m_out_bl.clear();
+ m_image_ctx->md_ctx.aio_operate(m_image_ctx->header_oid, comp, &op,
+ &m_out_bl);
+ comp->release();
+}
+
+template <typename I>
+Context *OpenRequest<I>::handle_v2_get_immutable_metadata(int *result) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << __func__ << ": r=" << *result << dendl;
+
+ if (*result == 0) {
+ bufferlist::iterator it = m_out_bl.begin();
+ *result = cls_client::get_immutable_metadata_finish(
+ &it, &m_image_ctx->object_prefix, &m_image_ctx->order);
+ }
+ if (*result < 0) {
+ lderr(cct) << "failed to retreive immutable metadata: "
+ << cpp_strerror(*result) << dendl;
+ send_close_image(*result);
+ } else {
+ send_v2_get_stripe_unit_count();
+ }
+
+ return nullptr;
+}
+
+template <typename I>
+void OpenRequest<I>::send_v2_get_stripe_unit_count() {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ librados::ObjectReadOperation op;
+ cls_client::get_stripe_unit_count_start(&op);
+
+ using klass = OpenRequest<I>;
+ librados::AioCompletion *comp = create_rados_ack_callback<
+ klass, &klass::handle_v2_get_stripe_unit_count>(this);
+ m_out_bl.clear();
+ m_image_ctx->md_ctx.aio_operate(m_image_ctx->header_oid, comp, &op,
+ &m_out_bl);
+ comp->release();
+}
+
+template <typename I>
+Context *OpenRequest<I>::handle_v2_get_stripe_unit_count(int *result) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << __func__ << ": r=" << *result << dendl;
+
+ if (*result == 0) {
+ bufferlist::iterator it = m_out_bl.begin();
+ *result = cls_client::get_stripe_unit_count_finish(
+ &it, &m_image_ctx->stripe_unit, &m_image_ctx->stripe_count);
+ }
+
+ if (*result == -ENOEXEC || *result == -EINVAL) {
+ *result = 0;
+ }
+
+ if (*result < 0) {
+ lderr(cct) << "failed to read striping metadata: " << cpp_strerror(*result)
+ << dendl;
+ send_close_image(*result);
+ } else {
+ send_register_watch();
+ }
+ return nullptr;
+}
+
+template <typename I>
+void OpenRequest<I>::send_register_watch() {
+ m_image_ctx->init();
+
+ if (!m_image_ctx->read_only) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ // no librados async version of watch
+ using klass = OpenRequest<I>;
+ Context *ctx = new C_RegisterWatch<I>(
+ *m_image_ctx,
+ create_context_callback<klass, &klass::handle_register_watch>(this));
+ m_image_ctx->op_work_queue->queue(ctx);
+ } else {
+ send_refresh();
+ }
+}
+
+template <typename I>
+Context *OpenRequest<I>::handle_register_watch(int *result) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to register watch: " << cpp_strerror(*result)
+ << dendl;
+ send_close_image(*result);
+ } else {
+ send_refresh();
+ }
+ return nullptr;
+}
+
+template <typename I>
+void OpenRequest<I>::send_refresh() {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ using klass = OpenRequest<I>;
+ RefreshRequest<I> *ctx = RefreshRequest<I>::create(
+ *m_image_ctx,
+ create_context_callback<klass, &klass::handle_refresh>(this));
+ ctx->send();
+}
+
+template <typename I>
+Context *OpenRequest<I>::handle_refresh(int *result) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to refresh image: " << cpp_strerror(*result)
+ << dendl;
+ send_close_image(*result);
+ return nullptr;
+ } else {
+ return send_set_snap(result);
+ }
+}
+
+template <typename I>
+Context *OpenRequest<I>::send_set_snap(int *result) {
+ if (m_image_ctx->snap_name.empty()) {
+ *result = 0;
+ return m_on_finish;
+ }
+
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ using klass = OpenRequest<I>;
+ SetSnapRequest<I> *ctx = SetSnapRequest<I>::create(
+ *m_image_ctx, m_image_ctx->snap_name,
+ create_context_callback<klass, &klass::handle_set_snap>(this));
+ ctx->send();
+ return nullptr;
+}
+
+template <typename I>
+Context *OpenRequest<I>::handle_set_snap(int *result) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to set image snapshot: " << cpp_strerror(*result)
+ << dendl;
+ send_close_image(*result);
+ return nullptr;
+ }
+
+ return m_on_finish;
+}
+
+template <typename I>
+void OpenRequest<I>::send_close_image(int error_result) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ m_error_result = error_result;
+
+ using klass = OpenRequest<I>;
+ Context *ctx = create_context_callback<klass, &klass::handle_close_image>(
+ this);
+ CloseRequest<I> *req = CloseRequest<I>::create(m_image_ctx, ctx);
+ req->send();
+}
+
+template <typename I>
+Context *OpenRequest<I>::handle_close_image(int *result) {
+ CephContext *cct = m_image_ctx->cct;
+ ldout(cct, 10) << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to close image: " << cpp_strerror(*result) << dendl;
+ }
+ if (m_error_result < 0) {
+ *result = m_error_result;
+ }
+ return m_on_finish;
+}
+
+} // namespace image
+} // namespace librbd
+
+template class librbd::image::OpenRequest<librbd::ImageCtx>;
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_IMAGE_OPEN_REQUEST_H
+#define CEPH_LIBRBD_IMAGE_OPEN_REQUEST_H
+
+#include "include/int_types.h"
+#include "include/buffer.h"
+
+class Context;
+
+namespace librbd {
+
+class ImageCtx;
+
+namespace image {
+
+template <typename ImageCtxT = ImageCtx>
+class OpenRequest {
+public:
+ static OpenRequest *create(ImageCtxT *image_ctx, Context *on_finish) {
+ return new OpenRequest(image_ctx, on_finish);
+ }
+
+ void send();
+
+private:
+ /**
+ * @verbatim
+ *
+ * <start>
+ * |
+ * | (v1) (read only)
+ * |-----> V1_DETECT_HEADER . . . . . . . . . . . . . . . . .
+ * | | .
+ * | \-------------------------------\ .
+ * | (v2) | .
+ * \-----> V2_DETECT_HEADER | .
+ * | | .
+ * v | .
+ * V2_GET_ID | .
+ * | | .
+ * v | .
+ * V2_GET_IMMUTABLE_METADATA | .
+ * | | .
+ * v v .
+ * V2_GET_STRIPE_UNIT_COUNT ----> REGISTER_WATCH .
+ * . | .
+ * . (read only) v .
+ * . . . . . . . . . . . . . > REFRESH < . . . . .
+ * . |
+ * . |
+ * . \--> SET_SNAP
+ * (no snap) . |
+ * . v
+ * . . . > <finish>
+ * ^
+ * (on error) |
+ * * * * * * * > CLOSE --------------------------------/
+ *
+ * @endverbatim
+ */
+
+ OpenRequest(ImageCtxT *image_ctx, Context *on_finish);
+
+ ImageCtxT *m_image_ctx;
+ Context *m_on_finish;
+
+ bufferlist m_out_bl;
+ int m_error_result;
+
+ void send_v1_detect_header();
+ Context *handle_v1_detect_header(int *result);
+
+ void send_v2_detect_header();
+ Context *handle_v2_detect_header(int *result);
+
+ void send_v2_get_id();
+ Context *handle_v2_get_id(int *result);
+
+ void send_v2_get_immutable_metadata();
+ Context *handle_v2_get_immutable_metadata(int *result);
+
+ void send_v2_get_stripe_unit_count();
+ Context *handle_v2_get_stripe_unit_count(int *result);
+
+ void send_register_watch();
+ Context *handle_register_watch(int *result);
+
+ void send_refresh();
+ Context *handle_refresh(int *result);
+
+ Context *send_set_snap(int *result);
+ Context *handle_set_snap(int *result);
+
+ void send_close_image(int error_result);
+ Context *handle_close_image(int *result);
+
+};
+
+} // namespace image
+} // namespace librbd
+
+extern template class librbd::image::OpenRequest<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_IMAGE_OPEN_REQUEST_H
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/image/RefreshParentRequest.h"
+#include "include/rados/librados.hpp"
+#include "common/dout.h"
+#include "common/errno.h"
+#include "common/WorkQueue.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/Utils.h"
+#include "librbd/image/CloseRequest.h"
+#include "librbd/image/OpenRequest.h"
+#include "librbd/image/SetSnapRequest.h"
+
+#define dout_subsys ceph_subsys_rbd
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::image::RefreshParentRequest: "
+
+namespace librbd {
+namespace image {
+
+using util::create_async_context_callback;
+using util::create_context_callback;
+
+template <typename I>
+RefreshParentRequest<I>::RefreshParentRequest(I &child_image_ctx,
+ const parent_info &parent_md,
+ Context *on_finish)
+ : m_child_image_ctx(child_image_ctx), m_parent_md(parent_md),
+ m_on_finish(on_finish), m_parent_image_ctx(nullptr),
+ m_parent_snap_id(CEPH_NOSNAP), m_error_result(0) {
+}
+
+template <typename I>
+bool RefreshParentRequest<I>::is_refresh_required(I &child_image_ctx,
+ const parent_info &parent_md) {
+ assert(child_image_ctx.snap_lock.is_locked());
+ assert(child_image_ctx.parent_lock.is_locked());
+ return (is_open_required(child_image_ctx, parent_md) ||
+ is_close_required(child_image_ctx, parent_md));
+}
+
+template <typename I>
+bool RefreshParentRequest<I>::is_close_required(I &child_image_ctx,
+ const parent_info &parent_md) {
+ return (child_image_ctx.parent != nullptr &&
+ (parent_md.spec.pool_id == -1 || parent_md.overlap == 0));
+}
+
+template <typename I>
+bool RefreshParentRequest<I>::is_open_required(I &child_image_ctx,
+ const parent_info &parent_md) {
+ return (parent_md.spec.pool_id > -1 && parent_md.overlap > 0 &&
+ (child_image_ctx.parent == nullptr ||
+ child_image_ctx.parent->md_ctx.get_id() != parent_md.spec.pool_id ||
+ child_image_ctx.parent->id != parent_md.spec.image_id ||
+ child_image_ctx.parent->snap_id != parent_md.spec.snap_id));
+}
+
+template <typename I>
+void RefreshParentRequest<I>::send() {
+ if (is_open_required(m_child_image_ctx, m_parent_md)) {
+ send_open_parent();
+ } else {
+ // parent will be closed (if necessary) during finalize
+ send_complete(0);
+ }
+}
+
+template <typename I>
+void RefreshParentRequest<I>::apply() {
+ if (m_child_image_ctx.parent != nullptr) {
+ // closing parent image
+ m_child_image_ctx.clear_nonexistence_cache();
+ }
+ assert(m_child_image_ctx.snap_lock.is_wlocked());
+ assert(m_child_image_ctx.parent_lock.is_wlocked());
+ std::swap(m_child_image_ctx.parent, m_parent_image_ctx);
+}
+
+template <typename I>
+void RefreshParentRequest<I>::finalize(Context *on_finish) {
+ CephContext *cct = m_child_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ m_on_finish = on_finish;
+ if (m_parent_image_ctx != nullptr) {
+ send_close_parent();
+ } else {
+ send_complete(0);
+ }
+}
+
+template <typename I>
+void RefreshParentRequest<I>::send_open_parent() {
+ assert(m_parent_md.spec.pool_id >= 0);
+
+ CephContext *cct = m_child_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ librados::Rados rados(m_child_image_ctx.md_ctx);
+
+ librados::IoCtx parent_io_ctx;
+ int r = rados.ioctx_create2(m_parent_md.spec.pool_id, parent_io_ctx);
+ assert(r == 0);
+
+ // since we don't know the image and snapshot name, set their ids and
+ // reset the snap_name and snap_exists fields after we read the header
+ m_parent_image_ctx = new I("", m_parent_md.spec.image_id, NULL, parent_io_ctx,
+ true);
+
+ // set rados flags for reading the parent image
+ if (m_child_image_ctx.balance_parent_reads) {
+ m_parent_image_ctx->set_read_flag(librados::OPERATION_BALANCE_READS);
+ } else if (m_child_image_ctx.localize_parent_reads) {
+ m_parent_image_ctx->set_read_flag(librados::OPERATION_LOCALIZE_READS);
+ }
+
+ using klass = RefreshParentRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_open_parent, false>(this);
+ OpenRequest<I> *req = OpenRequest<I>::create(m_parent_image_ctx, ctx);
+ req->send();
+}
+
+template <typename I>
+Context *RefreshParentRequest<I>::handle_open_parent(int *result) {
+ CephContext *cct = m_child_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << " r=" << *result << dendl;
+
+ save_result(result);
+ if (*result < 0) {
+ lderr(cct) << "failed to open parent image: " << cpp_strerror(*result)
+ << dendl;
+ send_close_parent();
+ return nullptr;
+ }
+
+ send_set_parent_snap();
+ return nullptr;
+}
+
+template <typename I>
+void RefreshParentRequest<I>::send_set_parent_snap() {
+ assert(m_parent_md.spec.snap_id != CEPH_NOSNAP);
+
+ CephContext *cct = m_child_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ int r;
+ std::string snap_name;
+ {
+ RWLock::RLocker snap_locker(m_parent_image_ctx->snap_lock);
+ r = m_parent_image_ctx->get_snap_name(m_parent_md.spec.snap_id, &snap_name);
+ }
+
+ if (r < 0) {
+ lderr(cct) << "failed to located snapshot: " << cpp_strerror(r) << dendl;
+ send_complete(r);
+ return;
+ }
+
+ using klass = RefreshParentRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_set_parent_snap, false>(this);
+ SetSnapRequest<I> *req = SetSnapRequest<I>::create(
+ *m_parent_image_ctx, snap_name, ctx);
+ req->send();
+}
+
+template <typename I>
+Context *RefreshParentRequest<I>::handle_set_parent_snap(int *result) {
+ CephContext *cct = m_child_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << " r=" << *result << dendl;
+
+ save_result(result);
+ if (*result < 0) {
+ lderr(cct) << "failed to set parent snapshot: " << cpp_strerror(*result)
+ << dendl;
+ send_close_parent();
+ return nullptr;
+ }
+
+ return m_on_finish;
+}
+
+template <typename I>
+void RefreshParentRequest<I>::send_close_parent() {
+ assert(m_parent_image_ctx != nullptr);
+
+ CephContext *cct = m_child_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ using klass = RefreshParentRequest<I>;
+ Context *ctx = create_async_context_callback(
+ m_child_image_ctx, create_context_callback<
+ klass, &klass::handle_close_parent, false>(this));
+ CloseRequest<I> *req = CloseRequest<I>::create(m_parent_image_ctx, ctx);
+ req->send();
+}
+
+template <typename I>
+Context *RefreshParentRequest<I>::handle_close_parent(int *result) {
+ CephContext *cct = m_child_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << " r=" << *result << dendl;
+
+ delete m_parent_image_ctx;
+ if (*result < 0) {
+ lderr(cct) << "failed to close parent image: " << cpp_strerror(*result)
+ << dendl;
+ }
+
+ if (m_error_result < 0) {
+ // propagate errors from opening the image
+ *result = m_error_result;
+ } else {
+ // ignore errors from closing the image
+ *result = 0;
+ }
+
+ return m_on_finish;
+}
+
+template <typename I>
+void RefreshParentRequest<I>::send_complete(int r) {
+ CephContext *cct = m_child_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ m_on_finish->complete(r);
+}
+
+} // namespace image
+} // namespace librbd
+
+template class librbd::image::RefreshParentRequest<librbd::ImageCtx>;
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_IMAGE_REFRESH_PARENT_REQUEST_H
+#define CEPH_LIBRBD_IMAGE_REFRESH_PARENT_REQUEST_H
+
+#include "include/int_types.h"
+#include "librbd/parent_types.h"
+
+class Context;
+
+namespace librbd {
+
+class ImageCtx;
+
+namespace image {
+
+template <typename ImageCtxT = ImageCtx>
+class RefreshParentRequest {
+public:
+ static RefreshParentRequest *create(ImageCtxT &child_image_ctx,
+ const parent_info &parent_md,
+ Context *on_finish) {
+ return new RefreshParentRequest(child_image_ctx, parent_md, on_finish);
+ }
+
+ static bool is_refresh_required(ImageCtxT &child_image_ctx,
+ const parent_info &parent_md);
+
+ void send();
+ void apply();
+ void finalize(Context *on_finish);
+
+private:
+ /**
+ * @verbatim
+ *
+ * <start>
+ * |
+ * | (open required)
+ * |----------------> OPEN_PARENT * * * * * * * *
+ * | | *
+ * | v * (on error)
+ * | SET_PARENT_SNAP * * * * * *
+ * | | *
+ * | v *
+ * \----------------> <apply> *
+ * | *
+ * | (close required) v
+ * |-----------------> CLOSE_PARENT
+ * | |
+ * | v
+ * \-----------------> <finish>
+ *
+ * @endverbatim
+ */
+
+ RefreshParentRequest(ImageCtxT &child_image_ctx, const parent_info &parent_md,
+ Context *on_finish);
+
+ ImageCtxT &m_child_image_ctx;
+ parent_info m_parent_md;
+ Context *m_on_finish;
+
+ ImageCtxT *m_parent_image_ctx;
+ uint64_t m_parent_snap_id;
+
+ int m_error_result;
+
+ static bool is_close_required(ImageCtxT &child_image_ctx,
+ const parent_info &parent_md);
+ static bool is_open_required(ImageCtxT &child_image_ctx,
+ const parent_info &parent_md);
+
+ void send_open_parent();
+ Context *handle_open_parent(int *result);
+
+ void send_set_parent_snap();
+ Context *handle_set_parent_snap(int *result);
+
+ void send_close_parent();
+ Context *handle_close_parent(int *result);
+
+ void send_complete(int r);
+
+ void save_result(int *result) {
+ if (m_error_result == 0 && *result < 0) {
+ m_error_result = *result;
+ }
+ }
+
+};
+
+} // namespace image
+} // namespace librbd
+
+extern template class librbd::image::RefreshParentRequest<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_IMAGE_REFRESH_PARENT_REQUEST_H
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/image/RefreshRequest.h"
+#include "include/stringify.h"
+#include "common/dout.h"
+#include "common/errno.h"
+#include "cls/lock/cls_lock_client.h"
+#include "cls/rbd/cls_rbd_client.h"
+#include "librbd/ExclusiveLock.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/Journal.h"
+#include "librbd/ObjectMap.h"
+#include "librbd/Utils.h"
+#include "librbd/image/RefreshParentRequest.h"
+
+#define dout_subsys ceph_subsys_rbd
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::image::RefreshRequest: "
+
+namespace librbd {
+namespace image {
+
+using util::create_rados_ack_callback;
+using util::create_context_callback;
+
+template <typename I>
+RefreshRequest<I>::RefreshRequest(I &image_ctx, Context *on_finish)
+ : m_image_ctx(image_ctx), m_on_finish(on_finish), m_error_result(0),
+ m_flush_aio(false), m_exclusive_lock(nullptr), m_object_map(nullptr),
+ m_journal(nullptr), m_refresh_parent(nullptr) {
+}
+
+template <typename I>
+RefreshRequest<I>::~RefreshRequest() {
+ delete m_object_map;
+
+ // these require state machine to close
+ assert(m_exclusive_lock == nullptr);
+ assert(m_journal == nullptr);
+ assert(m_refresh_parent == nullptr);
+}
+
+template <typename I>
+void RefreshRequest<I>::send() {
+ if (m_image_ctx.old_format) {
+ send_v1_read_header();
+ } else {
+ send_v2_get_mutable_metadata();
+ }
+}
+
+template <typename I>
+void RefreshRequest<I>::send_v1_read_header() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ librados::ObjectReadOperation op;
+ op.read(0, 0, nullptr, nullptr);
+
+ using klass = RefreshRequest<I>;
+ librados::AioCompletion *comp = create_rados_ack_callback<
+ klass, &klass::handle_v1_read_header>(this);
+ m_out_bl.clear();
+ int r = m_image_ctx.md_ctx.aio_operate(m_image_ctx.header_oid, comp, &op,
+ &m_out_bl);
+ assert(r == 0);
+ comp->release();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_v1_read_header(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": " << "r=" << *result << dendl;
+
+ rbd_obj_header_ondisk v1_header;
+ if (*result < 0) {
+ return m_on_finish;
+ } else if (m_out_bl.length() < sizeof(v1_header)) {
+ lderr(cct) << "v1 header too small" << dendl;
+ *result = -EIO;
+ return m_on_finish;
+ } else if (memcmp(RBD_HEADER_TEXT, m_out_bl.c_str(),
+ sizeof(RBD_HEADER_TEXT)) != 0) {
+ lderr(cct) << "unrecognized v1 header" << dendl;
+ *result = -ENXIO;
+ return m_on_finish;
+ }
+
+ memcpy(&v1_header, m_out_bl.c_str(), sizeof(v1_header));
+ m_order = v1_header.options.order;
+ m_size = v1_header.image_size;
+ m_object_prefix = v1_header.block_name;
+ send_v1_get_snapshots();
+ return nullptr;
+}
+
+template <typename I>
+void RefreshRequest<I>::send_v1_get_snapshots() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ librados::ObjectReadOperation op;
+ cls_client::old_snapshot_list_start(&op);
+
+ using klass = RefreshRequest<I>;
+ librados::AioCompletion *comp = create_rados_ack_callback<
+ klass, &klass::handle_v1_get_snapshots>(this);
+ m_out_bl.clear();
+ int r = m_image_ctx.md_ctx.aio_operate(m_image_ctx.header_oid, comp, &op,
+ &m_out_bl);
+ assert(r == 0);
+ comp->release();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_v1_get_snapshots(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": " << "r=" << *result << dendl;
+
+ if (*result == 0) {
+ bufferlist::iterator it = m_out_bl.begin();
+ *result = cls_client::old_snapshot_list_finish(
+ &it, &m_snap_names, &m_snap_sizes, &m_snapc);
+ }
+
+ if (*result < 0) {
+ lderr(cct) << "failed to retrieve v1 snapshots: " << cpp_strerror(*result)
+ << dendl;
+ return m_on_finish;
+ }
+
+ if (!m_snapc.is_valid()) {
+ lderr(cct) << "v1 image snap context is invalid" << dendl;
+ *result = -EIO;
+ return m_on_finish;
+ }
+
+ send_v1_get_locks();
+ return nullptr;
+}
+
+template <typename I>
+void RefreshRequest<I>::send_v1_get_locks() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ librados::ObjectReadOperation op;
+ rados::cls::lock::get_lock_info_start(&op, RBD_LOCK_NAME);
+
+ using klass = RefreshRequest<I>;
+ librados::AioCompletion *comp = create_rados_ack_callback<
+ klass, &klass::handle_v1_get_locks>(this);
+ m_out_bl.clear();
+ int r = m_image_ctx.md_ctx.aio_operate(m_image_ctx.header_oid, comp, &op,
+ &m_out_bl);
+ assert(r == 0);
+ comp->release();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_v1_get_locks(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": "
+ << "r=" << *result << dendl;
+
+ // If EOPNOTSUPP, treat image as if there are no locks (we can't
+ // query them).
+ if (*result == -EOPNOTSUPP) {
+ *result = 0;
+ } else if (*result == 0) {
+ bufferlist::iterator it = m_out_bl.begin();
+ ClsLockType lock_type;
+ *result = rados::cls::lock::get_lock_info_finish(&it, &m_lockers,
+ &lock_type, &m_lock_tag);
+ if (*result == 0) {
+ m_exclusive_locked = (lock_type == LOCK_EXCLUSIVE);
+ }
+ }
+ if (*result < 0) {
+ lderr(cct) << "failed to retrieve locks: " << cpp_strerror(*result)
+ << dendl;
+ return m_on_finish;
+ }
+
+ apply();
+
+ return send_flush_aio();
+}
+
+template <typename I>
+void RefreshRequest<I>::send_v2_get_mutable_metadata() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ uint64_t snap_id;
+ {
+ RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
+ snap_id = m_image_ctx.snap_id;
+ }
+
+ bool read_only = m_image_ctx.read_only || snap_id != CEPH_NOSNAP;
+ librados::ObjectReadOperation op;
+ cls_client::get_mutable_metadata_start(&op, read_only);
+
+ using klass = RefreshRequest<I>;
+ librados::AioCompletion *comp = create_rados_ack_callback<
+ klass, &klass::handle_v2_get_mutable_metadata>(this);
+ m_out_bl.clear();
+ int r = m_image_ctx.md_ctx.aio_operate(m_image_ctx.header_oid, comp, &op,
+ &m_out_bl);
+ assert(r == 0);
+ comp->release();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_v2_get_mutable_metadata(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": "
+ << "r=" << *result << dendl;
+
+ if (*result == 0) {
+ bufferlist::iterator it = m_out_bl.begin();
+ *result = cls_client::get_mutable_metadata_finish(&it, &m_size, &m_features,
+ &m_incompatible_features,
+ &m_lockers,
+ &m_exclusive_locked,
+ &m_lock_tag, &m_snapc,
+ &m_parent_md);
+ }
+ if (*result < 0) {
+ lderr(cct) << "failed to retrieve mutable metadata: "
+ << cpp_strerror(*result) << dendl;
+ return m_on_finish;
+ }
+
+ uint64_t unsupported = m_incompatible_features & ~RBD_FEATURES_ALL;
+ if (unsupported != 0ULL) {
+ lderr(cct) << "Image uses unsupported features: " << unsupported << dendl;
+ *result = -ENOSYS;
+ return m_on_finish;
+ }
+
+ send_v2_get_flags();
+ return nullptr;
+}
+
+template <typename I>
+void RefreshRequest<I>::send_v2_get_flags() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ librados::ObjectReadOperation op;
+ cls_client::get_flags_start(&op, m_snapc.snaps);
+
+ using klass = RefreshRequest<I>;
+ librados::AioCompletion *comp = create_rados_ack_callback<
+ klass, &klass::handle_v2_get_flags>(this);
+ m_out_bl.clear();
+ int r = m_image_ctx.md_ctx.aio_operate(m_image_ctx.header_oid, comp, &op,
+ &m_out_bl);
+ assert(r == 0);
+ comp->release();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_v2_get_flags(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": "
+ << "r=" << *result << dendl;
+
+ if (*result == 0) {
+ bufferlist::iterator it = m_out_bl.begin();
+ cls_client::get_flags_finish(&it, &m_flags, m_snapc.snaps, &m_snap_flags);
+ }
+ if (*result == -EOPNOTSUPP) {
+ // Older OSD doesn't support RBD flags, need to assume the worst
+ ldout(cct, 10) << "OSD does not support RBD flags, disabling object map "
+ << "optimizations" << dendl;
+ m_flags = RBD_FLAG_OBJECT_MAP_INVALID;
+ if ((m_features & RBD_FEATURE_FAST_DIFF) != 0) {
+ m_flags |= RBD_FLAG_FAST_DIFF_INVALID;
+ }
+
+ std::vector<uint64_t> default_flags(m_snapc.snaps.size(), m_flags);
+ m_snap_flags = std::move(default_flags);
+ } else if (*result == -ENOENT) {
+ ldout(cct, 10) << "out-of-sync snapshot state detected" << dendl;
+ send_v2_get_mutable_metadata();
+ return nullptr;
+ } else if (*result < 0) {
+ lderr(cct) << "failed to retrieve flags: " << cpp_strerror(*result)
+ << dendl;
+ return m_on_finish;
+ }
+
+ return send_v2_get_snapshots();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::send_v2_get_snapshots() {
+ if (m_snapc.snaps.empty()) {
+ m_snap_names.clear();
+ m_snap_sizes.clear();
+ m_snap_parents.clear();
+ m_snap_protection.clear();
+ return send_v2_refresh_parent();
+ }
+
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ librados::ObjectReadOperation op;
+ cls_client::snapshot_list_start(&op, m_snapc.snaps);
+
+ using klass = RefreshRequest<I>;
+ librados::AioCompletion *comp = create_rados_ack_callback<
+ klass, &klass::handle_v2_get_snapshots>(this);
+ m_out_bl.clear();
+ int r = m_image_ctx.md_ctx.aio_operate(m_image_ctx.header_oid, comp, &op,
+ &m_out_bl);
+ assert(r == 0);
+ comp->release();
+ return nullptr;
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_v2_get_snapshots(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": "
+ << "r=" << *result << dendl;
+
+ if (*result == 0) {
+ bufferlist::iterator it = m_out_bl.begin();
+ *result = cls_client::snapshot_list_finish(&it, m_snapc.snaps,
+ &m_snap_names, &m_snap_sizes,
+ &m_snap_parents,
+ &m_snap_protection);
+ }
+ if (*result == -ENOENT) {
+ ldout(cct, 10) << "out-of-sync snapshot state detected" << dendl;
+ send_v2_get_mutable_metadata();
+ return nullptr;
+ } else if (*result < 0) {
+ lderr(cct) << "failed to retrieve snapshots: " << cpp_strerror(*result)
+ << dendl;
+ return m_on_finish;
+ }
+
+ if (!m_snapc.is_valid()) {
+ lderr(cct) << "image snap context is invalid!" << dendl;
+ *result = -EIO;
+ return m_on_finish;
+ }
+
+ return send_v2_refresh_parent();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::send_v2_refresh_parent() {
+ {
+ RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
+ RWLock::RLocker parent_locker(m_image_ctx.parent_lock);
+
+ parent_info parent_md;
+ int r = get_parent_info(m_image_ctx.snap_id, &parent_md);
+ if (r < 0 ||
+ RefreshParentRequest<I>::is_refresh_required(m_image_ctx, parent_md)) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ using klass = RefreshRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_v2_refresh_parent>(this);
+ m_refresh_parent = RefreshParentRequest<I>::create(
+ m_image_ctx, parent_md, ctx);
+ }
+ }
+
+ if (m_refresh_parent != nullptr) {
+ m_refresh_parent->send();
+ return nullptr;
+ } else {
+ return send_v2_init_exclusive_lock();
+ }
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_v2_refresh_parent(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to refresh parent image: " << cpp_strerror(*result)
+ << dendl;
+ save_result(result);
+ return send_v2_finalize_refresh_parent();
+ }
+
+ return send_v2_init_exclusive_lock();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::send_v2_init_exclusive_lock() {
+ if ((m_features & RBD_FEATURE_EXCLUSIVE_LOCK) == 0 ||
+ !m_image_ctx.snap_name.empty() ||
+ m_image_ctx.exclusive_lock != nullptr) {
+ return send_v2_open_journal();
+ }
+
+ // implies exclusive lock dynamically enabled or image open in-progress
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ // TODO need safe shut down
+ m_exclusive_lock = ExclusiveLock<I>::create(m_image_ctx);
+
+ using klass = RefreshRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_v2_init_exclusive_lock>(this);
+
+ RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
+ m_exclusive_lock->init(ctx);
+ return nullptr;
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_v2_init_exclusive_lock(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to initialize exclusive lock: "
+ << cpp_strerror(*result) << dendl;
+ save_result(result);
+ return send_v2_finalize_refresh_parent();
+ }
+
+ return send_v2_open_journal();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::send_v2_open_journal() {
+ if ((m_features & RBD_FEATURE_JOURNALING) == 0 ||
+ m_image_ctx.read_only ||
+ m_image_ctx.journal != nullptr ||
+ m_image_ctx.exclusive_lock == nullptr ||
+ !m_image_ctx.exclusive_lock->is_lock_owner()) {
+ return send_v2_open_object_map();
+ }
+
+ // implies journal dynamically enabled since ExclusiveLock will init
+ // the journal upon acquiring the lock
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ using klass = RefreshRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_v2_open_journal>(this);
+
+ // TODO need safe close
+ m_journal = new Journal(m_image_ctx);
+ m_journal->open(ctx);
+ return nullptr;
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_v2_open_journal(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to initialize journal: " << cpp_strerror(*result)
+ << dendl;
+ save_result(result);
+ return send_v2_finalize_refresh_parent();
+ }
+
+ return send_v2_shut_down_exclusive_lock();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::send_v2_open_object_map() {
+ if ((m_features & RBD_FEATURE_OBJECT_MAP) == 0 ||
+ m_image_ctx.object_map != nullptr || m_image_ctx.snap_name.empty()) {
+ return send_v2_finalize_refresh_parent();
+ }
+
+ // implies object map dynamically enabled or image open in-progress
+ // since SetSnapRequest loads the object map for a snapshot and
+ // ExclusiveLock loads the object map for HEAD
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ for (size_t snap_idx = 0; snap_idx < m_snap_names.size(); ++snap_idx) {
+ if (m_snap_names[snap_idx] == m_image_ctx.snap_name) {
+ using klass = RefreshRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_v2_open_object_map>(this);
+
+ m_object_map = m_image_ctx.create_object_map(m_snapc.snaps[snap_idx].val);
+ m_object_map->open(ctx);
+ return nullptr;
+ }
+ }
+
+ lderr(cct) << "failed to locate snapshot: " << m_image_ctx.snap_name
+ << dendl;
+ return send_v2_finalize_refresh_parent();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_v2_open_object_map(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << *result << dendl;
+
+ assert(*result == 0);
+ return send_v2_finalize_refresh_parent();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::send_v2_finalize_refresh_parent() {
+ apply();
+
+ if (m_refresh_parent == nullptr) {
+ return send_v2_shut_down_exclusive_lock();
+ }
+
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ using klass = RefreshRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_v2_finalize_refresh_parent>(this);
+ m_refresh_parent->finalize(ctx);
+ return nullptr;
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_v2_finalize_refresh_parent(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << *result << dendl;
+
+ assert(m_refresh_parent != nullptr);
+ delete m_refresh_parent;
+ m_refresh_parent = nullptr;
+
+ return send_v2_shut_down_exclusive_lock();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::send_v2_shut_down_exclusive_lock() {
+ if (m_exclusive_lock == nullptr) {
+ return send_v2_close_journal();
+ }
+
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ // exclusive lock feature was dynamically disabled
+ using klass = RefreshRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_v2_shut_down_exclusive_lock>(this);
+ m_exclusive_lock->shut_down(ctx);
+ return nullptr;
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_v2_shut_down_exclusive_lock(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to shut down exclusive lock: "
+ << cpp_strerror(*result) << dendl;
+ save_result(result);
+ }
+
+ assert(m_exclusive_lock != nullptr);
+ delete m_exclusive_lock;
+ m_exclusive_lock = nullptr;
+
+ return send_v2_close_journal();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::send_v2_close_journal() {
+ if (m_journal == nullptr) {
+ return send_flush_aio();
+ }
+
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ // journal feature was dynamically disabled
+ using klass = RefreshRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_v2_close_journal>(this);
+ m_journal->close(ctx);
+ return nullptr;
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_v2_close_journal(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ save_result(result);
+ lderr(cct) << "failed to close journal: " << cpp_strerror(*result)
+ << dendl;
+ }
+
+ return send_flush_aio();
+}
+
+template <typename I>
+Context *RefreshRequest<I>::send_flush_aio() {
+ if (m_flush_aio) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ RWLock::RLocker owner_lock(m_image_ctx.owner_lock);
+ using klass = RefreshRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_flush_aio>(this);
+ m_image_ctx.flush(ctx);
+ return nullptr;
+ }
+ return m_on_finish;
+}
+
+template <typename I>
+Context *RefreshRequest<I>::handle_flush_aio(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to flush pending AIO: " << cpp_strerror(*result)
+ << dendl;
+ }
+
+ if (m_error_result < 0) {
+ *result = m_error_result;
+ }
+ return m_on_finish;
+}
+
+template <typename I>
+void RefreshRequest<I>::apply() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 20) << this << " " << __func__ << dendl;
+
+ RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
+ RWLock::WLocker md_locker(m_image_ctx.md_lock);
+
+ {
+ Mutex::Locker cache_locker(m_image_ctx.cache_lock);
+ RWLock::WLocker snap_locker(m_image_ctx.snap_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;
+
+ 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.object_prefix = std::move(m_object_prefix);
+ m_image_ctx.init_layout();
+ } else {
+ m_image_ctx.features = m_features;
+ m_image_ctx.flags = m_flags;
+ m_image_ctx.parent_md = m_parent_md;
+ }
+
+ 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_names[i]
+ << " size=" << m_snap_sizes[i]
+ << dendl;
+ }
+ }
+
+ m_image_ctx.snaps.clear();
+ m_image_ctx.snap_info.clear();
+ m_image_ctx.snap_ids.clear();
+ 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];
+ parent_info parent;
+ if (!m_image_ctx.old_format) {
+ parent = m_snap_parents[i];
+ }
+
+ m_image_ctx.add_snap(m_snap_names[i], m_snapc.snaps[i].val,
+ m_snap_sizes[i], parent, protection_status, flags);
+ }
+ m_image_ctx.snapc = m_snapc;
+
+ if (m_image_ctx.snap_id != CEPH_NOSNAP &&
+ m_image_ctx.get_snap_id(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;
+ }
+
+ 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.test_features(RBD_FEATURE_EXCLUSIVE_LOCK,
+ m_image_ctx.snap_lock) ||
+ m_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.snap_lock) ||
+ m_journal != nullptr) {
+ std::swap(m_journal, m_image_ctx.journal);
+ }
+ if (!m_image_ctx.test_features(RBD_FEATURE_OBJECT_MAP,
+ m_image_ctx.snap_lock) ||
+ m_object_map != nullptr) {
+ std::swap(m_object_map, m_image_ctx.object_map);
+ }
+ }
+}
+
+template <typename I>
+int RefreshRequest<I>::get_parent_info(uint64_t snap_id,
+ parent_info *parent_md) {
+ if (snap_id == CEPH_NOSNAP) {
+ *parent_md = m_parent_md;
+ return 0;
+ } else {
+ for (size_t i = 0; i < m_snapc.snaps.size(); ++i) {
+ if (m_snapc.snaps[i].val == snap_id) {
+ *parent_md = m_snap_parents[i];
+ return 0;
+ }
+ }
+ }
+ return -ENOENT;
+}
+
+} // namespace image
+} // namespace librbd
+
+template class librbd::image::RefreshRequest<librbd::ImageCtx>;
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_IMAGE_REFRESH_REQUEST_H
+#define CEPH_LIBRBD_IMAGE_REFRESH_REQUEST_H
+
+#include "include/int_types.h"
+#include "include/buffer.h"
+#include "include/rbd_types.h"
+#include "common/snap_types.h"
+#include "cls/lock/cls_lock_types.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/parent_types.h"
+#include <string>
+#include <vector>
+
+class Context;
+
+namespace librbd {
+
+template <typename> class ExclusiveLock;
+class ImageCtx;
+class Journal;
+class ObjectMap;
+
+namespace image {
+
+template<typename> class RefreshParentRequest;
+
+template<typename ImageCtxT = ImageCtx>
+class RefreshRequest {
+public:
+ static RefreshRequest *create(ImageCtxT &image_ctx, Context *on_finish) {
+ return new RefreshRequest(image_ctx, on_finish);
+ }
+
+ ~RefreshRequest();
+
+ void send();
+
+private:
+ /**
+ * @verbatim
+ *
+ * <start>
+ * |
+ * | (v1)
+ * |-----> V1_READ_HEADER ---> V1_GET_SNAPSHOTS ---> V1_GET_LOCKS
+ * | |
+ * | (v2) v
+ * \-----> V2_GET_MUTABLE_METADATA <apply>
+ * | |
+ * v |
+ * V2_GET_FLAGS |
+ * | |
+ * v |
+ * V2_GET_SNAPSHOTS (skip if no snaps) |
+ * | |
+ * v |
+ * V2_REFRESH_PARENT (skip if no parent or |
+ * | refresh not needed) |
+ * v |
+ * V2_INIT_EXCLUSIVE_LOCK (skip if lock |
+ * | active or disabled) |
+ * v |
+ * V2_OPEN_JOURNAL (skip if journal |
+ * | active or disabled) |
+ * v |
+ * V2_OPEN_OBJECT_MAP (skip if map |
+ * | active or disabled) |
+ * v |
+ * <apply> |
+ * | |
+ * v |
+ * V2_FINALIZE_REFRESH_PARENT (skip if refresh |
+ * | not needed) |
+ * (error) v |
+ * * * * * > V2_SHUT_DOWN_EXCLUSIVE_LOCK (skip if lock |
+ * | active or enabled) |
+ * v |
+ * V2_CLOSE_JOURNAL (skip if journal inactive |
+ * | or enabled) |
+ * v |
+ * V2_CLOSE_OBJECT_MAP (skip if map inactive |
+ * | or enabled) |
+ * | |
+ * \-------------------\/--------------------/
+ * |
+ * v
+ * FLUSH (skip if no new
+ * | snapshots)
+ * v
+ * <finish>
+ *
+ * @endverbatim
+ */
+
+ ImageCtxT &m_image_ctx;
+ Context *m_on_finish;
+
+ int m_error_result;
+ bool m_flush_aio;
+ decltype(m_image_ctx.exclusive_lock) m_exclusive_lock;
+ decltype(m_image_ctx.object_map) m_object_map;
+ decltype(m_image_ctx.journal) m_journal;
+ RefreshParentRequest<ImageCtxT> *m_refresh_parent;
+
+ bufferlist m_out_bl;
+
+ uint8_t m_order;
+ uint64_t m_size;
+ uint64_t m_features;
+ uint64_t m_incompatible_features;
+ uint64_t m_flags;
+ std::string m_object_prefix;
+ parent_info m_parent_md;
+
+ ::SnapContext m_snapc;
+ std::vector<std::string> m_snap_names;
+ std::vector<uint64_t> m_snap_sizes;
+ std::vector<parent_info> m_snap_parents;
+ std::vector<uint8_t> m_snap_protection;
+ std::vector<uint64_t> m_snap_flags;
+
+ std::map<rados::cls::lock::locker_id_t,
+ rados::cls::lock::locker_info_t> m_lockers;
+ std::string m_lock_tag;
+ bool m_exclusive_locked;
+
+ RefreshRequest(ImageCtxT &image_ctx, Context *on_finish);
+
+ void send_v1_read_header();
+ Context *handle_v1_read_header(int *result);
+
+ void send_v1_get_snapshots();
+ Context *handle_v1_get_snapshots(int *result);
+
+ void send_v1_get_locks();
+ Context *handle_v1_get_locks(int *result);
+
+ void send_v2_get_mutable_metadata();
+ Context *handle_v2_get_mutable_metadata(int *result);
+
+ void send_v2_get_flags();
+ Context *handle_v2_get_flags(int *result);
+
+ Context *send_v2_get_snapshots();
+ Context *handle_v2_get_snapshots(int *result);
+
+ Context *send_v2_refresh_parent();
+ Context *handle_v2_refresh_parent(int *result);
+
+ Context *send_v2_init_exclusive_lock();
+ Context *handle_v2_init_exclusive_lock(int *result);
+
+ Context *send_v2_open_journal();
+ Context *handle_v2_open_journal(int *result);
+
+ Context *send_v2_open_object_map();
+ Context *handle_v2_open_object_map(int *result);
+
+ Context *send_v2_finalize_refresh_parent();
+ Context *handle_v2_finalize_refresh_parent(int *result);
+
+ Context *send_v2_shut_down_exclusive_lock();
+ Context *handle_v2_shut_down_exclusive_lock(int *result);
+
+ Context *send_v2_close_journal();
+ Context *handle_v2_close_journal(int *result);
+
+ Context *send_flush_aio();
+ Context *handle_flush_aio(int *result);
+
+ void save_result(int *result) {
+ if (m_error_result == 0 && *result < 0) {
+ m_error_result = *result;
+ }
+ }
+
+ void apply();
+ int get_parent_info(uint64_t snap_id, parent_info *parent_md);
+};
+
+} // namespace image
+} // namespace librbd
+
+extern template class librbd::image::RefreshRequest<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_IMAGE_REFRESH_REQUEST_H
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/image/SetSnapRequest.h"
+#include "common/dout.h"
+#include "common/errno.h"
+#include "librbd/AioImageRequestWQ.h"
+#include "librbd/ExclusiveLock.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/ObjectMap.h"
+#include "librbd/Utils.h"
+#include "librbd/image/RefreshParentRequest.h"
+
+#define dout_subsys ceph_subsys_rbd
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::image::SetSnapRequest: "
+
+namespace librbd {
+namespace image {
+
+using util::create_context_callback;
+
+template <typename I>
+SetSnapRequest<I>::SetSnapRequest(I &image_ctx, const std::string &snap_name,
+ Context *on_finish)
+ : m_image_ctx(image_ctx), m_snap_name(snap_name), m_on_finish(on_finish),
+ m_snap_id(CEPH_NOSNAP), m_exclusive_lock(nullptr), m_object_map(nullptr),
+ m_refresh_parent(nullptr), m_writes_blocked(false) {
+}
+
+template <typename I>
+SetSnapRequest<I>::~SetSnapRequest() {
+ delete m_refresh_parent;
+ delete m_object_map;
+ delete m_exclusive_lock;
+ if (m_writes_blocked) {
+ m_image_ctx.aio_work_queue->unblock_writes();
+ }
+}
+
+template <typename I>
+void SetSnapRequest<I>::send() {
+ if (m_snap_name.empty()) {
+ send_init_exclusive_lock();
+ } else {
+ send_block_writes();
+ }
+}
+
+template <typename I>
+void SetSnapRequest<I>::send_init_exclusive_lock() {
+ {
+ RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
+ if (m_image_ctx.exclusive_lock != nullptr) {
+ assert(m_image_ctx.snap_id == CEPH_NOSNAP);
+ send_complete();
+ return;
+ }
+ }
+
+ if (!m_image_ctx.test_features(RBD_FEATURE_EXCLUSIVE_LOCK)) {
+ int r = 0;
+ if (send_refresh_parent(&r) != nullptr) {
+ send_complete();
+ return;
+ }
+ }
+
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << __func__ << dendl;
+
+ m_exclusive_lock = ExclusiveLock<I>::create(m_image_ctx);
+
+ using klass = SetSnapRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_init_exclusive_lock>(this);
+
+ RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
+ m_exclusive_lock->init(ctx);
+}
+
+template <typename I>
+Context *SetSnapRequest<I>::handle_init_exclusive_lock(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to initialize exclusive lock: "
+ << cpp_strerror(*result) << dendl;
+ return m_on_finish;
+ }
+ return send_refresh_parent(result);
+}
+
+template <typename I>
+void SetSnapRequest<I>::send_block_writes() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << __func__ << dendl;
+
+ m_writes_blocked = true;
+
+ using klass = SetSnapRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_block_writes>(this);
+
+ RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
+ m_image_ctx.aio_work_queue->block_writes(ctx);
+}
+
+template <typename I>
+Context *SetSnapRequest<I>::handle_block_writes(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to block writes: " << cpp_strerror(*result)
+ << dendl;
+ return m_on_finish;
+ }
+
+ {
+ RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
+ m_snap_id = m_image_ctx.get_snap_id(m_snap_name);
+ if (m_snap_id == CEPH_NOSNAP) {
+ lderr(cct) << "failed to locate snapshot '" << m_snap_name << "'"
+ << dendl;
+
+ *result = -ENOENT;
+ return m_on_finish;
+ }
+ }
+
+ return send_shut_down_exclusive_lock(result);
+}
+
+template <typename I>
+Context *SetSnapRequest<I>::send_shut_down_exclusive_lock(int *result) {
+ ExclusiveLock<I> *exclusive_lock;
+ {
+ RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
+ exclusive_lock = m_image_ctx.exclusive_lock;
+ }
+
+ if (exclusive_lock == nullptr) {
+ return send_refresh_parent(result);
+ }
+
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << __func__ << dendl;
+
+ using klass = SetSnapRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_shut_down_exclusive_lock>(this);
+ exclusive_lock->shut_down(ctx);
+ return nullptr;
+}
+
+template <typename I>
+Context *SetSnapRequest<I>::handle_shut_down_exclusive_lock(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to shut down exclusive lock: "
+ << cpp_strerror(*result) << dendl;
+ return m_on_finish;
+ }
+
+ return send_refresh_parent(result);
+}
+
+template <typename I>
+Context *SetSnapRequest<I>::send_refresh_parent(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+
+ parent_info parent_md;
+ bool refresh_parent;
+ {
+ RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
+ RWLock::RLocker parent_locker(m_image_ctx.parent_lock);
+
+ const parent_info *parent_info = m_image_ctx.get_parent_info(m_snap_id);
+ if (parent_info == nullptr) {
+ *result = -ENOENT;
+ lderr(cct) << "failed to retrieve snapshot parent info" << dendl;
+ return m_on_finish;
+ }
+
+ parent_md = *parent_info;
+ refresh_parent = RefreshParentRequest<I>::is_refresh_required(m_image_ctx,
+ parent_md);
+ }
+
+ if (!refresh_parent) {
+ if (m_snap_id == CEPH_NOSNAP) {
+ // object map is loaded when exclusive lock is acquired
+ *result = apply();
+ return m_on_finish;
+ } else {
+ // load snapshot object map
+ return send_open_object_map(result);
+ }
+ }
+
+ ldout(cct, 10) << __func__ << dendl;
+
+ using klass = SetSnapRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_refresh_parent>(this);
+ m_refresh_parent = RefreshParentRequest<I>::create(m_image_ctx, parent_md,
+ ctx);
+ m_refresh_parent->send();
+ return nullptr;
+}
+
+template <typename I>
+Context *SetSnapRequest<I>::handle_refresh_parent(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to refresh snapshot parent: " << cpp_strerror(*result)
+ << dendl;
+ return m_on_finish;
+ }
+
+ if (m_snap_id == CEPH_NOSNAP) {
+ // object map is loaded when exclusive lock is acquired
+ *result = apply();
+ if (*result < 0) {
+ return m_on_finish;
+ }
+
+ return send_finalize_refresh_parent(result);
+ } else {
+ // load snapshot object map
+ return send_open_object_map(result);
+ }
+}
+
+template <typename I>
+Context *SetSnapRequest<I>::send_open_object_map(int *result) {
+ if (!m_image_ctx.test_features(RBD_FEATURE_OBJECT_MAP)) {
+ *result = apply();
+ if (*result < 0) {
+ return m_on_finish;
+ }
+
+ return send_finalize_refresh_parent(result);
+ }
+
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << __func__ << dendl;
+
+ using klass = SetSnapRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_open_object_map>(this);
+ m_object_map = new ObjectMap(m_image_ctx, m_snap_id);
+ m_object_map->open(ctx);
+ return nullptr;
+}
+
+template <typename I>
+Context *SetSnapRequest<I>::handle_open_object_map(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << __func__ << ": r=" << *result << dendl;
+
+ // object map should never report errors
+ assert(*result == 0);
+
+ *result = apply();
+ if (*result < 0) {
+ return m_on_finish;
+ }
+
+ return send_finalize_refresh_parent(result);
+}
+
+template <typename I>
+Context *SetSnapRequest<I>::send_finalize_refresh_parent(int *result) {
+ if (m_refresh_parent == nullptr) {
+ return m_on_finish;
+ }
+
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << dendl;
+
+ using klass = SetSnapRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_finalize_refresh_parent>(this);
+ m_refresh_parent->finalize(ctx);
+ return nullptr;
+}
+
+template <typename I>
+Context *SetSnapRequest<I>::handle_finalize_refresh_parent(int *result) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << this << " " << __func__ << ": r=" << *result << dendl;
+
+ if (*result < 0) {
+ lderr(cct) << "failed to close parent image: " << cpp_strerror(*result)
+ << dendl;
+ }
+ return m_on_finish;
+}
+
+template <typename I>
+int SetSnapRequest<I>::apply() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << __func__ << dendl;
+
+ RWLock::WLocker snap_locker(m_image_ctx.snap_lock);
+ RWLock::WLocker parent_locker(m_image_ctx.parent_lock);
+
+ if (m_snap_id != CEPH_NOSNAP) {
+ int r = m_image_ctx.snap_set(m_snap_name);
+ if (r < 0) {
+ return r;
+ }
+ } else {
+ m_image_ctx.snap_unset();
+ }
+
+ if (m_refresh_parent != nullptr) {
+ m_refresh_parent->apply();
+ }
+
+ std::swap(m_exclusive_lock, m_image_ctx.exclusive_lock);
+ std::swap(m_object_map, m_image_ctx.object_map_ptr);
+ return 0;
+}
+
+template <typename I>
+void SetSnapRequest<I>::send_complete() {
+ m_on_finish->complete(0);
+ delete this;
+}
+
+} // namespace image
+} // namespace librbd
+
+template class librbd::image::SetSnapRequest<librbd::ImageCtx>;
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_IMAGE_SNAP_SET_REQUEST_H
+#define CEPH_LIBRBD_IMAGE_SNAP_SET_REQUEST_H
+
+#include "include/int_types.h"
+#include "librbd/parent_types.h"
+#include <string>
+
+class Context;
+
+namespace librbd {
+
+template <typename> class ExclusiveLock;
+class ImageCtx;
+class ObjectMap;
+
+namespace image {
+
+template <typename> class RefreshParentRequest;
+
+template <typename ImageCtxT = ImageCtx>
+class SetSnapRequest {
+public:
+ static SetSnapRequest *create(ImageCtxT &image_ctx,
+ const std::string &snap_name,
+ Context *on_finish) {
+ return new SetSnapRequest(image_ctx, snap_name, on_finish);
+ }
+
+ ~SetSnapRequest();
+
+ void send();
+
+private:
+ /**
+ * @verbatim
+ *
+ * <start>
+ * |
+ * | (set snap)
+ * |-----------> BLOCK_WRITES
+ * | |
+ * | v
+ * | SHUTDOWN_EXCLUSIVE_LOCK (skip if lock inactive
+ * | | or disabled)
+ * | v
+ * | REFRESH_PARENT (skip if no parent
+ * | | or refresh not needed)
+ * | v
+ * | OPEN_OBJECT_MAP (skip if map disabled)
+ * | |
+ * | v
+ * | <apply>
+ * | |
+ * | v
+ * | FINALIZE_REFRESH_PARENT (skip if no parent
+ * | | or refresh not needed)
+ * | v
+ * | <finish>
+ * |
+ * \-----------> INIT_EXCLUSIVE_LOCK (skip if active or
+ * | disabled)
+ * v
+ * REFRESH_PARENT (skip if no parent
+ * | or refresh not needed)
+ * v
+ * <apply>
+ * |
+ * v
+ * FINALIZE_REFRESH_PARENT (skip if no parent
+ * | or refresh not needed)
+ * v
+ * <finish>
+ *
+ * @endverbatim
+ */
+
+ SetSnapRequest(ImageCtxT &image_ctx, const std::string &snap_name,
+ Context *on_finish);
+
+ ImageCtxT &m_image_ctx;
+ std::string m_snap_name;
+ Context *m_on_finish;
+
+ uint64_t m_snap_id;
+ ExclusiveLock<ImageCtxT> *m_exclusive_lock;
+ ObjectMap *m_object_map;
+ RefreshParentRequest<ImageCtxT> *m_refresh_parent;
+
+ bool m_writes_blocked;
+
+ void send_block_writes();
+ Context *handle_block_writes(int *result);
+
+ void send_init_exclusive_lock();
+ Context *handle_init_exclusive_lock(int *result);
+
+ Context *send_shut_down_exclusive_lock(int *result);
+ Context *handle_shut_down_exclusive_lock(int *result);
+
+ Context *send_refresh_parent(int *result);
+ Context *handle_refresh_parent(int *result);
+
+ Context *send_open_object_map(int *result);
+ Context *handle_open_object_map(int *result);
+
+ Context *send_finalize_refresh_parent(int *result);
+ Context *handle_finalize_refresh_parent(int *result);
+
+ int apply();
+ void send_complete();
+};
+
+} // namespace image
+} // namespace librbd
+
+extern template class librbd::image::SetSnapRequest<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_IMAGE_SNAP_SET_REQUEST_H
#include "librbd/Journal.h"
#include "librbd/ObjectMap.h"
#include "librbd/parent_types.h"
+#include "librbd/Utils.h"
+#include "librbd/image/CloseRequest.h"
+#include "librbd/image/OpenRequest.h"
+#include "librbd/image/RefreshParentRequest.h"
+#include "librbd/image/RefreshRequest.h"
+#include "librbd/image/SetSnapRequest.h"
#include "librbd/operation/FlattenRequest.h"
#include "librbd/operation/RebuildObjectMapRequest.h"
#include "librbd/operation/RenameRequest.h"
#include "librbd/operation/SnapshotRollbackRequest.h"
#include "librbd/operation/SnapshotUnprotectRequest.h"
#include "librbd/operation/TrimRequest.h"
-#include "librbd/Utils.h"
#include "include/util.h"
#include <boost/bind.hpp>
<< "' id = '" << ictx->id
<< "' snap_name = '"
<< ictx->snap_name << "'" << dendl;
- int r = ictx->init();
+ int r = ictx->init_legacy();
if (r < 0)
goto err_close;