]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/mon/MDSMonitor.cc
update ceph source to reef 18.2.0
[ceph.git] / ceph / src / mon / MDSMonitor.cc
index 40cb07127ac339fd0885cbc7796d2e28013ffb95..091206a684a9194b78a93d90febeaa57a24f647d 100644 (file)
@@ -86,19 +86,19 @@ static const string MDS_HEALTH_PREFIX("mds_health");
  */
 namespace TOPNSPC::common {
 template<> bool cmd_getval(const cmdmap_t& cmdmap,
-                          const std::string& k, mds_gid_t &val)
+                          std::string_view k, mds_gid_t &val)
 {
   return cmd_getval(cmdmap, k, (int64_t&)val);
 }
 
 template<> bool cmd_getval(const cmdmap_t& cmdmap,
-                          const std::string& k, mds_rank_t &val)
+                          std::string_view k, mds_rank_t &val)
 {
   return cmd_getval(cmdmap, k, (int64_t&)val);
 }
 
 template<> bool cmd_getval(const cmdmap_t& cmdmap,
-                          const std::string& k, MDSMap::DaemonState &val)
+                          std::string_view k, MDSMap::DaemonState &val)
 {
   return cmd_getval(cmdmap, k, (int64_t&)val);
 }
@@ -146,7 +146,12 @@ void MDSMonitor::update_from_paxos(bool *need_bootstrap)
 
   ceph_assert(fsmap_bl.length() > 0);
   dout(10) << __func__ << " got " << version << dendl;
-  PaxosFSMap::decode(fsmap_bl);
+  try {
+    PaxosFSMap::decode(fsmap_bl);
+  } catch (const ceph::buffer::malformed_input& e) {
+    derr << "unable to decode FSMap: " << e.what() << dendl;
+    throw;
+  }
 
   // new map
   dout(0) << "new map" << dendl;
@@ -185,7 +190,7 @@ void MDSMonitor::encode_pending(MonitorDBStore::TransactionRef t)
   // print map iff 'debug mon = 30' or higher
   print_map<30>(pending);
   if (!g_conf()->mon_mds_skip_sanity) {
-    pending.sanity();
+    pending.sanity(true);
   }
 
   // Set 'modified' on maps modified this epoch
@@ -243,6 +248,9 @@ void MDSMonitor::encode_pending(MonitorDBStore::TransactionRef t)
       health.decode(bl_i);
     }
     for (const auto &metric : health.metrics) {
+      if (metric.type == MDS_HEALTH_DUMMY) {
+        continue;
+      }
       const auto rank = info.rank;
       health_check_t *check = &new_checks.get_or_add(
        mds_metric_name(metric.type),
@@ -290,7 +298,7 @@ version_t MDSMonitor::get_trim_to() const
 {
   version_t floor = 0;
   if (g_conf()->mon_mds_force_trim_to > 0 &&
-      g_conf()->mon_mds_force_trim_to < (int)get_last_committed()) {
+      g_conf()->mon_mds_force_trim_to <= (int)get_last_committed()) {
     floor = g_conf()->mon_mds_force_trim_to;
     dout(10) << __func__ << " explicit mon_mds_force_trim_to = "
              << floor << dendl;
@@ -299,8 +307,11 @@ version_t MDSMonitor::get_trim_to() const
   unsigned max = g_conf()->mon_max_mdsmap_epochs;
   version_t last = get_last_committed();
 
-  if (last - get_first_committed() > max && floor < last - max)
-    return last - max;
+  if (last - get_first_committed() > max && floor < last - max) {
+    floor = last-max;
+  }
+
+  dout(20) << __func__ << " = " << floor << dendl;
   return floor;
 }
 
@@ -384,14 +395,6 @@ bool MDSMonitor::preprocess_beacon(MonOpRequestRef op)
     goto ignore;
   }
 
-  // check compat
-  if (!m->get_compat().writeable(fsmap.compat)) {
-    dout(1) << " mds " << m->get_orig_source()
-           << " " << m->get_orig_source_addrs()
-           << " can't write to fsmap " << fsmap.compat << dendl;
-    goto ignore;
-  }
-
   // fw to leader?
   if (!is_leader())
     return false;
@@ -406,14 +409,18 @@ bool MDSMonitor::preprocess_beacon(MonOpRequestRef op)
        * know which FS it was part of. Nor does this matter. Sending an empty
        * MDSMap is sufficient for getting the MDS to respawn.
        */
-      MDSMap null_map;
-      null_map.epoch = fsmap.epoch;
-      null_map.compat = fsmap.compat;
-      auto m = make_message<MMDSMap>(mon.monmap->fsid, null_map);
+      auto m = make_message<MMDSMap>(mon.monmap->fsid, MDSMap::create_null_mdsmap());
       mon.send_reply(op, m.detach());
       return true;
     } else {
-      return false;  // not booted yet.
+      /* check if we've already recorded its entry in pending */
+      const auto& pending = get_pending_fsmap();
+      if (pending.gid_exists(gid)) {
+        /* MDS is already booted. */
+        goto ignore;
+      } else {
+        return false;  // not booted yet.
+      }
     }
   }
   dout(10) << __func__ << ": GID exists in map: " << gid << dendl;
