]> git-server-git.apps.pok.os.sepia.ceph.com Git - ceph.git/commitdiff
MDSMonitor: cleanup and protect fsmap access 21458/head
authorPatrick Donnelly <pdonnell@redhat.com>
Tue, 17 Apr 2018 03:46:39 +0000 (20:46 -0700)
committerPatrick Donnelly <pdonnell@redhat.com>
Tue, 17 Apr 2018 17:56:52 +0000 (10:56 -0700)
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 <pdonnell@redhat.com>
src/mds/FSMap.h
src/messages/MMDSMap.h
src/mon/MDSMonitor.cc
src/mon/MDSMonitor.h
src/mon/OSDMonitor.cc
src/mon/PaxosFSMap.h [new file with mode: 0644]

index 36f8a6f796e4eda6c47cf9beec608963ca674788..83a85e6f9ed6ebb625999447cec96a6d48332430 100644 (file)
@@ -103,6 +103,7 @@ protected:
 public:
 
   friend class MDSMonitor;
+  friend class PaxosFSMap;
 
   FSMap() 
     : epoch(0),
index 4f254d42d57d73779a93683bf39e4bd98df4c9bc..b34c7719dc42f7a424e1061f1227da71789720de 100644 (file)
@@ -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();
index 14c5e10511825716d609cecce49e30649e466d72..a00fb9721f0c3f26f08c386085a71d0bd972f037 100644 (file)
@@ -41,8 +41,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
 
 template <int dblV>
-void MDSMonitor::print_map(FSMap &m)
+void MDSMonitor::print_map(const FSMap& m)
 {
   dout(dblV) << "print_map\n";
   m.print(*_dout);
@@ -100,12 +100,12 @@ void MDSMonitor::get_store_prefixes(std::set<string>& s) const
 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<0>(fsmap);
+  print_map<0>(get_fsmap());
   if (!g_conf->mon_mds_skip_sanity) {
-    fsmap.sanity();
+    get_fsmap().sanity();
   }
 
   check_subs();
@@ -136,43 +136,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<30>(pending_fsmap);
+  print_map<30>(pending);
   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<uint64_t, MDSHealth>::iterator i = pending_daemon_health.begin();
@@ -191,7 +192,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;
@@ -231,7 +232,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 = std::regex_replace(
       p.second.summary,
@@ -314,6 +315,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();
   if (!session)
@@ -347,11 +350,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;
@@ -366,7 +369,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) {
@@ -376,11 +379,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
@@ -450,6 +453,8 @@ bool MDSMonitor::preprocess_offload_targets(MonOpRequestRef op)
   op->mark_mdsmon_event(__func__);
   MMDSLoadTargets *m = static_cast<MMDSLoadTargets*>(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();
@@ -507,6 +512,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
@@ -556,13 +563,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;
@@ -574,7 +581,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();
@@ -586,19 +593,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;
@@ -611,22 +618,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();
@@ -643,7 +650,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();
         }
@@ -656,15 +663,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) {
@@ -693,14 +700,14 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
       until += g_conf->get_val<double>("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()) {
@@ -718,7 +725,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
@@ -736,8 +743,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;
@@ -745,7 +752,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;
       });
@@ -753,7 +760,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)
@@ -770,12 +777,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<MMDSLoadTargets*>(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;
   }
@@ -790,6 +799,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<MMDSBeacon*>(op->get_req());
   dout(10) << "_updated " << m->get_orig_source() << " " << *m << dendl;
@@ -817,15 +827,15 @@ void MDSMonitor::on_active()
 {
   tick();
 
-  if (mon->is_leader()) {
-    mon->clog->debug() << "fsmap " << fsmap;
+  if (is_leader()) {
+    mon->clog->debug() << "fsmap " << get_fsmap();
   }
 }
 
 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());
@@ -840,6 +850,8 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
   bufferlist rdata;
   stringstream ss, ds;
 
+  const auto &fsmap = get_working_fsmap();
+
   cmdmap_t cmdmap;
   if (!cmdmap_from_json(m->cmd, &cmdmap, ss)) {
     // ss has reason for failure
@@ -874,40 +886,37 @@ 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;
-      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"));
@@ -976,7 +985,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;
@@ -996,8 +1005,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;
@@ -1034,8 +1043,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);
@@ -1056,6 +1065,7 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
     r = 0;
   }
 
+out:
   if (r != -1) {
     rdata.append(ds);
     string rs;
@@ -1068,7 +1078,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;
@@ -1078,7 +1090,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);
@@ -1090,7 +1102,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;
@@ -1098,7 +1110,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
@@ -1112,7 +1124,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";
@@ -1126,7 +1138,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);
     }
   }
@@ -1151,7 +1163,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;
@@ -1185,14 +1197,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();
       }
@@ -1204,7 +1218,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;
@@ -1224,8 +1238,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";
@@ -1274,11 +1288,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(
@@ -1293,13 +1303,15 @@ int MDSMonitor::filesystem_command(
   string whostr;
   cmd_getval(g_ceph_context, cmdmap, "role", whostr);
 
+  auto &pending = get_pending_fsmap_writeable();
+
   if (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;
@@ -1322,9 +1334,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;
       });
     }
