From 63a408ef06d08038e481978389527c3e5b9c6da9 Mon Sep 17 00:00:00 2001 From: Patrick Donnelly Date: Mon, 16 Apr 2018 20:46:39 -0700 Subject: [PATCH] MDSMonitor: cleanup and protect fsmap access This commit is designed to permanently protect the current epoch FSMap from accidental changes. Modifications to the pending_fsmap are protected by checking if the monitor is the leader. Additionally, code must explicitly request a writeable reference. Fixes http://tracker.ceph.com/issues/23762 Signed-off-by: Patrick Donnelly (cherry picked from commit 624efc64323f99b2e843f376879c1080276e036f) Conflicts: src/mon/MDSMonitor.cc src/mon/MDSMonitor.h --- src/mds/FSMap.h | 1 + src/messages/MMDSMap.h | 2 +- src/mon/MDSMonitor.cc | 447 ++++++++++++++++++++++------------------- src/mon/MDSMonitor.h | 20 +- src/mon/OSDMonitor.cc | 8 +- src/mon/PaxosFSMap.h | 63 ++++++ 6 files changed, 312 insertions(+), 229 deletions(-) create mode 100644 src/mon/PaxosFSMap.h diff --git a/src/mds/FSMap.h b/src/mds/FSMap.h index d1757f860122..720a22f0be78 100644 --- a/src/mds/FSMap.h +++ b/src/mds/FSMap.h @@ -103,6 +103,7 @@ protected: public: friend class MDSMonitor; + friend class PaxosFSMap; FSMap() : epoch(0), diff --git a/src/messages/MMDSMap.h b/src/messages/MMDSMap.h index 92b4e1cafd9c..4fd9cc797dd4 100644 --- a/src/messages/MMDSMap.h +++ b/src/messages/MMDSMap.h @@ -34,7 +34,7 @@ public: MMDSMap() : Message(CEPH_MSG_MDS_MAP, HEAD_VERSION, COMPAT_VERSION) {} - MMDSMap(const uuid_d &f, MDSMap *mm) : + MMDSMap(const uuid_d &f, const MDSMap *mm) : Message(CEPH_MSG_MDS_MAP, HEAD_VERSION, COMPAT_VERSION), fsid(f) { epoch = mm->get_epoch(); diff --git a/src/mon/MDSMonitor.cc b/src/mon/MDSMonitor.cc index 5c51d2f84d55..ad8df52c0490 100644 --- a/src/mon/MDSMonitor.cc +++ b/src/mon/MDSMonitor.cc @@ -42,8 +42,8 @@ #define dout_subsys ceph_subsys_mon #undef dout_prefix -#define dout_prefix _prefix(_dout, mon, fsmap) -static ostream& _prefix(std::ostream *_dout, Monitor *mon, FSMap const& fsmap) { +#define dout_prefix _prefix(_dout, mon, get_fsmap()) +static ostream& _prefix(std::ostream *_dout, Monitor *mon, const FSMap& fsmap) { return *_dout << "mon." << mon->name << "@" << mon->rank << "(" << mon->get_state_name() << ").mds e" << fsmap.get_epoch() << " "; @@ -77,7 +77,7 @@ template<> bool cmd_getval(CephContext *cct, const cmdmap_t& cmdmap, // my methods -void MDSMonitor::print_map(FSMap &m, int dbl) +void MDSMonitor::print_map(const FSMap &m, int dbl) { dout(dbl) << "print_map\n"; m.print(*_dout); @@ -100,12 +100,12 @@ void MDSMonitor::get_store_prefixes(std::set& s) void MDSMonitor::update_from_paxos(bool *need_bootstrap) { version_t version = get_last_committed(); - if (version == fsmap.epoch) + if (version == get_fsmap().epoch) return; dout(10) << __func__ << " version " << version - << ", my e " << fsmap.epoch << dendl; - assert(version > fsmap.epoch); + << ", my e " << get_fsmap().epoch << dendl; + assert(version > get_fsmap().epoch); load_health(); @@ -117,13 +117,13 @@ void MDSMonitor::update_from_paxos(bool *need_bootstrap) assert(fsmap_bl.length() > 0); dout(10) << __func__ << " got " << version << dendl; - fsmap.decode(fsmap_bl); + PaxosFSMap::decode(fsmap_bl); // new map dout(4) << "new map" << dendl; - print_map(fsmap, 0); + print_map(get_fsmap(), 0); if (!g_conf->mon_mds_skip_sanity) { - fsmap.sanity(); + get_fsmap().sanity(); } check_subs(); @@ -137,43 +137,44 @@ void MDSMonitor::init() void MDSMonitor::create_pending() { - pending_fsmap = fsmap; - pending_fsmap.epoch++; + auto &fsmap = PaxosFSMap::create_pending(); if (mon->osdmon()->is_readable()) { - auto &osdmap = mon->osdmon()->osdmap; - pending_fsmap.sanitize([&osdmap](int64_t pool){return osdmap.have_pg_pool(pool);}); + const auto &osdmap = mon->osdmon()->osdmap; + fsmap.sanitize([&osdmap](int64_t pool){return osdmap.have_pg_pool(pool);}); } - dout(10) << "create_pending e" << pending_fsmap.epoch << dendl; + dout(10) << "create_pending e" << fsmap.epoch << dendl; } void MDSMonitor::encode_pending(MonitorDBStore::TransactionRef t) { - dout(10) << "encode_pending e" << pending_fsmap.epoch << dendl; + auto &pending = get_pending_fsmap_writeable(); + auto &epoch = pending.epoch; + dout(10) << "encode_pending e" << epoch << dendl; // print map iff 'debug mon = 30' or higher - print_map(pending_fsmap, 30); + print_map(get_pending_fsmap(), 30); if (!g_conf->mon_mds_skip_sanity) { - pending_fsmap.sanity(); + pending.sanity(); } // Set 'modified' on maps modified this epoch - for (auto &i : fsmap.filesystems) { - if (i.second->mds_map.epoch == fsmap.epoch) { - i.second->mds_map.modified = ceph_clock_now(); + for (auto &p : pending.filesystems) { + if (p.second->mds_map.epoch == epoch) { + p.second->mds_map.modified = ceph_clock_now(); } } // apply to paxos - assert(get_last_committed() + 1 == pending_fsmap.epoch); - bufferlist fsmap_bl; - pending_fsmap.encode(fsmap_bl, mon->get_quorum_con_features()); + assert(get_last_committed() + 1 == pending.epoch); + bufferlist pending_bl; + pending.encode(pending_bl, mon->get_quorum_con_features()); /* put everything in the transaction */ - put_version(t, pending_fsmap.epoch, fsmap_bl); - put_last_committed(t, pending_fsmap.epoch); + put_version(t, pending.epoch, pending_bl); + put_last_committed(t, pending.epoch); // Encode MDSHealth data for (std::map::iterator i = pending_daemon_health.begin(); @@ -192,7 +193,7 @@ void MDSMonitor::encode_pending(MonitorDBStore::TransactionRef t) // health health_check_map_t new_checks; - const auto info_map = pending_fsmap.get_mds_info(); + const auto &info_map = pending.get_mds_info(); for (const auto &i : info_map) { const auto &gid = i.first; const auto &info = i.second; @@ -232,7 +233,7 @@ void MDSMonitor::encode_pending(MonitorDBStore::TransactionRef t) check->detail.push_back(ss.str()); } } - pending_fsmap.get_health_checks(&new_checks); + pending.get_health_checks(&new_checks); for (auto& p : new_checks.checks) { p.second.summary = boost::regex_replace( p.second.summary, @@ -276,6 +277,8 @@ void MDSMonitor::update_logger() { dout(10) << "update_logger" << dendl; + const auto &fsmap = get_fsmap(); + uint64_t up = 0; uint64_t in = 0; uint64_t failed = 0; @@ -335,6 +338,8 @@ bool MDSMonitor::preprocess_beacon(MonOpRequestRef op) MDSMap::mds_info_t info; epoch_t effective_epoch = 0; + const auto &fsmap = get_working_fsmap(); + // check privileges, ignore if fails MonSession *session = m->get_session(); assert(session); @@ -367,11 +372,11 @@ bool MDSMonitor::preprocess_beacon(MonOpRequestRef op) } // fw to leader? - if (!mon->is_leader()) + if (!is_leader()) return false; // booted, but not in map? - if (!pending_fsmap.gid_exists(gid)) { + if (!fsmap.gid_exists(gid)) { if (state != MDSMap::STATE_BOOT) { dout(7) << "mds_beacon " << *m << " is not in fsmap (state " << ceph_mds_state_name(state) << ")" << dendl; @@ -386,7 +391,7 @@ bool MDSMonitor::preprocess_beacon(MonOpRequestRef op) } } dout(10) << __func__ << ": GID exists in map: " << gid << dendl; - info = pending_fsmap.get_info_gid(gid); + info = fsmap.get_info_gid(gid); // old seq? if (info.state_seq > seq) { @@ -396,11 +401,11 @@ bool MDSMonitor::preprocess_beacon(MonOpRequestRef op) // Work out the latest epoch that this daemon should have seen { - fs_cluster_id_t fscid = pending_fsmap.mds_roles.at(gid); + fs_cluster_id_t fscid = fsmap.mds_roles.at(gid); if (fscid == FS_CLUSTER_ID_NONE) { - effective_epoch = pending_fsmap.standby_epochs.at(gid); + effective_epoch = fsmap.standby_epochs.at(gid); } else { - effective_epoch = pending_fsmap.get_filesystem(fscid)->mds_map.epoch; + effective_epoch = fsmap.get_filesystem(fscid)->mds_map.epoch; } if (effective_epoch != m->get_last_epoch_seen()) { dout(10) << "mds_beacon " << *m @@ -470,6 +475,8 @@ bool MDSMonitor::preprocess_offload_targets(MonOpRequestRef op) op->mark_mdsmon_event(__func__); MMDSLoadTargets *m = static_cast(op->get_req()); dout(10) << "preprocess_offload_targets " << *m << " from " << m->get_orig_source() << dendl; + + auto &fsmap = get_working_fsmap(); // check privileges, ignore message if fails MonSession *session = m->get_session(); @@ -527,6 +534,8 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op) MDSMap::DaemonState state = m->get_state(); version_t seq = m->get_seq(); + auto &pending = get_pending_fsmap_writeable(); + dout(20) << __func__ << " got health from gid " << gid << " with " << m->get_health().metrics.size() << " metrics." << dendl; // Calculate deltas of health metrics created and removed @@ -576,13 +585,13 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op) // zap previous instance of this name? if (g_conf->mds_enforce_unique_name) { bool failed_mds = false; - while (mds_gid_t existing = pending_fsmap.find_mds_gid_by_name(m->get_name())) { + while (mds_gid_t existing = pending.find_mds_gid_by_name(m->get_name())) { if (!mon->osdmon()->is_writeable()) { mon->osdmon()->wait_for_writeable(op, new C_RetryMessage(this, op)); return false; } const MDSMap::mds_info_t &existing_info = - pending_fsmap.get_info_gid(existing); + pending.get_info_gid(existing); mon->clog->info() << existing_info.human_name() << " restarted"; fail_mds_gid(existing); failed_mds = true; @@ -594,7 +603,7 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op) } // Add this daemon to the map - if (pending_fsmap.mds_roles.count(gid) == 0) { + if (pending.mds_roles.count(gid) == 0) { MDSMap::mds_info_t new_info; new_info.global_id = gid; new_info.name = m->get_name(); @@ -606,19 +615,19 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op) new_info.standby_for_name = m->get_standby_for_name(); new_info.standby_for_fscid = m->get_standby_for_fscid(); new_info.standby_replay = m->get_standby_replay(); - pending_fsmap.insert(new_info); + pending.insert(new_info); } // Resolve standby_for_name to a rank - const MDSMap::mds_info_t &info = pending_fsmap.get_info_gid(gid); + const MDSMap::mds_info_t &info = pending.get_info_gid(gid); if (!info.standby_for_name.empty()) { - const MDSMap::mds_info_t *leaderinfo = fsmap.find_by_name( + const MDSMap::mds_info_t *leaderinfo = pending.find_by_name( info.standby_for_name); if (leaderinfo && (leaderinfo->rank >= 0)) { - auto fscid = pending_fsmap.mds_roles.at(leaderinfo->global_id); - auto fs = pending_fsmap.get_filesystem(fscid); + const auto &fscid = pending.mds_roles.at(leaderinfo->global_id); + const auto &fs = pending.get_filesystem(fscid); - pending_fsmap.modify_daemon(gid, [fscid, leaderinfo]( + pending.modify_daemon(gid, [fscid, leaderinfo]( MDSMap::mds_info_t *info) { info->standby_for_rank = leaderinfo->rank; info->standby_for_fscid = fscid; @@ -631,22 +640,22 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op) last_beacon[gid].seq = seq; // new incompat? - if (!pending_fsmap.compat.writeable(m->get_compat())) { - dout(10) << " fsmap " << pending_fsmap.compat + if (!pending.compat.writeable(m->get_compat())) { + dout(10) << " fsmap " << pending.compat << " can't write to new mds' " << m->get_compat() << ", updating fsmap and killing old mds's" << dendl; - pending_fsmap.update_compat(m->get_compat()); + pending.update_compat(m->get_compat()); } update_metadata(m->get_global_id(), m->get_sys_info()); } else { // state update - const MDSMap::mds_info_t &info = pending_fsmap.get_info_gid(gid); + const MDSMap::mds_info_t &info = pending.get_info_gid(gid); // Old MDS daemons don't mention that they're standby replay until // after they've sent their boot beacon, so update this field. if (info.standby_replay != m->get_standby_replay()) { - pending_fsmap.modify_daemon(info.global_id, [&m]( + pending.modify_daemon(info.global_id, [&m]( MDSMap::mds_info_t *i) { i->standby_replay = m->get_standby_replay(); @@ -663,7 +672,7 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op) if (info.laggy()) { dout(10) << "prepare_beacon clearing laggy flag on " << addr << dendl; - pending_fsmap.modify_daemon(info.global_id, [](MDSMap::mds_info_t *info) + pending.modify_daemon(info.global_id, [](MDSMap::mds_info_t *info) { info->clear_laggy(); } @@ -676,15 +685,15 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op) << " standby_for_rank=" << m->get_standby_for_rank() << dendl; if (state == MDSMap::STATE_STOPPED) { - const auto fscid = pending_fsmap.mds_roles.at(gid); - auto fs = pending_fsmap.get_filesystem(fscid); + const auto fscid = pending.mds_roles.at(gid); + const auto &fs = pending.get_filesystem(fscid); mon->clog->info() << info.human_name() << " finished " << "deactivating rank " << info.rank << " in filesystem " << fs->mds_map.fs_name << " (now has " << fs->mds_map.get_num_in_mds() - 1 << " ranks)"; - auto erased = pending_fsmap.stop(gid); + auto erased = pending.stop(gid); erased.push_back(gid); for (const auto &erased_gid : erased) { @@ -713,14 +722,14 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op) until += g_conf->get_val("mon_mds_blacklist_interval"); const auto blacklist_epoch = mon->osdmon()->blacklist(info.addr, until); request_proposal(mon->osdmon()); - pending_fsmap.damaged(gid, blacklist_epoch); + pending.damaged(gid, blacklist_epoch); last_beacon.erase(gid); // Respond to MDS, so that it knows it can continue to shut down mon->send_reply(op, new MMDSBeacon( mon->monmap->fsid, m->get_global_id(), - m->get_name(), fsmap.get_epoch(), state, seq, + m->get_name(), pending.get_epoch(), state, seq, CEPH_FEATURES_SUPPORTED_DEFAULT)); } else if (state == MDSMap::STATE_DNE) { if (!mon->osdmon()->is_writeable()) { @@ -738,7 +747,7 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op) mon->send_reply(op, new MMDSBeacon( mon->monmap->fsid, m->get_global_id(), - m->get_name(), fsmap.get_epoch(), state, seq, + m->get_name(), pending.get_epoch(), state, seq, CEPH_FEATURES_SUPPORTED_DEFAULT)); } else if (info.state == MDSMap::STATE_STANDBY && state != info.state) { // Standby daemons should never modify their own @@ -756,8 +765,8 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op) return true; } else { if (info.state != MDSMap::STATE_ACTIVE && state == MDSMap::STATE_ACTIVE) { - auto fscid = pending_fsmap.mds_roles.at(gid); - auto fs = pending_fsmap.get_filesystem(fscid); + const auto &fscid = pending.mds_roles.at(gid); + const auto &fs = pending.get_filesystem(fscid); mon->clog->info() << info.human_name() << " is now active in " << "filesystem " << fs->mds_map.fs_name << " as rank " << info.rank; @@ -765,7 +774,7 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op) // Made it through special cases and validations, record the // daemon's reported state to the FSMap. - pending_fsmap.modify_daemon(gid, [state, seq](MDSMap::mds_info_t *info) { + pending.modify_daemon(gid, [state, seq](MDSMap::mds_info_t *info) { info->state = state; info->state_seq = seq; }); @@ -773,7 +782,7 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op) } dout(7) << "prepare_beacon pending map now:" << dendl; - print_map(pending_fsmap); + print_map(pending); wait_for_finished_proposal(op, new FunctionContext([op, this](int r){ if (r >= 0) @@ -790,12 +799,14 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op) bool MDSMonitor::prepare_offload_targets(MonOpRequestRef op) { + auto &pending = get_pending_fsmap_writeable(); + op->mark_mdsmon_event(__func__); MMDSLoadTargets *m = static_cast(op->get_req()); mds_gid_t gid = m->global_id; - if (pending_fsmap.gid_has_rank(gid)) { + if (pending.gid_has_rank(gid)) { dout(10) << "prepare_offload_targets " << gid << " " << m->targets << dendl; - pending_fsmap.update_export_targets(gid, m->targets); + pending.update_export_targets(gid, m->targets); } else { dout(10) << "prepare_offload_targets " << gid << " not in map" << dendl; } @@ -810,6 +821,7 @@ bool MDSMonitor::should_propose(double& delay) void MDSMonitor::_updated(MonOpRequestRef op) { + const auto &fsmap = get_fsmap(); op->mark_mdsmon_event(__func__); MMDSBeacon *m = static_cast(op->get_req()); dout(10) << "_updated " << m->get_orig_source() << " " << *m << dendl; @@ -838,8 +850,8 @@ void MDSMonitor::on_active() tick(); update_logger(); - if (mon->is_leader()) { - mon->clog->debug() << "fsmap " << fsmap; + if (is_leader()) { + mon->clog->debug() << "fsmap " << get_fsmap(); } } @@ -847,10 +859,12 @@ void MDSMonitor::get_health(list >& summary, list > *detail, CephContext* cct) const { + const auto &fsmap = get_fsmap(); + fsmap.get_health(summary, detail); // For each MDS GID... - const auto info_map = fsmap.get_mds_info(); + const auto &info_map = fsmap.get_mds_info(); for (const auto &i : info_map) { const auto &gid = i.first; const auto &info = i.second; @@ -898,7 +912,7 @@ void MDSMonitor::get_health(list >& summary, void MDSMonitor::dump_info(Formatter *f) { f->open_object_section("fsmap"); - fsmap.dump(f); + get_fsmap().dump(f); f->close_section(); f->dump_unsigned("mdsmap_first_committed", get_first_committed()); @@ -914,6 +928,8 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op) stringstream ss, ds; map cmdmap; + const auto &fsmap = get_working_fsmap(); + if (!cmdmap_from_json(m->cmd, &cmdmap, ss)) { // ss has reason for failure string rs = ss.str(); @@ -947,87 +963,80 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op) int64_t epocharg; epoch_t epoch; - FSMap *p = &fsmap; + const FSMap *fsmapp = &get_fsmap(); + FSMap dummy; if (cmd_getval(g_ceph_context, cmdmap, "epoch", epocharg)) { epoch = epocharg; bufferlist b; int err = get_version(epoch, b); if (err == -ENOENT) { - p = 0; r = -ENOENT; + goto out; } else { assert(err == 0); assert(b.length()); - p = new FSMap; - p->decode(b); + dummy.decode(b); + fsmapp = &dummy; } } - if (p) { - stringstream ds; - const MDSMap *mdsmap = nullptr; - MDSMap blank; - blank.epoch = fsmap.epoch; - if (fsmap.legacy_client_fscid != FS_CLUSTER_ID_NONE) { - mdsmap = &(fsmap.filesystems[fsmap.legacy_client_fscid]->mds_map); - } else { - mdsmap = ␣ - } - if (f != NULL) { - f->open_object_section("mdsmap"); - mdsmap->dump(f.get()); - f->close_section(); - f->flush(ds); - r = 0; - } else { - mdsmap->print(ds); - r = 0; - } - rdata.append(ds); - ss << "dumped fsmap epoch " << p->get_epoch(); - - if (p != &fsmap) { - delete p; - } + stringstream ds; + const MDSMap *mdsmapp = nullptr; + MDSMap blank; + blank.epoch = fsmapp->epoch; + if (fsmapp->legacy_client_fscid != FS_CLUSTER_ID_NONE) { + mdsmapp = &fsmapp->filesystems.at(fsmapp->legacy_client_fscid)->mds_map; + } else { + mdsmapp = ␣ + } + if (f != NULL) { + f->open_object_section("mdsmap"); + mdsmapp->dump(f.get()); + f->close_section(); + f->flush(ds); + r = 0; + } else { + mdsmapp->print(ds); + r = 0; } + + rdata.append(ds); + ss << "dumped fsmap epoch " << fsmapp->get_epoch(); } else if (prefix == "fs dump") { int64_t epocharg; epoch_t epoch; - FSMap *p = &fsmap; + const FSMap *fsmapp = &get_fsmap(); + FSMap dummy; if (cmd_getval(g_ceph_context, cmdmap, "epoch", epocharg)) { epoch = epocharg; bufferlist b; int err = get_version(epoch, b); if (err == -ENOENT) { - p = 0; r = -ENOENT; + goto out; } else { assert(err == 0); assert(b.length()); - p = new FSMap; - p->decode(b); + dummy.decode(b); + fsmapp = &dummy; } } - if (p) { - stringstream ds; - if (f != NULL) { - f->open_object_section("fsmap"); - p->dump(f.get()); - f->close_section(); - f->flush(ds); - r = 0; - } else { - p->print(ds); - r = 0; - } - rdata.append(ds); - ss << "dumped fsmap epoch " << p->get_epoch(); - - if (p != &fsmap) - delete p; + stringstream ds; + if (f != NULL) { + f->open_object_section("fsmap"); + fsmapp->dump(f.get()); + f->close_section(); + f->flush(ds); + r = 0; + } else { + fsmapp->print(ds); + r = 0; } + + rdata.append(ds); + ss << "dumped fsmap epoch " << fsmapp->get_epoch(); } else if (prefix == "mds metadata") { if (!f) f.reset(Formatter::create("json-pretty")); @@ -1119,7 +1128,7 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op) } else if (prefix == "fs get") { string fs_name; cmd_getval(g_ceph_context, cmdmap, "fs_name", fs_name); - auto fs = fsmap.get_filesystem(fs_name); + const auto &fs = fsmap.get_filesystem(fs_name); if (fs == nullptr) { ss << "filesystem '" << fs_name << "' not found"; r = -ENOENT; @@ -1139,8 +1148,8 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op) if (f) { f->open_array_section("filesystems"); { - for (const auto i : fsmap.filesystems) { - const auto fs = i.second; + for (const auto &p : fsmap.filesystems) { + const auto &fs = p.second; f->open_object_section("filesystem"); { const MDSMap &mds_map = fs->mds_map; @@ -1177,8 +1186,8 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op) f->close_section(); f->flush(ds); } else { - for (const auto i : fsmap.filesystems) { - const auto fs = i.second; + for (const auto &p : fsmap.filesystems) { + const auto &fs = p.second; const MDSMap &mds_map = fs->mds_map; const string &md_pool_name = mon->osdmon()->osdmap.get_pool_name( mds_map.metadata_pool); @@ -1199,6 +1208,7 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op) r = 0; } +out: if (r != -1) { rdata.append(ds); string rs; @@ -1211,7 +1221,9 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op) bool MDSMonitor::fail_mds_gid(mds_gid_t gid) { - const MDSMap::mds_info_t info = pending_fsmap.get_info_gid(gid); + auto &pending = get_pending_fsmap_writeable(); + + const MDSMap::mds_info_t &info = pending.get_info_gid(gid); dout(10) << "fail_mds_gid " << gid << " mds." << info.name << " role " << info.rank << dendl; epoch_t blacklist_epoch = 0; @@ -1221,7 +1233,7 @@ bool MDSMonitor::fail_mds_gid(mds_gid_t gid) blacklist_epoch = mon->osdmon()->blacklist(info.addr, until); } - pending_fsmap.erase(gid, blacklist_epoch); + pending.erase(gid, blacklist_epoch); last_beacon.erase(gid); if (pending_daemon_health.count(gid)) { pending_daemon_health.erase(gid); @@ -1233,7 +1245,7 @@ bool MDSMonitor::fail_mds_gid(mds_gid_t gid) mds_gid_t MDSMonitor::gid_from_arg(const std::string& arg, std::ostream &ss) { - const FSMap *relevant_fsmap = mon->is_leader() ? &pending_fsmap : &fsmap; + const auto &fsmap = get_working_fsmap(); // Try parsing as a role mds_role_t role; @@ -1241,7 +1253,7 @@ mds_gid_t MDSMonitor::gid_from_arg(const std::string& arg, std::ostream &ss) int r = parse_role(arg, &role, ignore_err); if (r == 0) { // See if a GID is assigned to this role - auto fs = relevant_fsmap->get_filesystem(role.fscid); + const auto &fs = fsmap.get_filesystem(role.fscid); assert(fs != nullptr); // parse_role ensures it exists if (fs->mds_map.is_up(role.rank)) { dout(10) << __func__ << ": validated rank/GID " << role @@ -1255,7 +1267,7 @@ mds_gid_t MDSMonitor::gid_from_arg(const std::string& arg, std::ostream &ss) unsigned long long maybe_gid = strict_strtoll(arg.c_str(), 10, &err); if (!err.empty()) { // Not a role or a GID, try as a daemon name - const MDSMap::mds_info_t *mds_info = relevant_fsmap->find_by_name(arg); + const MDSMap::mds_info_t *mds_info = fsmap.find_by_name(arg); if (!mds_info) { ss << "MDS named '" << arg << "' does not exist, or is not up"; @@ -1269,7 +1281,7 @@ mds_gid_t MDSMonitor::gid_from_arg(const std::string& arg, std::ostream &ss) dout(10) << __func__ << ": treating MDS reference '" << arg << "' as an integer " << maybe_gid << dendl; - if (relevant_fsmap->gid_exists(mds_gid_t(maybe_gid))) { + if (fsmap.gid_exists(mds_gid_t(maybe_gid))) { return mds_gid_t(maybe_gid); } } @@ -1294,7 +1306,7 @@ int MDSMonitor::fail_mds(std::ostream &ss, const std::string &arg, // Take a copy of the info before removing the MDS from the map, // so that the caller knows which mds (if any) they ended up removing. - *failed_info = pending_fsmap.get_info_gid(gid); + *failed_info = get_pending_fsmap().get_info_gid(gid); fail_mds_gid(gid); ss << "failed mds gid " << gid; @@ -1328,14 +1340,16 @@ bool MDSMonitor::prepare_command(MonOpRequestRef op) return true; } + auto &pending = get_pending_fsmap_writeable(); + bool batched_propose = false; - for (auto h : handlers) { + for (const auto &h : handlers) { if (h->can_handle(prefix)) { batched_propose = h->batched_propose(); if (batched_propose) { paxos->plug(); } - r = h->handle(mon, pending_fsmap, op, cmdmap, ss); + r = h->handle(mon, pending, op, cmdmap, ss); if (batched_propose) { paxos->unplug(); } @@ -1347,7 +1361,7 @@ bool MDSMonitor::prepare_command(MonOpRequestRef op) } else { if (r == 0) { // On successful updates, print the updated map - print_map(pending_fsmap); + print_map(pending); } // Successful or not, we're done: respond. goto out; @@ -1367,8 +1381,8 @@ bool MDSMonitor::prepare_command(MonOpRequestRef op) } // Only handle legacy commands if there is a filesystem configured - if (pending_fsmap.legacy_client_fscid == FS_CLUSTER_ID_NONE) { - if (pending_fsmap.filesystems.size() == 0) { + if (pending.legacy_client_fscid == FS_CLUSTER_ID_NONE) { + if (pending.filesystems.size() == 0) { ss << "No filesystem configured: use `ceph fs new` to create a filesystem"; } else { ss << "No filesystem set for use with legacy commands"; @@ -1419,11 +1433,7 @@ int MDSMonitor::parse_role( mds_role_t *role, std::ostream &ss) { - const FSMap *relevant_fsmap = &fsmap; - if (mon->is_leader()) { - relevant_fsmap = &pending_fsmap; - } - return relevant_fsmap->parse_role(role_str, role, ss); + return get_working_fsmap().parse_role(role_str, role, ss); } int MDSMonitor::filesystem_command( @@ -1438,15 +1448,15 @@ int MDSMonitor::filesystem_command( string whostr; cmd_getval(g_ceph_context, cmdmap, "who", whostr); + auto &pending = get_pending_fsmap_writeable(); if (prefix == "mds stop" || prefix == "mds deactivate") { - mds_role_t role; r = parse_role(whostr, &role, ss); if (r < 0 ) { return r; } - auto fs = pending_fsmap.get_filesystem(role.fscid); + const auto &fs = pending.get_filesystem(role.fscid); if (!fs->mds_map.is_active(role.rank)) { r = -EEXIST; @@ -1469,9 +1479,9 @@ int MDSMonitor::filesystem_command( r = 0; mds_gid_t gid = fs->mds_map.up.at(role.rank); ss << "telling mds." << role << " " - << pending_fsmap.get_info_gid(gid).addr << " to deactivate"; + << pending.get_info_gid(gid).addr << " to deactivate"; - pending_fsmap.modify_daemon(gid, [](MDSMap::mds_info_t *info) { + pending.modify_daemon(gid, [](MDSMap::mds_info_t *info) { info->state = MDSMap::STATE_STOPPING; }); } @@ -1488,8 +1498,8 @@ int MDSMonitor::filesystem_command( << cmd_vartype_stringify(cmdmap["state"]) << "'"; return -EINVAL; } - if (pending_fsmap.gid_exists(gid)) { - pending_fsmap.modify_daemon(gid, [state](MDSMap::mds_info_t *info) { + if (pending.gid_exists(gid)) { + pending.modify_daemon(gid, [state](MDSMap::mds_info_t *info) { info->state = state; }); ss << "set mds gid " << gid << " to state " << state << " " @@ -1519,17 +1529,18 @@ int MDSMonitor::filesystem_command( << cmd_vartype_stringify(cmdmap["gid"]) << "'"; return -EINVAL; } - if (!pending_fsmap.gid_exists(gid)) { + if (!pending.gid_exists(gid)) { ss << "mds gid " << gid << " dne"; r = 0; } else { - MDSMap::DaemonState state = pending_fsmap.get_info_gid(gid).state; + const auto &info = pending.get_info_gid(gid); + MDSMap::DaemonState state = info.state; if (state > 0) { - ss << "cannot remove active mds." << pending_fsmap.get_info_gid(gid).name - << " rank " << pending_fsmap.get_info_gid(gid).rank; + ss << "cannot remove active mds." << info.name + << " rank " << info.rank; return -EBUSY; } else { - pending_fsmap.erase(gid, {}); + pending.erase(gid, {}); ss << "removed mds gid " << gid; return 0; } @@ -1552,7 +1563,7 @@ int MDSMonitor::filesystem_command( return -EINVAL; } - pending_fsmap.modify_filesystem( + pending.modify_filesystem( role.fscid, [role](std::shared_ptr fs) { @@ -1568,13 +1579,13 @@ int MDSMonitor::filesystem_command( << cmd_vartype_stringify(cmdmap["feature"]) << "'"; return -EINVAL; } - if (pending_fsmap.compat.compat.contains(f)) { + if (pending.compat.compat.contains(f)) { ss << "removing compat feature " << f; - CompatSet modified = pending_fsmap.compat; + CompatSet modified = pending.compat; modified.compat.remove(f); - pending_fsmap.update_compat(modified); + pending.update_compat(modified); } else { - ss << "compat feature " << f << " not present in " << pending_fsmap.compat; + ss << "compat feature " << f << " not present in " << pending.compat; } r = 0; } else if (prefix == "mds compat rm_incompat") { @@ -1584,13 +1595,13 @@ int MDSMonitor::filesystem_command( << cmd_vartype_stringify(cmdmap["feature"]) << "'"; return -EINVAL; } - if (pending_fsmap.compat.incompat.contains(f)) { + if (pending.compat.incompat.contains(f)) { ss << "removing incompat feature " << f; - CompatSet modified = pending_fsmap.compat; + CompatSet modified = pending.compat; modified.incompat.remove(f); - pending_fsmap.update_compat(modified); + pending.update_compat(modified); } else { - ss << "incompat feature " << f << " not present in " << pending_fsmap.compat; + ss << "incompat feature " << f << " not present in " << pending.compat; } r = 0; } else if (prefix == "mds repaired") { @@ -1602,7 +1613,7 @@ int MDSMonitor::filesystem_command( return r; } - bool modified = pending_fsmap.undamaged(role.fscid, role.rank); + bool modified = pending.undamaged(role.fscid, role.rank); if (modified) { dout(4) << "repaired: restoring rank " << role << dendl; } else { @@ -1623,6 +1634,7 @@ int MDSMonitor::filesystem_command( void MDSMonitor::modify_legacy_filesystem( std::function )> fn) { + auto &pending_fsmap = get_pending_fsmap_writeable(); pending_fsmap.modify_filesystem( pending_fsmap.legacy_client_fscid, fn @@ -1652,6 +1664,8 @@ int MDSMonitor::legacy_filesystem_command( string whostr; cmd_getval(g_ceph_context, cmdmap, "who", whostr); + auto &pending_fsmap = get_pending_fsmap_writeable(); + assert (pending_fsmap.legacy_client_fscid != FS_CLUSTER_ID_NONE); if (prefix == "mds set_max_mds") { @@ -1721,8 +1735,8 @@ void MDSMonitor::check_subs() types.push_back("fsmap"); types.push_back("fsmap.user"); types.push_back("mdsmap"); - for (const auto &i : fsmap.filesystems) { - auto fscid = i.first; + for (const auto &p : get_fsmap().filesystems) { + const auto &fscid = p.first; std::ostringstream oss; oss << "mdsmap." << fscid; types.push_back(oss.str()); @@ -1745,6 +1759,8 @@ void MDSMonitor::check_sub(Subscription *sub) { dout(20) << __func__ << ": " << sub->type << dendl; + const auto &fsmap = get_fsmap(); + if (sub->type == "fsmap") { if (sub->next <= fsmap.get_epoch()) { sub->session->con->send_message(new MFSMap(mon->monmap->fsid, fsmap)); @@ -1759,12 +1775,10 @@ void MDSMonitor::check_sub(Subscription *sub) FSMapUser fsmap_u; fsmap_u.epoch = fsmap.get_epoch(); fsmap_u.legacy_client_fscid = fsmap.legacy_client_fscid; - for (auto p = fsmap.filesystems.begin(); - p != fsmap.filesystems.end(); - ++p) { - FSMapUser::fs_info_t& fs_info = fsmap_u.filesystems[p->first]; - fs_info.cid = p->first; - fs_info.name= p->second->mds_map.fs_name; + for (const auto &p : fsmap.filesystems) { + FSMapUser::fs_info_t& fs_info = fsmap_u.filesystems[p.second->fscid]; + fs_info.cid = p.second->fscid; + fs_info.name = p.second->mds_map.fs_name; } sub->session->con->send_message(new MFSMapUser(mon->monmap->fsid, fsmap_u)); if (sub->onetime) { @@ -1828,24 +1842,25 @@ void MDSMonitor::check_sub(Subscription *sub) dout(10) << __func__ << ": is_mds=" << is_mds << ", fscid= " << fscid << dendl; // Work out the effective latest epoch - MDSMap *mds_map = nullptr; + const MDSMap *mds_map = nullptr; MDSMap null_map; null_map.compat = fsmap.compat; if (fscid == FS_CLUSTER_ID_NONE) { // For a client, we should have already dropped out assert(is_mds); - if (fsmap.standby_daemons.count(mds_gid)) { + auto it = fsmap.standby_daemons.find(mds_gid); + if (it != fsmap.standby_daemons.end()) { // For an MDS, we need to feed it an MDSMap with its own state in - null_map.mds_info[mds_gid] = fsmap.standby_daemons[mds_gid]; - null_map.epoch = fsmap.standby_epochs[mds_gid]; + null_map.mds_info[mds_gid] = it->second; + null_map.epoch = fsmap.standby_epochs.at(mds_gid); } else { null_map.epoch = fsmap.epoch; } mds_map = &null_map; } else { // Check the effective epoch - mds_map = &(fsmap.filesystems.at(fscid)->mds_map); + mds_map = &fsmap.get_filesystem(fscid)->mds_map; } assert(mds_map != nullptr); @@ -1888,7 +1903,7 @@ void MDSMonitor::remove_from_metadata(MonitorDBStore::TransactionRef t) bool update = false; for (map::iterator i = pending_metadata.begin(); i != pending_metadata.end(); ) { - if (!pending_fsmap.gid_exists(i->first)) { + if (!get_pending_fsmap().gid_exists(i->first)) { pending_metadata.erase(i++); update = true; } else { @@ -1985,11 +2000,11 @@ int MDSMonitor::print_nodes(Formatter *f) continue; } const mds_gid_t gid = it->first; - if (!fsmap.gid_exists(gid)) { + if (!get_fsmap().gid_exists(gid)) { dout(5) << __func__ << ": GID " << gid << " not existent" << dendl; continue; } - const MDSMap::mds_info_t& mds_info = fsmap.get_info_gid(gid); + const MDSMap::mds_info_t& mds_info = get_fsmap().get_info_gid(gid); // FIXME: include filesystem name with rank here mdses[hostname->second].push_back(mds_info.rank); } @@ -2002,9 +2017,10 @@ int MDSMonitor::print_nodes(Formatter *f) * If a cluster is undersized (with respect to max_mds), then * attempt to find daemons to grow it. */ -bool MDSMonitor::maybe_expand_cluster(std::shared_ptr fs) +bool MDSMonitor::maybe_expand_cluster(std::shared_ptr &fs) { bool do_propose = false; + auto &pending = get_pending_fsmap_writeable(); if (fs->mds_map.test_flag(CEPH_MDSMAP_DOWN)) { return do_propose; @@ -2017,13 +2033,13 @@ bool MDSMonitor::maybe_expand_cluster(std::shared_ptr fs) while (fs->mds_map.is_in(mds)) { mds++; } - mds_gid_t newgid = pending_fsmap.find_replacement_for({fs->fscid, mds}, + mds_gid_t newgid = pending.find_replacement_for({fs->fscid, mds}, name, g_conf->mon_force_standby_active); if (newgid == MDS_GID_NONE) { break; } - const auto &new_info = pending_fsmap.get_info_gid(newgid); + const auto &new_info = pending.get_info_gid(newgid); dout(1) << "assigned standby " << new_info.addr << " as mds." << mds << dendl; @@ -2031,7 +2047,7 @@ bool MDSMonitor::maybe_expand_cluster(std::shared_ptr fs) "filesystem " << fs->mds_map.fs_name << " as rank " << mds << " (now has " << fs->mds_map.get_num_in_mds() + 1 << " ranks)"; - pending_fsmap.promote(newgid, fs, mds); + pending.promote(newgid, fs, mds); do_propose = true; } @@ -2050,7 +2066,8 @@ void MDSMonitor::maybe_replace_gid(mds_gid_t gid, const MDSMap::mds_info_t& info assert(mds_propose != nullptr); assert(osd_propose != nullptr); - const auto fscid = pending_fsmap.mds_roles.at(gid); + auto &pending = get_pending_fsmap_writeable(); + const auto fscid = pending.mds_roles.at(gid); // We will only take decisive action (replacing/removing a daemon) // if we have some indicating that some other daemon(s) are successfully @@ -2070,12 +2087,12 @@ void MDSMonitor::maybe_replace_gid(mds_gid_t gid, const MDSMap::mds_info_t& info info.state != MDSMap::STATE_STANDBY && info.state != MDSMap::STATE_STANDBY_REPLAY && may_replace && - !pending_fsmap.get_filesystem(fscid)->mds_map.test_flag(CEPH_MDSMAP_DOWN) && - (sgid = pending_fsmap.find_replacement_for({fscid, info.rank}, info.name, + !pending.get_filesystem(fscid)->mds_map.test_flag(CEPH_MDSMAP_DOWN) && + (sgid = pending.find_replacement_for({fscid, info.rank}, info.name, g_conf->mon_force_standby_active)) != MDS_GID_NONE) { - MDSMap::mds_info_t si = pending_fsmap.get_info_gid(sgid); + MDSMap::mds_info_t si = pending.get_info_gid(sgid); dout(10) << " replacing " << gid << " " << info.addr << " mds." << info.rank << "." << info.inc << " " << ceph_mds_state_name(info.state) @@ -2087,14 +2104,14 @@ void MDSMonitor::maybe_replace_gid(mds_gid_t gid, const MDSMap::mds_info_t& info << " with standby " << si.human_name(); // Remember what NS the old one was in - const fs_cluster_id_t fscid = pending_fsmap.mds_roles.at(gid); + const fs_cluster_id_t fscid = pending.mds_roles.at(gid); // Remove the old one *osd_propose |= fail_mds_gid(gid); // Promote the replacement - auto fs = pending_fsmap.filesystems.at(fscid); - pending_fsmap.promote(sgid, fs, info.rank); + auto fs = pending.filesystems.at(fscid); + pending.promote(sgid, fs, info.rank); *mds_propose = true; } else if ((info.state == MDSMap::STATE_STANDBY_REPLAY || @@ -2110,17 +2127,19 @@ void MDSMonitor::maybe_replace_gid(mds_gid_t gid, const MDSMap::mds_info_t& info dout(10) << " marking " << gid << " " << info.addr << " mds." << info.rank << "." << info.inc << " " << ceph_mds_state_name(info.state) << " laggy" << dendl; - pending_fsmap.modify_daemon(info.global_id, [](MDSMap::mds_info_t *info) { + pending.modify_daemon(info.global_id, [](MDSMap::mds_info_t *info) { info->laggy_since = ceph_clock_now(); }); *mds_propose = true; } } -bool MDSMonitor::maybe_promote_standby(std::shared_ptr fs) +bool MDSMonitor::maybe_promote_standby(std::shared_ptr &fs) { assert(!fs->mds_map.test_flag(CEPH_MDSMAP_DOWN)); + auto &pending = get_pending_fsmap_writeable(); + bool do_propose = false; // have a standby take over? @@ -2130,17 +2149,17 @@ bool MDSMonitor::maybe_promote_standby(std::shared_ptr fs) set::iterator p = failed.begin(); while (p != failed.end()) { mds_rank_t f = *p++; - mds_gid_t sgid = pending_fsmap.find_replacement_for({fs->fscid, f}, {}, + mds_gid_t sgid = pending.find_replacement_for({fs->fscid, f}, {}, g_conf->mon_force_standby_active); if (sgid) { - const MDSMap::mds_info_t si = pending_fsmap.get_info_gid(sgid); + const MDSMap::mds_info_t si = pending.get_info_gid(sgid); dout(0) << " taking over failed mds." << f << " with " << sgid << "/" << si.name << " " << si.addr << dendl; mon->clog->info() << "Standby " << si.human_name() << " assigned to filesystem " << fs->mds_map.fs_name << " as rank " << f; - pending_fsmap.promote(sgid, fs, f); + pending.promote(sgid, fs, f); do_propose = true; } } @@ -2152,12 +2171,12 @@ bool MDSMonitor::maybe_promote_standby(std::shared_ptr fs) // them while perhaps-modifying standby_daemons during the loop // (if we promote anyone they are removed from standby_daemons) std::vector standby_gids; - for (const auto &j : pending_fsmap.standby_daemons) { + for (const auto &j : pending.standby_daemons) { standby_gids.push_back(j.first); } for (const auto &gid : standby_gids) { - const auto &info = pending_fsmap.standby_daemons.at(gid); + const auto &info = pending.standby_daemons.at(gid); assert(info.state == MDSMap::STATE_STANDBY); if (!info.standby_replay) { @@ -2176,14 +2195,14 @@ bool MDSMonitor::maybe_promote_standby(std::shared_ptr fs) // the standby_for_rank refers to: lookup via legacy_client_fscid mds_role_t target_role = { info.standby_for_fscid == FS_CLUSTER_ID_NONE ? - pending_fsmap.legacy_client_fscid : info.standby_for_fscid, + pending.legacy_client_fscid : info.standby_for_fscid, info.standby_for_rank}; // It is possible that the map contains a standby_for_fscid // that doesn't correspond to an existing filesystem, especially // if we loaded from a version with a bug (#17466) if (info.standby_for_fscid != FS_CLUSTER_ID_NONE - && !pending_fsmap.filesystem_exists(info.standby_for_fscid)) { + && !pending.filesystem_exists(info.standby_for_fscid)) { derr << "gid " << gid << " has invalid standby_for_fscid " << info.standby_for_fscid << dendl; continue; @@ -2191,7 +2210,7 @@ bool MDSMonitor::maybe_promote_standby(std::shared_ptr fs) // If we managed to resolve a full target role if (target_role.fscid != FS_CLUSTER_ID_NONE) { - auto fs = pending_fsmap.get_filesystem(target_role.fscid); + const auto &fs = pending.get_filesystem(target_role.fscid); if (fs->mds_map.is_followable(target_role.rank)) { do_propose |= try_standby_replay( info, @@ -2204,17 +2223,18 @@ bool MDSMonitor::maybe_promote_standby(std::shared_ptr fs) } // check everyone - for (auto fs_i : pending_fsmap.filesystems) { - const MDSMap &mds_map = fs_i.second->mds_map; - for (auto mds_i : mds_map.mds_info) { - MDSMap::mds_info_t &cand_info = mds_i.second; + for (const auto &p : pending.filesystems) { + const auto &fs = p.second; + const MDSMap &mds_map = fs->mds_map; + for (const auto &mds_i : mds_map.mds_info) { + const MDSMap::mds_info_t &cand_info = mds_i.second; if (cand_info.rank >= 0 && mds_map.is_followable(cand_info.rank)) { if ((info.standby_for_name.length() && info.standby_for_name != cand_info.name) || info.standby_for_rank != MDS_RANK_NONE) { continue; // we're supposed to follow someone else } - if (try_standby_replay(info, *(fs_i.second), cand_info)) { + if (try_standby_replay(info, *fs, cand_info)) { do_propose = true; break; } @@ -2232,19 +2252,22 @@ void MDSMonitor::tick() { // make sure mds's are still alive // ...if i am an active leader + if (!is_active()) return; - dout(10) << fsmap << dendl; + dout(10) << get_working_fsmap() << dendl; - bool do_propose = false; + if (!is_leader()) return; + + auto &pending = get_pending_fsmap_writeable(); - if (!mon->is_leader()) return; + bool do_propose = false; - do_propose |= pending_fsmap.check_health(); + do_propose |= pending.check_health(); // expand mds cluster (add new nodes to @in)? - for (auto i : pending_fsmap.filesystems) { - do_propose |= maybe_expand_cluster(i.second); + for (auto &p : pending.filesystems) { + do_propose |= maybe_expand_cluster(p.second); } const auto now = ceph_clock_now(); @@ -2270,7 +2293,7 @@ void MDSMonitor::tick() cutoff -= g_conf->mds_beacon_grace; // make sure last_beacon is fully populated - for (const auto &p : pending_fsmap.mds_roles) { + for (auto &p : pending.mds_roles) { auto &gid = p.first; if (last_beacon.count(gid) == 0) { last_beacon[gid].stamp = now; @@ -2286,14 +2309,14 @@ void MDSMonitor::tick() auto beacon_info = p->second; ++p; - if (!pending_fsmap.gid_exists(gid)) { + if (!pending.gid_exists(gid)) { // clean it out last_beacon.erase(gid); continue; } if (beacon_info.stamp < cutoff) { - auto &info = pending_fsmap.get_info_gid(gid); + auto &info = pending.get_info_gid(gid); dout(1) << "no beacon from mds." << info.rank << "." << info.inc << " (gid: " << gid << " addr: " << info.addr << " state: " << ceph_mds_state_name(info.state) << ")" @@ -2309,8 +2332,8 @@ void MDSMonitor::tick() request_proposal(mon->osdmon()); } - for (auto i : pending_fsmap.filesystems) { - auto fs = i.second; + for (auto &p : pending.filesystems) { + auto &fs = p.second; if (!fs->mds_map.test_flag(CEPH_MDSMAP_DOWN)) { do_propose |= maybe_promote_standby(fs); } @@ -2338,7 +2361,7 @@ bool MDSMonitor::try_standby_replay( } else { // Assign the new role to the standby dout(10) << " setting to follow mds rank " << ainfo.rank << dendl; - pending_fsmap.assign_standby_replay(finfo.global_id, leader_fs.fscid, ainfo.rank); + get_pending_fsmap_writeable().assign_standby_replay(finfo.global_id, leader_fs.fscid, ainfo.rank); return true; } } diff --git a/src/mon/MDSMonitor.h b/src/mon/MDSMonitor.h index 3d84f92a811f..88e9decf266c 100644 --- a/src/mon/MDSMonitor.h +++ b/src/mon/MDSMonitor.h @@ -23,8 +23,7 @@ using namespace std; #include "include/types.h" -#include "mds/FSMap.h" -#include "mds/MDSMap.h" +#include "PaxosFSMap.h" #include "PaxosService.h" #include "msg/Messenger.h" #include "messages/MMDSBeacon.h" @@ -34,7 +33,7 @@ class MMDSLoadTargets; class MMDSMap; class FileSystemCommandHandler; -class MDSMonitor : public PaxosService { +class MDSMonitor : public PaxosService, public PaxosFSMap { public: MDSMonitor(Monitor *mn, Paxos *p, string service_name); @@ -59,8 +58,6 @@ class MDSMonitor : public PaxosService { void check_subs(); void check_sub(Subscription *sub); - const FSMap &get_pending() const { return pending_fsmap; } - const FSMap &get_fsmap() const { return fsmap; } void dump_info(Formatter *f); int print_nodes(Formatter *f); @@ -68,13 +65,12 @@ class MDSMonitor : public PaxosService { * Return true if a blacklist was done (i.e. OSD propose needed) */ bool fail_mds_gid(mds_gid_t gid); - protected: - // mds maps - FSMap fsmap; // current - FSMap pending_fsmap; // current + pending updates + bool is_leader() const override { return mon->is_leader(); } + + protected: // my helpers - void print_map(FSMap &m, int dbl=7); + void print_map(const FSMap &m, int dbl=7); void update_logger(); void _updated(MonOpRequestRef op); @@ -127,8 +123,8 @@ class MDSMonitor : public PaxosService { std::list > handlers; - bool maybe_promote_standby(std::shared_ptr fs); - bool maybe_expand_cluster(std::shared_ptr fs); + bool maybe_promote_standby(std::shared_ptr &fs); + bool maybe_expand_cluster(std::shared_ptr &fs); void maybe_replace_gid(mds_gid_t gid, const MDSMap::mds_info_t& info, bool *mds_propose, bool *osd_propose); void tick() override; // check state, take actions diff --git a/src/mon/OSDMonitor.cc b/src/mon/OSDMonitor.cc index 17921fd077e2..adce63713d91 100644 --- a/src/mon/OSDMonitor.cc +++ b/src/mon/OSDMonitor.cc @@ -1240,7 +1240,7 @@ void OSDMonitor::encode_pending(MonitorDBStore::TransactionRef t) uint32_t match_count = 0; // CephFS - FSMap const &pending_fsmap = mon->mdsmon()->get_pending(); + const FSMap &pending_fsmap = mon->mdsmon()->get_pending_fsmap(); if (pending_fsmap.pool_in_use(pool_id)) { dout(10) << __func__ << " auto-enabling CephFS on pool '" << pool_name << "'" << dendl; @@ -11744,7 +11744,7 @@ int OSDMonitor::_check_remove_pool(int64_t pool_id, const pg_pool_t& pool, const string& poolstr = osdmap.get_pool_name(pool_id); // If the Pool is in use by CephFS, refuse to delete it - FSMap const &pending_fsmap = mon->mdsmon()->get_pending(); + FSMap const &pending_fsmap = mon->mdsmon()->get_pending_fsmap(); if (pending_fsmap.pool_in_use(pool_id)) { *ss << "pool '" << poolstr << "' is in use by CephFS"; return -EBUSY; @@ -11793,7 +11793,7 @@ bool OSDMonitor::_check_become_tier( const std::string &tier_pool_name = osdmap.get_pool_name(tier_pool_id); const std::string &base_pool_name = osdmap.get_pool_name(base_pool_id); - const FSMap &pending_fsmap = mon->mdsmon()->get_pending(); + const FSMap &pending_fsmap = mon->mdsmon()->get_pending_fsmap(); if (pending_fsmap.pool_in_use(tier_pool_id)) { *ss << "pool '" << tier_pool_name << "' is in use by CephFS"; *err = -EBUSY; @@ -11853,7 +11853,7 @@ bool OSDMonitor::_check_remove_tier( const std::string &base_pool_name = osdmap.get_pool_name(base_pool_id); // Apply CephFS-specific checks - const FSMap &pending_fsmap = mon->mdsmon()->get_pending(); + const FSMap &pending_fsmap = mon->mdsmon()->get_pending_fsmap(); if (pending_fsmap.pool_in_use(base_pool_id)) { if (base_pool->is_erasure() && !base_pool->allows_ecoverwrites()) { // If the underlying pool is erasure coded and does not allow EC diff --git a/src/mon/PaxosFSMap.h b/src/mon/PaxosFSMap.h new file mode 100644 index 000000000000..8d7c8c1f1cc7 --- /dev/null +++ b/src/mon/PaxosFSMap.h @@ -0,0 +1,63 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab +/* + * Ceph - scalable distributed file system + * + * Copyright (C) 2004-2006 Sage Weil + * + * This is free software; you can redistribute it and/or + * modify it under the terms of the GNU Lesser General Public + * License version 2.1, as published by the Free Software + * Foundation. See file COPYING. + * + */ + +#ifndef CEPH_PAXOS_FSMAP_H +#define CEPH_PAXOS_FSMAP_H + +#include "mds/FSMap.h" +#include "mds/MDSMap.h" + +#include "include/assert.h" + +class PaxosFSMap { +public: + virtual ~PaxosFSMap() {} + + const FSMap &get_pending_fsmap() const { assert(is_leader()); return pending_fsmap; } + const FSMap &get_fsmap() const { return fsmap; } + + virtual bool is_leader() const = 0; + +protected: + FSMap &get_pending_fsmap_writeable() { assert(is_leader()); return pending_fsmap; } + + /* get_working_fsmap returns the "relevant" version of the fsmap (see MDSMonitor.cc history) + * used depending in helper methods of MDSMonitor.cc. + * + * This is technically evil and will be removed in the future. + * + * See discussion: https://github.com/ceph/ceph/pull/21458#discussion_r182081366 + */ + const FSMap &get_working_fsmap() const { return is_leader() ? pending_fsmap : fsmap; } + + FSMap &create_pending() { + assert(is_leader()); + pending_fsmap = fsmap; + pending_fsmap.epoch++; + return pending_fsmap; + } + + void decode(bufferlist &bl) { + fsmap.decode(bl); + pending_fsmap = FSMap(); /* nuke it to catch invalid access */ + } + +private: + /* Keep these PRIVATE to prevent unprotected manipulation. */ + FSMap fsmap; /* the current epoch */ + FSMap pending_fsmap; /* the next epoch */ +}; + + +#endif -- 2.47.3