@@ -476,23 +483,6 @@ bool MDSMonitor::preprocess_beacon(MonOpRequestRef op)
 
   // is there a state change here?
   if (info.state != state) {
-    // legal state change?
-    if ((info.state == MDSMap::STATE_STANDBY ||
-        info.state == MDSMap::STATE_STANDBY_REPLAY) && state > 0) {
-      dout(10) << "mds_beacon mds can't activate itself (" << ceph_mds_state_name(info.state)
-              << " -> " << ceph_mds_state_name(state) << ")" << dendl;
-      goto reply;
-    }
-
-    if ((state == MDSMap::STATE_STANDBY || state == MDSMap::STATE_STANDBY_REPLAY)
-        && info.rank != MDS_RANK_NONE)
-    {
-      dout(4) << "mds_beacon MDS can't go back into standby after taking rank: "
-                 "held rank " << info.rank << " while requesting state "
-              << ceph_mds_state_name(state) << dendl;
-      goto reply;
-    }
-    
     _note_beacon(m);
     return false;
   }
@@ -571,7 +561,7 @@ bool MDSMonitor::prepare_update(MonOpRequestRef op)
     } catch (const bad_cmd_get& e) {
       bufferlist bl;
       mon.reply_command(op, -EINVAL, e.what(), bl, get_last_committed());
-      return true;
+      return false; /* nothing to propose */
     }
 
   case MSG_MDS_OFFLOAD_TARGETS:
@@ -581,7 +571,7 @@ bool MDSMonitor::prepare_update(MonOpRequestRef op)
     ceph_abort();
   }
 
-  return true;
+  return false; /* nothing to propose! */
 }
 
 bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
