class MOSDFailure : public PaxosServiceMessage {
- static const int HEAD_VERSION = 3;
- static const int COMPAT_VERSION = 3;
+ static const int HEAD_VERSION = 4;
+ static const int COMPAT_VERSION = 4;
public:
enum {
};
uuid_d fsid;
- entity_inst_t target_osd;
+ int32_t target_osd;
+ entity_addrvec_t target_addrs;
__u8 flags = 0;
- epoch_t epoch = 0;
+ epoch_t epoch = 0;
int32_t failed_for = 0; // known to be failed since at least this long
MOSDFailure() : PaxosServiceMessage(MSG_OSD_FAILURE, 0, HEAD_VERSION) { }
- MOSDFailure(const uuid_d &fs, const entity_inst_t& f, int duration, epoch_t e)
+ MOSDFailure(const uuid_d &fs, int osd, const entity_addrvec_t& av,
+ int duration, epoch_t e)
: PaxosServiceMessage(MSG_OSD_FAILURE, e, HEAD_VERSION, COMPAT_VERSION),
- fsid(fs), target_osd(f),
+ fsid(fs),
+ target_osd(osd),
+ target_addrs(av),
flags(FLAG_FAILED),
epoch(e), failed_for(duration) { }
- MOSDFailure(const uuid_d &fs, const entity_inst_t& f, int duration,
+ MOSDFailure(const uuid_d &fs, int osd, const entity_addrvec_t& av,
+ int duration,
epoch_t e, __u8 extra_flags)
: PaxosServiceMessage(MSG_OSD_FAILURE, e, HEAD_VERSION, COMPAT_VERSION),
- fsid(fs), target_osd(f),
+ fsid(fs),
+ target_osd(osd),
+ target_addrs(av),
flags(extra_flags),
epoch(e), failed_for(duration) { }
private:
~MOSDFailure() override {}
-public:
- entity_inst_t get_target() { return target_osd; }
+public:
+ int get_target_osd() { return target_osd; }
+ const entity_addrvec_t& get_target_addrs() { return target_addrs; }
bool if_osd_failed() const {
return flags & FLAG_FAILED;
}
auto p = payload.cbegin();
paxos_decode(p);
decode(fsid, p);
- decode(target_osd, p);
+ if (header.version < 4) {
+ entity_inst_t i;
+ decode(i, p);
+ target_osd = i.name.num();
+ target_addrs.v.push_back(i.addr);
+ } else {
+ decode(target_osd, p);
+ decode(target_addrs, p);
+ }
decode(epoch, p);
decode(flags, p);
decode(failed_for, p);
void encode_payload(uint64_t features) override {
using ceph::encode;
paxos_encode();
+ if (!HAVE_FEATURE(features, SERVER_NAUTILUS)) {
+ header.version = 3;
+ header.compat_version = 3;
+ encode(fsid, payload);
+ encode(entity_inst_t(entity_name_t::OSD(target_osd),
+ target_addrs.legacy_addr()), payload, features);
+ encode(epoch, payload);
+ encode(flags, payload);
+ encode(failed_for, payload);
+ return;
+ }
+ header.version = HEAD_VERSION;
+ header.compat_version = COMPAT_VERSION;
encode(fsid, payload);
encode(target_osd, payload, features);
+ encode(target_addrs, payload, features);
encode(epoch, payload);
encode(flags, payload);
encode(failed_for, payload);
out << "osd_failure("
<< (if_osd_failed() ? "failed " : "recovered ")
<< (is_immediate() ? "immediate " : "timeout ")
- << target_osd << " for " << failed_for << "sec e" << epoch
+ << "osd." << target_osd << " " << target_addrs
+ << " for " << failed_for << "sec e" << epoch
<< " v" << version << ")";
}
};
op->mark_osdmon_event(__func__);
MOSDFailure *m = static_cast<MOSDFailure*>(op->get_req());
// who is target_osd
- int badboy = m->get_target().name.num();
+ int badboy = m->get_target_osd();
// check permissions
if (check_source(m, m->fsid))
if (m->get_orig_source().is_osd()) {
int from = m->get_orig_source().num();
if (!osdmap.exists(from) ||
- osdmap.get_addr(from) != m->get_orig_source_inst().addr ||
+ osdmap.get_addrs(from) != m->get_orig_source_addrs() ||
(osdmap.is_down(from) && m->if_osd_failed())) {
- dout(5) << "preprocess_failure from dead osd." << from << ", ignoring" << dendl;
+ dout(5) << "preprocess_failure from dead osd." << from
+ << ", ignoring" << dendl;
send_incremental(op, m->get_epoch()+1);
goto didit;
}
// weird?
if (osdmap.is_down(badboy)) {
- dout(5) << "preprocess_failure dne(/dup?): " << m->get_target() << ", from " << m->get_orig_source_inst() << dendl;
+ dout(5) << "preprocess_failure dne(/dup?): osd." << m->get_target_osd()
+ << " " << m->get_target_addrs()
+ << ", from " << m->get_orig_source() << dendl;
if (m->get_epoch() < osdmap.get_epoch())
send_incremental(op, m->get_epoch()+1);
goto didit;
}
- if (osdmap.get_inst(badboy) != m->get_target()) {
- dout(5) << "preprocess_failure wrong osd: report " << m->get_target() << " != map's " << osdmap.get_inst(badboy)
- << ", from " << m->get_orig_source_inst() << dendl;
+ if (osdmap.get_addrs(badboy) != m->get_target_addrs()) {
+ dout(5) << "preprocess_failure wrong osd: report osd." << m->get_target_osd()
+ << " " << m->get_target_addrs()
+ << " != map's " << osdmap.get_addrs(badboy)
+ << ", from " << m->get_orig_source() << dendl;
if (m->get_epoch() < osdmap.get_epoch())
send_incremental(op, m->get_epoch()+1);
goto didit;
// already reported?
if (osdmap.is_down(badboy) ||
osdmap.get_up_from(badboy) > m->get_epoch()) {
- dout(5) << "preprocess_failure dup/old: " << m->get_target() << ", from " << m->get_orig_source_inst() << dendl;
+ dout(5) << "preprocess_failure dup/old: osd." << m->get_target_osd()
+ << " " << m->get_target_addrs()
+ << ", from " << m->get_orig_source() << dendl;
if (m->get_epoch() < osdmap.get_epoch())
send_incremental(op, m->get_epoch()+1);
goto didit;
}
if (!can_mark_down(badboy)) {
- dout(5) << "preprocess_failure ignoring report of " << m->get_target() << " from " << m->get_orig_source_inst() << dendl;
+ dout(5) << "preprocess_failure ignoring report of osd."
+ << m->get_target_osd() << " " << m->get_target_addrs()
+ << " from " << m->get_orig_source() << dendl;
goto didit;
}
- dout(10) << "preprocess_failure new: " << m->get_target() << ", from " << m->get_orig_source_inst() << dendl;
+ dout(10) << "preprocess_failure new: osd." << m->get_target_osd()
+ << " " << m->get_target_addrs()
+ << ", from " << m->get_orig_source() << dendl;
return false;
didit:
{
op->mark_osdmon_event(__func__);
MOSDFailure *m = static_cast<MOSDFailure*>(op->get_req());
- dout(1) << "prepare_failure " << m->get_target()
- << " from " << m->get_orig_source_inst()
+ dout(1) << "prepare_failure osd." << m->get_target_osd()
+ << " " << m->get_target_addrs()
+ << " from " << m->get_orig_source()
<< " is reporting failure:" << m->if_osd_failed() << dendl;
- int target_osd = m->get_target().name.num();
+ int target_osd = m->get_target_osd();
int reporter = m->get_orig_source().num();
assert(osdmap.is_up(target_osd));
- assert(osdmap.get_addr(target_osd) == m->get_target().addr);
+ assert(osdmap.get_addrs(target_osd) == m->get_target_addrs());
if (m->if_osd_failed()) {
// calculate failure time
// add a report
if (m->is_immediate()) {
- mon->clog->debug() << m->get_target() << " reported immediately failed by "
- << m->get_orig_source_inst();
+ mon->clog->debug() << "osd." << m->get_target_osd()
+ << " reported immediately failed by "
+ << m->get_orig_source();
force_failure(target_osd, reporter);
mon->no_reply(op);
return true;
}
- mon->clog->debug() << m->get_target() << " reported failed by "
- << m->get_orig_source_inst();
+ mon->clog->debug() << "osd." << m->get_target_osd() << " reported failed by "
+ << m->get_orig_source();
failure_info_t& fi = failure_info[target_osd];
MonOpRequestRef old_op = fi.add_report(reporter, failed_since, op);
return check_failure(now, target_osd, fi);
} else {
// remove the report
- mon->clog->debug() << m->get_target() << " failure report canceled by "
- << m->get_orig_source_inst();
+ mon->clog->debug() << "osd." << m->get_target_osd()
+ << " failure report canceled by "
+ << m->get_orig_source();
if (failure_info.count(target_osd)) {
failure_info_t& fi = failure_info[target_osd];
MonOpRequestRef report_op = fi.cancel_report(reporter);
dout(10) << "handle_osd_ping canceling in-flight "
<< "failure report for osd." << from << dendl;
send_still_alive(curmap->get_epoch(),
+ from,
failure_pending_entry->second.second);
failure_pending.erase(failure_pending_entry);
}
if (id >= 0 && osdmap->is_up(id)) {
// I'm cheating mon heartbeat grace logic, because we know it's not going
// to respawn alone. +1 so we won't hit any boundary case.
- monc->send_mon_message(new MOSDFailure(monc->get_fsid(),
- osdmap->get_inst(id),
- cct->_conf->osd_heartbeat_grace + 1,
- osdmap->get_epoch(),
- MOSDFailure::FLAG_IMMEDIATE | MOSDFailure::FLAG_FAILED
- ));
+ monc->send_mon_message(
+ new MOSDFailure(
+ monc->get_fsid(),
+ id,
+ osdmap->get_addrs(id),
+ cct->_conf->osd_heartbeat_grace + 1,
+ osdmap->get_epoch(),
+ MOSDFailure::FLAG_IMMEDIATE | MOSDFailure::FLAG_FAILED
+ ));
}
}
}
Mutex::Locker l(heartbeat_lock);
unsigned old_queue = failure_queue.size();
unsigned old_pending = failure_pending.size();
- for (map<int,pair<utime_t,entity_inst_t> >::iterator p =
- failure_pending.begin();
- p != failure_pending.end(); ) {
+ for (auto p = failure_pending.begin(); p != failure_pending.end(); ) {
failure_queue[p->first] = p->second.first;
failure_pending.erase(p++);
}
while (!failure_queue.empty()) {
int osd = failure_queue.begin()->first;
if (!failure_pending.count(osd)) {
- entity_inst_t i = osdmap->get_inst(osd);
int failed_for = (int)(double)(now - failure_queue.begin()->second);
- monc->send_mon_message(new MOSDFailure(monc->get_fsid(), i, failed_for,
- osdmap->get_epoch()));
- failure_pending[osd] = make_pair(failure_queue.begin()->second, i);
+ monc->send_mon_message(
+ new MOSDFailure(
+ monc->get_fsid(),
+ osd,
+ osdmap->get_addrs(osd),
+ failed_for,
+ osdmap->get_epoch()));
+ failure_pending[osd] = make_pair(failure_queue.begin()->second,
+ osdmap->get_addrs(osd));
}
failure_queue.erase(osd);
}
}
-void OSD::send_still_alive(epoch_t epoch, const entity_inst_t &i)
+void OSD::send_still_alive(epoch_t epoch, int osd, const entity_addrvec_t &addrs)
{
- MOSDFailure *m = new MOSDFailure(monc->get_fsid(), i, 0, epoch, MOSDFailure::FLAG_ALIVE);
+ MOSDFailure *m = new MOSDFailure(monc->get_fsid(), osd, addrs, 0, epoch,
+ MOSDFailure::FLAG_ALIVE);
monc->send_mon_message(m);
}
if (do_shutdown) {
if (network_error) {
Mutex::Locker l(heartbeat_lock);
- map<int,pair<utime_t,entity_inst_t>>::iterator it =
- failure_pending.begin();
+ auto it = failure_pending.begin();
while (it != failure_pending.end()) {
dout(10) << "handle_osd_ping canceling in-flight failure report for osd."
<< it->first << dendl;
- send_still_alive(osdmap->get_epoch(), it->second.second);
+ send_still_alive(osdmap->get_epoch(), it->first, it->second.second);
failure_pending.erase(it++);
}
}
// -- failures --
map<int,utime_t> failure_queue;
- map<int,pair<utime_t,entity_inst_t> > failure_pending;
+ map<int,pair<utime_t,entity_addrvec_t> > failure_pending;
void requeue_failures();
void send_failures();
- void send_still_alive(epoch_t epoch, const entity_inst_t &i);
+ void send_still_alive(epoch_t epoch, int osd, const entity_addrvec_t &addrs);
ceph::coarse_mono_clock::time_point last_sent_beacon;
Mutex min_last_epoch_clean_lock{"OSD::min_last_epoch_clean_lock"};