notify_lock_owner(std::move(bl), on_finish);
}
+template <typename I>
+void ImageWatcher<I>::notify_update_features(uint64_t features, bool enabled,
+ Context *on_finish) {
+ assert(m_image_ctx.owner_lock.is_locked());
+ assert(m_image_ctx.exclusive_lock &&
+ !m_image_ctx.exclusive_lock->is_lock_owner());
+
+ bufferlist bl;
+ ::encode(NotifyMessage(UpdateFeaturesPayload(features, enabled)), bl);
+ notify_lock_owner(std::move(bl), 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 UpdateFeaturesPayload& payload,
+ C_NotifyAck *ack_ctx) {
+ RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
+ if (m_image_ctx.exclusive_lock != nullptr) {
+ int r;
+ if (m_image_ctx.exclusive_lock->accept_requests(&r)) {
+ ldout(m_image_ctx.cct, 10) << this << " remote update_features request: "
+ << payload.features << " "
+ << (payload.enabled ? "enabled" : "disabled")
+ << dendl;
+
+ m_image_ctx.operations->execute_update_features(
+ payload.features, payload.enabled, new C_ResponseMessage(ack_ctx), 0);
+ return false;
+ } 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) {
ProgressContext &prog_ctx, Context *on_finish);
void notify_rename(const std::string &image_name, Context *on_finish);
+ void notify_update_features(uint64_t features, bool enabled,
+ Context *on_finish);
+
void notify_acquired_lock();
void notify_released_lock();
void notify_request_lock();
C_NotifyAck *ctx);
bool handle_payload(const watch_notify::RenamePayload& payload,
C_NotifyAck *ctx);
+ bool handle_payload(const watch_notify::UpdateFeaturesPayload& 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 "include/rbd_types.h"
#include "include/rados/librados.hpp"
#include "common/errno.h"
+#include "librbd/Utils.h"
#define dout_subsys ceph_subsys_rbd
#undef dout_prefix
int MirroringWatcher<I>::notify_image_updated(
librados::IoCtx &io_ctx, cls::rbd::MirrorImageState mirror_image_state,
const std::string &image_id, const std::string &global_image_id) {
+ C_SaferCond cond;
+ notify_image_updated(io_ctx, mirror_image_state, image_id, global_image_id,
+ &cond);
+ return cond.wait();
+}
+
+template <typename I>
+int MirroringWatcher<I>::notify_image_updated(
+ librados::IoCtx &io_ctx, cls::rbd::MirrorImageState mirror_image_state,
+ const std::string &image_id, const std::string &global_image_id,
+ Context *on_finish) {
CephContext *cct = reinterpret_cast<CephContext*>(io_ctx.cct());
ldout(cct, 20) << dendl;
::encode(NotifyMessage{ImageUpdatedPayload{mirror_image_state, image_id,
global_image_id}},
bl);
-
- int r = io_ctx.notify2(RBD_MIRRORING, bl, NOTIFY_TIMEOUT_MS, nullptr);
+ librados::AioCompletion *comp = util::create_rados_ack_callback(on_finish);
+ int r = io_ctx.aio_notify(RBD_MIRRORING, comp, bl, NOTIFY_TIMEOUT_MS,
+ nullptr);
if (r < 0) {
lderr(cct) << ": error encountered sending image updated notification: "
<< cpp_strerror(r) << dendl;
cls::rbd::MirrorImageState mirror_image_state,
const std::string &image_id,
const std::string &global_image_id);
+ static int notify_image_updated(librados::IoCtx &io_ctx,
+ cls::rbd::MirrorImageState mirror_image_state,
+ const std::string &image_id,
+ const std::string &global_image_id,
+ Context *on_finish);
virtual void handle_mode_updated(cls::rbd::MirrorMode mirror_mode,
Context *on_ack) = 0;
#include "librbd/ImageWatcher.h"
#include "librbd/ObjectMap.h"
#include "librbd/Utils.h"
+#include "librbd/operation/DisableFeaturesRequest.h"
+#include "librbd/operation/EnableFeaturesRequest.h"
#include "librbd/operation/FlattenRequest.h"
#include "librbd/operation/RebuildObjectMapRequest.h"
#include "librbd/operation/ObjectMapIterate.h"
request->send();
}
+template <typename I>
+int Operations<I>::update_features(uint64_t features, bool enabled) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 5) << this << " " << __func__ << ": features=" << features
+ << ", enabled=" << enabled << dendl;
+
+ int r = m_image_ctx.state->refresh_if_required();
+ if (r < 0) {
+ return r;
+ }
+
+ if (m_image_ctx.read_only) {
+ return -EROFS;
+ } else if (m_image_ctx.old_format) {
+ lderr(cct) << "old-format images do not support features" << dendl;
+ return -EINVAL;
+ }
+
+ uint64_t disable_mask = (RBD_FEATURES_MUTABLE |
+ RBD_FEATURES_DISABLE_ONLY);
+ if ((enabled && (features & RBD_FEATURES_MUTABLE) != features) ||
+ (!enabled && (features & disable_mask) != features)) {
+ lderr(cct) << "cannot update immutable features" << dendl;
+ return -EINVAL;
+ }
+ if (features == 0) {
+ lderr(cct) << "update requires at least one feature" << dendl;
+ return -EINVAL;
+ }
+ {
+ RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
+ if (enabled && (features & m_image_ctx.features) != 0) {
+ lderr(cct) << "one or more requested features are already enabled"
+ << dendl;
+ return -EINVAL;
+ }
+ if (!enabled && (features & ~m_image_ctx.features) != 0) {
+ lderr(cct) << "one or more requested features are already disabled"
+ << dendl;
+ return -EINVAL;
+ }
+ }
+
+ r = invoke_async_request("update_features", false,
+ boost::bind(&Operations<I>::execute_update_features,
+ this, features, enabled, _1, 0),
+ boost::bind(&ImageWatcher<I>::notify_update_features,
+ m_image_ctx.image_watcher, features,
+ enabled, _1));
+ ldout(cct, 2) << "update_features finished" << dendl;
+ return r;
+}
+
+template <typename I>
+void Operations<I>::execute_update_features(uint64_t features, bool enabled,
+ Context *on_finish,
+ uint64_t journal_op_tid) {
+ assert(m_image_ctx.owner_lock.is_locked());
+ assert(m_image_ctx.exclusive_lock == nullptr ||
+ m_image_ctx.exclusive_lock->is_lock_owner());
+
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 5) << this << " " << __func__ << ": features=" << features
+ << ", enabled=" << enabled << dendl;
+
+ if (enabled) {
+ operation::EnableFeaturesRequest<I> *req =
+ new operation::EnableFeaturesRequest<I>(
+ m_image_ctx, on_finish, journal_op_tid, features);
+ req->send();
+ } else {
+ operation::DisableFeaturesRequest<I> *req =
+ new operation::DisableFeaturesRequest<I>(
+ m_image_ctx, on_finish, journal_op_tid, features);
+ req->send();
+ }
+}
+
template <typename I>
int Operations<I>::prepare_image_update() {
assert(m_image_ctx.owner_lock.is_locked() &&
int snap_set_limit(uint64_t limit);
void execute_snap_set_limit(uint64_t limit, Context *on_finish);
+ int update_features(uint64_t features, bool enabled);
+ void execute_update_features(uint64_t features, bool enabled,
+ Context *on_finish, uint64_t journal_op_tid);
+
int prepare_image_update();
private:
f->dump_string("image_name", image_name);
}
+void UpdateFeaturesPayload::encode(bufferlist &bl) const {
+ ::encode(features, bl);
+ ::encode(enabled, bl);
+}
+
+void UpdateFeaturesPayload::decode(__u8 version, bufferlist::iterator &iter) {
+ ::decode(features, iter);
+ ::decode(enabled, iter);
+}
+
+void UpdateFeaturesPayload::dump(Formatter *f) const {
+ f->dump_unsigned("features", features);
+ f->dump_bool("enabled", enabled);
+}
+
void UnknownPayload::encode(bufferlist &bl) const {
assert(false);
}
case NOTIFY_OP_RENAME:
payload = RenamePayload();
break;
+ case NOTIFY_OP_UPDATE_FEATURES:
+ payload = UpdateFeaturesPayload();
+ break;
default:
payload = UnknownPayload();
break;
o.push_back(new NotifyMessage(SnapUnprotectPayload("foo")));
o.push_back(new NotifyMessage(RebuildObjectMapPayload(AsyncRequestId(ClientId(0, 1), 2))));
o.push_back(new NotifyMessage(RenamePayload("foo")));
+ o.push_back(new NotifyMessage(UpdateFeaturesPayload(1, true)));
}
void ResponseMessage::encode(bufferlist& bl) const {
case NOTIFY_OP_RENAME:
out << "Rename";
break;
+ case NOTIFY_OP_UPDATE_FEATURES:
+ out << "UpdateFeatures";
+ break;
default:
out << "Unknown (" << static_cast<uint32_t>(op) << ")";
break;
NOTIFY_OP_SNAP_RENAME = 11,
NOTIFY_OP_SNAP_PROTECT = 12,
NOTIFY_OP_SNAP_UNPROTECT = 13,
- NOTIFY_OP_RENAME = 14
+ NOTIFY_OP_RENAME = 14,
+ NOTIFY_OP_UPDATE_FEATURES = 15,
};
struct AcquiredLockPayload {
void dump(Formatter *f) const;
};
+struct UpdateFeaturesPayload {
+ static const NotifyOp NOTIFY_OP = NOTIFY_OP_UPDATE_FEATURES;
+ static const bool CHECK_FOR_REFRESH = true;
+
+ UpdateFeaturesPayload() : features(0), enabled(false) {}
+ UpdateFeaturesPayload(uint64_t features_, bool enabled_)
+ : features(features_), enabled(enabled_) {}
+
+ uint64_t features;
+ bool enabled;
+
+ void encode(bufferlist &bl) const;
+ void decode(__u8 version, bufferlist::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;
SnapUnprotectPayload,
RebuildObjectMapPayload,
RenamePayload,
+ UpdateFeaturesPayload,
UnknownPayload> Payload;
struct NotifyMessage {
#include "librbd/ImageState.h"
#include "librbd/internal.h"
#include "librbd/Journal.h"
-#include "librbd/journal/DisabledPolicy.h"
-#include "librbd/journal/StandardPolicy.h"
#include "librbd/journal/Types.h"
+#include "librbd/mirror/DisableRequest.h"
+#include "librbd/mirror/EnableRequest.h"
#include "librbd/MirroringWatcher.h"
#include "librbd/ObjectMap.h"
#include "librbd/Operations.h"
namespace {
-int remove_object_map(ImageCtx *ictx) {
- assert(ictx->snap_lock.is_locked());
- CephContext *cct = ictx->cct;
-
- int r;
- for (std::map<snap_t, SnapInfo>::iterator it = ictx->snap_info.begin();
- it != ictx->snap_info.end(); ++it) {
- std::string oid(ObjectMap::object_map_name(ictx->id, it->first));
- r = ictx->md_ctx.remove(oid);
- if (r < 0 && r != -ENOENT) {
- lderr(cct) << "failed to remove object map " << oid << ": "
- << cpp_strerror(r) << dendl;
- return r;
- }
- }
-
- r = ictx->md_ctx.remove(ObjectMap::object_map_name(ictx->id, CEPH_NOSNAP));
- if (r < 0 && r != -ENOENT) {
- lderr(cct) << "failed to remove object map: " << cpp_strerror(r) << dendl;
- return r;
- }
- return 0;
-}
-
-int create_object_map(ImageCtx *ictx) {
- assert(ictx->snap_lock.is_locked());
- CephContext *cct = ictx->cct;
-
- int r;
- uint64_t max_size = ictx->size;
- std::vector<uint64_t> snap_ids;
- snap_ids.push_back(CEPH_NOSNAP);
- for (std::map<snap_t, SnapInfo>::iterator it = ictx->snap_info.begin();
- it != ictx->snap_info.end(); ++it) {
- max_size = MAX(max_size, it->second.size);
- snap_ids.push_back(it->first);
- }
-
- if (!ObjectMap::is_compatible(ictx->layout, max_size)) {
- lderr(cct) << "image size not compatible with object map" << dendl;
- return -EINVAL;
- }
-
- for (std::vector<uint64_t>::iterator it = snap_ids.begin();
- it != snap_ids.end(); ++it) {
- librados::ObjectWriteOperation op;
- std::string oid(ObjectMap::object_map_name(ictx->id, *it));
- uint64_t snap_size = ictx->get_image_size(*it);
- cls_client::object_map_resize(&op, Striper::get_num_objects(ictx->layout, snap_size),
- OBJECT_NONEXISTENT);
- r = ictx->md_ctx.operate(oid, &op);
- if (r < 0) {
- lderr(cct) << "failed to create object map " << oid << ": "
- << cpp_strerror(r) << dendl;
- return r;
- }
- }
-
- return 0;
-}
-
-int update_all_flags(ImageCtx *ictx, uint64_t flags, uint64_t mask) {
- assert(ictx->snap_lock.is_locked());
- CephContext *cct = ictx->cct;
-
- std::vector<uint64_t> snap_ids;
- snap_ids.push_back(CEPH_NOSNAP);
- for (std::map<snap_t, SnapInfo>::iterator it = ictx->snap_info.begin();
- it != ictx->snap_info.end(); ++it) {
- snap_ids.push_back(it->first);
- }
-
- for (size_t i=0; i<snap_ids.size(); ++i) {
- librados::ObjectWriteOperation op;
- cls_client::set_flags(&op, snap_ids[i], flags, mask);
- int r = ictx->md_ctx.operate(ictx->header_oid, &op);
- if (r < 0) {
- lderr(cct) << "failed to update image flags: " << cpp_strerror(r)
- << dendl;
- return r;
- }
- }
- return 0;
-}
-
int validate_pool(IoCtx &io_ctx, CephContext *cct) {
if (!cct->_conf->rbd_validate_pool) {
return 0;
return 0;
}
-int mirror_image_enable(CephContext *cct, librados::IoCtx &io_ctx,
- const std::string &id,
- const std::string &global_image_id) {
- cls::rbd::MirrorImage mirror_image_internal;
- int r = cls_client::mirror_image_get(&io_ctx, id, &mirror_image_internal);
- if (r < 0 && r != -ENOENT) {
- lderr(cct) << "cannot enable mirroring: " << cpp_strerror(r) << dendl;
- return r;
- }
-
- if (mirror_image_internal.state == cls::rbd::MIRROR_IMAGE_STATE_ENABLED) {
- // mirroring is already enabled
- return 0;
- } else if (r != -ENOENT) {
- lderr(cct) << "cannot enable mirroring: currently disabling" << dendl;
- return -EINVAL;
- }
-
- mirror_image_internal.state = cls::rbd::MIRROR_IMAGE_STATE_ENABLED;
- if (global_image_id.empty()) {
- uuid_d uuid_gen;
- uuid_gen.generate_random();
- mirror_image_internal.global_image_id = uuid_gen.to_string();
- } else {
- mirror_image_internal.global_image_id = global_image_id;
- }
-
- r = cls_client::mirror_image_set(&io_ctx, id, mirror_image_internal);
- if (r < 0) {
- lderr(cct) << "cannot enable mirroring: " << cpp_strerror(r) << dendl;
- return r;
- }
-
- r = MirroringWatcher<>::notify_image_updated(
- io_ctx, cls::rbd::MIRROR_IMAGE_STATE_ENABLED, id,
- mirror_image_internal.global_image_id);
- if (r < 0) {
- lderr(cct) << "failed to send update notification: " << cpp_strerror(r)
- << dendl;
- }
-
- ldout(cct, 20) << "image mirroring is enabled: global_id=" <<
- mirror_image_internal.global_image_id << dendl;
-
- return 0;
-}
-
int mirror_image_enable_internal(ImageCtx *ictx) {
CephContext *cct = ictx->cct;
+ C_SaferCond cond;
if ((ictx->features & RBD_FEATURE_JOURNALING) == 0) {
- lderr(cct) << "cannot enable mirroring: journaling is not enabled"
- << dendl;
+ lderr(cct) << "cannot enable mirroring: journaling is not enabled" << dendl;
return -EINVAL;
}
- bool is_primary;
- int r = Journal<>::is_tag_owner(ictx, &is_primary);
- if (r < 0) {
- lderr(cct) << "cannot enable mirroring: failed to check tag ownership: "
- << cpp_strerror(r) << dendl;
- return r;
- }
+ mirror::EnableRequest<ImageCtx> *req =
+ mirror::EnableRequest<ImageCtx>::create(ictx, &cond);
+ req->send();
- if (!is_primary) {
- lderr(cct) <<
- "cannot enable mirroring: last journal tag not owned by local cluster"
- << dendl;
- return -EINVAL;
- }
-
- r = mirror_image_enable(cct, ictx->md_ctx, ictx->id, "");
+ int r = cond.wait();
if (r < 0) {
+ lderr(cct) << "cannot enable mirroring: " << cpp_strerror(r) << dendl;
return r;
}
int mirror_image_disable_internal(ImageCtx *ictx, bool force,
bool remove=true) {
CephContext *cct = ictx->cct;
+ C_SaferCond cond;
- cls::rbd::MirrorImage mirror_image_internal;
- std::vector<snap_info_t> snaps;
- std::set<cls::journal::Client> clients;
- std::string header_oid;
-
- int r = cls_client::mirror_image_get(&ictx->md_ctx, ictx->id,
- &mirror_image_internal);
- if (r == -ENOENT) {
- // mirroring is not enabled for this image
- ldout(cct, 20) << "ignoring disable command: mirroring is not enabled "
- "for this image" << dendl;
- return 0;
- } else if (r == -EOPNOTSUPP) {
- ldout(cct, 5) << "mirroring not supported by OSD" << dendl;
- return r;
- } else if (r < 0) {
- lderr(cct) << "cannot disable mirroring: " << cpp_strerror(r) << dendl;
- return r;
- }
+ mirror::DisableRequest<ImageCtx> *req =
+ mirror::DisableRequest<ImageCtx>::create(ictx, force, remove, &cond);
+ req->send();
- bool is_primary;
- r = Journal<>::is_tag_owner(ictx, &is_primary);
- if (r < 0) {
- lderr(cct) << "cannot disable mirroring: failed to check tag ownership: "
- << cpp_strerror(r) << dendl;
- return r;
- }
-
- if (!is_primary && !force) {
- lderr(cct) << "Mirrored image is not the primary, add force option to"
- " disable mirroring" << dendl;
- return -EINVAL;
- }
-
- mirror_image_internal.state = cls::rbd::MIRROR_IMAGE_STATE_DISABLING;
- r = cls_client::mirror_image_set(&ictx->md_ctx, ictx->id,
- mirror_image_internal);
+ int r = cond.wait();
if (r < 0) {
lderr(cct) << "cannot disable mirroring: " << cpp_strerror(r) << dendl;
return r;
}
-
- r = MirroringWatcher<>::notify_image_updated(
- ictx->md_ctx, cls::rbd::MIRROR_IMAGE_STATE_DISABLING,
- ictx->id, mirror_image_internal.global_image_id);
- if (r < 0) {
- lderr(cct) << "failed to send update notification: " << cpp_strerror(r)
- << dendl;
- }
-
- header_oid = ::journal::Journaler::header_oid(ictx->id);
-
- while(true) {
- clients.clear();
- r = cls::journal::client::client_list(ictx->md_ctx, header_oid, &clients);
- if (r < 0) {
- lderr(cct) << "cannot disable mirroring: " << cpp_strerror(r) << dendl;
- return r;
- }
-
- assert(clients.size() >= 1);
-
- if (clients.size() == 1) {
- // only local journal client remains
- break;
- }
-
- for (auto client : clients) {
- journal::ClientData client_data;
- bufferlist::iterator bl = client.data.begin();
- ::decode(client_data, bl);
- journal::ClientMetaType type = client_data.get_client_meta_type();
-
- if (type != journal::ClientMetaType::MIRROR_PEER_CLIENT_META_TYPE) {
- continue;
- }
-
- journal::MirrorPeerClientMeta client_meta =
- boost::get<journal::MirrorPeerClientMeta>(client_data.client_meta);
-
- for (const auto& sync : client_meta.sync_points) {
- r = ictx->operations->snap_remove(sync.snap_name.c_str());
- if (r < 0 && r != -ENOENT) {
- lderr(cct) << "cannot disable mirroring: failed to remove temporary"
- " snapshot created by remote peer: " << cpp_strerror(r) << dendl;
- return r;
- }
- }
-
- r = cls::journal::client::client_unregister(ictx->md_ctx, header_oid,
- client.id);
- if (r < 0 && r != -ENOENT) {
- lderr(cct) << "cannot disable mirroring: failed to unregister remote"
- " journal client: " << cpp_strerror(r) << dendl;
- return r;
- }
- }
- }
-
- if (remove) {
- r = cls_client::mirror_image_remove(&ictx->md_ctx, ictx->id);
- if (r < 0 && r != -ENOENT) {
- lderr(cct) << "failed to remove image from mirroring directory: "
- << cpp_strerror(r) << dendl;
- return r;
- }
-
- ldout(cct, 20) << "removed image state from rbd_mirroring object" << dendl;
-
- if (is_primary) {
- // TODO: send notification to mirroring object about update
- }
- }
-
return 0;
}
return 0;
}
- int update_features(ImageCtx *ictx, uint64_t features, bool enabled)
- {
- int r = ictx->state->refresh_if_required();
- if (r < 0) {
- return r;
- }
-
- CephContext *cct = ictx->cct;
- if (ictx->read_only) {
- return -EROFS;
- } else if (ictx->old_format) {
- lderr(cct) << "old-format images do not support features" << dendl;
- return -EINVAL;
- }
-
- uint64_t disable_mask = (RBD_FEATURES_MUTABLE |
- RBD_FEATURES_DISABLE_ONLY);
- if ((enabled && (features & RBD_FEATURES_MUTABLE) != features) ||
- (!enabled && (features & disable_mask) != features)) {
- lderr(cct) << "cannot update immutable features" << dendl;
- return -EINVAL;
- } else if (features == 0) {
- lderr(cct) << "update requires at least one feature" << dendl;
- return -EINVAL;
- }
-
- rbd_mirror_mode_t mirror_mode = RBD_MIRROR_MODE_DISABLED;
- if ((features & RBD_FEATURE_JOURNALING) != 0) {
- r = librbd::mirror_mode_get(ictx->md_ctx, &mirror_mode);
- if (r < 0) {
- lderr(cct) << "error in retrieving pool mirroring status: "
- << cpp_strerror(r) << dendl;
- return r;
- }
-
- // mark mirroring as disabling and prune all sync snapshots
- // before acquiring locks
- if (mirror_mode == RBD_MIRROR_MODE_POOL && !enabled) {
- r = mirror_image_disable_internal(ictx, false, false);
- if (r < 0) {
- lderr(cct) << "error disabling image mirroring: "
- << cpp_strerror(r) << dendl;
- }
- }
- }
-
- RWLock::RLocker owner_locker(ictx->owner_lock);
- r = ictx->aio_work_queue->block_writes();
- BOOST_SCOPE_EXIT_ALL( (ictx) ) {
- ictx->aio_work_queue->unblock_writes();
- };
- if (r < 0) {
- return r;
- }
-
- // avoid accepting new requests from peers while we manipulate
- // the image features
- if (ictx->exclusive_lock != nullptr) {
- ictx->exclusive_lock->block_requests(0);
- }
- BOOST_SCOPE_EXIT_ALL( (ictx) ) {
- if (ictx->exclusive_lock != nullptr) {
- ictx->exclusive_lock->unblock_requests();
- }
- };
-
- // if disabling journaling, avoid attempting to open the journal
- // when acquiring the exclusive lock in case the journal is corrupt
- bool disabling_journal = false;
- if (!enabled && ((features & RBD_FEATURE_JOURNALING) != 0)) {
- RWLock::WLocker snap_locker(ictx->snap_lock);
- ictx->set_journal_policy(new journal::DisabledPolicy());
- disabling_journal = true;
- }
- BOOST_SCOPE_EXIT_ALL( (ictx)(disabling_journal) ) {
- if (disabling_journal) {
- RWLock::WLocker snap_locker(ictx->snap_lock);
- ictx->set_journal_policy(new journal::StandardPolicy(ictx));
- }
- };
-
- // if disabling features w/ exclusive lock supported, we need to
- // acquire the lock to temporarily block IO against the image
- bool acquired_lock = false;
- if (ictx->exclusive_lock != nullptr &&
- !ictx->exclusive_lock->is_lock_owner() && !enabled) {
- acquired_lock = true;
-
- C_SaferCond lock_ctx;
- ictx->exclusive_lock->request_lock(&lock_ctx);
-
- // don't block holding lock since refresh might be required
- ictx->owner_lock.put_read();
- r = lock_ctx.wait();
- ictx->owner_lock.get_read();
-
- if (r < 0) {
- lderr(cct) << "failed to lock image: " << cpp_strerror(r) << dendl;
- return r;
- } else if (ictx->exclusive_lock == nullptr ||
- !ictx->exclusive_lock->is_lock_owner()) {
- lderr(cct) << "failed to acquire exclusive lock" << dendl;
- return -EROFS;
- }
- }
-
- {
- RWLock::WLocker snap_locker(ictx->snap_lock);
- uint64_t new_features;
- if (enabled) {
- if ((features & ictx->features) != 0) {
- lderr(cct) << "one or more requested features are already enabled" << dendl;
- return -EINVAL;
- }
- features &= ~ictx->features;
- new_features = ictx->features | features;
- } else {
- if ((features & ~ictx->features) != 0) {
- lderr(cct) << "one or more requested features are already disabled" << dendl;
- return -EINVAL;
- }
- features &= ictx->features;
- new_features = ictx->features & ~features;
- }
-
- bool enable_mirroring = false;
- uint64_t features_mask = features;
- uint64_t disable_flags = 0;
- if (enabled) {
- uint64_t enable_flags = 0;
-
- if ((features & RBD_FEATURE_OBJECT_MAP) != 0) {
- if ((new_features & RBD_FEATURE_EXCLUSIVE_LOCK) == 0) {
- lderr(cct) << "cannot enable object map" << dendl;
- return -EINVAL;
- }
- enable_flags |= RBD_FLAG_OBJECT_MAP_INVALID;
- features_mask |= RBD_FEATURE_EXCLUSIVE_LOCK;
- }
- if ((features & RBD_FEATURE_FAST_DIFF) != 0) {
- if ((new_features & RBD_FEATURE_OBJECT_MAP) == 0) {
- lderr(cct) << "cannot enable fast diff" << dendl;
- return -EINVAL;
- }
- enable_flags |= RBD_FLAG_FAST_DIFF_INVALID;
- features_mask |= (RBD_FEATURE_OBJECT_MAP | RBD_FEATURE_EXCLUSIVE_LOCK);
- }
- if ((features & RBD_FEATURE_JOURNALING) != 0) {
- if ((new_features & RBD_FEATURE_EXCLUSIVE_LOCK) == 0) {
- lderr(cct) << "cannot enable journaling" << dendl;
- return -EINVAL;
- }
- features_mask |= RBD_FEATURE_EXCLUSIVE_LOCK;
-
- r = Journal<>::create(ictx->md_ctx, ictx->id, ictx->journal_order,
- ictx->journal_splay_width, ictx->journal_pool);
- if (r < 0) {
- lderr(cct) << "error creating image journal: " << cpp_strerror(r)
- << dendl;
- return r;
- }
-
- enable_mirroring = (mirror_mode == RBD_MIRROR_MODE_POOL);
- }
-
- if (enable_flags != 0) {
- r = update_all_flags(ictx, enable_flags, enable_flags);
- if (r < 0) {
- return r;
- }
- }
- } else {
- if ((features & RBD_FEATURE_EXCLUSIVE_LOCK) != 0) {
- if ((new_features & RBD_FEATURE_OBJECT_MAP) != 0 ||
- (new_features & RBD_FEATURE_JOURNALING) != 0) {
- lderr(cct) << "cannot disable exclusive lock" << dendl;
- return -EINVAL;
- }
- features_mask |= (RBD_FEATURE_OBJECT_MAP |
- RBD_FEATURE_JOURNALING);
- }
- if ((features & RBD_FEATURE_OBJECT_MAP) != 0) {
- if ((new_features & RBD_FEATURE_FAST_DIFF) != 0) {
- lderr(cct) << "cannot disable object map" << dendl;
- return -EINVAL;
- }
-
- disable_flags |= RBD_FLAG_OBJECT_MAP_INVALID;
- r = remove_object_map(ictx);
- if (r < 0) {
- lderr(cct) << "failed to remove object map" << dendl;
- return r;
- }
- }
- if ((features & RBD_FEATURE_FAST_DIFF) != 0) {
- disable_flags |= RBD_FLAG_FAST_DIFF_INVALID;
- }
- if ((features & RBD_FEATURE_JOURNALING) != 0) {
- if (mirror_mode == RBD_MIRROR_MODE_IMAGE) {
- cls::rbd::MirrorImage mirror_image;
- r = cls_client::mirror_image_get(&ictx->md_ctx, ictx->id,
- &mirror_image);
- if (r < 0 && r != -ENOENT) {
- lderr(cct) << "error retrieving mirroring state: "
- << cpp_strerror(r) << dendl;
- return r;
- }
-
- if (mirror_image.state == cls::rbd::MIRROR_IMAGE_STATE_ENABLED) {
- lderr(cct) << "cannot disable journaling: image mirroring "
- << "enabled and mirror pool mode set to image"
- << dendl;
- return -EINVAL;
- }
- } else if (mirror_mode == RBD_MIRROR_MODE_POOL) {
- r = cls_client::mirror_image_remove(&ictx->md_ctx, ictx->id);
- if (r < 0 && r != -ENOENT) {
- lderr(cct) << "failed to remove image from mirroring directory: "
- << cpp_strerror(r) << dendl;
- return r;
- }
- }
-
- if (ictx->journal != nullptr) {
- C_SaferCond cond;
- ictx->journal->close(&cond);
- r = cond.wait();
- if (r < 0) {
- lderr(cct) << "error closing image journal: " << cpp_strerror(r)
- << dendl;
- return r;
- }
- }
-
- r = Journal<>::remove(ictx->md_ctx, ictx->id);
- if (r < 0) {
- lderr(cct) << "error removing image journal: " << cpp_strerror(r)
- << dendl;
- return r;
- }
- }
- }
-
- ldout(cct, 10) << "update_features: features=" << new_features << ", "
- << "mask=" << features_mask << dendl;
- r = librbd::cls_client::set_features(&ictx->md_ctx, ictx->header_oid,
- new_features, features_mask);
- if (!enabled && r == -EINVAL) {
- // NOTE: infernalis OSDs will not accept a mask with new features, so
- // re-attempt with a reduced mask.
- features_mask &= ~RBD_FEATURE_JOURNALING;
- r = librbd::cls_client::set_features(&ictx->md_ctx, ictx->header_oid,
- new_features, features_mask);
- }
- if (r < 0) {
- lderr(cct) << "failed to update features: " << cpp_strerror(r)
- << dendl;
- return r;
- }
- if (((ictx->features & RBD_FEATURE_OBJECT_MAP) == 0) &&
- ((features & RBD_FEATURE_OBJECT_MAP) != 0)) {
- r = create_object_map(ictx);
- if (r < 0) {
- lderr(cct) << "failed to create object map" << dendl;
- return r;
- }
- }
-
- if (disable_flags != 0) {
- r = update_all_flags(ictx, 0, disable_flags);
- if (r < 0) {
- return r;
- }
- }
-
- if (enable_mirroring) {
- ImageCtx *img_ctx = new ImageCtx("", ictx->id, nullptr,
- ictx->md_ctx, false);
- r = img_ctx->state->open();
- if (r < 0) {
- lderr(cct) << "error opening image: " << cpp_strerror(r) << dendl;
- delete img_ctx;
- } else {
- r = mirror_image_enable_internal(img_ctx);
- if (r < 0) {
- lderr(cct) << "error enabling mirroring: " << cpp_strerror(r)
- << dendl;
- }
- img_ctx->state->close();
- }
- }
- }
-
- ictx->notify_update();
-
- if (ictx->exclusive_lock != nullptr && acquired_lock) {
- C_SaferCond lock_ctx;
- ictx->exclusive_lock->release_lock(&lock_ctx);
- r = lock_ctx.wait();
- if (r < 0) {
- lderr(cct) << "failed to unlock image: " << cpp_strerror(r) << dendl;
- return r;
- }
- }
- return 0;
- }
-
int get_overlap(ImageCtx *ictx, uint64_t *overlap)
{
int r = ictx->state->refresh_if_required();
int get_old_format(ImageCtx *ictx, uint8_t *old);
int get_size(ImageCtx *ictx, uint64_t *size);
int get_features(ImageCtx *ictx, uint64_t *features);
- int update_features(ImageCtx *ictx, uint64_t features, bool enabled);
int get_overlap(ImageCtx *ictx, uint64_t *overlap);
int get_parent_info(ImageCtx *ictx, std::string *parent_pool_name,
std::string *parent_name, std::string *parent_snap_name);
FlattenEvent,
DemoteEvent,
SnapLimitEvent,
- UpdateFeaturesEvent,
+ UpdateFeaturesEvent,
UnknownEvent> Event;
struct EventEntry {
{
ImageCtx *ictx = reinterpret_cast<ImageCtx *>(ctx);
tracepoint(librbd, update_features_enter, ictx, features, enabled);
- int r = librbd::update_features(ictx, features, enabled);
+ int r = ictx->operations->update_features(features, enabled);
tracepoint(librbd, update_features_exit, r);
return r;
}
librbd::ImageCtx *ictx = reinterpret_cast<librbd::ImageCtx *>(image);
bool features_enabled = enabled != 0;
tracepoint(librbd, update_features_enter, ictx, features, features_enabled);
- int r = librbd::update_features(ictx, features, features_enabled);
+ int r = ictx->operations->update_features(features, features_enabled);
tracepoint(librbd, update_features_exit, r);
return r;
}
mock_image_ctx.journal = &mock_journal;
}
- if (ictx->test_features(RBD_FEATURE_JOURNALING)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_JOURNALING, false));
+ if (ictx->test_features(RBD_FEATURE_JOURNALING)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_JOURNALING,
+ false));
}
- if (ictx->test_features(RBD_FEATURE_FAST_DIFF)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_FAST_DIFF , false));
+ if (ictx->test_features(RBD_FEATURE_FAST_DIFF)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_FAST_DIFF,
+ false));
}
- if (ictx->test_features(RBD_FEATURE_OBJECT_MAP)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_OBJECT_MAP , false));
+ if (ictx->test_features(RBD_FEATURE_OBJECT_MAP)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_OBJECT_MAP,
+ false));
}
-
- if (ictx->test_features(RBD_FEATURE_EXCLUSIVE_LOCK)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_EXCLUSIVE_LOCK , false));
+
+ if (ictx->test_features(RBD_FEATURE_EXCLUSIVE_LOCK)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_EXCLUSIVE_LOCK,
+ false));
}
-
+
expect_op_work_queue(mock_image_ctx);
expect_test_features(mock_image_ctx);
MockExclusiveLock mock_exclusive_lock;
mock_image_ctx.exclusive_lock = &mock_exclusive_lock;
- if (ictx->test_features(RBD_FEATURE_JOURNALING)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_JOURNALING, false));
+ if (ictx->test_features(RBD_FEATURE_JOURNALING)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_JOURNALING,
+ false));
}
- if (ictx->test_features(RBD_FEATURE_FAST_DIFF)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_FAST_DIFF , false));
+ if (ictx->test_features(RBD_FEATURE_FAST_DIFF)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_FAST_DIFF,
+ false));
}
- if (ictx->test_features(RBD_FEATURE_OBJECT_MAP)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_OBJECT_MAP , false));
+ if (ictx->test_features(RBD_FEATURE_OBJECT_MAP)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_OBJECT_MAP,
+ false));
}
-
- if (ictx->test_features(RBD_FEATURE_EXCLUSIVE_LOCK)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_EXCLUSIVE_LOCK , false));
+
+ if (ictx->test_features(RBD_FEATURE_EXCLUSIVE_LOCK)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_EXCLUSIVE_LOCK,
+ false));
}
expect_op_work_queue(mock_image_ctx);
ASSERT_EQ(0, open_image(m_image_name, &ictx));
if (ictx->test_features(RBD_FEATURE_FAST_DIFF)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_FAST_DIFF , false));
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_FAST_DIFF,
+ false));
}
if (ictx->test_features(RBD_FEATURE_OBJECT_MAP)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_OBJECT_MAP , false));
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_OBJECT_MAP,
+ false));
}
MockRefreshImageCtx mock_image_ctx(*ictx);
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
- if (ictx->test_features(RBD_FEATURE_FAST_DIFF)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_FAST_DIFF , false));
+ if (ictx->test_features(RBD_FEATURE_FAST_DIFF)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_FAST_DIFF,
+ false));
}
- if (ictx->test_features(RBD_FEATURE_OBJECT_MAP)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_OBJECT_MAP , false));
+ if (ictx->test_features(RBD_FEATURE_OBJECT_MAP)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_OBJECT_MAP,
+ false));
}
MockRefreshImageCtx mock_image_ctx(*ictx);
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
- if (ictx->test_features(RBD_FEATURE_FAST_DIFF)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_FAST_DIFF , false));
+ if (ictx->test_features(RBD_FEATURE_FAST_DIFF)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_FAST_DIFF,
+ false));
}
- if (ictx->test_features(RBD_FEATURE_OBJECT_MAP)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_OBJECT_MAP , false));
+ if (ictx->test_features(RBD_FEATURE_OBJECT_MAP)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_OBJECT_MAP,
+ false));
}
MockRefreshImageCtx mock_image_ctx(*ictx);
MockJournal *mock_journal = new MockJournal();
mock_image_ctx.journal = mock_journal;
- if (ictx->test_features(RBD_FEATURE_JOURNALING)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_JOURNALING , false));
+ if (ictx->test_features(RBD_FEATURE_JOURNALING)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_JOURNALING,
+ false));
}
expect_op_work_queue(mock_image_ctx);
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
- if (ictx->test_features(RBD_FEATURE_JOURNALING)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_JOURNALING , false));
+ if (ictx->test_features(RBD_FEATURE_JOURNALING)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_JOURNALING,
+ false));
}
MockRefreshImageCtx mock_image_ctx(*ictx);
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
- if (ictx->test_features(RBD_FEATURE_JOURNALING)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_JOURNALING , false));
+ if (ictx->test_features(RBD_FEATURE_JOURNALING)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_JOURNALING,
+ false));
}
MockRefreshImageCtx mock_image_ctx(*ictx);
mock_image_ctx.journal = &mock_journal;
}
- if (ictx->test_features(RBD_FEATURE_FAST_DIFF)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_FAST_DIFF , false));
+ if (ictx->test_features(RBD_FEATURE_FAST_DIFF)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_FAST_DIFF,
+ false));
}
- if (ictx->test_features(RBD_FEATURE_OBJECT_MAP)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_OBJECT_MAP , false));
+ if (ictx->test_features(RBD_FEATURE_OBJECT_MAP)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_OBJECT_MAP,
+ false));
}
expect_op_work_queue(mock_image_ctx);
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
- if (ictx->test_features(RBD_FEATURE_JOURNALING)) {
- ASSERT_EQ(0, update_features(ictx, RBD_FEATURE_JOURNALING , false));
+ if (ictx->test_features(RBD_FEATURE_JOURNALING)) {
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_JOURNALING,
+ false));
}
MockRefreshImageCtx mock_image_ctx(*ictx);
ASSERT_EQ(-EINVAL, ictx2->operations->flatten(no_op));
}
+TEST_F(TestJournalReplay, UpdateFeatures) {
+ REQUIRE_FEATURE(RBD_FEATURE_JOURNALING);
+
+ librbd::ImageCtx *ictx;
+
+ ASSERT_EQ(0, open_image(m_image_name, &ictx));
+ ASSERT_EQ(0, when_acquired_lock(ictx));
+
+ uint64_t features = RBD_FEATURE_OBJECT_MAP | RBD_FEATURE_FAST_DIFF;
+ bool enabled = !ictx->test_features(features);
+
+ // get current commit position
+ int64_t initial_tag;
+ int64_t initial_entry;
+ get_journal_commit_position(ictx, &initial_tag, &initial_entry);
+
+ // inject update_features op into journal
+ inject_into_journal(ictx, librbd::journal::UpdateFeaturesEvent(1, features,
+ enabled));
+ close_image(ictx);
+
+ // replay journal
+ ASSERT_EQ(0, open_image(m_image_name, &ictx));
+ ASSERT_EQ(0, when_acquired_lock(ictx));
+
+ int64_t current_tag;
+ int64_t current_entry;
+ get_journal_commit_position(ictx, ¤t_tag, ¤t_entry);
+ ASSERT_EQ(initial_tag + 1, current_tag);
+ ASSERT_EQ(0, current_entry);
+
+ ASSERT_EQ(enabled, ictx->test_features(features));
+
+ // verify lock ordering constraints
+ ASSERT_EQ(0, ictx->operations->update_features(features, !enabled));
+}
+
TEST_F(TestJournalReplay, ObjectPosition) {
REQUIRE_FEATURE(RBD_FEATURE_JOURNALING);
NotifyInvoke(&m_invoke_lock, &m_invoke_cond)));
}
+ void expect_update_features(MockReplayImageCtx &mock_image_ctx, Context **on_finish,
+ uint64_t features, bool enabled, uint64_t op_tid) {
+ EXPECT_CALL(*mock_image_ctx.operations, execute_update_features(features, enabled, _, op_tid))
+ .WillOnce(DoAll(SaveArg<2>(on_finish),
+ NotifyInvoke(&m_invoke_lock, &m_invoke_cond)));
+ }
+
void expect_refresh_image(MockReplayImageCtx &mock_image_ctx, bool required,
int r) {
EXPECT_CALL(*mock_image_ctx.state, is_refresh_required())
ASSERT_EQ(0, on_finish_safe.wait());
}
+TEST_F(TestMockJournalReplay, UpdateFeaturesEvent) {
+ REQUIRE_FEATURE(RBD_FEATURE_JOURNALING);
+
+ librbd::ImageCtx *ictx;
+ ASSERT_EQ(0, open_image(m_image_name, &ictx));
+
+ uint64_t features = RBD_FEATURE_OBJECT_MAP | RBD_FEATURE_FAST_DIFF;
+ bool enabled = !ictx->test_features(features);
+
+ MockReplayImageCtx mock_image_ctx(*ictx);
+ MockJournalReplay mock_journal_replay(mock_image_ctx);
+ expect_op_work_queue(mock_image_ctx);
+
+ InSequence seq;
+ Context *on_finish = nullptr;
+ expect_refresh_image(mock_image_ctx, false, 0);
+ expect_update_features(mock_image_ctx, &on_finish, features, enabled, 123);
+
+ C_SaferCond on_start_ready;
+ C_SaferCond on_start_safe;
+ when_process(mock_journal_replay,
+ EventEntry{UpdateFeaturesEvent(123, features, enabled)},
+ &on_start_ready, &on_start_safe);
+
+ C_SaferCond on_resume;
+ when_replay_op_ready(mock_journal_replay, 123, &on_resume);
+ ASSERT_EQ(0, on_start_ready.wait());
+
+ C_SaferCond on_finish_ready;
+ C_SaferCond on_finish_safe;
+ when_process(mock_journal_replay, EventEntry{OpFinishEvent(123, 0)},
+ &on_finish_ready, &on_finish_safe);
+
+ ASSERT_EQ(0, on_resume.wait());
+ wait_for_op_invoked(&on_finish, 0);
+
+ ASSERT_EQ(0, on_start_safe.wait());
+ ASSERT_EQ(0, on_finish_ready.wait());
+ ASSERT_EQ(0, on_finish_safe.wait());
+}
+
TEST_F(TestMockJournalReplay, UnknownEvent) {
REQUIRE_FEATURE(RBD_FEATURE_JOURNALING);
Context *on_finish));
MOCK_METHOD2(execute_snap_set_limit, void(uint64_t limit,
Context *on_finish));
+ MOCK_METHOD4(execute_update_features, void(uint64_t features, bool enabled,
+ Context *on_finish,
+ uint64_t journal_op_tid));
};
} // namespace librbd
#include "test/librbd/mock/MockImageCtx.h"
#include "cls/rbd/cls_rbd_client.h"
#include "librbd/AioCompletion.h"
+#include "librbd/Operations.h"
#include "librbd/internal.h"
#include "librbd/image/SetFlagsRequest.h"
#include "librbd/mirror/EnableRequest.h"
if (!features_to_disable) {
return;
}
- ASSERT_EQ(0, librbd::update_features(ictx, features_to_disable, false));
+ ASSERT_EQ(0, ictx->operations->update_features(features_to_disable, false));
ASSERT_EQ(0, librbd::get_features(ictx, &features));
ASSERT_EQ(0U, features & features_to_disable);
}
ASSERT_EQ(0, image.update_features(disable_features, false));
}
+ ASSERT_EQ(0, image.features(&features));
+ ASSERT_EQ(0U, features & disable_features);
+
// cannot enable object map nor journaling w/o exclusive lock
ASSERT_EQ(-EINVAL, image.update_features(RBD_FEATURE_OBJECT_MAP, true));
ASSERT_EQ(-EINVAL, image.update_features(RBD_FEATURE_JOURNALING, true));
ASSERT_EQ(0, image.update_features(RBD_FEATURE_EXCLUSIVE_LOCK, true));
+ ASSERT_EQ(0, image.features(&features));
+ ASSERT_NE(0U, features & RBD_FEATURE_EXCLUSIVE_LOCK);
+
// cannot enable fast diff w/o object map
ASSERT_EQ(-EINVAL, image.update_features(RBD_FEATURE_FAST_DIFF, true));
+ ASSERT_EQ(0, image.update_features(RBD_FEATURE_OBJECT_MAP, true));
+ ASSERT_EQ(0, image.features(&features));
+ ASSERT_NE(0U, features & RBD_FEATURE_OBJECT_MAP);
+
+ uint64_t expected_flags = RBD_FLAG_OBJECT_MAP_INVALID;
+ uint64_t flags;
+ ASSERT_EQ(0, image.get_flags(&flags));
+ ASSERT_EQ(expected_flags, flags);
+
+ ASSERT_EQ(0, image.update_features(RBD_FEATURE_OBJECT_MAP, false));
+ ASSERT_EQ(0, image.features(&features));
+ ASSERT_EQ(0U, features & RBD_FEATURE_OBJECT_MAP);
+
ASSERT_EQ(0, image.update_features(RBD_FEATURE_OBJECT_MAP |
RBD_FEATURE_FAST_DIFF |
RBD_FEATURE_JOURNALING, true));
- uint64_t expected_flags = RBD_FLAG_OBJECT_MAP_INVALID |
- RBD_FLAG_FAST_DIFF_INVALID;
- uint64_t flags;
+ expected_flags = RBD_FLAG_OBJECT_MAP_INVALID | RBD_FLAG_FAST_DIFF_INVALID;
ASSERT_EQ(0, image.get_flags(&flags));
ASSERT_EQ(expected_flags, flags);
// cannot disable object map w/ fast diff
ASSERT_EQ(-EINVAL, image.update_features(RBD_FEATURE_OBJECT_MAP, false));
ASSERT_EQ(0, image.update_features(RBD_FEATURE_FAST_DIFF, false));
+ ASSERT_EQ(0, image.features(&features));
+ ASSERT_EQ(0U, features & RBD_FEATURE_FAST_DIFF);
expected_flags = RBD_FLAG_OBJECT_MAP_INVALID;
ASSERT_EQ(0, image.get_flags(&flags));
ASSERT_EQ(0, image.update_features(RBD_FEATURE_EXCLUSIVE_LOCK, false));
+ ASSERT_EQ(0, image.features(&features));
if ((features & RBD_FEATURE_DEEP_FLATTEN) != 0) {
ASSERT_EQ(0, image.update_features(RBD_FEATURE_DEEP_FLATTEN, false));
}
open_remote_image(&ictx);
uint64_t features;
ASSERT_EQ(0, librbd::get_features(ictx, &features));
- ASSERT_EQ(0, librbd::update_features(ictx, RBD_FEATURE_JOURNALING, false));
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_JOURNALING,
+ false));
close_image(ictx);
create_replayer<>();
open_remote_image(&ictx);
uint64_t features;
ASSERT_EQ(0, librbd::get_features(ictx, &features));
- ASSERT_EQ(0, librbd::update_features(ictx, RBD_FEATURE_JOURNALING, false));
+ ASSERT_EQ(0, ictx->operations->update_features(RBD_FEATURE_JOURNALING,
+ false));
close_image(ictx);
C_SaferCond cond;
stop();
}
+
+TEST_F(TestImageReplayer, UpdateFeatures)
+{
+ const uint64_t FEATURES_TO_UPDATE =
+ RBD_FEATURE_OBJECT_MAP | RBD_FEATURE_FAST_DIFF;
+
+ uint64_t features;
+ librbd::ImageCtx *ictx;
+
+ // Make sure the features we will update are disabled initially
+
+ open_remote_image(&ictx);
+ ASSERT_EQ(0, librbd::get_features(ictx, &features));
+ features &= FEATURES_TO_UPDATE;
+ if (features) {
+ ASSERT_EQ(0, ictx->operations->update_features(FEATURES_TO_UPDATE,
+ false));
+ }
+ ASSERT_EQ(0, librbd::get_features(ictx, &features));
+ ASSERT_EQ(0U, features & FEATURES_TO_UPDATE);
+ close_image(ictx);
+
+ bootstrap();
+
+ open_remote_image(&ictx);
+ ASSERT_EQ(0, librbd::get_features(ictx, &features));
+ ASSERT_EQ(0U, features & FEATURES_TO_UPDATE);
+ close_image(ictx);
+
+ open_local_image(&ictx);
+ ASSERT_EQ(0, librbd::get_features(ictx, &features));
+ ASSERT_EQ(0U, features & FEATURES_TO_UPDATE);
+ close_image(ictx);
+
+ // Start replay and update features
+
+ start();
+
+ open_remote_image(&ictx);
+ ASSERT_EQ(0, ictx->operations->update_features(FEATURES_TO_UPDATE,
+ true));
+ ASSERT_EQ(0, librbd::get_features(ictx, &features));
+ ASSERT_EQ(FEATURES_TO_UPDATE, features & FEATURES_TO_UPDATE);
+ close_image(ictx);
+
+ wait_for_replay_complete();
+
+ open_local_image(&ictx);
+ ASSERT_EQ(0, librbd::get_features(ictx, &features));
+ ASSERT_EQ(FEATURES_TO_UPDATE, features & FEATURES_TO_UPDATE);
+ close_image(ictx);
+
+ open_remote_image(&ictx);
+ ASSERT_EQ(0, ictx->operations->update_features(FEATURES_TO_UPDATE,
+ false));
+ ASSERT_EQ(0, librbd::get_features(ictx, &features));
+ ASSERT_EQ(0U, features & FEATURES_TO_UPDATE);
+ close_image(ictx);
+
+ wait_for_replay_complete();
+
+ open_local_image(&ictx);
+ ASSERT_EQ(0, librbd::get_features(ictx, &features));
+ ASSERT_EQ(0U, features & FEATURES_TO_UPDATE);
+ close_image(ictx);
+
+ // Test update_features error does not stop replication
+
+ open_remote_image(&ictx);
+ ASSERT_EQ(0, librbd::get_features(ictx, &features));
+ ASSERT_NE(0U, features & RBD_FEATURE_EXCLUSIVE_LOCK);
+ ASSERT_EQ(-EINVAL, ictx->operations->update_features(RBD_FEATURE_EXCLUSIVE_LOCK,
+ false));
+ generate_test_data();
+ for (int i = 0; i < TEST_IO_COUNT; ++i) {
+ write_test_data(ictx, m_test_data, TEST_IO_SIZE * i, TEST_IO_SIZE);
+ }
+ flush(ictx);
+ close_image(ictx);
+
+ wait_for_replay_complete();
+
+ open_local_image(&ictx);
+ for (int i = 0; i < TEST_IO_COUNT; ++i) {
+ read_test_data(ictx, m_test_data, TEST_IO_SIZE * i, TEST_IO_SIZE);
+ }
+ close_image(ictx);
+
+ stop();
+}