@@ -613,10 +603,16 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
 
   std::set<mds_metric_t> new_types;
   for (const auto &i : new_health) {
+    if (i.type == MDS_HEALTH_DUMMY) {
+      continue;
+    }
     new_types.insert(i.type);
   }
 
   for (const auto &new_metric: new_health) {
+    if (new_metric.type == MDS_HEALTH_DUMMY) {
+      continue;
+    }
     if (old_types.count(new_metric.type) == 0) {
       dout(10) << "MDS health message (" << m->get_orig_source()
               << "): " << new_metric.sev << " " << new_metric.message << dendl;
@@ -634,7 +630,7 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
   // Store health
   pending_daemon_health[gid] = m->get_health();
 
-  // boot?
+  const auto& cs = m->get_compat();
   if (state == MDSMap::STATE_BOOT) {
     // zap previous instance of this name?
     if (g_conf()->mds_enforce_unique_name) {
@@ -644,8 +640,7 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
           mon.osdmon()->wait_for_writeable(op, new C_RetryMessage(this, op));
           return false;
         }
-        const MDSMap::mds_info_t &existing_info =
-          pending.get_info_gid(existing);
+        const auto& existing_info = pending.get_info_gid(existing);
         mon.clog->info() << existing_info.human_name() << " restarted";
        fail_mds_gid(pending, existing);
         failed_mds = true;
@@ -665,7 +660,7 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
       new_info.mds_features = m->get_mds_features();
       new_info.state = MDSMap::STATE_STANDBY;
       new_info.state_seq = seq;
-      pending.insert(new_info);
+      new_info.compat = cs;
       if (m->get_fs().size()) {
        fs_cluster_id_t fscid = FS_CLUSTER_ID_NONE;
        auto f = pending.get_filesystem(m->get_fs());
@@ -674,6 +669,7 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
        }
         new_info.join_fscid = fscid;
       }
+      pending.insert(new_info);
     }
 
     // initialize the beacon timer
@@ -681,15 +677,6 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
     beacon.stamp = mono_clock::now();
     beacon.seq = seq;
 
-    // new incompat?
-    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.update_compat(m->get_compat());
-    }
-
     update_metadata(m->get_global_id(), m->get_sys_info());
   } else {
     // state update
@@ -703,32 +690,47 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
        * know which FS it was part of. Nor does this matter. Sending an empty
        * MDSMap is sufficient for getting the MDS to respawn.
        */
-      wait_for_finished_proposal(op, new LambdaContext([op, this](int r){
-        if (r >= 0) {
-          const auto& fsmap = get_fsmap();
-          MDSMap null_map;
-          null_map.epoch = fsmap.epoch;
-          null_map.compat = fsmap.compat;
-          auto m = make_message<MMDSMap>(mon.monmap->fsid, null_map);
-          mon.send_reply(op, m.detach());
-        } else {
-          dispatch(op);        // try again
-        }
-      }));
-      return true;
+      goto null;
     }
 
     const auto& info = pending.get_info_gid(gid);
