librbd: online re-sparsify of images

Signed-off-by: Mykola Golub <mgolub@suse.com>
This commit is contained in:
Mykola Golub 2019-01-29 11:42:43 +00:00
parent 7b8a95ef78
commit a96feac2e4
15 changed files with 915 additions and 0 deletions

View File

@ -734,6 +734,13 @@ CEPH_RBD_API int rbd_flatten_with_progress(rbd_image_t image,
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().

View File

@ -469,6 +469,9 @@ public:
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.

View File

@ -128,6 +128,7 @@ set(librbd_internal_srcs
operation/SnapshotRollbackRequest.cc
operation/SnapshotUnprotectRequest.cc
operation/SnapshotLimitRequest.cc
operation/SparsifyRequest.cc
operation/TrimRequest.cc
trash/MoveRequest.cc
watcher/Notifier.cc

View File

@ -295,6 +295,21 @@ void ImageWatcher<I>::notify_migrate(uint64_t request_id,
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;
@ -954,6 +969,33 @@ bool ImageWatcher<I>::handle_payload(const MigratePayload &payload,
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) {

View File

@ -68,6 +68,9 @@ public:
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();
@ -238,6 +241,8 @@ private:
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,

View File

@ -36,6 +36,7 @@
#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>
@ -1637,6 +1638,56 @@ void Operations<I>::execute_migrate(ProgressContext &prog_ctx,
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() &&

View File

@ -103,6 +103,10 @@ public:
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:

View File

@ -292,6 +292,23 @@ void UpdateFeaturesPayload::dump(Formatter *f) const {
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();
}
@ -371,6 +388,9 @@ void NotifyMessage::decode(bufferlist::const_iterator& iter) {
case NOTIFY_OP_MIGRATE:
payload = MigratePayload();
break;
case NOTIFY_OP_SPARSIFY:
payload = SparsifyPayload();
break;
default:
payload = UnknownPayload();
break;
@ -406,6 +426,7 @@ void NotifyMessage::generate_test_instances(std::list<NotifyMessage *> &o) {
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 {
@ -484,6 +505,9 @@ std::ostream &operator<<(std::ostream &out,
case NOTIFY_OP_MIGRATE:
out << "Migrate";
break;
case NOTIFY_OP_SPARSIFY:
out << "Sparsify";
break;
default:
out << "Unknown (" << static_cast<uint32_t>(op) << ")";
break;

View File

@ -66,6 +66,7 @@ enum NotifyOp {
NOTIFY_OP_RENAME = 14,
NOTIFY_OP_UPDATE_FEATURES = 15,
NOTIFY_OP_MIGRATE = 16,
NOTIFY_OP_SPARSIFY = 17,
};
struct AcquiredLockPayload {
@ -310,6 +311,21 @@ struct MigratePayload : public AsyncRequestPayloadBase {
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;
@ -336,6 +352,7 @@ typedef boost::variant<AcquiredLockPayload,
RenamePayload,
UpdateFeaturesPayload,
MigratePayload,
SparsifyPayload,
UnknownPayload> Payload;
struct NotifyMessage {

View File

@ -1732,6 +1732,28 @@ namespace librbd {
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;
@ -3539,6 +3561,29 @@ extern "C" int rbd_flatten_with_progress(rbd_image_t image,
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)
{

View File

@ -0,0 +1,344 @@
// -*- 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>;

View File

@ -0,0 +1,64 @@
// -*- 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

View File

@ -2,6 +2,7 @@
// 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"
@ -130,6 +131,38 @@ void generate_random_iomap(librbd::Image &image, int num_objects, int object_siz
}
}
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();
@ -1341,3 +1374,144 @@ TEST_F(TestInternal, PoolMetadataConfApply) {
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));
}

View File

@ -4523,6 +4523,82 @@ TEST_F(TestLibRBD, Flatten)
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;
@ -5054,6 +5130,42 @@ TEST_F(TestLibRBD, ResizeViaLockOwner)
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);

View File

@ -1501,6 +1501,28 @@ TRACEPOINT_EVENT(librbd, flatten_exit,
)
)
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,