@@ -1341,8 +1353,8 @@ int MDSMonitor::filesystem_command(
          << cmd_vartype_stringify(cmdmap.at("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 << " "
@@ -1372,17 +1384,18 @@ int MDSMonitor::filesystem_command(
          << cmd_vartype_stringify(cmdmap.at("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;
       }
@@ -1405,7 +1418,7 @@ int MDSMonitor::filesystem_command(
       return -EINVAL;
     }
 
-    pending_fsmap.modify_filesystem(
+    pending.modify_filesystem(
         role.fscid,
         [role](std::shared_ptr<Filesystem> fs)
     {
@@ -1421,13 +1434,13 @@ int MDSMonitor::filesystem_command(
          << cmd_vartype_stringify(cmdmap.at("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") {
@@ -1437,13 +1450,13 @@ int MDSMonitor::filesystem_command(
          << cmd_vartype_stringify(cmdmap.at("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") {
@@ -1455,7 +1468,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 {
@@ -1481,8 +1494,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());
@@ -1505,6 +1518,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));
@@ -1519,12 +1534,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) {
@@ -1588,24 +1601,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);
@@ -1648,7 +1662,7 @@ void MDSMonitor::remove_from_metadata(MonitorDBStore::TransactionRef t)
   bool update = false;
   for (map<mds_gid_t, Metadata>::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 {
@@ -1672,7 +1686,7 @@ int MDSMonitor::load_metadata(map<mds_gid_t, Metadata>& m)
   }
 
   bufferlist::iterator it = bl.begin();
-  decode(m, it);
+  ceph::decode(m, it);
   return 0;
 }
 
@@ -1745,11 +1759,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);
     mdses[hostname->second].push_back(mds_info.name);
   }
 
@@ -1761,9 +1775,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<Filesystem> fs)
+bool MDSMonitor::maybe_expand_cluster(std::shared_ptr<Filesystem> &fs)
 {
   bool do_propose = false;
+  auto &pending = get_pending_fsmap_writeable();
 
   if (fs->mds_map.test_flag(CEPH_MDSMAP_DOWN)) {
     return do_propose;
@@ -1776,13 +1791,13 @@ bool MDSMonitor::maybe_expand_cluster(std::shared_ptr<Filesystem> 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;
 
@@ -1790,7 +1805,7 @@ bool MDSMonitor::maybe_expand_cluster(std::shared_ptr<Filesystem> 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;
   }
 
@@ -1809,7 +1824,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
@@ -1829,12 +1845,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)
@@ -1846,14 +1862,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 ||
@@ -1869,17 +1885,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<Filesystem> fs)
+bool MDSMonitor::maybe_promote_standby(std::shared_ptr<Filesystem> &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?
@@ -1889,17 +1907,17 @@ bool MDSMonitor::maybe_promote_standby(std::shared_ptr<Filesystem> fs)
     set<mds_rank_t>::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;
       }
     }
@@ -1911,12 +1929,12 @@ bool MDSMonitor::maybe_promote_standby(std::shared_ptr<Filesystem> fs)
     // them while perhaps-modifying standby_daemons during the loop
     // (if we promote anyone they are removed from standby_daemons)
     std::vector<mds_gid_t> 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) {
@@ -1935,14 +1953,14 @@ bool MDSMonitor::maybe_promote_standby(std::shared_ptr<Filesystem> 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;
@@ -1950,7 +1968,7 @@ bool MDSMonitor::maybe_promote_standby(std::shared_ptr<Filesystem> 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,
@@ -1963,17 +1981,18 @@ bool MDSMonitor::maybe_promote_standby(std::shared_ptr<Filesystem> 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;
             }
@@ -1991,19 +2010,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;
 
-  if (!mon->is_leader()) return;
+  auto &pending = get_pending_fsmap_writeable();
+
+  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();
@@ -2029,7 +2051,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;
@@ -2045,14 +2067,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) << ")"
@@ -2068,8 +2090,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);
     }
@@ -2097,7 +2119,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;
   }
 }
index f503b225b9e50cfd111ebddc7c7cca3f7ba993c2..d02af4b2d94fef3e8c8a9b932d35e4928a5d383d 100644 (file)
@@ -22,8 +22,7 @@
 #include <set>
 
 #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"
@@ -33,7 +32,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);
 
@@ -58,8 +57,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);
 
@@ -67,14 +64,13 @@ 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
   template<int dblV = 7>
-  void print_map(FSMap &m);
+  void print_map(const FSMap &m);
 
   void _updated(MonOpRequestRef op);
 
@@ -116,8 +112,8 @@ class MDSMonitor : public PaxosService {
 
   std::list<std::shared_ptr<FileSystemCommandHandler> > handlers;
 
-  bool maybe_promote_standby(std::shared_ptr<Filesystem> fs);
-  bool maybe_expand_cluster(std::shared_ptr<Filesystem> fs);
+  bool maybe_promote_standby(std::shared_ptr<Filesystem> &fs);
+  bool maybe_expand_cluster(std::shared_ptr<Filesystem> &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
index c30f1e34049f07898c6f957458f0f1feb9d7ddf0..a2b55534bc7227c2c429274c627e3a4becf43c6c 100644 (file)
@@ -11858,7 +11858,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;
@@ -11907,7 +11907,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;
@@ -11967,7 +11967,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 (file)
index 0000000..8d7c8c1
--- /dev/null
@@ -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 <sage@newdream.net>
+ *
+ * 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