-    if (info.state == MDSMap::STATE_STOPPING &&
-        state != MDSMap::STATE_STOPPING &&
-        state != MDSMap::STATE_STOPPED) {
-      // we can't transition to any other states from STOPPING
-      dout(0) << "got beacon for MDS in STATE_STOPPING, ignoring requested state change"
-              << dendl;
-      _note_beacon(m);
+
+    // did the reported compat change? That's illegal!
+    if (cs.compare(info.compat) != 0) {
+      if (!mon.osdmon()->is_writeable()) {
+        mon.osdmon()->wait_for_writeable(op, new C_RetryMessage(this, op));
+        return false;
+      }
+      mon.clog->warn() << info.human_name() << " compat changed unexpectedly";
+      fail_mds_gid(pending, gid);
+      request_proposal(mon.osdmon());
       return true;
     }
 
+    if (state == MDSMap::STATE_DNE) {
+      dout(1) << __func__ << ": DNE from " << info << dendl;
+      goto evict;
+    }
+
+    // legal state change?
+    if ((info.state == MDSMap::STATE_STANDBY && state != info.state) ||
+        (info.state == MDSMap::STATE_STANDBY_REPLAY && state != info.state && state != MDSMap::STATE_DAMAGED)) {
+      // Standby daemons should never modify their own state.
+      // Except that standby-replay can indicate the rank is damaged due to failure to replay.
+      // Reject any attempts to do so.
+      derr << "standby " << gid << " attempted to change state to "
+           << ceph_mds_state_name(state) << ", rejecting" << dendl;
+      goto evict;
+    } else if (info.state != MDSMap::STATE_STANDBY && state != info.state &&
+               !MDSMap::state_transition_valid(info.state, state)) {
+      // Validate state transitions for daemons that hold a rank
+      derr << "daemon " << gid << " (rank " << info.rank << ") "
+           << "reported invalid state transition "
+           << ceph_mds_state_name(info.state) << " -> "
+           << ceph_mds_state_name(state) << dendl;
+      goto evict;
+    }
+
     if (info.laggy()) {
       dout(1) << "prepare_beacon clearing laggy flag on " << addrs << dendl;
       pending.modify_daemon(info.global_id, [](auto& info)
@@ -773,8 +775,6 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
           pending_daemon_health_rm.insert(erased_gid);
         }
       }
-
-
     } else if (state == MDSMap::STATE_DAMAGED) {
       if (!mon.osdmon()->is_writeable()) {
         dout(1) << __func__ << ": DAMAGED from rank " << info.rank
@@ -783,55 +783,31 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
         return false;
       }
 
+      auto rank = info.rank;
+
       // Record this MDS rank as damaged, so that other daemons
       // won't try to run it.
-      dout(0) << __func__ << ": marking rank "
-              << info.rank << " damaged" << dendl;
+      dout(0) << __func__ << ": marking rank " << rank << " damaged" << dendl;
+
+      auto fs = pending.get_filesystem(gid);
+      auto rankgid = fs->mds_map.get_gid(rank);
+      auto rankinfo = pending.get_info_gid(rankgid);
+      auto followergid = fs->mds_map.get_standby_replay(rank);
+
+      ceph_assert(gid == rankgid || gid == followergid);
 
       utime_t until = ceph_clock_now();
       until += g_conf().get_val<double>("mon_mds_blocklist_interval");
-      const auto blocklist_epoch = mon.osdmon()->blocklist(info.addrs, until);
-      request_proposal(mon.osdmon());
-      pending.damaged(gid, blocklist_epoch);
-      last_beacon.erase(gid);
-
-      // Respond to MDS, so that it knows it can continue to shut down
-      auto beacon = make_message<MMDSBeacon>(
-                       mon.monmap->fsid, m->get_global_id(),
-                       m->get_name(), pending.get_epoch(), state, seq,
-                       CEPH_FEATURES_SUPPORTED_DEFAULT);
-      mon.send_reply(op, beacon.detach());
-    } else if (state == MDSMap::STATE_DNE) {
-      if (!mon.osdmon()->is_writeable()) {
-        dout(1) << __func__ << ": DNE from rank " << info.rank
-                << " waiting for osdmon writeable to blocklist it" << dendl;
-        mon.osdmon()->wait_for_writeable(op, new C_RetryMessage(this, op));
-        return false;
+      const auto blocklist_epoch = mon.osdmon()->blocklist(rankinfo.addrs, until);
+      if (followergid != MDS_GID_NONE) {
+        fail_mds_gid(pending, followergid);
+        last_beacon.erase(followergid);
       }
-
-      fail_mds_gid(pending, gid);
-      ceph_assert(mon.osdmon()->is_writeable());
       request_proposal(mon.osdmon());
+      pending.damaged(rankgid, blocklist_epoch);
+      last_beacon.erase(rankgid);
 
-      // Respond to MDS, so that it knows it can continue to shut down
-      auto beacon = make_message<MMDSBeacon>(mon.monmap->fsid,
-          m->get_global_id(), m->get_name(), pending.get_epoch(), state, seq,
-          CEPH_FEATURES_SUPPORTED_DEFAULT);
-      mon.send_reply(op, beacon.detach());
-    } else if (info.state == MDSMap::STATE_STANDBY && state != info.state) {
-      // Standby daemons should never modify their own
-      // state.  Reject any attempts to do so.
-      derr << "standby " << gid << " attempted to change state to "
-           << ceph_mds_state_name(state) << ", rejecting" << dendl;
-      return true;
-    } else if (info.state != MDSMap::STATE_STANDBY && state != info.state &&
-               !MDSMap::state_transition_valid(info.state, state)) {
-      // Validate state transitions for daemons that hold a rank
-      derr << "daemon " << gid << " (rank " << info.rank << ") "
-           << "reported invalid state transition "
-           << ceph_mds_state_name(info.state) << " -> "
-           << ceph_mds_state_name(state) << dendl;
-      return true;
+      /* MDS expects beacon reply back */
     } else {
       if (info.state != MDSMap::STATE_ACTIVE && state == MDSMap::STATE_ACTIVE) {
         const auto &fscid = pending.mds_roles.at(gid);
@@ -863,12 +839,39 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
     }
   }));
 
