librbd_progress_fn_t cb,
void *cbdata);
+CEPH_RBD_API int rbd_sparsify(rbd_image_t image, size_t sparse_size);
+
+CEPH_RBD_API int rbd_sparsify_with_progress(rbd_image_t image,
+ size_t sparse_size,
+ librbd_progress_fn_t cb,
+ void *cbdata);
+
/**
* List all images that are cloned from the image at the
* snapshot that is set via rbd_snap_set().
int flatten();
int flatten_with_progress(ProgressContext &prog_ctx);
+
+ int sparsify(size_t sparse_size);
+ int sparsify_with_progress(size_t sparse_size, ProgressContext &prog_ctx);
/**
* Returns a pair of poolname, imagename for each clone
* of this image at the currently set snapshot.
operation/SnapshotRollbackRequest.cc
operation/SnapshotUnprotectRequest.cc
operation/SnapshotLimitRequest.cc
+ operation/SparsifyRequest.cc
operation/TrimRequest.cc
trash/MoveRequest.cc
watcher/Notifier.cc
prog_ctx, on_finish);
}
+template <typename I>
+void ImageWatcher<I>::notify_sparsify(uint64_t request_id, size_t sparse_size,
+ ProgressContext &prog_ctx,
+ Context *on_finish) {
+ ceph_assert(m_image_ctx.owner_lock.is_locked());
+ ceph_assert(m_image_ctx.exclusive_lock &&
+ !m_image_ctx.exclusive_lock->is_lock_owner());
+
+ AsyncRequestId async_request_id(get_client_id(), request_id);
+
+ notify_async_request(async_request_id,
+ SparsifyPayload(async_request_id, sparse_size), prog_ctx,
+ on_finish);
+}
+
template <typename I>
void ImageWatcher<I>::notify_header_update(Context *on_finish) {
ldout(m_image_ctx.cct, 10) << this << ": " << __func__ << dendl;
return true;
}
+template <typename I>
+bool ImageWatcher<I>::handle_payload(const SparsifyPayload &payload,
+ C_NotifyAck *ack_ctx) {
+ RWLock::RLocker l(m_image_ctx.owner_lock);
+ if (m_image_ctx.exclusive_lock != nullptr) {
+ int r;
+ if (m_image_ctx.exclusive_lock->accept_requests(&r)) {
+ bool new_request;
+ Context *ctx;
+ ProgressContext *prog_ctx;
+ r = prepare_async_request(payload.async_request_id, &new_request,
+ &ctx, &prog_ctx);
+ if (r == 0 && new_request) {
+ ldout(m_image_ctx.cct, 10) << this << " remote sparsify request: "
+ << payload.async_request_id << dendl;
+ m_image_ctx.operations->execute_sparsify(payload.sparse_size, *prog_ctx,
+ ctx);
+ }
+
+ encode(ResponseMessage(r), ack_ctx->out);
+ } else if (r < 0) {
+ encode(ResponseMessage(r), ack_ctx->out);
+ }
+ }
+ return true;
+}
+
template <typename I>
bool ImageWatcher<I>::handle_payload(const UnknownPayload &payload,
C_NotifyAck *ack_ctx) {
void notify_migrate(uint64_t request_id, ProgressContext &prog_ctx,
Context *on_finish);
+ void notify_sparsify(uint64_t request_id, size_t sparse_size,
+ ProgressContext &prog_ctx, Context *on_finish);
+
void notify_acquired_lock();
void notify_released_lock();
void notify_request_lock();
C_NotifyAck *ctx);
bool handle_payload(const watch_notify::MigratePayload& payload,
C_NotifyAck *ctx);
+ bool handle_payload(const watch_notify::SparsifyPayload& payload,
+ C_NotifyAck *ctx);
bool handle_payload(const watch_notify::UnknownPayload& payload,
C_NotifyAck *ctx);
void process_payload(uint64_t notify_id, uint64_t handle,
#include "librbd/operation/SnapshotRollbackRequest.h"
#include "librbd/operation/SnapshotUnprotectRequest.h"
#include "librbd/operation/SnapshotLimitRequest.h"
+#include "librbd/operation/SparsifyRequest.h"
#include <set>
#include <boost/bind.hpp>
#include <boost/scope_exit.hpp>
req->send();
}
+template <typename I>
+int Operations<I>::sparsify(size_t sparse_size, ProgressContext &prog_ctx) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 20) << "sparsify" << dendl;
+
+ if (sparse_size < 4096 || sparse_size > m_image_ctx.get_object_size() ||
+ (sparse_size & (sparse_size - 1)) != 0) {
+ lderr(cct) << "sparse size should be power of two not less than 4096"
+ << " and not larger image object size" << dendl;
+ return -EINVAL;
+ }
+
+ uint64_t request_id = ++m_async_request_seq;
+ int r = invoke_async_request("sparsify", false,
+ boost::bind(&Operations<I>::execute_sparsify,
+ this, sparse_size,
+ boost::ref(prog_ctx), _1),
+ boost::bind(&ImageWatcher<I>::notify_sparsify,
+ m_image_ctx.image_watcher,
+ request_id, sparse_size,
+ boost::ref(prog_ctx), _1));
+ if (r < 0 && r != -EINVAL) {
+ return r;
+ }
+ ldout(cct, 20) << "resparsify finished" << dendl;
+ return 0;
+}
+
+template <typename I>
+void Operations<I>::execute_sparsify(size_t sparse_size,
+ ProgressContext &prog_ctx,
+ Context *on_finish) {
+ ceph_assert(m_image_ctx.owner_lock.is_locked());
+ ceph_assert(m_image_ctx.exclusive_lock == nullptr ||
+ m_image_ctx.exclusive_lock->is_lock_owner());
+
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 20) << "sparsify" << dendl;
+
+ if (m_image_ctx.operations_disabled) {
+ on_finish->complete(-EROFS);
+ return;
+ }
+
+ auto req = new operation::SparsifyRequest<I>(
+ m_image_ctx, sparse_size, new C_NotifyUpdate<I>(m_image_ctx, on_finish),
+ prog_ctx);
+ req->send();
+}
+
template <typename I>
int Operations<I>::prepare_image_update(bool request_lock) {
ceph_assert(m_image_ctx.owner_lock.is_locked() &&
int migrate(ProgressContext &prog_ctx);
void execute_migrate(ProgressContext &prog_ctx, Context *on_finish);
+ int sparsify(size_t sparse_size, ProgressContext &prog_ctx);
+ void execute_sparsify(size_t sparse_size, ProgressContext &prog_ctx,
+ Context *on_finish);
+
int prepare_image_update(bool request_lock);
private:
f->dump_bool("enabled", enabled);
}
+void SparsifyPayload::encode(bufferlist &bl) const {
+ using ceph::encode;
+ AsyncRequestPayloadBase::encode(bl);
+ encode(sparse_size, bl);
+}
+
+void SparsifyPayload::decode(__u8 version, bufferlist::const_iterator &iter) {
+ using ceph::decode;
+ AsyncRequestPayloadBase::decode(version, iter);
+ decode(sparse_size, iter);
+}
+
+void SparsifyPayload::dump(Formatter *f) const {
+ AsyncRequestPayloadBase::dump(f);
+ f->dump_unsigned("sparse_size", sparse_size);
+}
+
void UnknownPayload::encode(bufferlist &bl) const {
ceph_abort();
}
case NOTIFY_OP_MIGRATE:
payload = MigratePayload();
break;
+ case NOTIFY_OP_SPARSIFY:
+ payload = SparsifyPayload();
+ break;
default:
payload = UnknownPayload();
break;
o.push_back(new NotifyMessage(RenamePayload("foo")));
o.push_back(new NotifyMessage(UpdateFeaturesPayload(1, true)));
o.push_back(new NotifyMessage(MigratePayload(AsyncRequestId(ClientId(0, 1), 2))));
+ o.push_back(new NotifyMessage(SparsifyPayload(AsyncRequestId(ClientId(0, 1), 2), 1)));
}
void ResponseMessage::encode(bufferlist& bl) const {
case NOTIFY_OP_MIGRATE:
out << "Migrate";
break;
+ case NOTIFY_OP_SPARSIFY:
+ out << "Sparsify";
+ break;
default:
out << "Unknown (" << static_cast<uint32_t>(op) << ")";
break;
NOTIFY_OP_RENAME = 14,
NOTIFY_OP_UPDATE_FEATURES = 15,
NOTIFY_OP_MIGRATE = 16,
+ NOTIFY_OP_SPARSIFY = 17,
};
struct AcquiredLockPayload {
MigratePayload(const AsyncRequestId &id) : AsyncRequestPayloadBase(id) {}
};
+struct SparsifyPayload : public AsyncRequestPayloadBase {
+ static const NotifyOp NOTIFY_OP = NOTIFY_OP_SPARSIFY;
+ static const bool CHECK_FOR_REFRESH = true;
+
+ SparsifyPayload() {}
+ SparsifyPayload(const AsyncRequestId &id, size_t sparse_size)
+ : AsyncRequestPayloadBase(id), sparse_size(sparse_size) {}
+
+ size_t sparse_size = 0;
+
+ void encode(bufferlist &bl) const;
+ void decode(__u8 version, bufferlist::const_iterator &iter);
+ void dump(Formatter *f) const;
+};
+
struct UnknownPayload {
static const NotifyOp NOTIFY_OP = static_cast<NotifyOp>(-1);
static const bool CHECK_FOR_REFRESH = false;
RenamePayload,
UpdateFeaturesPayload,
MigratePayload,
+ SparsifyPayload,
UnknownPayload> Payload;
struct NotifyMessage {
return r;
}
+ int Image::sparsify(size_t sparse_size)
+ {
+ ImageCtx *ictx = (ImageCtx *)ctx;
+ tracepoint(librbd, sparsify_enter, ictx, ictx->name.c_str(), sparse_size,
+ ictx->id.c_str());
+ librbd::NoOpProgressContext prog_ctx;
+ int r = ictx->operations->sparsify(sparse_size, prog_ctx);
+ tracepoint(librbd, sparsify_exit, r);
+ return r;
+ }
+
+ int Image::sparsify_with_progress(size_t sparse_size,
+ librbd::ProgressContext& prog_ctx)
+ {
+ ImageCtx *ictx = (ImageCtx *)ctx;
+ tracepoint(librbd, sparsify_enter, ictx, ictx->name.c_str(), sparse_size,
+ ictx->id.c_str());
+ int r = ictx->operations->sparsify(sparse_size, prog_ctx);
+ tracepoint(librbd, sparsify_exit, r);
+ return r;
+ }
+
int Image::list_children(set<pair<string, string> > *children)
{
std::vector<linked_image_spec_t> images;
return r;
}
+extern "C" int rbd_sparsify(rbd_image_t image, size_t sparse_size)
+{
+ librbd::ImageCtx *ictx = (librbd::ImageCtx *)image;
+ tracepoint(librbd, sparsify_enter, ictx, ictx->name.c_str(), sparse_size,
+ ictx->id.c_str());
+ librbd::NoOpProgressContext prog_ctx;
+ int r = ictx->operations->sparsify(sparse_size, prog_ctx);
+ tracepoint(librbd, sparsify_exit, r);
+ return r;
+}
+
+extern "C" int rbd_sparsify_with_progress(rbd_image_t image, size_t sparse_size,
+ librbd_progress_fn_t cb, void *cbdata)
+{
+ librbd::ImageCtx *ictx = (librbd::ImageCtx *)image;
+ tracepoint(librbd, sparsify_enter, ictx, ictx->name.c_str(), sparse_size,
+ ictx->id.c_str());
+ librbd::CProgressContext prog_ctx(cb, cbdata);
+ int r = ictx->operations->sparsify(sparse_size, prog_ctx);
+ tracepoint(librbd, sparsify_exit, r);
+ return r;
+}
+
extern "C" int rbd_rename(rados_ioctx_t src_p, const char *srcname,
const char *destname)
{
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/operation/SparsifyRequest.h"
+#include "cls/rbd/cls_rbd_client.h"
+#include "common/dout.h"
+#include "common/errno.h"
+#include "librbd/AsyncObjectThrottle.h"
+#include "librbd/ExclusiveLock.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/Types.h"
+#include "librbd/io/ObjectRequest.h"
+#include "osdc/Striper.h"
+#include <boost/lambda/bind.hpp>
+#include <boost/lambda/construct.hpp>
+
+#define dout_subsys ceph_subsys_rbd
+
+namespace librbd {
+namespace operation {
+
+using util::create_context_callback;
+using util::create_rados_callback;
+
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::operation::SparsifyObject: " << this \
+ << " " << m_oid << " " << __func__ << ": "
+
+template <typename I>
+class C_SparsifyObject : public C_AsyncObjectThrottle<I> {
+public:
+
+ /**
+ * @verbatim
+ *
+ * <start>
+ * |
+ * v (object map disabled)
+ * SPARSIFY -----------------------\
+ * | |
+ * | (object map enabled) |
+ * v |
+ * PRE UPDATE OBJECT MAP |
+ * | |
+ * v |
+ * CHECK EXISTS |
+ * | |
+ * v |
+ * POST UPDATE OBJECT MAP |
+ * | |
+ * v |
+ * <finish> <----------------------/
+ *
+ * @endverbatim
+ *
+ */
+
+ C_SparsifyObject(AsyncObjectThrottle<I> &throttle, I *image_ctx,
+ uint64_t object_no, size_t sparse_size)
+ : C_AsyncObjectThrottle<I>(throttle, *image_ctx), m_cct(image_ctx->cct),
+ m_object_no(object_no), m_sparse_size(sparse_size),
+ m_oid(image_ctx->get_object_name(object_no)) {
+ }
+
+ int send() override {
+ I &image_ctx = this->m_image_ctx;
+ ceph_assert(image_ctx.owner_lock.is_locked());
+
+ ldout(m_cct, 20) << dendl;
+
+ if (image_ctx.exclusive_lock != nullptr &&
+ !image_ctx.exclusive_lock->is_lock_owner()) {
+ ldout(m_cct, 1) << "lost exclusive lock during sparsify" << dendl;
+ return -ERESTART;
+ }
+
+ {
+ RWLock::RLocker snap_locker(image_ctx.snap_lock);
+ if (image_ctx.object_map != nullptr &&
+ !image_ctx.object_map->object_may_exist(m_object_no)) {
+ // can skip because the object does not exist
+ return 1;
+ }
+
+ RWLock::RLocker parent_locker(image_ctx.parent_lock);
+ uint64_t overlap_objects = 0;
+ uint64_t overlap;
+ int r = image_ctx.get_parent_overlap(CEPH_NOSNAP, &overlap);
+ if (r == 0 && overlap > 0) {
+ overlap_objects = Striper::get_num_objects(image_ctx.layout, overlap);
+ }
+ m_remove_empty = (m_object_no >= overlap_objects);
+ }
+
+ send_sparsify();
+ return 0;
+ }
+
+ void send_sparsify() {
+ I &image_ctx = this->m_image_ctx;
+ ldout(m_cct, 20) << dendl;
+
+ librados::ObjectWriteOperation op;
+ cls_client::sparsify(&op, m_sparse_size, m_remove_empty);
+ auto comp = create_rados_callback<
+ C_SparsifyObject, &C_SparsifyObject::handle_sparsify>(this);
+ int r = image_ctx.data_ctx.aio_operate(m_oid, comp, &op);
+ ceph_assert(r == 0);
+ comp->release();
+ }
+
+ void handle_sparsify(int r) {
+ ldout(m_cct, 20) << "r=" << r << dendl;
+
+ if (r < 0 && r != -ENOENT) {
+ lderr(m_cct) << "failed to sparsify: " << cpp_strerror(r) << dendl;
+ }
+
+ if (r == -ENOENT) {
+ this->complete(0);
+ return;
+ }
+
+ send_pre_update_object_map();
+ }
+
+ void send_pre_update_object_map() {
+ I *image_ctx = &this->m_image_ctx;
+
+ if (!m_remove_empty || !image_ctx->test_features(RBD_FEATURE_OBJECT_MAP)) {
+ this->complete(0);
+ return;
+ }
+
+ ldout(m_cct, 20) << dendl;
+
+ image_ctx->owner_lock.get_read();
+ image_ctx->snap_lock.get_read();
+ if (image_ctx->object_map == nullptr) {
+ // possible that exclusive lock was lost in background
+ lderr(m_cct) << "object map is not initialized" << dendl;
+
+ image_ctx->snap_lock.put_read();
+ image_ctx->owner_lock.put_read();
+ this->complete(-EINVAL);
+ return;
+ }
+
+ int r;
+ m_finish_op_ctx = image_ctx->exclusive_lock->start_op(&r);
+ if (m_finish_op_ctx == nullptr) {
+ lderr(m_cct) << "lost exclusive lock" << dendl;
+ image_ctx->snap_lock.put_read();
+ image_ctx->owner_lock.put_read();
+ this->complete(r);
+ return;
+ }
+
+ auto ctx = create_context_callback<
+ C_SparsifyObject<I>,
+ &C_SparsifyObject<I>::handle_pre_update_object_map>(this);
+
+ image_ctx->object_map_lock.get_write();
+ bool sent = image_ctx->object_map->template aio_update<
+ Context, &Context::complete>(CEPH_NOSNAP, m_object_no, OBJECT_PENDING,
+ OBJECT_EXISTS, {}, false, ctx);
+
+ // NOTE: state machine might complete before we reach here
+ image_ctx->object_map_lock.put_write();
+ image_ctx->snap_lock.put_read();
+ image_ctx->owner_lock.put_read();
+ if (!sent) {
+ ctx->complete(0);
+ }
+ }
+
+ void handle_pre_update_object_map(int r) {
+ ldout(m_cct, 20) << "r=" << r << dendl;
+
+ if (r < 0) {
+ lderr(m_cct) << "failed to update object map: " << cpp_strerror(r)
+ << dendl;
+ finish_op(r);
+ return;
+ }
+
+ send_check_exists();
+ }
+
+ void send_check_exists() {
+ I &image_ctx = this->m_image_ctx;
+
+ ldout(m_cct, 20) << dendl;
+
+ librados::ObjectReadOperation op;
+ op.stat(NULL, NULL, NULL);
+ m_out_bl.clear();
+ auto comp = create_rados_callback<
+ C_SparsifyObject, &C_SparsifyObject::handle_check_exists>(this);
+ int r = image_ctx.data_ctx.aio_operate(m_oid, comp, &op, &m_out_bl);
+ ceph_assert(r == 0);
+ comp->release();
+ }
+
+ void handle_check_exists(int r) {
+ ldout(m_cct, 20) << "r=" << r << dendl;
+
+ if (r < 0 && r != -ENOENT) {
+ lderr(m_cct) << "stat failed: " << cpp_strerror(r) << dendl;
+ finish_op(r);
+ return;
+ }
+
+ send_post_update_object_map(r == 0);
+ }
+
+ void send_post_update_object_map(bool exists) {
+ I &image_ctx = this->m_image_ctx;
+
+ auto ctx = create_context_callback<
+ C_SparsifyObject<I>,
+ &C_SparsifyObject<I>::handle_post_update_object_map>(this);
+ bool sent;
+ {
+ RWLock::RLocker owner_locker(image_ctx.owner_lock);
+ RWLock::RLocker snap_locker(image_ctx.snap_lock);
+
+ assert(image_ctx.exclusive_lock->is_lock_owner());
+ assert(image_ctx.object_map != nullptr);
+
+ RWLock::WLocker object_map_locker(image_ctx.object_map_lock);
+
+ sent = image_ctx.object_map->template aio_update<
+ Context, &Context::complete>(CEPH_NOSNAP, m_object_no,
+ exists ? OBJECT_EXISTS : OBJECT_NONEXISTENT,
+ OBJECT_PENDING, {}, false, ctx);
+ }
+ if (!sent) {
+ ctx->complete(0);
+ }
+ }
+
+ void handle_post_update_object_map(int r) {
+ ldout(m_cct, 20) << "r=" << r << dendl;
+
+ if (r < 0) {
+ lderr(m_cct) << "failed to update object map: " << cpp_strerror(r)
+ << dendl;
+ finish_op(r);
+ return;
+ }
+
+ finish_op(0);
+ }
+
+ void finish_op(int r) {
+ ldout(m_cct, 20) << "r=" << r << dendl;
+
+ m_finish_op_ctx->complete(0);
+ this->complete(r);
+ }
+
+private:
+ CephContext *m_cct;
+ uint64_t m_object_no;
+ size_t m_sparse_size;
+ std::string m_oid;
+
+ bool m_remove_empty = false;
+ bufferlist m_out_bl;
+ Context *m_finish_op_ctx = nullptr;
+};
+
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::operation::SparsifyRequest: " << this \
+ << " " << __func__ << ": "
+
+template <typename I>
+bool SparsifyRequest<I>::should_complete(int r) {
+ I &image_ctx = this->m_image_ctx;
+ CephContext *cct = image_ctx.cct;
+ ldout(cct, 5) << "r=" << r << dendl;
+ if (r < 0) {
+ lderr(cct) << "encountered error: " << cpp_strerror(r) << dendl;
+ }
+ return true;
+}
+
+template <typename I>
+void SparsifyRequest<I>::send_op() {
+ sparsify_objects();
+}
+
+template <typename I>
+void SparsifyRequest<I>::sparsify_objects() {
+ I &image_ctx = this->m_image_ctx;
+ ceph_assert(image_ctx.owner_lock.is_locked());
+
+ CephContext *cct = image_ctx.cct;
+ ldout(cct, 5) << dendl;
+
+ assert(image_ctx.owner_lock.is_locked());
+
+ uint64_t objects = 0;
+ {
+ RWLock::RLocker snap_locker(image_ctx.snap_lock);
+ objects = image_ctx.get_object_count(CEPH_NOSNAP);
+ }
+
+ auto ctx = create_context_callback<
+ SparsifyRequest<I>,
+ &SparsifyRequest<I>::handle_sparsify_objects>(this);
+ typename AsyncObjectThrottle<I>::ContextFactory context_factory(
+ boost::lambda::bind(boost::lambda::new_ptr<C_SparsifyObject<I> >(),
+ boost::lambda::_1, &image_ctx, boost::lambda::_2, m_sparse_size));
+ AsyncObjectThrottle<I> *throttle = new AsyncObjectThrottle<I>(
+ this, image_ctx, context_factory, ctx, &m_prog_ctx, 0, objects);
+ throttle->start_ops(
+ image_ctx.config.template get_val<uint64_t>("rbd_concurrent_management_ops"));
+}
+
+template <typename I>
+void SparsifyRequest<I>::handle_sparsify_objects(int r) {
+ I &image_ctx = this->m_image_ctx;
+ CephContext *cct = image_ctx.cct;
+ ldout(cct, 5) << "r=" << r << dendl;
+
+ if (r == -ERESTART) {
+ ldout(cct, 5) << "sparsify operation interrupted" << dendl;
+ this->complete(r);
+ return;
+ } else if (r < 0) {
+ lderr(cct) << "sparsify encountered an error: " << cpp_strerror(r) << dendl;
+ this->complete(r);
+ return;
+ }
+
+ this->complete(0);
+}
+
+} // namespace operation
+} // namespace librbd
+
+template class librbd::operation::SparsifyRequest<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_OPERATION_SPARSIFY_REQUEST_H
+#define CEPH_LIBRBD_OPERATION_SPARSIFY_REQUEST_H
+
+#include "librbd/operation/Request.h"
+#include "common/snap_types.h"
+
+namespace librbd {
+
+class ImageCtx;
+class ProgressContext;
+
+namespace operation {
+
+template <typename ImageCtxT = ImageCtx>
+class SparsifyRequest : public Request<ImageCtxT>
+{
+public:
+ SparsifyRequest(ImageCtxT &image_ctx, size_t sparse_size, Context *on_finish,
+ ProgressContext &prog_ctx)
+ : Request<ImageCtxT>(image_ctx, on_finish), m_sparse_size(sparse_size),
+ m_prog_ctx(prog_ctx) {
+ }
+
+protected:
+ void send_op() override;
+ bool should_complete(int r) override;
+ bool can_affect_io() const override {
+ return true;
+ }
+ journal::Event create_event(uint64_t op_tid) const override {
+ ceph_abort();
+ return journal::UnknownEvent();
+ }
+
+private:
+ /**
+ * @verbatim
+ *
+ * <start>
+ * |
+ * v
+ * SPARSIFY OBJECTS
+ * |
+ * v
+ * <finish>
+ *
+ * @endverbatim
+ */
+
+ size_t m_sparse_size;
+ ProgressContext &m_prog_ctx;
+
+ void sparsify_objects();
+ void handle_sparsify_objects(int r);
+};
+
+} // namespace operation
+} // namespace librbd
+
+extern template class librbd::operation::SparsifyRequest<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_OPERATION_SPARSIFY_REQUEST_H
// vim: ts=8 sw=2 smarttab
#include "cls/journal/cls_journal_client.h"
+#include "cls/rbd/cls_rbd_client.h"
#include "cls/rbd/cls_rbd_types.h"
#include "test/librbd/test_fixture.h"
#include "test/librbd/test_support.h"
}
}
+static bool is_sparsify_supported(librados::IoCtx &ioctx,
+ const std::string &oid) {
+ EXPECT_EQ(0, ioctx.create(oid, true));
+ int r = librbd::cls_client::sparsify(&ioctx, oid, 16, true);
+ EXPECT_TRUE(r == 0 || r == -EOPNOTSUPP);
+ ioctx.remove(oid);
+
+ return (r == 0);
+}
+
+static bool is_sparse_read_supported(librados::IoCtx &ioctx,
+ const std::string &oid) {
+ EXPECT_EQ(0, ioctx.create(oid, true));
+ bufferlist inbl;
+ inbl.append(std::string(1, 'X'));
+ EXPECT_EQ(0, ioctx.write(oid, inbl, inbl.length(), 1));
+ EXPECT_EQ(0, ioctx.write(oid, inbl, inbl.length(), 3));
+
+ std::map<uint64_t, uint64_t> m;
+ bufferlist outbl;
+ int r = ioctx.sparse_read(oid, m, outbl, 4, 0);
+ ioctx.remove(oid);
+
+ int expected_r = 2;
+ std::map<uint64_t, uint64_t> expected_m = {{1, 1}, {3, 1}};
+ bufferlist expected_outbl;
+ expected_outbl.append(std::string(2, 'X'));
+
+ return (r == expected_r && m == expected_m &&
+ outbl.contents_equal(expected_outbl));
+}
+
TEST_F(TestInternal, OpenByID) {
REQUIRE_FORMAT_V2();
ASSERT_EQ(0, librbd::api::PoolMetadata<>::remove(m_ioctx,
"conf_rbd_journal_order"));
}
+
+TEST_F(TestInternal, Sparsify) {
+ librbd::ImageCtx *ictx;
+ ASSERT_EQ(0, open_image(m_image_name, &ictx));
+
+ REQUIRE(is_sparsify_supported(ictx->data_ctx, ictx->get_object_name(10)));
+
+ bool sparse_read_supported = is_sparse_read_supported(
+ ictx->data_ctx, ictx->get_object_name(10));
+
+ librbd::NoOpProgressContext no_op;
+ ASSERT_EQ(0, ictx->operations->resize((1 << ictx->order) * 20, true, no_op));
+
+ bufferlist bl;
+ bl.append(std::string(4096, '\0'));
+
+ ASSERT_EQ((ssize_t)bl.length(),
+ ictx->io_work_queue->write(0, bl.length(), bufferlist{bl}, 0));
+
+ bl.append(std::string(4096, '1'));
+ bl.append(std::string(4096, '\0'));
+ bl.append(std::string(4096, '2'));
+ bl.append(std::string(4096, '\0'));
+ ASSERT_EQ((ssize_t)bl.length(),
+ ictx->io_work_queue->write((1 << ictx->order) * 10, bl.length(),
+ bufferlist{bl}, 0));
+ ASSERT_EQ(0, ictx->io_work_queue->flush());
+
+ ASSERT_EQ(0, ictx->operations->sparsify(4096, no_op));
+
+ bufferptr read_ptr(bl.length());
+ bufferlist read_bl;
+ read_bl.push_back(read_ptr);
+
+ librbd::io::ReadResult read_result{&read_bl};
+ ASSERT_EQ((ssize_t)read_bl.length(),
+ ictx->io_work_queue->read((1 << ictx->order) * 10, read_bl.length(),
+ librbd::io::ReadResult{read_result}, 0));
+ ASSERT_TRUE(bl.contents_equal(read_bl));
+
+ std::string oid = ictx->get_object_name(0);
+ uint64_t size;
+ ASSERT_EQ(-ENOENT, ictx->data_ctx.stat(oid, &size, NULL));
+
+ if (!sparse_read_supported) {
+ return;
+ }
+
+ oid = ictx->get_object_name(10);
+ std::map<uint64_t, uint64_t> m;
+ read_bl.clear();
+ ASSERT_EQ(2, ictx->data_ctx.sparse_read(oid, m, read_bl, bl.length(), 0));
+ std::map<uint64_t, uint64_t> expected_m =
+ {{4096 * 1, 4096}, {4096 * 3, 4096}};
+ ASSERT_EQ(m, expected_m);
+ bl.clear();
+ bl.append(std::string(4096, '1'));
+ bl.append(std::string(4096, '2'));
+ ASSERT_TRUE(bl.contents_equal(read_bl));
+}
+
+
+TEST_F(TestInternal, SparsifyClone) {
+ REQUIRE_FEATURE(RBD_FEATURE_LAYERING);
+
+ librbd::ImageCtx *ictx;
+ ASSERT_EQ(0, open_image(m_image_name, &ictx));
+
+ REQUIRE(is_sparsify_supported(ictx->data_ctx, ictx->get_object_name(10)));
+
+ bool sparse_read_supported = is_sparse_read_supported(
+ ictx->data_ctx, ictx->get_object_name(10));
+
+ librbd::NoOpProgressContext no_op;
+ ASSERT_EQ(0, ictx->operations->resize((1 << ictx->order) * 10, true, no_op));
+
+ ASSERT_EQ(0, create_snapshot("snap", true));
+ std::string clone_name = get_temp_image_name();
+ int order = ictx->order;
+ ASSERT_EQ(0, librbd::clone(m_ioctx, m_image_name.c_str(), "snap", m_ioctx,
+ clone_name.c_str(), ictx->features, &order, 0, 0));
+ close_image(ictx);
+
+ ASSERT_EQ(0, open_image(clone_name, &ictx));
+
+ BOOST_SCOPE_EXIT_ALL(this, &ictx, clone_name) {
+ close_image(ictx);
+ librbd::NoOpProgressContext no_op;
+ EXPECT_EQ(0, librbd::api::Image<>::remove(m_ioctx, clone_name, "", no_op));
+ };
+
+ ASSERT_EQ(0, ictx->operations->resize((1 << ictx->order) * 20, true, no_op));
+
+ bufferlist bl;
+ bl.append(std::string(4096, '\0'));
+
+ ASSERT_EQ((ssize_t)bl.length(),
+ ictx->io_work_queue->write(0, bl.length(), bufferlist{bl}, 0));
+
+ bl.append(std::string(4096, '1'));
+ bl.append(std::string(4096, '\0'));
+ bl.append(std::string(4096, '2'));
+ bl.append(std::string(4096, '\0'));
+ ASSERT_EQ((ssize_t)bl.length(),
+ ictx->io_work_queue->write((1 << ictx->order) * 10, bl.length(),
+ bufferlist{bl}, 0));
+ ASSERT_EQ(0, ictx->io_work_queue->flush());
+
+ ASSERT_EQ(0, ictx->operations->sparsify(4096, no_op));
+
+ bufferptr read_ptr(bl.length());
+ bufferlist read_bl;
+ read_bl.push_back(read_ptr);
+
+ librbd::io::ReadResult read_result{&read_bl};
+ ASSERT_EQ((ssize_t)read_bl.length(),
+ ictx->io_work_queue->read((1 << ictx->order) * 10, read_bl.length(),
+ librbd::io::ReadResult{read_result}, 0));
+ ASSERT_TRUE(bl.contents_equal(read_bl));
+
+ std::string oid = ictx->get_object_name(0);
+ uint64_t size;
+ ASSERT_EQ(0, ictx->data_ctx.stat(oid, &size, NULL));
+ ASSERT_EQ(0, ictx->data_ctx.read(oid, read_bl, 4096, 0));
+
+ if (!sparse_read_supported) {
+ return;
+ }
+
+ oid = ictx->get_object_name(10);
+ std::map<uint64_t, uint64_t> m;
+ read_bl.clear();
+ ASSERT_EQ(2, ictx->data_ctx.sparse_read(oid, m, read_bl, bl.length(), 0));
+ std::map<uint64_t, uint64_t> expected_m =
+ {{4096 * 1, 4096}, {4096 * 3, 4096}};
+ ASSERT_EQ(m, expected_m);
+ bl.clear();
+ bl.append(std::string(4096, '1'));
+ bl.append(std::string(4096, '2'));
+ ASSERT_TRUE(bl.contents_equal(read_bl));
+}
ASSERT_PASSED(validate_object_map, clone_image);
}
+TEST_F(TestLibRBD, Sparsify)
+{
+ rados_ioctx_t ioctx;
+ ASSERT_EQ(0, rados_ioctx_create(_cluster, m_pool_name.c_str(), &ioctx));
+ BOOST_SCOPE_EXIT_ALL(&ioctx) {
+ rados_ioctx_destroy(ioctx);
+ };
+
+ const size_t CHUNK_SIZE = 4096 * 2;
+ rbd_image_t image;
+ int order = 0;
+ std::string name = get_temp_image_name();
+ uint64_t size = CHUNK_SIZE * 1024;
+
+ ASSERT_EQ(0, create_image(ioctx, name.c_str(), size, &order));
+ ASSERT_EQ(0, rbd_open(ioctx, name.c_str(), &image, NULL));
+ BOOST_SCOPE_EXIT_ALL(&image) {
+ rbd_close(image);
+ };
+
+ char test_data[4 * CHUNK_SIZE + 1];
+ for (size_t i = 0; i < 4 ; ++i) {
+ for (size_t j = 0; j < CHUNK_SIZE; j++) {
+ if (i % 2) {
+ test_data[i * CHUNK_SIZE + j] = (char)(rand() % (126 - 33) + 33);
+ } else {
+ test_data[i * CHUNK_SIZE + j] = '\0';
+ }
+ }
+ }
+ test_data[4 * CHUNK_SIZE] = '\0';
+
+ ASSERT_PASSED(write_test_data, image, test_data, 0, 4 * CHUNK_SIZE, 0);
+ ASSERT_EQ(0, rbd_flush(image));
+
+ ASSERT_EQ(-EINVAL, rbd_sparsify(image, 16));
+ ASSERT_EQ(-EINVAL, rbd_sparsify(image, 1 << (order + 1)));
+ ASSERT_EQ(-EINVAL, rbd_sparsify(image, 4096 + 1));
+ ASSERT_EQ(0, rbd_sparsify(image, 4096));
+
+ ASSERT_PASSED(read_test_data, image, test_data, 0, 4 * CHUNK_SIZE, 0);
+}
+
+TEST_F(TestLibRBD, SparsifyPP)
+{
+ librados::IoCtx ioctx;
+ ASSERT_EQ(0, _rados.ioctx_create(m_pool_name.c_str(), ioctx));
+
+ librbd::RBD rbd;
+ std::string name = get_temp_image_name();
+ uint64_t size = 12 * 1024 * 1024;
+ int order = 0;
+ ASSERT_EQ(0, create_image_pp(rbd, ioctx, name.c_str(), size, &order));
+
+ librbd::Image image;
+ ASSERT_EQ(0, rbd.open(ioctx, image, name.c_str(), nullptr));
+
+ bufferlist bl;
+ bl.append(std::string(4096, '\0'));
+ bl.append(std::string(4096, '1'));
+ bl.append(std::string(4096, '\0'));
+ ASSERT_EQ((ssize_t)bl.length(), image.write(0, bl.length(), bl));
+ ASSERT_EQ(0, image.flush());
+
+ ASSERT_EQ(-EINVAL, image.sparsify(16));
+ ASSERT_EQ(-EINVAL, image.sparsify(1 << (order + 1)));
+ ASSERT_EQ(-EINVAL, image.sparsify(4096 + 1));
+ ASSERT_EQ(0, image.sparsify(4096));
+
+ bufferlist read_bl;
+ ASSERT_EQ((ssize_t)bl.length(), image.read(0, bl.length(), read_bl));
+ ASSERT_TRUE(bl.contents_equal(read_bl));
+
+ ASSERT_PASSED(validate_object_map, image);
+}
+
TEST_F(TestLibRBD, SnapshotLimit)
{
rados_ioctx_t ioctx;
ASSERT_PASSED(validate_object_map, image1);
}
+TEST_F(TestLibRBD, SparsifyViaLockOwner)
+{
+ REQUIRE_FEATURE(RBD_FEATURE_EXCLUSIVE_LOCK);
+
+ librados::IoCtx ioctx;
+ ASSERT_EQ(0, _rados.ioctx_create(m_pool_name.c_str(), ioctx));
+
+ librbd::RBD rbd;
+ std::string name = get_temp_image_name();
+ uint64_t size = 2 << 20;
+ int order = 0;
+ ASSERT_EQ(0, create_image_pp(rbd, ioctx, name.c_str(), size, &order));
+
+ librbd::Image image1;
+ ASSERT_EQ(0, rbd.open(ioctx, image1, name.c_str(), NULL));
+
+ bufferlist bl;
+ ASSERT_EQ(0, image1.write(0, 0, bl));
+
+ bool lock_owner;
+ ASSERT_EQ(0, image1.is_exclusive_lock_owner(&lock_owner));
+ ASSERT_TRUE(lock_owner);
+
+ librbd::Image image2;
+ ASSERT_EQ(0, rbd.open(ioctx, image2, name.c_str(), NULL));
+
+ ASSERT_EQ(0, image2.is_exclusive_lock_owner(&lock_owner));
+ ASSERT_FALSE(lock_owner);
+
+ ASSERT_EQ(0, image2.sparsify(4096));
+
+ ASSERT_EQ(0, image1.is_exclusive_lock_owner(&lock_owner));
+ ASSERT_TRUE(lock_owner);
+ ASSERT_PASSED(validate_object_map, image1);
+}
+
TEST_F(TestLibRBD, ObjectMapConsistentSnap)
{
REQUIRE_FEATURE(RBD_FEATURE_OBJECT_MAP);
)
)
+TRACEPOINT_EVENT(librbd, sparsify_enter,
+ TP_ARGS(
+ void*, imagectx,
+ const char*, name,
+ size_t, sparse_size,
+ const char*, id),
+ TP_FIELDS(
+ ctf_integer_hex(void*, imagectx, imagectx)
+ ctf_string(name, name)
+ ctf_integer(size_t, sparse_size, sparse_size)
+ ctf_string(id, id)
+ )
+)
+
+TRACEPOINT_EVENT(librbd, sparsify_exit,
+ TP_ARGS(
+ int, retval),
+ TP_FIELDS(
+ ctf_integer(int, retval, retval)
+ )
+)
+
TRACEPOINT_EVENT(librbd, snap_create_enter,
TP_ARGS(
void*, imagectx,