mon: add override in headers

Signed-off-by: liuchang0812 <liuchang0812@gmail.com>
This commit is contained in:
liuchang0812 2017-02-22 18:07:02 +08:00
parent a15e5045ff
commit 2199315c40
21 changed files with 160 additions and 160 deletions

View File

@ -109,7 +109,7 @@ private:
uint64_t max_global_id;
uint64_t last_allocated_id;
void upgrade_format();
void upgrade_format() override;
void export_keyring(KeyRing& keyring);
int import_keyring(KeyRing& keyring);
@ -136,21 +136,21 @@ private:
return true;
}
void on_active();
bool should_propose(double& delay);
void create_initial();
void update_from_paxos(bool *need_bootstrap);
void create_pending(); // prepare a new pending
void on_active() override;
bool should_propose(double& delay) override;
void create_initial() override;
void update_from_paxos(bool *need_bootstrap) override;
void create_pending() override; // prepare a new pending
bool prepare_global_id(MonOpRequestRef op);
void increase_max_global_id();
uint64_t assign_global_id(MonOpRequestRef op, bool should_increase_max);
// propose pending update to peers
void encode_pending(MonitorDBStore::TransactionRef t);
virtual void encode_full(MonitorDBStore::TransactionRef t);
version_t get_trim_to();
void encode_pending(MonitorDBStore::TransactionRef t) override;
void encode_full(MonitorDBStore::TransactionRef t) override;
version_t get_trim_to() override;
bool preprocess_query(MonOpRequestRef op); // true if processed.
bool prepare_update(MonOpRequestRef op);
bool preprocess_query(MonOpRequestRef op) override; // true if processed.
bool prepare_update(MonOpRequestRef op) override;
bool prep_auth(MonOpRequestRef op, bool paxos_writable);
@ -168,7 +168,7 @@ private:
void pre_auth(MAuth *m);
void tick(); // check state, take actions
void tick() override; // check state, take actions
void dump_info(Formatter *f);
};

View File