+  return true;
+
+evict:
+  if (!mon.osdmon()->is_writeable()) {
+    dout(1) << __func__ << ": waiting for writeable OSDMap to evict" << dendl;
+    mon.osdmon()->wait_for_writeable(op, new C_RetryMessage(this, op));
+    return false;
+  }
+
+  fail_mds_gid(pending, gid);
+  request_proposal(mon.osdmon());
+  dout(5) << __func__ << ": pending map now:" << dendl;
+  print_map(pending);
+
+  goto null;
+
+null:
+  wait_for_finished_proposal(op, new LambdaContext([op, this](int r){
+    if (r >= 0) {
+      auto m = make_message<MMDSMap>(mon.monmap->fsid, MDSMap::create_null_mdsmap());
+      mon.send_reply(op, m.detach());
+    } else {
+      dispatch(op);        // try again
+    }
+  }));
+
   return true;
 }
 
 bool MDSMonitor::prepare_offload_targets(MonOpRequestRef op)
 {
   auto &pending = get_pending_fsmap_writeable();
+  bool propose = false;
 
   op->mark_mdsmon_event(__func__);
   auto m = op->get_req<MMDSLoadTargets>();
@@ -876,11 +879,12 @@ bool MDSMonitor::prepare_offload_targets(MonOpRequestRef op)
   if (pending.gid_has_rank(gid)) {
     dout(10) << "prepare_offload_targets " << gid << " " << m->targets << dendl;
     pending.update_export_targets(gid, m->targets);
+    propose = true;
   } else {
     dout(10) << "prepare_offload_targets " << gid << " not in map" << dendl;
   }
   mon.no_reply(op);
-  return true;
+  return propose;
 }
 
 bool MDSMonitor::should_propose(double& delay)
@@ -901,10 +905,7 @@ void MDSMonitor::_updated(MonOpRequestRef op)
 
   if (m->get_state() == MDSMap::STATE_STOPPED) {
     // send the map manually (they're out of the map, so they won't get it automatic)
-    MDSMap null_map;
-    null_map.epoch = fsmap.epoch;
-    null_map.compat = fsmap.compat;
-    auto m = make_message<MMDSMap>(mon.monmap->fsid, null_map);
+    auto m = make_message<MMDSMap>(mon.monmap->fsid, MDSMap::create_null_mdsmap());
     mon.send_reply(op, m.detach());
   } else {
     auto beacon = make_message<MMDSBeacon>(mon.monmap->fsid,
@@ -951,8 +952,7 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
 
   string prefix;
   cmd_getval(cmdmap, "prefix", prefix);
-  string format;
-  cmd_getval(cmdmap, "format", format, string("plain"));
+  string format = cmd_getval_or<string>(cmdmap, "format", "plain");
   std::unique_ptr<Formatter> f(Formatter::create(format));
 
   MonSession *session = op->get_session();
@@ -1117,14 +1117,33 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
     count_metadata(field, f.get());
     f->flush(ds);
     r = 0;
+  } else if (prefix == "fs compat show") {
+    string fs_name;
+    cmd_getval(cmdmap, "fs_name", fs_name);
+    const auto &fs = fsmap.get_filesystem(fs_name);
+    if (fs == nullptr) {
+      ss << "filesystem '" << fs_name << "' not found";
+      r = -ENOENT;
+      goto out;
+    }
+
+    if (f) {
+      f->open_object_section("mds_compat");
+      fs->mds_map.compat.dump(f.get());
+      f->close_section();
+      f->flush(ds);
+    } else {
+      ds << fs->mds_map.compat;
+    }
+    r = 0;
   } else if (prefix == "mds compat show") {
       if (f) {
        f->open_object_section("mds_compat");
-       fsmap.compat.dump(f.get());
+       fsmap.default_compat.dump(f.get());
        f->close_section();
        f->flush(ds);
       } else {
-       ds << fsmap.compat;
+       ds << fsmap.default_compat;
       }
       r = 0;
   } else if (prefix == "fs get") {
@@ -1215,6 +1234,24 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
       }
     }
     r = 0;
