librbd: add override in header files

Fixes: http://tracker.ceph.com/issues/19012

Signed-off-by: liuchang0812 <liuchang0812@gmail.com>
This commit is contained in:
liuchang0812 2017-02-20 22:23:47 +08:00
parent a4fcdb6133
commit f72b881249
72 changed files with 356 additions and 354 deletions

View File

@ -33,7 +33,7 @@ public:
protected:
ImageCtxT &m_image_ctx;
virtual void finish(int r) {
void finish(int r) override {
m_finisher.finish_op(r);
}
@ -55,7 +55,7 @@ public:
uint64_t end_object_no);
void start_ops(uint64_t max_concurrent);
virtual void finish_op(int r);
void finish_op(int r) override;
private:
Mutex m_lock;

View File

@ -29,13 +29,13 @@ public:
void handle_peer_notification(int r);
protected:
virtual void shutdown_handler(int r, Context *on_finish);
virtual void pre_acquire_lock_handler(Context *on_finish);
virtual void post_acquire_lock_handler(int r, Context *on_finish);
virtual void pre_release_lock_handler(bool shutting_down,
Context *on_finish);
virtual void post_release_lock_handler(bool shutting_down, int r,
Context *on_finish);
void shutdown_handler(int r, Context *on_finish) override;
void pre_acquire_lock_handler(Context *on_finish) override;
void post_acquire_lock_handler(int r, Context *on_finish) override;
void pre_release_lock_handler(bool shutting_down,
Context *on_finish) override;
void post_release_lock_handler(bool shutting_down, int r,
Context *on_finish) override;
private:

View File