@ -35,36 +35,36 @@ class ConfigKeyService : public QuorumService
static const string STORE_PREFIX;
protected:
virtual void service_shutdown() { }
void service_shutdown() override { }
public:
ConfigKeyService(Monitor *m, Paxos *p) :
QuorumService(m),
paxos(p)
{ }
virtual ~ConfigKeyService() { }
~ConfigKeyService() override { }
/**
* @defgroup ConfigKeyService_Inherited_h Inherited abstract methods
* @{
*/
virtual void init() { }
virtual void get_health(Formatter *f,
list<pair<health_status_t,string> >& summary,
list<pair<health_status_t,string> > *detail) { }
virtual bool service_dispatch(MonOpRequestRef op);
void init() override { }
void get_health(Formatter *f,
list<pair<health_status_t,string> >& summary,
list<pair<health_status_t,string> > *detail) override { }
bool service_dispatch(MonOpRequestRef op) override;
virtual void start_epoch() { }
virtual void finish_epoch() { }
virtual void cleanup() { }
virtual void service_tick() { }
void start_epoch() override { }
void finish_epoch() override { }
void cleanup() override { }
void service_tick() override { }
virtual int get_type() {
int get_type() override {
return QuorumService::SERVICE_CONFIG_KEY;
}
virtual string get_name() const {
string get_name() const override {
return "config_key";
}
virtual void get_store_prefixes(set<string>& s);

View File

@ -43,13 +43,13 @@ class DataHealthService :
}
protected:
virtual void service_tick();
virtual bool service_dispatch_op(MonOpRequestRef op);
virtual void service_shutdown() { }
void service_tick() override;
bool service_dispatch_op(MonOpRequestRef op) override;
void service_shutdown() override { }
virtual void start_epoch();
virtual void finish_epoch() { }
virtual void cleanup() { }
void start_epoch() override;
void finish_epoch() override { }
void cleanup() override { }
public:
DataHealthService(Monitor *m) :
@ -58,22 +58,22 @@ public:
{
set_update_period(g_conf->mon_health_data_update_interval);
}
virtual ~DataHealthService() { }
~DataHealthService() override { }
virtual void init() {
void init() override {
generic_dout(20) << "data_health " << __func__ << dendl;
start_tick();
}
virtual void get_health(Formatter *f,
void get_health(Formatter *f,
list<pair<health_status_t,string> >& summary,
list<pair<health_status_t,string> > *detail);
list<pair<health_status_t,string> > *detail) override;
virtual int get_type() {
int get_type() override {
return HealthService::SERVICE_HEALTH_DATA;
}
virtual string get_name() const {
string get_name() const override {
return "data_health";
}
};

View File

@ -29,7 +29,7 @@ protected:
public:
HealthMonitor(Monitor *m) : QuorumService(m) { }
virtual ~HealthMonitor() {
~HealthMonitor() override {
assert(services.empty());
}

View File

@ -28,20 +28,20 @@ struct HealthService : public QuorumService
};
HealthService(Monitor *m) : QuorumService(m) { }
virtual ~HealthService() { }
~HealthService() override { }
virtual bool service_dispatch(MonOpRequestRef op) {
bool service_dispatch(MonOpRequestRef op) override {
return service_dispatch_op(op);
}
virtual bool service_dispatch_op(MonOpRequestRef op) = 0;
public:
virtual void get_health(Formatter *f,
void get_health(Formatter *f,
list<pair<health_status_t,string> >& summary,
list<pair<health_status_t,string> > *detail) = 0;
virtual int get_type() = 0;
virtual string get_name() const = 0;
list<pair<health_status_t,string> > *detail) override = 0;
int get_type() override = 0;
string get_name() const override = 0;
};
#endif // CEPH_MON_HEALTH_SERVICE_H

View File

@ -130,23 +130,23 @@ private:
void update_log_channels();
void create_initial();
void update_from_paxos(bool *need_bootstrap);
void create_pending(); // prepare a new pending
void create_initial() override;
void update_from_paxos(bool *need_bootstrap) override;
void create_pending() override; // prepare a new pending
// propose pending update to peers
void encode_pending(MonitorDBStore::TransactionRef t);
virtual void encode_full(MonitorDBStore::TransactionRef t);
version_t get_trim_to();
bool preprocess_query(MonOpRequestRef op); // true if processed.
bool prepare_update(MonOpRequestRef op);
void encode_pending(MonitorDBStore::TransactionRef t) override;
void encode_full(MonitorDBStore::TransactionRef t) override;
version_t get_trim_to() override;
bool preprocess_query(MonOpRequestRef op) override; // true if processed.
bool prepare_update(MonOpRequestRef op) override;
bool preprocess_log(MonOpRequestRef op);
bool prepare_log(MonOpRequestRef op);
void _updated_log(MonOpRequestRef op);
bool should_propose(double& delay);
bool should_propose(double& delay) override;
bool should_stash_full() {
bool should_stash_full() override {
// commit a LogSummary on every commit
return true;
}
@ -163,13 +163,13 @@ private:
LogMonitor(Monitor *mn, Paxos *p, const string& service_name)
: PaxosService(mn, p, service_name) { }
void init() {
void init() override {
generic_dout(10) << "LogMonitor::init" << dendl;
g_conf->add_observer(this);
update_log_channels();
}
void tick(); // check state, take actions
void tick() override; // check state, take actions
void check_subs();
void check_sub(Subscription *s);
@ -182,11 +182,11 @@ private:
*/
int sub_name_to_id(const string& n);
void on_shutdown() {
void on_shutdown() override {
g_conf->remove_observer(this);
}
const char **get_tracked_conf_keys() const {
const char **get_tracked_conf_keys() const override {
static const char* KEYS[] = {
"mon_cluster_log_to_syslog",
"mon_cluster_log_to_syslog_level",
@ -201,6 +201,6 @@ private:
return KEYS;
}
void handle_conf_change(const struct md_config_t *conf,
const std::set<std::string> &changed);
const std::set<std::string> &changed) override;
};
#endif

View File

@ -41,21 +41,21 @@ class MDSMonitor : public PaxosService {
MDSMonitor(Monitor *mn, Paxos *p, string service_name);
// service methods
void create_initial();
void update_from_paxos(bool *need_bootstrap);
void init();
void create_pending();
void encode_pending(MonitorDBStore::TransactionRef t);
void create_initial() override;
void update_from_paxos(bool *need_bootstrap) override;
void init() override;
void create_pending() override;
void encode_pending(MonitorDBStore::TransactionRef t) override;
// we don't require full versions; don't encode any.
virtual void encode_full(MonitorDBStore::TransactionRef t) { }
version_t get_trim_to();
void encode_full(MonitorDBStore::TransactionRef t) override { }
version_t get_trim_to() override;
bool preprocess_query(MonOpRequestRef op); // true if processed.
bool prepare_update(MonOpRequestRef op);
bool should_propose(double& delay);
bool preprocess_query(MonOpRequestRef op) override; // true if processed.
bool prepare_update(MonOpRequestRef op) override;
bool should_propose(double& delay) override;
void on_active();
void on_restart();
void on_active() override;
void on_restart() override;
void check_subs();
void check_sub(Subscription *sub);
@ -132,7 +132,7 @@ class MDSMonitor : public PaxosService {
bool maybe_expand_cluster(std::shared_ptr<Filesystem> fs);
void maybe_replace_gid(mds_gid_t gid, const beacon_info_t &beacon,
bool *mds_propose, bool *osd_propose);
void tick(); // check state, take actions
void tick() override; // check state, take actions
int dump_metadata(const string& who, Formatter *f, ostream& err);

View File

@ -41,25 +41,25 @@ public:
: PaxosService(mn, p, service_name), digest_callback(nullptr)
{}
void init();
void on_shutdown();
void init() override;
void on_shutdown() override;
const MgrMap &get_map() const { return map; }
bool in_use() const { return map.epoch > 0; }
void create_initial();
void update_from_paxos(bool *need_bootstrap);
void create_pending();
void encode_pending(MonitorDBStore::TransactionRef t);
void create_initial() override;
void update_from_paxos(bool *need_bootstrap) override;
void create_pending() override;
void encode_pending(MonitorDBStore::TransactionRef t) override;
bool preprocess_query(MonOpRequestRef op);
bool prepare_update(MonOpRequestRef op);
bool preprocess_query(MonOpRequestRef op) override;
bool prepare_update(MonOpRequestRef op) override;
bool preprocess_command(MonOpRequestRef op);
bool prepare_command(MonOpRequestRef op);
void encode_full(MonitorDBStore::TransactionRef t) { }
void encode_full(MonitorDBStore::TransactionRef t) override { }
bool preprocess_beacon(MonOpRequestRef op);
bool prepare_beacon(MonOpRequestRef op);
@ -68,7 +68,7 @@ public:
void check_subs();
void send_digests();
void tick();
void tick() override;
void print_summary(Formatter *f, std::ostream *ss) const;

View File

@ -67,7 +67,7 @@ struct MonClientPinger : public Dispatcher {
return ret;
}
bool ms_dispatch(Message *m) {
bool ms_dispatch(Message *m) override {
Mutex::Locker l(lock);
if (m->get_type() != CEPH_MSG_PING)
return false;
@ -82,14 +82,14 @@ struct MonClientPinger : public Dispatcher {
m->put();
return true;
}
bool ms_handle_reset(Connection *con) {
bool ms_handle_reset(Connection *con) override {
Mutex::Locker l(lock);
done = true;
ping_recvd_cond.SignalAll();
return true;
}
void ms_handle_remote_reset(Connection *con) {}
bool ms_handle_refused(Connection *con) {
void ms_handle_remote_reset(Connection *con) override {}
bool ms_handle_refused(Connection *con) override {
return false;
}
};
@ -171,10 +171,10 @@ private:
std::unique_ptr<AuthMethodList> auth_supported;
bool ms_dispatch(Message *m);
bool ms_handle_reset(Connection *con);
void ms_handle_remote_reset(Connection *con) {}
bool ms_handle_refused(Connection *con) { return false; }
bool ms_dispatch(Message *m) override;
bool ms_handle_reset(Connection *con) override;
void ms_handle_remote_reset(Connection *con) override {}
bool ms_handle_refused(Connection *con) override { return false; }
void handle_monmap(MMonMap *m);
@ -324,7 +324,7 @@ public:
explicit MonClient(CephContext *cct_);
MonClient(const MonClient &) = delete;
MonClient& operator=(const MonClient &) = delete;
~MonClient();
~MonClient() override;
int init();
void shutdown();

View File

@ -108,7 +108,7 @@ private:
}
}
void _dump(Formatter *f) const {
void _dump(Formatter *f) const override {
{
f->open_array_section("events");
Mutex::Locker l(lock);
@ -126,12 +126,12 @@ private:
}
protected:
void _dump_op_descriptor_unlocked(ostream& stream) const {
void _dump_op_descriptor_unlocked(ostream& stream) const override {
get_req()->print(stream);
}
public:
~MonOpRequest() {
~MonOpRequest() override {
request->put();
// certain ops may not have a session (e.g., AUTH or PING)
if (session)

View File

@ -332,7 +332,7 @@ private:
struct C_SyncTimeout : public Context {
Monitor *mon;
explicit C_SyncTimeout(Monitor *m) : mon(m) {}
void finish(int r) {
void finish(int r) override {
mon->sync_timeout();
}
};
@ -499,7 +499,7 @@ private:
struct C_TimeCheck : public Context {
Monitor *mon;
explicit C_TimeCheck(Monitor *m) : mon(m) { }
void finish(int r) {
void finish(int r) override {
mon->timecheck_start_round();
}
};
@ -544,7 +544,7 @@ private:
struct C_ProbeTimeout : public Context {
Monitor *mon;
explicit C_ProbeTimeout(Monitor *m) : mon(m) {}
void finish(int r) {
void finish(int r) override {
mon->probe_timeout(r);
}
};
@ -702,7 +702,7 @@ public:
struct C_HealthToClogTick : public Context {
Monitor *mon;
explicit C_HealthToClogTick(Monitor *m) : mon(m) { }
void finish(int r) {
void finish(int r) override {
if (r < 0)
return;
mon->do_health_to_clog();
@ -713,7 +713,7 @@ public:
struct C_HealthToClogInterval : public Context {
Monitor *mon;
explicit C_HealthToClogInterval(Monitor *m) : mon(m) { }
void finish(int r) {
void finish(int r) override {
if (r < 0)
return;
mon->do_health_to_clog_interval();
@ -812,7 +812,7 @@ public:
C_Command(Monitor *_mm, MonOpRequestRef _op, int r, string s, bufferlist rd, version_t v) :
C_MonOp(_op), mon(_mm), rc(r), rs(s), rdata(rd), version(v){}
virtual void _finish(int r) {
void _finish(int r) override {
MMonCommand *m = static_cast<MMonCommand*>(op->get_req());
if (r >= 0) {
ostringstream ss;
@ -850,7 +850,7 @@ public:
C_RetryMessage(Monitor *m, MonOpRequestRef op) :
C_MonOp(op), mon(m) { }
virtual void _finish(int r) {
void _finish(int r) override {
if (r == -EAGAIN || r >= 0)
mon->dispatch_op(op);
else if (r == -ECANCELED)
@ -872,10 +872,10 @@ public:
void dispatch_op(MonOpRequestRef op);
//mon_caps is used for un-connected messages from monitors
MonCap * mon_caps;
bool ms_get_authorizer(int dest_type, AuthAuthorizer **authorizer, bool force_new);
bool ms_get_authorizer(int dest_type, AuthAuthorizer **authorizer, bool force_new) override;
bool ms_verify_authorizer(Connection *con, int peer_type,
int protocol, bufferlist& authorizer_data, bufferlist& authorizer_reply,
bool& isvalid, CryptoKey& session_key);
bool& isvalid, CryptoKey& session_key) override;
bool ms_handle_reset(Connection *con) override;
void ms_handle_remote_reset(Connection *con) override {}
bool ms_handle_refused(Connection *con) override;
@ -900,7 +900,7 @@ public:
public:
Monitor(CephContext *cct_, string nm, MonitorDBStore *s,
Messenger *m, MonMap *map);
~Monitor();
~Monitor() override;
static int check_features(MonitorDBStore *store);

View File

@ -320,7 +320,7 @@ class MonitorDBStore
Context *f)
: store(s), t(t), oncommit(f)
{}
void finish(int r) {
void finish(int r) override {
/* The store serializes writes. Each transaction is handled
* sequentially by the io_work Finisher. If a transaction takes longer
* to apply its state to permanent storage, then no other transaction
@ -434,7 +434,7 @@ class MonitorDBStore
sync_prefixes(prefixes)
{ }
virtual ~WholeStoreIteratorImpl() { }
~WholeStoreIteratorImpl() override { }
/**
* Obtain a chunk of the store
@ -445,7 +445,7 @@ class MonitorDBStore
* differ from the one passed on to the function)
* @param last_key[out] Last key in the chunk
*/
virtual void get_chunk_tx(TransactionRef tx, uint64_t max) {
void get_chunk_tx(TransactionRef tx, uint64_t max) override {
assert(done == false);
assert(iter->valid() == true);
@ -463,7 +463,7 @@ class MonitorDBStore
done = true;
}
virtual pair<string,string> get_next_key() {
pair<string,string> get_next_key() override {
assert(iter->valid());
for (; iter->valid(); iter->next()) {
@ -476,7 +476,7 @@ class MonitorDBStore
return pair<string,string>();
}
virtual bool _is_valid() {
bool _is_valid() override {
return iter->valid();
}
};

View File

@ -44,23 +44,23 @@ class MonmapMonitor : public PaxosService {
}
MonMap pending_map; //the pending map awaiting passage
void create_initial();
void create_initial() override;
void update_from_paxos(bool *need_bootstrap);
void update_from_paxos(bool *need_bootstrap) override;
void create_pending();
void create_pending() override;
void encode_pending(MonitorDBStore::TransactionRef t);
void encode_pending(MonitorDBStore::TransactionRef t) override;
// we always encode the full map; we have no use for full versions
virtual void encode_full(MonitorDBStore::TransactionRef t) { }
void encode_full(MonitorDBStore::TransactionRef t) override { }
void on_active();
void on_active() override;
void apply_mon_features(const mon_feature_t& features);
void dump_info(Formatter *f);
bool preprocess_query(MonOpRequestRef op);
bool prepare_update(MonOpRequestRef op);
bool preprocess_query(MonOpRequestRef op) override;
bool prepare_update(MonOpRequestRef op) override;
bool preprocess_join(MonOpRequestRef op);
bool prepare_join(MonOpRequestRef op);
@ -78,9 +78,9 @@ class MonmapMonitor : public PaxosService {
* Since monitors are pretty
* important, this implementation will just write 0.0.
*/
bool should_propose(double& delay);
bool should_propose(double& delay) override;
void tick();
void tick() override;
void check_sub(Subscription *sub);

View File

@ -72,7 +72,7 @@ struct C_PrintTime : public Context {
utime_t start;
epoch_t epoch;
C_PrintTime(epoch_t e) : start(ceph_clock_now()), epoch(e) {}
void finish(int r) {
void finish(int r) override {
if (r >= 0) {
utime_t end = ceph_clock_now();
dout(10) << "osdmap epoch " << epoch << " mapping took "
@ -1591,7 +1591,7 @@ public:
m->get_epoch(),
false)); // ACK itself does not request an ack
}
~C_AckMarkedDown() {
~C_AckMarkedDown() override {
}
};

View File

@ -151,24 +151,24 @@ public:
// svc
public:
void create_initial();
void create_initial() override;
private:
void update_from_paxos(bool *need_bootstrap);
void create_pending(); // prepare a new pending
void encode_pending(MonitorDBStore::TransactionRef t);
void on_active();
void on_shutdown();
void update_from_paxos(bool *need_bootstrap) override;
void create_pending() override; // prepare a new pending
void encode_pending(MonitorDBStore::TransactionRef t) override;
void on_active() override;
void on_shutdown() override;
/**
* we haven't delegated full version stashing to paxosservice for some time
* now, making this function useless in current context.
*/
virtual void encode_full(MonitorDBStore::TransactionRef t) { }
void encode_full(MonitorDBStore::TransactionRef t) override { }
/**
* do not let paxosservice periodically stash full osdmaps, or we will break our
* locally-managed full maps. (update_from_paxos loads the latest and writes them
* out going forward from there, but if we just synced that may mean we skip some.)
*/
virtual bool should_stash_full() {
bool should_stash_full() override {
return false;
}
@ -178,7 +178,7 @@ private:
* This ensures that anyone post-sync will have enough to rebuild their
* full osdmaps.
*/
void encode_trim_extra(MonitorDBStore::TransactionRef tx, version_t first);
void encode_trim_extra(MonitorDBStore::TransactionRef tx, version_t first) override;
void update_msgr_features();
int check_cluster_features(uint64_t features, stringstream &ss);
@ -213,11 +213,11 @@ private:
void update_logger();
void handle_query(PaxosServiceMessage *m);
bool preprocess_query(MonOpRequestRef op); // true if processed.
bool prepare_update(MonOpRequestRef op);
bool should_propose(double &delay);
bool preprocess_query(MonOpRequestRef op) override; // true if processed.
bool prepare_update(MonOpRequestRef op) override;
bool should_propose(double &delay) override;
version_t get_trim_to();
version_t get_trim_to() override;
bool can_mark_down(int o);
bool can_mark_up(int o);
@ -354,7 +354,7 @@ private:
bool logit;
C_Booted(OSDMonitor *cm, MonOpRequestRef op_, bool l=true) :
C_MonOp(op_), cmon(cm), logit(l) {}
void _finish(int r) {
void _finish(int r) override {
if (r >= 0)
cmon->_booted(op, logit);
else if (r == -ECANCELED)
@ -371,7 +371,7 @@ private:
epoch_t e;
C_ReplyMap(OSDMonitor *o, MonOpRequestRef op_, epoch_t ee)
: C_MonOp(op_), osdmon(o), e(ee) {}
void _finish(int r) {
void _finish(int r) override {
if (r >= 0)
osdmon->_reply_map(op, e);
else if (r == -ECANCELED)
@ -392,7 +392,7 @@ private:
if (rd)
reply_data = *rd;
}
void _finish(int r) {
void _finish(int r) override {
if (r >= 0)
osdmon->_pool_op_reply(op, replyCode, epoch, &reply_data);
else if (r == -ECANCELED)
@ -414,7 +414,7 @@ private:
public:
OSDMonitor(CephContext *cct, Monitor *mn, Paxos *p, const string& service_name);
void tick(); // check state, take actions
void tick() override; // check state, take actions
int parse_osd_id(const char *s, stringstream *pss);

View File

@ -52,24 +52,24 @@ private:
const char *pgmap_pg_prefix;
const char *pgmap_osd_prefix;
void create_initial();
void update_from_paxos(bool *need_bootstrap);
void upgrade_format();
void on_upgrade();
void post_paxos_update();
void create_initial() override;
void update_from_paxos(bool *need_bootstrap) override;
void upgrade_format() override;
void on_upgrade() override;
void post_paxos_update() override;
void handle_osd_timeouts();
void create_pending(); // prepare a new pending
void create_pending() override; // prepare a new pending
// propose pending update to peers
version_t get_trim_to();
version_t get_trim_to() override;
void update_logger();
void encode_pending(MonitorDBStore::TransactionRef t);
void encode_pending(MonitorDBStore::TransactionRef t) override;
void read_pgmap_meta();
void read_pgmap_full();
void apply_pgmap_delta(bufferlist& bl);
bool preprocess_query(MonOpRequestRef op); // true if processed.
bool prepare_update(MonOpRequestRef op);
bool preprocess_query(MonOpRequestRef op) override; // true if processed.
bool prepare_update(MonOpRequestRef op) override;
bool preprocess_pg_stats(MonOpRequestRef op);
bool pg_stats_have_changed(int from, const MPGStats *stats) const;
@ -109,31 +109,31 @@ public:
pgmap_pg_prefix("pgmap_pg"),
pgmap_osd_prefix("pgmap_osd")
{ }
~PGMonitor() { }
~PGMonitor() override { }
virtual void get_store_prefixes(set<string>& s) {
void get_store_prefixes(set<string>& s) override {
s.insert(get_service_name());
s.insert(pgmap_meta_prefix);
s.insert(pgmap_pg_prefix);
s.insert(pgmap_osd_prefix);
}
virtual void on_restart();
void on_restart() override;
/* Courtesy function provided by PaxosService, called when an election
* finishes and the cluster goes active. We use it here to make sure we
* haven't lost any PGs from new pools. */
virtual void on_active();
void on_active() override;
bool should_stash_full() {
bool should_stash_full() override {
return false; // never
}
virtual void encode_full(MonitorDBStore::TransactionRef t) {
void encode_full(MonitorDBStore::TransactionRef t) override {
assert(0 == "unimplemented encode_full");
}
void tick(); // check state, take actions
void tick() override; // check state, take actions
void check_osd_map(epoch_t epoch);

View File

@ -661,7 +661,7 @@ public:
proposal_time(ceph_clock_now())
{ }
void finish(int r) {
void finish(int r) override {
if (proposer_context) {
proposer_context->complete(r);
proposer_context = NULL;

View File

@ -104,7 +104,7 @@ protected:
public:
C_RetryMessage(PaxosService *s, MonOpRequestRef op_) :
C_MonOp(op_), svc(s) { }
void _finish(int r) {
void _finish(int r) override {
if (r == -EAGAIN || r >= 0)
svc->dispatch(op);
else if (r == -ECANCELED)

View File

@ -31,7 +31,7 @@ class QuorumService
struct C_Tick : public Context {
QuorumService *s;
C_Tick(QuorumService *qs) : s(qs) { }
void finish(int r) {
void finish(int r) override {
if (r < 0)
return;
s->tick();

View File

@ -68,7 +68,7 @@ struct MonSession : public RefCountedObject {
proxy_con(NULL), proxy_tid(0) {
time_established = ceph_clock_now();
}
~MonSession() {
~MonSession() override {
//generic_dout(0) << "~MonSession " << this << dendl;
// we should have been removed before we get destructed; see MonSessionMap::remove_session()
assert(!item.is_on_list());

View File

@ -218,7 +218,7 @@ struct C_MonOp : public Context
explicit C_MonOp(MonOpRequestRef o) :
op(o) { }
void finish(int r) {
void finish(int r) override {
if (op && r == -ECANCELED) {
op->mark_event("callback canceled");
} else if (op && r == -EAGAIN) {