+  } else if (prefix == "fs lsflags") {
+    string fs_name;
+    cmd_getval(cmdmap, "fs_name", fs_name);
+    const auto &fs = fsmap.get_filesystem(fs_name);
+    if (!fs) {
+      ss << "filesystem '" << fs_name << "' not found";
+      r = -ENOENT;
+    } else {
+      const MDSMap &mds_map = fs->mds_map;
+      if (f) {
+        mds_map.dump_flags_state(f.get());
+        f->flush(ds);
+      }
+      else {
+        mds_map.print_flags(ds);
+      }
+      r = 0;
+    }
   }
 
 out:
@@ -1334,7 +1371,7 @@ bool MDSMonitor::prepare_command(MonOpRequestRef op)
   if (!cmdmap_from_json(m->cmd, &cmdmap, ss)) {
     string rs = ss.str();
     mon.reply_command(op, -EINVAL, rs, rdata, get_last_committed());
-    return true;
+    return false;
   }
 
   string prefix;
@@ -1344,7 +1381,7 @@ bool MDSMonitor::prepare_command(MonOpRequestRef op)
   MonSession *session = op->get_session();
   if (!session) {
     mon.reply_command(op, -EACCES, "access denied", rdata, get_last_committed());
-    return true;
+    return false;
   }
 
   auto &pending = get_pending_fsmap_writeable();
@@ -1546,6 +1583,7 @@ int MDSMonitor::filesystem_command(
 
     ss << "removed failed mds." << role;
     return 0;
+    /* TODO: convert to fs commands to update defaults */
   } else if (prefix == "mds compat rm_compat") {
     int64_t f;
     if (!cmd_getval(cmdmap, "feature", f)) {
@@ -1553,13 +1591,11 @@ int MDSMonitor::filesystem_command(
          << cmd_vartype_stringify(cmdmap.at("feature")) << "'";
       return -EINVAL;
     }
-    if (fsmap.compat.compat.contains(f)) {
+    if (fsmap.default_compat.compat.contains(f)) {
       ss << "removing compat feature " << f;
-      CompatSet modified = fsmap.compat;
-      modified.compat.remove(f);
-      fsmap.update_compat(modified);
+      fsmap.default_compat.compat.remove(f);
     } else {
-      ss << "compat feature " << f << " not present in " << fsmap.compat;
+      ss << "compat feature " << f << " not present in " << fsmap.default_compat;
     }
     r = 0;
   } else if (prefix == "mds compat rm_incompat") {
@@ -1569,13 +1605,11 @@ int MDSMonitor::filesystem_command(
          << cmd_vartype_stringify(cmdmap.at("feature")) << "'";
       return -EINVAL;
     }
-    if (fsmap.compat.incompat.contains(f)) {
+    if (fsmap.default_compat.incompat.contains(f)) {
       ss << "removing incompat feature " << f;
-      CompatSet modified = fsmap.compat;
-      modified.incompat.remove(f);
-      fsmap.update_compat(modified);
+      fsmap.default_compat.incompat.remove(f);
     } else {
-      ss << "incompat feature " << f << " not present in " << fsmap.compat;
+      ss << "incompat feature " << f << " not present in " << fsmap.default_compat;
     }
     r = 0;
   } else if (prefix == "mds repaired") {
@@ -1738,15 +1772,6 @@ void MDSMonitor::check_sub(Subscription *sub)
                   << "'" << dendl;
           return;
         }
-        if (fsmap.filesystems.count(fscid) == 0) {
-          // Client asked for a non-existent namespace, send them nothing
-          // TODO: something more graceful for when a client has a filesystem
-          // mounted, and the fileysstem is deleted.  Add a "shut down you fool"
-          // flag to MMDSMap?
-          dout(1) << "Client subscribed to non-existent namespace '" <<
-                  fscid << "'" << dendl;
-          return;
-        }
       } else {
         // Unqualified request for "mdsmap": give it the one marked
         // for use by legacy clients.
@@ -1758,13 +1783,21 @@ void MDSMonitor::check_sub(Subscription *sub)
           return;
         }
       }
+      if (!fsmap.filesystem_exists(fscid)) {
+        // Client asked for a non-existent namespace, send them nothing
+        // TODO: something more graceful for when a client has a filesystem
+        // mounted, and the fileysstem is deleted.  Add a "shut down you fool"
+        // flag to MMDSMap?
+        dout(1) << "Client subscribed to non-existent namespace '" <<
+                fscid << "'" << dendl;
+        return;
+      }
     }