@ -32,7 +32,7 @@ class ImageWatcher : public Watcher {
public:
ImageWatcher(ImageCtxT& image_ctx);
virtual ~ImageWatcher();
~ImageWatcher() override;
void unregister_watch(Context *on_finish);
@ -104,7 +104,7 @@ private:
{
}
virtual int update_progress(uint64_t offset, uint64_t total) {
int update_progress(uint64_t offset, uint64_t total) override {
m_image_watcher.schedule_async_progress(m_async_request_id, offset,
total);
return 0;
@ -125,11 +125,11 @@ private:
{
}
virtual ~RemoteContext() {
~RemoteContext() override {
delete m_prog_ctx;
}
virtual void finish(int r);
void finish(int r) override;
private:
ImageWatcher &m_image_watcher;
@ -149,7 +149,7 @@ private:
payload(payload) {
}
virtual void finish(int r) override {
void finish(int r) override {
image_watcher->process_payload(notify_id, handle, payload, r);
}
};
@ -159,7 +159,7 @@ private:
C_ResponseMessage(watcher::C_NotifyAck *notify_ack) : notify_ack(notify_ack) {
}
virtual void finish(int r);
void finish(int r) override;
};
ImageCtxT &m_image_ctx;
@ -245,10 +245,10 @@ private:
void process_payload(uint64_t notify_id, uint64_t handle,
const watch_notify::Payload &payload, int r);
virtual void handle_notify(uint64_t notify_id, uint64_t handle,
uint64_t notifier_id, bufferlist &bl);
virtual void handle_error(uint64_t cookie, int err);
virtual void handle_rewatch_complete(int r);
void handle_notify(uint64_t notify_id, uint64_t handle,
uint64_t notifier_id, bufferlist &bl) override;
void handle_error(uint64_t cookie, int err) override;
void handle_rewatch_complete(int r) override;
void send_notify(const watch_notify::Payload& payload,
Context *ctx = nullptr);

View File

@ -221,7 +221,7 @@ private:
: journal(_journal), tid(_tid) {
}
virtual void finish(int r) {
void finish(int r) override {
journal->handle_io_event_safe(r, tid);
}
};
@ -239,7 +239,7 @@ private:
op_finish_future(op_finish_future), on_safe(on_safe) {
}
virtual void finish(int r) {
void finish(int r) override {
journal->handle_op_event_safe(r, tid, op_start_future, op_finish_future,
on_safe);
}
@ -252,7 +252,7 @@ private:
C_ReplayProcessSafe(Journal *journal, ReplayEntry &&replay_entry) :
journal(journal), replay_entry(std::move(replay_entry)) {
}
virtual void finish(int r) {
void finish(int r) override {
journal->handle_replay_process_safe(replay_entry, r);
}
};
@ -262,17 +262,17 @@ private:
ReplayHandler(Journal *_journal) : journal(_journal) {
}
virtual void get() {
void get() override {
// TODO
}
virtual void put() {
void put() override {
// TODO
}
virtual void handle_entries_available() {
void handle_entries_available() override {
journal->handle_replay_ready();
}
virtual void handle_complete(int r) {
void handle_complete(int r) override {
journal->handle_replay_complete(r);
}
};
@ -315,7 +315,7 @@ private:
MetadataListener(Journal<ImageCtxT> *journal) : journal(journal) { }
void handle_update(::journal::JournalMetadata *) {
void handle_update(::journal::JournalMetadata *) override {
FunctionContext *ctx = new FunctionContext([this](int r) {
journal->handle_metadata_updated();
});

View File

@ -15,10 +15,10 @@ namespace librbd {
class LibrbdAdminSocketHook : public AdminSocketHook {
public:
LibrbdAdminSocketHook(ImageCtx *ictx);
~LibrbdAdminSocketHook();
~LibrbdAdminSocketHook() override;
bool call(std::string command, cmdmap_t& cmdmap, std::string format,
bufferlist& out);
bufferlist& out) override;
private:
typedef std::map<std::string,LibrbdAdminSocketCommand*> Commands;

View File

@ -21,28 +21,28 @@ namespace librbd {
LibrbdWriteback(ImageCtx *ictx, Mutex& lock);
// Note that oloc, trunc_size, and trunc_seq are ignored
virtual void read(const object_t& oid, uint64_t object_no,
const object_locator_t& oloc, uint64_t off, uint64_t len,
snapid_t snapid, bufferlist *pbl, uint64_t trunc_size,
__u32 trunc_seq, int op_flags, Context *onfinish);
void read(const object_t& oid, uint64_t object_no,
const object_locator_t& oloc, uint64_t off, uint64_t len,
snapid_t snapid, bufferlist *pbl, uint64_t trunc_size,
__u32 trunc_seq, int op_flags, Context *onfinish) override;
// Determine whether a read to this extent could be affected by a
// write-triggered copy-on-write
virtual bool may_copy_on_write(const object_t& oid, uint64_t read_off,
uint64_t read_len, snapid_t snapid);
bool may_copy_on_write(const object_t& oid, uint64_t read_off,
uint64_t read_len, snapid_t snapid) override;
// Note that oloc, trunc_size, and trunc_seq are ignored
virtual ceph_tid_t write(const object_t& oid, const object_locator_t& oloc,
uint64_t off, uint64_t len,
const SnapContext& snapc, const bufferlist &bl,
ceph::real_time mtime, uint64_t trunc_size,
__u32 trunc_seq, ceph_tid_t journal_tid,
Context *oncommit);
ceph_tid_t write(const object_t& oid, const object_locator_t& oloc,
uint64_t off, uint64_t len,
const SnapContext& snapc, const bufferlist &bl,
ceph::real_time mtime, uint64_t trunc_size,
__u32 trunc_seq, ceph_tid_t journal_tid,
Context *oncommit) override;
using WritebackHandler::write;
virtual void overwrite_extent(const object_t& oid, uint64_t off,
uint64_t len, ceph_tid_t original_journal_tid,
ceph_tid_t new_journal_tid);
void overwrite_extent(const object_t& oid, uint64_t off,
uint64_t len, ceph_tid_t original_journal_tid,
ceph_tid_t new_journal_tid) override;
struct write_result_d {
bool done;
@ -50,7 +50,7 @@ namespace librbd {
std::string oid;
Context *oncommit;
write_result_d(const std::string& oid, Context *oncommit) :
done(false), ret(0), oid(oid), oncommit(oncommit) {}
done(false), ret(0), oid(oid), oncommit(oncommit) {}
private:
write_result_d(const write_result_d& rhs);
const write_result_d& operator=(const write_result_d& rhs);

View File

@ -199,7 +199,7 @@ private:
C_ShutDownRelease(ManagedLock *lock)
: lock(lock) {
}
virtual void finish(int r) override {
void finish(int r) override {
lock->send_shutdown_release();
}
};

View File

@ -58,8 +58,8 @@ private:
bool handle_payload(const mirroring_watcher::UnknownPayload &payload,
Context *on_notify_ack);
virtual void handle_notify(uint64_t notify_id, uint64_t handle,
uint64_t notifier_id, bufferlist &bl);
void handle_notify(uint64_t notify_id, uint64_t handle,
uint64_t notifier_id, bufferlist &bl) override;
};
} // namespace librbd

View File

@ -119,7 +119,7 @@ private:
{
}
protected:
virtual void finish(int r) {
void finish(int r) override {
m_task_finisher->complete(m_task);
}
private:

View File

@ -50,7 +50,7 @@ public:
}
protected:
virtual void finish(int r) {
void finish(int r) override {
(obj->*MF)(r);
}
};
@ -63,7 +63,7 @@ public:
}
protected:
virtual void complete(int r) override {
void complete(int r) override {
Context *on_finish = (obj->*MF)(&r);
if (on_finish != nullptr) {
on_finish->complete(r);
@ -73,7 +73,7 @@ protected:
}
Context::complete(r);
}
virtual void finish(int r) override {
void finish(int r) override {
}
};
@ -85,7 +85,7 @@ struct C_AsyncCallback : public Context {
C_AsyncCallback(WQ *op_work_queue, Context *on_finish)
: op_work_queue(op_work_queue), on_finish(on_finish) {
}
virtual void finish(int r) {
void finish(int r) override {
op_work_queue->queue(on_finish, r);
}
};

View File

@ -114,11 +114,11 @@ private:
WatchCtx(Watcher &parent) : watcher(parent) {}
virtual void handle_notify(uint64_t notify_id,
uint64_t handle,
uint64_t notifier_id,
bufferlist& bl);
virtual void handle_error(uint64_t handle, int err);
void handle_notify(uint64_t notify_id,
uint64_t handle,
uint64_t notifier_id,
bufferlist& bl) override;
void handle_error(uint64_t handle, int err) override;
};
struct C_RegisterWatch : public Context {
@ -128,7 +128,7 @@ private:
C_RegisterWatch(Watcher *watcher, Context *on_finish)
: watcher(watcher), on_finish(on_finish) {
}
virtual void finish(int r) override {
void finish(int r) override {
watcher->handle_register_watch(r, on_finish);
}
};

View File

@ -22,20 +22,20 @@ public:
PassthroughImageCache(ImageCtx &image_ctx);
/// client AIO methods
virtual void aio_read(Extents&& image_extents, ceph::bufferlist *bl,
int fadvise_flags, Context *on_finish);
virtual void aio_write(Extents&& image_extents, ceph::bufferlist&& bl,
int fadvise_flags, Context *on_finish);
virtual void aio_discard(uint64_t offset, uint64_t length,
Context *on_finish);
virtual void aio_flush(Context *on_finish);
void aio_read(Extents&& image_extents, ceph::bufferlist *bl,
int fadvise_flags, Context *on_finish) override;
void aio_write(Extents&& image_extents, ceph::bufferlist&& bl,
int fadvise_flags, Context *on_finish) override;
void aio_discard(uint64_t offset, uint64_t length,
Context *on_finish) override;
void aio_flush(Context *on_finish) override;
/// internal state methods
virtual void init(Context *on_finish);
virtual void shut_down(Context *on_finish);
void init(Context *on_finish) override;
void shut_down(Context *on_finish) override;
virtual void invalidate(Context *on_finish);
virtual void flush(Context *on_finish);
void invalidate(Context *on_finish) override;
void flush(Context *on_finish) override;
private:
ImageCtxT &m_image_ctx;

View File

@ -17,11 +17,11 @@ public:
AutomaticPolicy(ImageCtx *image_ctx) : m_image_ctx(image_ctx) {
}
virtual bool may_auto_request_lock() {
bool may_auto_request_lock() override {
return true;
}
virtual int lock_requested(bool force);
int lock_requested(bool force) override;
private:
ImageCtx *m_image_ctx;

View File

@ -17,11 +17,11 @@ public:
StandardPolicy(ImageCtx *image_ctx) : m_image_ctx(image_ctx) {
}
virtual bool may_auto_request_lock() {
bool may_auto_request_lock() override {
return false;
}
virtual int lock_requested(bool force);
int lock_requested(bool force) override;
private:
ImageCtx *m_image_ctx;

View File

@ -61,7 +61,7 @@ namespace librbd {
NoOpProgressContext()
{
}
int update_progress(uint64_t offset, uint64_t src_size)
int update_progress(uint64_t offset, uint64_t src_size) override
{
return 0;
}

View File

@ -212,8 +212,8 @@ public:
C_AioRequest(AioCompletion *completion) : m_completion(completion) {
m_completion->add_request();
}
virtual ~C_AioRequest() {}
virtual void finish(int r) {
~C_AioRequest() override {}
void finish(int r) override {
m_completion->complete_request(r);
}
protected:

View File

@ -83,13 +83,13 @@ public:
int op_flags);
protected:
virtual void send_request() override;
virtual void send_image_cache_request() override;
void send_request() override;
void send_image_cache_request() override;
virtual aio_type_t get_aio_type() const {
aio_type_t get_aio_type() const override {
return AIO_TYPE_READ;
}
virtual const char *get_request_type() const {
const char *get_request_type() const override {
return "aio_read";
}
private:
@ -101,7 +101,7 @@ private:
template <typename ImageCtxT = ImageCtx>
class AbstractImageWriteRequest : public ImageRequest<ImageCtxT> {
public:
virtual bool is_write_op() const {
bool is_write_op() const override {
return true;
}
@ -121,7 +121,7 @@ protected:
m_synchronous(false) {
}
virtual void send_request();
void send_request() override;
virtual void prune_object_extents(ObjectExtents &object_extents) {
}
@ -162,30 +162,32 @@ protected:
using typename ImageRequest<ImageCtxT>::ObjectRequests;
using typename AbstractImageWriteRequest<ImageCtxT>::ObjectExtents;
virtual aio_type_t get_aio_type() const {
aio_type_t get_aio_type() const override {
return AIO_TYPE_WRITE;
}
virtual const char *get_request_type() const {
const char *get_request_type() const override {
return "aio_write";
}
void assemble_extent(const ObjectExtent &object_extent, bufferlist *bl);
virtual void send_image_cache_request() override;
void send_image_cache_request() override;
virtual void send_object_cache_requests(const ObjectExtents &object_extents,
uint64_t journal_tid);
void send_object_cache_requests(const ObjectExtents &object_extents,
uint64_t journal_tid) override;
virtual void send_object_requests(const ObjectExtents &object_extents,
const ::SnapContext &snapc,
ObjectRequests *object_requests);
virtual ObjectRequestHandle *create_object_request(
void send_object_requests(const ObjectExtents &object_extents,
const ::SnapContext &snapc,
ObjectRequests *aio_object_requests) override;
ObjectRequestHandle *create_object_request(
const ObjectExtent &object_extent, const ::SnapContext &snapc,
Context *on_finish);
Context *on_finish) override;
uint64_t append_journal_event(const ObjectRequests &requests,
bool synchronous) override;
void update_stats(size_t length) override;
virtual uint64_t append_journal_event(const ObjectRequests &requests,
bool synchronous);
virtual void update_stats(size_t length);
private:
bufferlist m_bl;
int m_op_flags;
@ -203,28 +205,28 @@ protected:
using typename ImageRequest<ImageCtxT>::ObjectRequests;
using typename AbstractImageWriteRequest<ImageCtxT>::ObjectExtents;
virtual aio_type_t get_aio_type() const {
aio_type_t get_aio_type() const override {
return AIO_TYPE_DISCARD;
}
virtual const char *get_request_type() const {
const char *get_request_type() const override {
return "aio_discard";
}
virtual void prune_object_extents(ObjectExtents &object_extents) override;
void prune_object_extents(ObjectExtents &object_extents) override;
virtual void send_image_cache_request() override;
void send_image_cache_request() override;
virtual uint32_t get_object_cache_request_count(bool journaling) const override;
virtual void send_object_cache_requests(const ObjectExtents &object_extents,
uint64_t journal_tid);
uint32_t get_object_cache_request_count(bool journaling) const override;
void send_object_cache_requests(const ObjectExtents &object_extents,
uint64_t journal_tid) override;
virtual ObjectRequestHandle *create_object_request(
ObjectRequestHandle *create_object_request(
const ObjectExtent &object_extent, const ::SnapContext &snapc,
Context *on_finish);
Context *on_finish) override;
virtual uint64_t append_journal_event(const ObjectRequests &requests,
bool synchronous);
virtual void update_stats(size_t length);
uint64_t append_journal_event(const ObjectRequests &requests,
bool synchronous) override;
void update_stats(size_t length) override;
};
template <typename ImageCtxT = ImageCtx>
@ -234,23 +236,23 @@ public:
: ImageRequest<ImageCtxT>(image_ctx, aio_comp, {}) {
}
virtual bool is_write_op() const {
bool is_write_op() const override {
return true;
}
protected:
using typename ImageRequest<ImageCtxT>::ObjectRequests;
virtual int clip_request() {
int clip_request() override {
return 0;
}
virtual void send_request();
virtual void send_image_cache_request() override;
void send_request() override;
void send_image_cache_request() override;
virtual aio_type_t get_aio_type() const {
aio_type_t get_aio_type() const override {
return AIO_TYPE_FLUSH;
}
virtual const char *get_request_type() const {
const char *get_request_type() const override {
return "aio_flush";
}
};

View File

@ -58,8 +58,8 @@ public:
void clear_require_lock_on_read();
protected:
virtual void *_void_dequeue();
virtual void process(ImageRequest<ImageCtx> *req);
void *_void_dequeue() override;
void process(ImageRequest<ImageCtx> *req) override;
private:
typedef std::list<Context *> Contexts;
@ -72,7 +72,7 @@ private:
ImageRequest<ImageCtx> *aio_image_request)
: aio_work_queue(aio_work_queue), aio_image_request(aio_image_request) {
}
virtual void finish(int r) override {
void finish(int r) override {
aio_work_queue->handle_refreshed(r, aio_image_request);
}
};
@ -83,7 +83,7 @@ private:
: aio_work_queue(_aio_work_queue) {
}
virtual void finish(int r) {
void finish(int r) override {
aio_work_queue->handle_blocked_writes(r);
}
};

View File

@ -73,14 +73,14 @@ public:
uint64_t objectno, uint64_t off, uint64_t len,
librados::snap_t snap_id,
Context *completion, bool hide_enoent);
virtual ~ObjectRequest() {}
~ObjectRequest() override {}
virtual void add_copyup_ops(librados::ObjectWriteOperation *wr) {};
void complete(int r);
void complete(int r) override;
virtual bool should_complete(int r) = 0;
virtual void send() = 0;
void send() override = 0;
bool has_parent() const {
return m_has_parent;
@ -129,8 +129,8 @@ public:
Extents& buffer_extents, librados::snap_t snap_id,
bool sparse, Context *completion, int op_flags);
virtual bool should_complete(int r);
virtual void send();
bool should_complete(int r) override;
void send() override;
void guard_read();
inline uint64_t get_offset() const {
@ -149,11 +149,11 @@ public:
return m_ext_map;
}
const char *get_op_type() const {
const char *get_op_type() const override {
return "read";
}
bool pre_object_map_update(uint8_t *new_state) {
bool pre_object_map_update(uint8_t *new_state) override {
return false;
}
@ -201,13 +201,13 @@ public:
uint64_t len, const ::SnapContext &snapc,
Context *completion, bool hide_enoent);
virtual void add_copyup_ops(librados::ObjectWriteOperation *wr)
void add_copyup_ops(librados::ObjectWriteOperation *wr) override
{
add_write_ops(wr);
}
virtual bool should_complete(int r);
virtual void send();
bool should_complete(int r) override;
void send() override;
/**
* Writes go through the following state machine to deal with
@ -289,23 +289,23 @@ public:
m_write_data(data), m_op_flags(op_flags) {
}
bool is_op_payload_empty() const {
bool is_op_payload_empty() const override {
return (m_write_data.length() == 0);
}
virtual const char *get_op_type() const {
const char *get_op_type() const override {
return "write";
}
virtual bool pre_object_map_update(uint8_t *new_state) {
bool pre_object_map_update(uint8_t *new_state) override {
*new_state = OBJECT_EXISTS;
return true;
}
protected:
virtual void add_write_ops(librados::ObjectWriteOperation *wr);
void add_write_ops(librados::ObjectWriteOperation *wr) override;
virtual void send_write();
void send_write() override;
private:
ceph::bufferlist m_write_data;
@ -322,14 +322,14 @@ public:
m_object_state(OBJECT_NONEXISTENT) {
}
virtual const char* get_op_type() const {
const char* get_op_type() const override {
if (has_parent()) {
return "remove (trunc)";
}
return "remove";
}
virtual bool pre_object_map_update(uint8_t *new_state) {
bool pre_object_map_update(uint8_t *new_state) override {
if (has_parent()) {
m_object_state = OBJECT_EXISTS;
} else {
@ -339,18 +339,18 @@ public:
return true;
}
virtual bool post_object_map_update() {
bool post_object_map_update() override {
if (m_object_state == OBJECT_EXISTS) {
return false;
}
return true;
}
virtual void guard_write();
virtual void send_write();
void guard_write() override;
void send_write() override;
protected:
virtual void add_write_ops(librados::ObjectWriteOperation *wr) {
void add_write_ops(librados::ObjectWriteOperation *wr) override {
if (has_parent()) {
wr->truncate(0);
} else {
@ -375,21 +375,21 @@ public:
m_post_object_map_update(post_object_map_update) {
}
virtual const char* get_op_type() const {
const char* get_op_type() const override {
return "remove (trim)";
}
virtual bool pre_object_map_update(uint8_t *new_state) {
bool pre_object_map_update(uint8_t *new_state) override {
*new_state = OBJECT_PENDING;
return true;
}
virtual bool post_object_map_update() {
bool post_object_map_update() override {
return m_post_object_map_update;
}
protected:
virtual void add_write_ops(librados::ObjectWriteOperation *wr) {
void add_write_ops(librados::ObjectWriteOperation *wr) override {
wr->remove();
}
@ -406,11 +406,11 @@ public:
completion, true) {
}
virtual const char* get_op_type() const {
const char* get_op_type() const override {
return "truncate";
}
virtual bool pre_object_map_update(uint8_t *new_state) {
bool pre_object_map_update(uint8_t *new_state) override {
if (!m_object_exist && !has_parent())
*new_state = OBJECT_NONEXISTENT;
else
@ -418,10 +418,10 @@ public:
return true;
}
virtual void send_write();
void send_write() override;
protected:
virtual void add_write_ops(librados::ObjectWriteOperation *wr) {
void add_write_ops(librados::ObjectWriteOperation *wr) override {
wr->truncate(m_object_off);
}
};
@ -435,17 +435,17 @@ public:
snapc, completion, true) {
}
virtual const char* get_op_type() const {
const char* get_op_type() const override {
return "zero";
}
virtual bool pre_object_map_update(uint8_t *new_state) {
bool pre_object_map_update(uint8_t *new_state) override {
*new_state = OBJECT_EXISTS;
return true;
}
protected:
virtual void add_write_ops(librados::ObjectWriteOperation *wr) {
void add_write_ops(librados::ObjectWriteOperation *wr) override {
wr->zero(m_object_off, m_object_len);
}
};

View File

@ -14,13 +14,13 @@ namespace journal {
class DisabledPolicy : public Policy {
public:
virtual bool append_disabled() const {
bool append_disabled() const override {
return true;
}
virtual bool journal_disabled() const {
bool journal_disabled() const override {
return true;
}
virtual void allocate_tag_on_lock(Context *on_finish) {
void allocate_tag_on_lock(Context *on_finish) override {
assert(false);
}
};

View File

@ -68,7 +68,7 @@ private:
C_OpOnComplete(Replay *replay, uint64_t op_tid)
: replay(replay), op_tid(op_tid) {
}
virtual void finish(int r) override {
void finish(int r) override {
replay->handle_op_complete(op_tid, r);
}
};
@ -80,7 +80,7 @@ private:
C_AioModifyComplete(Replay *replay, Context *on_ready, Context *on_safe)
: replay(replay), on_ready(on_ready), on_safe(on_safe) {
}
virtual void finish(int r) {
void finish(int r) override {
replay->handle_aio_modify_complete(on_ready, on_safe, r);
}
};
@ -94,7 +94,7 @@ private:
: replay(replay), on_flush_safe(on_flush_safe),
on_safe_ctxs(on_safe_ctxs) {
}
virtual void finish(int r) {
void finish(int r) override {
replay->handle_aio_flush_complete(on_flush_safe, on_safe_ctxs, r);
}
};

View File

@ -18,13 +18,13 @@ public:
StandardPolicy(ImageCtxT *image_ctx) : m_image_ctx(image_ctx) {
}
virtual bool append_disabled() const {
bool append_disabled() const override {
return false;
}
virtual bool journal_disabled() const {
bool journal_disabled() const override {
return false;
}
virtual void allocate_tag_on_lock(Context *on_finish);
void allocate_tag_on_lock(Context *on_finish) override;
private:
ImageCtxT *m_image_ctx;

View File

@ -34,11 +34,11 @@ struct C_DecodeTag : public Context {
on_finish(on_finish) {
}
virtual void complete(int r) override {
void complete(int r) override {
on_finish->complete(process(r));
Context::complete(0);
}
virtual void finish(int r) override {
void finish(int r) override {
}
int process(int r);
@ -64,11 +64,11 @@ struct C_DecodeTags : public Context {
on_finish(on_finish) {
}
virtual void complete(int r) {
void complete(int r) override {
on_finish->complete(process(r));
Context::complete(0);
}
virtual void finish(int r) override {
void finish(int r) override {
}
int process(int r);

View File

@ -27,11 +27,11 @@ public:
m_snap_id(snap_id), m_force(force) {
}
virtual void send();
void send() override;
protected:
virtual bool should_complete(int r) override;
virtual int filter_return_code(int r) const override{
bool should_complete(int r) override;
int filter_return_code(int r) const override{
// never propagate an error back to the caller
return 0;
}

View File

@ -23,13 +23,13 @@ public:
{
}
virtual void send() = 0;
void send() override = 0;
protected:
const uint64_t m_snap_id;
virtual bool should_complete(int r);
virtual int filter_return_code(int r) const {
bool should_complete(int r) override;
int filter_return_code(int r) const override {
// never propagate an error back to the caller
return 0;
}

View File

@ -30,10 +30,10 @@ public:
static void resize(ceph::BitVector<2> *object_map, uint64_t num_objs,
uint8_t default_state);
virtual void send();
void send() override;
protected:
virtual void finish_request() override;
void finish_request() override;
private:
ceph::BitVector<2> *m_object_map;

View File

@ -50,10 +50,10 @@ public:
m_object_map(*object_map), m_ret_val(0) {
}
virtual void send();
void send() override;
protected:
virtual bool should_complete(int r);
bool should_complete(int r) override;
private:
State m_state;

View File

@ -52,12 +52,12 @@ public:
m_snap_id(snap_id), m_next_snap_id(CEPH_NOSNAP) {
}
virtual void send();
void send() override;
protected:
virtual bool should_complete(int r);
bool should_complete(int r) override;
virtual int filter_return_code(int r) const {
int filter_return_code(int r) const override {
if ((m_state == STATE_LOAD_MAP || m_state == STATE_REMOVE_MAP) &&
r == -ENOENT) {
return 0;

View File

@ -50,10 +50,10 @@ public:
assert(snap_id != CEPH_NOSNAP);
}
virtual void send();
void send() override;
protected:
virtual bool should_complete(int r);
bool should_complete(int r) override;
private:
State m_state;

View File

@ -42,10 +42,10 @@ public:
{
}
virtual void send();
void send() override;
protected:
virtual void finish_request() override;
void finish_request() override;
private:
ceph::BitVector<2> &m_object_map;

View File

@ -29,12 +29,12 @@ public:
uint64_t journal_op_tid, uint64_t features);
protected:
virtual void send_op();
virtual bool should_complete(int r);
virtual bool can_affect_io() const override {
void send_op() override;
bool should_complete(int r) override;
bool can_affect_io() const override {
return true;
}
virtual journal::Event create_event(uint64_t op_tid) const {
journal::Event create_event(uint64_t op_tid) const override {
return journal::UpdateFeaturesEvent(op_tid, m_features, false);
}

View File

@ -28,12 +28,12 @@ public:
uint64_t journal_op_tid, uint64_t features);
protected:
virtual void send_op();
virtual bool should_complete(int r);
virtual bool can_affect_io() const override {
void send_op() override;
bool should_complete(int r) override;
bool can_affect_io() const override {
return true;
}
virtual journal::Event create_event(uint64_t op_tid) const {
journal::Event create_event(uint64_t op_tid) const override {
return journal::UpdateFeaturesEvent(op_tid, m_features, true);
}

View File

@ -28,10 +28,10 @@ public:
}
protected:
virtual void send_op();
virtual bool should_complete(int r);
void send_op() override;
bool should_complete(int r) override;
virtual journal::Event create_event(uint64_t op_tid) const {
journal::Event create_event(uint64_t op_tid) const override {
return journal::FlattenEvent(op_tid);
}

View File

@ -23,10 +23,10 @@ public:
const std::string &key);
protected:
virtual void send_op();
virtual bool should_complete(int r);
void send_op() override;
bool should_complete(int r) override;
virtual journal::Event create_event(uint64_t op_tid) const {
journal::Event create_event(uint64_t op_tid) const override {
return journal::MetadataRemoveEvent(op_tid, m_key);
}

View File

@ -24,10 +24,10 @@ public:
const std::string &key, const std::string &value);
protected:
virtual void send_op();
virtual bool should_complete(int r);
void send_op() override;
bool should_complete(int r) override;
virtual journal::Event create_event(uint64_t op_tid) const {
journal::Event create_event(uint64_t op_tid) const override {
return journal::MetadataSetEvent(op_tid, m_key, m_value);
}

View File

@ -34,10 +34,10 @@ public:
{
}
virtual void send();
void send() override;
protected:
virtual bool should_complete(int r);
bool should_complete(int r) override;
private:
enum State {

View File

@ -24,10 +24,10 @@ public:
{
}
virtual void send();
void send() override;
protected:
virtual bool should_complete(int r);
bool should_complete(int r) override;
private:
/**

View File

@ -55,10 +55,10 @@ public:
const std::string &dest_name);
protected:
virtual void send_op();
virtual bool should_complete(int r);
void send_op() override;
bool should_complete(int r) override;
virtual journal::Event create_event(uint64_t op_tid) const {
journal::Event create_event(uint64_t op_tid) const override {
return journal::RenameEvent(op_tid, m_dest_name);
}

View File

@ -22,10 +22,10 @@ public:
Request(ImageCtxT &image_ctx, Context *on_finish,
uint64_t journal_op_tid = 0);
virtual void send();
void send();
protected:
virtual void finish(int r) override;
void finish(int r) override;
virtual void send_op() = 0;
virtual bool can_affect_io() const {
@ -58,7 +58,7 @@ protected:
// NOTE: temporary until converted to new state machine format
Context *create_context_finisher(int r);
virtual void finish_and_destroy(int r) override;
void finish_and_destroy(int r) override;
private:
struct C_AppendOpEvent : public Context {
@ -67,7 +67,7 @@ private:
C_AppendOpEvent(Request *request, Context *on_safe)
: request(request), on_safe(on_safe) {
}
virtual void finish(int r) override {
void finish(int r) override {
if (r >= 0) {
request->m_appended_op_event = true;
}
@ -81,7 +81,7 @@ private:
C_CommitOpEvent(Request *request, int ret_val)
: request(request), ret_val(ret_val) {
}
virtual void finish(int r) override {
void finish(int r) override {
request->handle_commit_op_event(r, ret_val);
delete request;
}

View File

@ -28,7 +28,7 @@ public:
ResizeRequest(ImageCtxT &image_ctx, Context *on_finish, uint64_t new_size,
bool allow_shrink, ProgressContext &prog_ctx, uint64_t journal_op_tid,
bool disable_journal);
virtual ~ResizeRequest();
~ResizeRequest() override;
inline bool shrinking() const {
return (m_shrink_size_visible && m_new_size < m_original_size);
@ -38,17 +38,17 @@ public:
return m_new_size;
}
virtual void send();
void send() override;
protected:
virtual void send_op();
virtual bool should_complete(int r) {
void send_op() override;
bool should_complete(int r) override {
return true;
}
virtual bool can_affect_io() const override {
bool can_affect_io() const override {
return true;
}
virtual journal::Event create_event(uint64_t op_tid) const {
journal::Event create_event(uint64_t op_tid) const override {
return journal::ResizeEvent(op_tid, m_new_size);
}

View File

@ -66,14 +66,14 @@ public:
bool skip_object_map);
protected:
virtual void send_op();
virtual bool should_complete(int r) {
void send_op() override;
bool should_complete(int r) override {
return true;
}
virtual bool can_affect_io() const override {
bool can_affect_io() const override {
return true;
}
virtual journal::Event create_event(uint64_t op_tid) const {
journal::Event create_event(uint64_t op_tid) const override {
return journal::SnapCreateEvent(op_tid, m_snap_name, m_snap_namespace);
}

View File

@ -23,10 +23,10 @@ public:
uint64_t limit);
protected:
virtual void send_op();
virtual bool should_complete(int r);
void send_op() override;
bool should_complete(int r) override;
virtual journal::Event create_event(uint64_t op_tid) const {
journal::Event create_event(uint64_t op_tid) const override {
return journal::SnapLimitEvent(op_tid, m_snap_limit);
}

View File

@ -42,10 +42,10 @@ public:
const std::string &snap_name);
protected:
virtual void send_op();
virtual bool should_complete(int r);
void send_op() override;
bool should_complete(int r) override;
virtual journal::Event create_event(uint64_t op_tid) const {
journal::Event create_event(uint64_t op_tid) const override {
return journal::SnapProtectEvent(op_tid, m_snap_name);
}

View File

@ -60,10 +60,10 @@ public:
const std::string &snap_name, uint64_t snap_id);
protected:
virtual void send_op();
virtual bool should_complete(int r);
void send_op() override;
bool should_complete(int r) override;
virtual journal::Event create_event(uint64_t op_tid) const {
journal::Event create_event(uint64_t op_tid) const override {
return journal::SnapRemoveEvent(op_tid, m_snap_name);
}

View File

@ -41,11 +41,11 @@ public:
SnapshotRenameRequest(ImageCtxT &image_ctx, Context *on_finish,
uint64_t snap_id, const std::string &snap_name);
virtual journal::Event create_event(uint64_t op_tid) const;
journal::Event create_event(uint64_t op_tid) const override;
protected:
virtual void send_op();
virtual bool should_complete(int r);
void send_op() override;
bool should_complete(int r) override;
private:
uint64_t m_snap_id;

View File

@ -59,15 +59,15 @@ public:
SnapshotRollbackRequest(ImageCtxT &image_ctx, Context *on_finish,
const std::string &snap_name, uint64_t snap_id,
uint64_t snap_size, ProgressContext &prog_ctx);
virtual ~SnapshotRollbackRequest();
~SnapshotRollbackRequest() override;
protected:
virtual void send_op();
virtual bool should_complete(int r) {
void send_op() override;
bool should_complete(int r) override {
return true;
}
virtual journal::Event create_event(uint64_t op_tid) const {
journal::Event create_event(uint64_t op_tid) const override {
return journal::SnapRollbackEvent(op_tid, m_snap_name);
}

View File

@ -53,17 +53,17 @@ public:
const std::string &snap_name);
protected:
virtual void send_op();
virtual bool should_complete(int r);
void send_op() override;
bool should_complete(int r) override;
virtual int filter_return_code(int r) const {
int filter_return_code(int r) const override {
if (m_ret_val < 0) {
return m_ret_val;
}
return 0;
}
virtual journal::Event create_event(uint64_t op_tid) const {
journal::Event create_event(uint64_t op_tid) const override {
return journal::SnapUnprotectEvent(op_tid, m_snap_name);
}

View File

@ -24,7 +24,7 @@ public:
prog_ctx);
}
virtual void send();
void send() override;
protected:
/**
@ -77,7 +77,7 @@ protected:
STATE_FINISHED
};
virtual bool should_complete(int r);
bool should_complete(int r) override;
State m_state;

View File

@ -37,7 +37,7 @@ private:
C_AioNotify(Notifier *notifier, Context *on_finish)
: notifier(notifier), on_finish(on_finish) {
}
virtual void finish(int r) override {
void finish(int r) override {
notifier->handle_notify(r, on_finish);
}
};

View File

@ -26,7 +26,7 @@ struct C_NotifyAck : public Context {
bufferlist out;
C_NotifyAck(Watcher *watcher, uint64_t notify_id, uint64_t handle);
void finish(int r);
void finish(int r) override;
};
template <typename Watcher>

View File

@ -36,31 +36,31 @@ public:
void start();
/// Should only be called by StopThreadAction
void stop();
void stop() override;
void join();
void send(Action::ptr action);
void add_pending(PendingIO::ptr io);
void add_pending(PendingIO::ptr io) override;
void remove_pending(PendingIO::ptr io);
void remove_pending(PendingIO::ptr io) override;
librbd::Image* get_image(imagectx_id_t imagectx_id);
librbd::Image* get_image(imagectx_id_t imagectx_id) override;
void put_image(imagectx_id_t imagectx_id, librbd::Image* image);
void put_image(imagectx_id_t imagectx_id, librbd::Image* image) override;
void erase_image(imagectx_id_t imagectx_id);
void erase_image(imagectx_id_t imagectx_id) override;
librbd::RBD* rbd();
librbd::RBD* rbd() override;
librados::IoCtx* ioctx();
librados::IoCtx* ioctx() override;
void set_action_complete(action_id_t id);
void set_action_complete(action_id_t id) override;
bool readonly() const;
bool readonly() const override;
rbd_loc map_image_name(std::string image_name, std::string snap_name) const;
rbd_loc map_image_name(std::string image_name, std::string snap_name) const override;
private:
void run();

View File

@ -127,19 +127,19 @@ public:
explicit TypedAction(const ActionType &action) : m_action(action) {
}
virtual action_id_t id() const {
action_id_t id() const override {
return m_action.id;
}
virtual thread_id_t thread_id() const {
thread_id_t thread_id() const override {
return m_action.thread_id;
}
virtual const action::Dependencies& predecessors() const {
const action::Dependencies& predecessors() const override {
return m_action.dependencies;
}
virtual std::ostream& dump(std::ostream& o) const {
std::ostream& dump(std::ostream& o) const override {
o << get_action_name() << ": ";
ceph::JSONFormatter formatter(false);
formatter.open_object_section("");
@ -165,13 +165,13 @@ public:
: TypedAction<action::StartThreadAction>(action) {
}
virtual bool is_start_thread() {
bool is_start_thread() override {
return true;
}
virtual void perform(ActionCtx &ctx);
void perform(ActionCtx &ctx) override;
protected:
virtual const char *get_action_name() const {
const char *get_action_name() const override {
return "StartThreadAction";
}
};
@ -182,10 +182,10 @@ public:
: TypedAction<action::StopThreadAction>(action) {
}
virtual void perform(ActionCtx &ctx);
void perform(ActionCtx &ctx) override;
protected:
virtual const char *get_action_name() const {
const char *get_action_name() const override {
return "StartThreadAction";
}
};
@ -197,10 +197,10 @@ public:
: TypedAction<action::AioReadAction>(action) {
}
virtual void perform(ActionCtx &ctx);
void perform(ActionCtx &ctx) override;
protected:
virtual const char *get_action_name() const {
const char *get_action_name() const override {
return "AioReadAction";
}
};
@ -212,10 +212,10 @@ public:
: TypedAction<action::ReadAction>(action) {
}
virtual void perform(ActionCtx &ctx);
void perform(ActionCtx &ctx) override;
protected:
virtual const char *get_action_name() const {
const char *get_action_name() const override {
return "ReadAction";
}
};
@ -227,10 +227,10 @@ public:
: TypedAction<action::AioWriteAction>(action) {
}
virtual void perform(ActionCtx &ctx);
void perform(ActionCtx &ctx) override;
protected:
virtual const char *get_action_name() const {
const char *get_action_name() const override {
return "AioWriteAction";
}
};
@ -242,10 +242,10 @@ public:
: TypedAction<action::WriteAction>(action) {
}
virtual void perform(ActionCtx &ctx);
void perform(ActionCtx &ctx) override;
protected:
virtual const char *get_action_name() const {
const char *get_action_name() const override {
return "WriteAction";
}
};
@ -257,10 +257,10 @@ public:
: TypedAction<action::AioDiscardAction>(action) {
}
virtual void perform(ActionCtx &ctx);
void perform(ActionCtx &ctx) override;
protected:
virtual const char *get_action_name() const {
const char *get_action_name() const override {
return "AioDiscardAction";
}
};
@ -272,10 +272,10 @@ public:
: TypedAction<action::DiscardAction>(action) {
}
virtual void perform(ActionCtx &ctx);
void perform(ActionCtx &ctx) override;
protected:
virtual const char *get_action_name() const {
const char *get_action_name() const override {
return "DiscardAction";
}
};
@ -287,10 +287,10 @@ public:
: TypedAction<action::OpenImageAction>(action) {
}
virtual void perform(ActionCtx &ctx);
void perform(ActionCtx &ctx) override;
protected:
virtual const char *get_action_name() const {
const char *get_action_name() const override {
return "OpenImageAction";
}
};
@ -302,10 +302,10 @@ public:
: TypedAction<action::CloseImageAction>(action) {
}
virtual void perform(ActionCtx &ctx);
void perform(ActionCtx &ctx) override;
protected:
virtual const char *get_action_name() const {
const char *get_action_name() const override {
return "CloseImageAction";
}
};
@ -316,10 +316,10 @@ public:
: TypedAction<action::AioOpenImageAction>(action) {
}
virtual void perform(ActionCtx &ctx);
void perform(ActionCtx &ctx) override;
protected:
virtual const char *get_action_name() const {
const char *get_action_name() const override {
return "AioOpenImageAction";
}
};
@ -331,10 +331,10 @@ public:
: TypedAction<action::AioCloseImageAction>(action) {
}
virtual void perform(ActionCtx &ctx);
void perform(ActionCtx &ctx) override;
protected:
virtual const char *get_action_name() const {
const char *get_action_name() const override {
return "AioCloseImageAction";
}
};

View File

@ -117,9 +117,9 @@ public:
: IO(ionum, start_time, thread_id, io_set_t()) {
}
virtual void encode(bufferlist &bl) const;
void encode(bufferlist &bl) const override;
void write_debug(std::ostream& out) const;
void write_debug(std::ostream& out) const override;
};
class StopThreadIO : public IO {
@ -131,9 +131,9 @@ public:
: IO(ionum, start_time, thread_id, deps) {
}
virtual void encode(bufferlist &bl) const;
void encode(bufferlist &bl) const override;
void write_debug(std::ostream& out) const;
void write_debug(std::ostream& out) const override;
};
class ReadIO : public IO {
@ -151,9 +151,9 @@ public:
m_length(length) {
}
virtual void encode(bufferlist &bl) const;
void encode(bufferlist &bl) const override;
void write_debug(std::ostream& out) const;
void write_debug(std::ostream& out) const override;
private:
imagectx_id_t m_imagectx;
@ -176,9 +176,9 @@ public:
m_length(length) {
}
virtual void encode(bufferlist &bl) const;
void encode(bufferlist &bl) const override;
void write_debug(std::ostream& out) const;
void write_debug(std::ostream& out) const override;
private:
imagectx_id_t m_imagectx;
@ -201,9 +201,9 @@ public:
m_length(length) {
}
virtual void encode(bufferlist &bl) const;
void encode(bufferlist &bl) const override;
void write_debug(std::ostream& out) const;
void write_debug(std::ostream& out) const override;
private:
imagectx_id_t m_imagectx;
@ -226,9 +226,9 @@ public:
m_length(length) {
}
virtual void encode(bufferlist &bl) const;
void encode(bufferlist &bl) const override;
void write_debug(std::ostream& out) const;
void write_debug(std::ostream& out) const override;
private:
imagectx_id_t m_imagectx;
@ -251,9 +251,9 @@ public:
m_length(length) {
}
virtual void encode(bufferlist &bl) const;
void encode(bufferlist &bl) const override;
void write_debug(std::ostream& out) const;
void write_debug(std::ostream& out) const override;
private:
imagectx_id_t m_imagectx;
@ -276,9 +276,9 @@ public:
m_length(length) {
}
virtual void encode(bufferlist &bl) const;
void encode(bufferlist &bl) const override;
void write_debug(std::ostream& out) const;
void write_debug(std::ostream& out) const override;
private:
imagectx_id_t m_imagectx;
@ -303,13 +303,13 @@ public:
m_readonly(readonly) {
}
virtual void encode(bufferlist &bl) const;
void encode(bufferlist &bl) const override;
imagectx_id_t imagectx() const {
return m_imagectx;
}
void write_debug(std::ostream& out) const;
void write_debug(std::ostream& out) const override;
private:
imagectx_id_t m_imagectx;
@ -329,13 +329,13 @@ public:
m_imagectx(imagectx) {
}
virtual void encode(bufferlist &bl) const;
void encode(bufferlist &bl) const override;
imagectx_id_t imagectx() const {
return m_imagectx;
}
void write_debug(std::ostream& out) const;
void write_debug(std::ostream& out) const override;
private:
imagectx_id_t m_imagectx;
@ -358,13 +358,13 @@ public:
m_readonly(readonly) {
}
virtual void encode(bufferlist &bl) const;
void encode(bufferlist &bl) const override;
imagectx_id_t imagectx() const {
return m_imagectx;
}
void write_debug(std::ostream& out) const;
void write_debug(std::ostream& out) const override;
private:
imagectx_id_t m_imagectx;
@ -384,13 +384,13 @@ public:
m_imagectx(imagectx) {
}
virtual void encode(bufferlist &bl) const;
void encode(bufferlist &bl) const override;
imagectx_id_t imagectx() const {
return m_imagectx;
}
void write_debug(std::ostream& out) const;
void write_debug(std::ostream& out) const override;
private:
imagectx_id_t m_imagectx;

View File

@ -82,7 +82,7 @@ struct ObjectRequest<librbd::MockTestImageCtx> : public ObjectRequestHandle {
assert(s_instance == nullptr);
s_instance = this;
}
~ObjectRequest() {
~ObjectRequest() override {
s_instance = nullptr;
}
@ -112,7 +112,7 @@ struct ObjectReadRequest<librbd::MockTestImageCtx> : public ObjectRequest<librbd
assert(s_instance == nullptr);
s_instance = this;
}
~ObjectReadRequest() {
~ObjectReadRequest() override {
s_instance = nullptr;
}

View File

@ -20,8 +20,8 @@ public:
static std::string get_temp_image_name();
virtual void SetUp();
virtual void TearDown();
void SetUp() override;
void TearDown() override;
int open_image(const std::string &image_name, librbd::ImageCtx **ictx);
void close_image(librbd::ImageCtx *ictx);

View File

@ -64,8 +64,8 @@ public:
static void SetUpTestCase();
static void TearDownTestCase();
virtual void SetUp();
virtual void TearDown();
void SetUp() override;
void TearDown() override;
::testing::NiceMock<librados::MockTestMemRadosClient> &get_mock_rados_client() {
return *s_mock_rados_client;

View File

@ -36,7 +36,7 @@ public:
m_cluster_watcher.reset(new ClusterWatcher(m_cluster, m_lock));
}
~TestClusterWatcher() {
~TestClusterWatcher() override {
m_cluster->wait_for_latest_osdmap();
for (auto& pool : m_pools) {
EXPECT_EQ(0, m_cluster->pool_delete(pool.c_str()));

View File

@ -48,7 +48,7 @@ TestPoolWatcher() : m_lock("TestPoolWatcherLock"),
EXPECT_EQ("", connect_cluster_pp(*m_cluster));
}
~TestPoolWatcher() {
~TestPoolWatcher() override {
m_cluster->wait_for_latest_osdmap();
for (auto& pool : m_pools) {
EXPECT_EQ(0, m_cluster->pool_delete(pool.c_str()));

View File

@ -26,8 +26,8 @@ public:
static void SetUpTestCase();
static void TearDownTestCase();
virtual void SetUp();
virtual void TearDown();
void SetUp() override;
void TearDown() override;
librados::IoCtx m_local_io_ctx;
librados::IoCtx m_remote_io_ctx;

View File

@ -39,8 +39,8 @@ public:
static void SetUpTestCase();
static void TearDownTestCase();
virtual void SetUp();
virtual void TearDown();
void SetUp() override;
void TearDown() override;
void expect_test_features(librbd::MockImageCtx &mock_image_ctx);

View File

@ -25,7 +25,7 @@ public:
}
protected:
virtual int overflow (int c);
int overflow (int c) override;
private:
size_t m_indent;

View File

@ -54,7 +54,7 @@ struct ProgressContext : public librbd::ProgressContext {
: operation(o), progress(!no_progress), last_pc(0) {
}
int update_progress(uint64_t offset, uint64_t total);
int update_progress(uint64_t offset, uint64_t total) override;
void finish();
void fail();
};

View File

@ -68,7 +68,7 @@ private:
public:
ImageDeleterThread(ImageDeleter *image_deleter) :
m_image_deleter(image_deleter) {}
void *entry() {
void *entry() override {
m_image_deleter->run();
return 0;
}

View File

@ -244,15 +244,15 @@ private:
: img_replayer(img_replayer) {
}
virtual void handle_close() {
void handle_close() override {
img_replayer->on_stop_journal_replay();
}
virtual void handle_promoted() {
void handle_promoted() override {
img_replayer->on_stop_journal_replay(0, "force promoted");
}
virtual void handle_resync() {
void handle_resync() override {
img_replayer->resync_image();
}
};
@ -263,8 +263,8 @@ private:
replayer(replayer) {
}
virtual void update_progress(const std::string &description,
bool flush = true);
void update_progress(const std::string &description,
bool flush = true) override;
private:
ImageReplayer<ImageCtxT> *replayer;
};
@ -334,7 +334,7 @@ private:
RemoteJournalerListener(ImageReplayer *replayer) : replayer(replayer) { }
void handle_update(::journal::JournalMetadata *);
void handle_update(::journal::JournalMetadata *) override;
} m_remote_listener;
struct C_ReplayCommitted : public Context {
@ -345,7 +345,7 @@ private:
ReplayEntry &&replay_entry)
: replayer(replayer), replay_entry(std::move(replay_entry)) {
}
virtual void finish(int r) {
void finish(int r) override {
replayer->handle_process_entry_safe(replay_entry, r);
}
};

View File

@ -51,10 +51,10 @@ public:
Journaler *journaler, MirrorPeerClientMeta *client_meta,
ContextWQ *work_queue, Context *on_finish,
ProgressContext *progress_ctx = nullptr);
~ImageSync();
~ImageSync() override;
void send();
void cancel();
void send() override;
void cancel() override;
private:
/**

View File

@ -49,7 +49,7 @@ public:
typedef librbd::journal::MirrorPeerClientMeta MirrorPeerClientMeta;
ImageSyncThrottler();
~ImageSyncThrottler();
~ImageSyncThrottler() override;
ImageSyncThrottler(const ImageSyncThrottler&) = delete;
ImageSyncThrottler& operator=(const ImageSyncThrottler&) = delete;
@ -74,9 +74,9 @@ private:
void handle_sync_finished(C_SyncHolder *sync_holder);
const char **get_tracked_conf_keys() const;
const char **get_tracked_conf_keys() const override;
void handle_conf_change(const struct md_config_t *conf,
const std::set<std::string> &changed);
const std::set<std::string> &changed) override;
uint32_t m_max_concurrent_syncs;
Mutex m_lock;

View File

@ -104,7 +104,7 @@ private:
}
protected:
virtual void post_acquire_lock_handler(int r, Context *on_finish) {
void post_acquire_lock_handler(int r, Context *on_finish) {
if (r == 0) {
// lock is owned at this point
Mutex::Locker locker(Parent::m_lock);
@ -112,12 +112,12 @@ private:
}
watcher->handle_post_acquire_leader_lock(r, on_finish);
}
virtual void pre_release_lock_handler(bool shutting_down,
Context *on_finish) {
void pre_release_lock_handler(bool shutting_down,
Context *on_finish) {
watcher->handle_pre_release_leader_lock(on_finish);
}
virtual void post_release_lock_handler(bool shutting_down, int r,
Context *on_finish) {
void post_release_lock_handler(bool shutting_down, int r,
Context *on_finish) {
watcher->handle_post_release_leader_lock(r, on_finish);
}
private:
@ -146,7 +146,7 @@ private:
: leader_watcher(leader_watcher) {
}
virtual void finish(int r) {
void finish(int r) override {
leader_watcher->handle_get_locker(r, locker);
}
};
@ -220,7 +220,7 @@ private:
void handle_post_release_leader_lock(int r, Context *on_finish);
void handle_notify(uint64_t notify_id, uint64_t handle,
uint64_t notifier_id, bufferlist &bl);
uint64_t notifier_id, bufferlist &bl) override;
void handle_heartbeat(Context *on_ack);
void handle_lock_acquired(Context *on_ack);

View File

@ -17,8 +17,8 @@ public:
void shut_down(Context *on_finish);
protected:
virtual void handle_notify(uint64_t notify_id, uint64_t handle,
uint64_t notifier_id, bufferlist &bl);
void handle_notify(uint64_t notify_id, uint64_t handle,
uint64_t notifier_id, bufferlist &bl) override;
};
} // namespace mirror

View File

@ -99,7 +99,7 @@ private:
Replayer *m_replayer;
public:
ReplayerThread(Replayer *replayer) : m_replayer(replayer) {}
void *entry() {
void *entry() override {
m_replayer->run();
return 0;
}
@ -111,11 +111,11 @@ private:
}
protected:
virtual void post_acquire_handler(Context *on_finish) {
void post_acquire_handler(Context *on_finish) override {
m_replayer->handle_post_acquire_leader(on_finish);
}
virtual void pre_release_handler(Context *on_finish) {
void pre_release_handler(Context *on_finish) override {
m_replayer->handle_pre_release_leader(on_finish);
}

View File

@ -75,10 +75,10 @@ public:
const std::string &remote_mirror_uuid, Journaler *journaler,
MirrorPeerClientMeta *client_meta, Context *on_finish,
bool *do_resync, ProgressContext *progress_ctx = nullptr);
~BootstrapRequest();
~BootstrapRequest() override;
void send();
void cancel();
void send() override;
void cancel() override;
private:
/**

View File

@ -55,8 +55,8 @@ public:
MirrorPeerSyncPoint *sync_point, Context *on_finish,
ProgressContext *progress_ctx = nullptr);
void send();
void cancel();
void send() override;
void cancel() override;
private:
/**

View File

@ -50,8 +50,8 @@ public:
librbd::journal::MirrorPeerClientMeta *client_meta,
ContextWQ *work_queue, Context *on_finish);
void send();
void cancel();
void send() override;
void cancel() override;
private:
/**