-    dout(10) << __func__ << ": is_mds=" << is_mds << ", fscid= " << fscid << dendl;
+    dout(10) << __func__ << ": is_mds=" << is_mds << ", fscid=" << fscid << dendl;
 
     // Work out the effective latest epoch
     const MDSMap *mds_map = nullptr;
-    MDSMap null_map;
-    null_map.compat = fsmap.compat;
+    MDSMap null_map = MDSMap::create_null_mdsmap();
     if (fscid == FS_CLUSTER_ID_NONE) {
       // For a client, we should have already dropped out
       ceph_assert(is_mds);
@@ -1791,8 +1824,7 @@ void MDSMonitor::check_sub(Subscription *sub)
     if (sub->next > mds_map->epoch) {
       return;
     }
-    auto msg = make_message<MMDSMap>(mon.monmap->fsid, *mds_map,
-                                    mds_map->fs_name);
+    auto msg = make_message<MMDSMap>(mon.monmap->fsid, *mds_map);
 
     sub->session->con->send_message(msg.detach());
     if (sub->onetime) {
@@ -1807,7 +1839,9 @@ void MDSMonitor::check_sub(Subscription *sub)
 void MDSMonitor::update_metadata(mds_gid_t gid,
                                 const map<string, string>& metadata)
 {
+  dout(20) << __func__ <<  ": mds." << gid << ": " << metadata << dendl;
   if (metadata.empty()) {
+    dout(5) << __func__ << ": mds." << gid << ": no metadata!" << dendl;
     return;
   }
   pending_metadata[gid] = metadata;
@@ -1816,7 +1850,6 @@ void MDSMonitor::update_metadata(mds_gid_t gid,
   bufferlist bl;
   encode(pending_metadata, bl);
   t->put(MDS_METADATA_PREFIX, "last_metadata", bl);
-  paxos.trigger_propose();
 }
 
 void MDSMonitor::remove_from_metadata(const FSMap &fsmap, MonitorDBStore::TransactionRef t)
@@ -1955,7 +1988,6 @@ int MDSMonitor::print_nodes(Formatter *f)
  */
 bool MDSMonitor::maybe_resize_cluster(FSMap &fsmap, fs_cluster_id_t fscid)
 {
-  auto &current_mds_map = get_fsmap().get_filesystem(fscid)->mds_map;
   auto&& fs = fsmap.get_filesystem(fscid);
   auto &mds_map = fs->mds_map;
 
@@ -1968,7 +2000,8 @@ bool MDSMonitor::maybe_resize_cluster(FSMap &fsmap, fs_cluster_id_t fscid)
    * current batch of changes in pending. This is important if an MDS is
    * becoming active in the next epoch.
    */
-  if (!current_mds_map.is_resizeable() ||
+  if (!get_fsmap().filesystem_exists(fscid) ||
+      !get_fsmap().get_filesystem(fscid)->mds_map.is_resizeable() ||
       !mds_map.is_resizeable()) {
     dout(5) << __func__ << " mds_map is not currently resizeable" << dendl;
     return false;
@@ -2111,6 +2144,11 @@ bool MDSMonitor::check_health(FSMap& fsmap, bool* propose_osdmap)
 
   // check beacon timestamps
   std::vector<mds_gid_t> to_remove;
+  const bool mon_down = mon.is_mon_down();
+  const auto mds_beacon_mon_down_grace =
+      g_conf().get_val<std::chrono::seconds>("mds_beacon_mon_down_grace");
+  const auto quorum_age = std::chrono::seconds(mon.quorum_age());
+  const bool new_quorum = quorum_age < mds_beacon_mon_down_grace;
   for (auto it = last_beacon.begin(); it != last_beacon.end(); ) {
     auto& [gid, beacon_info] = *it;
     auto since_last = std::chrono::duration<double>(now-beacon_info.stamp);
@@ -2127,6 +2165,14 @@ bool MDSMonitor::check_health(FSMap& fsmap, bool* propose_osdmap)
               << " (gid: " << gid << " addr: " << info.addrs
               << " state: " << ceph_mds_state_name(info.state) << ")"
               << " since " << since_last.count() << dendl;
+      if ((mon_down || new_quorum) && since_last < mds_beacon_mon_down_grace) {
+        /* The MDS may be sending beacons to a monitor not yet in quorum or
+         * temporarily partitioned. Hold off on removal for a little longer...
+         */
+        dout(10) << "deferring removal for mds_beacon_mon_down_grace during MON_DOWN" << dendl;
+        ++it;
+        continue;
+      }
       // If the OSDMap is writeable, we can blocklist things, so we can
       // try failing any laggy MDS daemons.  Consider each one for failure.
       if (!info.laggy()) {
@@ -2176,7 +2222,7 @@ bool MDSMonitor::check_health(FSMap& fsmap, bool* propose_osdmap)
           const auto state = info.state;
           const mds_info_t* rep_info = nullptr;
           if (state == MDSMap::STATE_STANDBY_REPLAY) {
-            rep_info = fsmap.get_available_standby(fscid);
+            rep_info = fsmap.get_available_standby(*fs);
           } else if (state == MDSMap::STATE_ACTIVE) {
             rep_info = fsmap.find_replacement_for({fscid, rank});
           } else {
@@ -2245,7 +2291,7 @@ bool MDSMonitor::maybe_promote_standby(FSMap &fsmap, Filesystem& fs)
     // as standby-replay daemons. Don't do this when the cluster is degraded
     // as a standby-replay daemon may try to read a journal being migrated.
     for (;;) {
-      auto info = fsmap.get_available_standby(fs.fscid);
+      auto info = fsmap.get_available_standby(fs);
       if (!info) break;
       dout(20) << "standby available mds." << info->global_id << dendl;
       bool changed = false;
@@ -2276,6 +2322,39 @@ void MDSMonitor::tick()
   bool do_propose = false;
   bool propose_osdmap = false;
 
+  if (check_fsmap_struct_version) {
+    /* Allow time for trimming otherwise PaxosService::is_writeable will always
+     * be false.
+     */
+
+    auto now = clock::now();
+    auto elapsed = now - last_fsmap_struct_flush;
+    if (elapsed > std::chrono::seconds(30)) {
+      FSMap fsmap;
+      bufferlist bl;
+      auto v = get_first_committed();
+      int err = get_version(v, bl);
+      if (err) {
+        derr << "could not get version " << v << dendl;
+        ceph_abort();
+      }
+      try {
+        fsmap.decode(bl);
+      } catch (const ceph::buffer::malformed_input& e) {
+        dout(5) << "flushing old fsmap struct because unable to decode FSMap: " << e.what() << dendl;
+      }
+      /* N.B. FSMap::is_struct_old is also true for undecoded (failed to decode) FSMap */
+      if (fsmap.is_struct_old()) {
+        dout(5) << "fsmap struct is too old; proposing to flush out old versions" << dendl;
+        do_propose = true;
+        last_fsmap_struct_flush = now;
+      } else {
+        dout(20) << "struct is recent" << dendl;
+        check_fsmap_struct_version = false;
+      }
+    }
+  }
+
   do_propose |= pending.check_health();
 
   /* Check health and affinity of ranks */