]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
MDSMonitor: clean up use of pending fsmap in uncommitted ops 22005/head
authorPatrick Donnelly <pdonnell@redhat.com>
Mon, 7 May 2018 02:06:47 +0000 (19:06 -0700)
committerPatrick Donnelly <pdonnell@redhat.com>
Tue, 15 May 2018 13:44:10 +0000 (06:44 -0700)
Gist of this commit is to eliminate use of PaxosFSMap::get_working_fsmap which
allowed looking at the pending FSMap depending if it's the leader. Instead,
pass the FSMap being worked on to helper functions.

Also, ensure that we are only looking at the committed (i.e. current) FSMap
in the preprocess_* service methods.

Fixes: http://tracker.ceph.com/issues/23768
Signed-off-by: Patrick Donnelly <pdonnell@redhat.com>
(cherry picked from commit dd68c2386cff7857c4466216db80cfdb4f0353b5)

src/mon/FSCommands.cc
src/mon/MDSMonitor.cc
src/mon/MDSMonitor.h
src/mon/PaxosFSMap.h

index 6a8d9ab15954cd343a5586d922050edb2c622ea3..7e98f9c0861f4eac5251fcc1fb41d67208f19e00 100644 (file)
@@ -703,7 +703,7 @@ class RemoveFilesystemHandler : public FileSystemCommandHandler
     for (const auto &gid : to_fail) {
       // Standby replays don't write, so it isn't important to
       // wait for an osdmap propose here: ignore return value.
-      mon->mdsmon()->fail_mds_gid(gid);
+      mon->mdsmon()->fail_mds_gid(fsmap, gid);
     }
 
     fsmap.erase_filesystem(fs->fscid);
index 72ea76058998a04d43dcf7a9fab43c1f5eb3e071..389bfb190a97dd6ae7f569c7b6c6e82f6801a11c 100644 (file)
@@ -188,7 +188,7 @@ void MDSMonitor::encode_pending(MonitorDBStore::TransactionRef t)
     t->erase(MDS_HEALTH_PREFIX, stringify(*i));
   }
   pending_daemon_health_rm.clear();
-  remove_from_metadata(t);
+  remove_from_metadata(pending, t);
 
   // health
   health_check_map_t new_checks;
@@ -318,7 +318,7 @@ bool MDSMonitor::preprocess_beacon(MonOpRequestRef op)
   MDSMap::mds_info_t info;
   epoch_t effective_epoch = 0;
 
-  const auto &fsmap = get_working_fsmap();
+  const auto &fsmap = get_fsmap();
 
   // check privileges, ignore if fails
   MonSession *session = m->get_session();
@@ -362,6 +362,10 @@ bool MDSMonitor::preprocess_beacon(MonOpRequestRef op)
       dout(7) << "mds_beacon " << *m << " is not in fsmap (state "
               << ceph_mds_state_name(state) << ")" << dendl;
 
+      /* We can't send an MDSMap this MDS was a part of because we no longer
+       * 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;
@@ -457,25 +461,25 @@ bool MDSMonitor::preprocess_offload_targets(MonOpRequestRef op)
   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();
+  const auto &fsmap = get_fsmap();
   
   // check privileges, ignore message if fails
   MonSession *session = m->get_session();
   if (!session)
-    goto done;
+    goto ignore;
   if (!session->is_capable("mds", MON_CAP_X)) {
     dout(0) << "preprocess_offload_targets got MMDSLoadTargets from entity with insufficient caps "
            << session->caps << dendl;
-    goto done;
+    goto ignore;
   }
 
   if (fsmap.gid_exists(m->global_id) &&
       m->targets == fsmap.get_info_gid(m->global_id).export_targets)
-    goto done;
+    goto ignore;
 
   return false;
 
done:
ignore:
   mon->no_reply(op);
   return true;
 }
@@ -575,7 +579,7 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
         const MDSMap::mds_info_t &existing_info =
           pending.get_info_gid(existing);
         mon->clog->info() << existing_info.human_name() << " restarted";
-       fail_mds_gid(existing);
+       fail_mds_gid(pending, existing);
         failed_mds = true;
       }
       if (failed_mds) {
@@ -722,7 +726,7 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
         return false;
       }
 
-      fail_mds_gid(gid);
+      fail_mds_gid(pending, gid);
       assert(mon->osdmon()->is_writeable());
       request_proposal(mon->osdmon());
 
@@ -856,7 +860,7 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
   bufferlist rdata;
   stringstream ss, ds;
 
-  const auto &fsmap = get_working_fsmap();
+  const auto &fsmap = get_fsmap();
 
   cmdmap_t cmdmap;
   if (!cmdmap_from_json(m->cmd, &cmdmap, ss)) {
@@ -870,7 +874,7 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
   cmd_getval(g_ceph_context, cmdmap, "prefix", prefix);
   string format;
   cmd_getval(g_ceph_context, cmdmap, "format", format, string("plain"));
-  boost::scoped_ptr<Formatter> f(Formatter::create(format));
+  std::unique_ptr<Formatter> f(Formatter::create(format));
 
   MonSession *session = m->get_session();
   if (!session) {
@@ -892,7 +896,7 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
     int64_t epocharg;
     epoch_t epoch;
 
-    const FSMap *fsmapp = &get_fsmap();
+    const FSMap *fsmapp = &fsmap;
     FSMap dummy;
     if (cmd_getval(g_ceph_context, cmdmap, "epoch", epocharg)) {
       epoch = epocharg;
@@ -942,7 +946,7 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
         f->open_object_section("mds");
         f->dump_string("name", info.name);
         std::ostringstream get_err;
-        r = dump_metadata(info.name, f.get(), get_err);
+        r = dump_metadata(fsmap, info.name, f.get(), get_err);
         if (r == -EINVAL || r == -ENOENT) {
           // Drop error, list what metadata we do have
           dout(1) << get_err.str() << dendl;
@@ -960,7 +964,7 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
     } else {
       // Dump a single daemon's metadata
       f->open_object_section("mds_metadata");
-      r = dump_metadata(who, f.get(), ss);
+      r = dump_metadata(fsmap, who, f.get(), ss);
       f->close_section();
     }
     f->flush(ds);
@@ -1010,41 +1014,31 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
   } else if (prefix == "fs ls") {
     if (f) {
       f->open_array_section("filesystems");
-      {
-        for (const auto &p : fsmap.filesystems) {
-          const auto &fs = p.second;
-          f->open_object_section("filesystem");
-          {
-            const MDSMap &mds_map = fs->mds_map;
-            f->dump_string("name", mds_map.fs_name);
-            /* Output both the names and IDs of pools, for use by
-             * humans and machines respectively */
-            f->dump_string("metadata_pool", mon->osdmon()->osdmap.get_pool_name(
-                  mds_map.metadata_pool));
-            f->dump_int("metadata_pool_id", mds_map.metadata_pool);
-            f->open_array_section("data_pool_ids");
-            {
-              for (auto dpi = mds_map.data_pools.begin();
-                   dpi != mds_map.data_pools.end(); ++dpi) {
-                f->dump_int("data_pool_id", *dpi);
-              }
-            }
-            f->close_section();
-
-            f->open_array_section("data_pools");
-            {
-                for (auto dpi = mds_map.data_pools.begin();
-                   dpi != mds_map.data_pools.end(); ++dpi) {
-                  const auto &name = mon->osdmon()->osdmap.get_pool_name(
-                      *dpi);
-                  f->dump_string("data_pool", name);
-                }
-            }
+      for (const auto &p : fsmap.filesystems) {
+        const auto &fs = p.second;
+        f->open_object_section("filesystem");
+        {
+          const MDSMap &mds_map = fs->mds_map;
+          f->dump_string("name", mds_map.fs_name);
+          /* Output both the names and IDs of pools, for use by
+           * humans and machines respectively */
+          f->dump_string("metadata_pool", mon->osdmon()->osdmap.get_pool_name(
+                mds_map.metadata_pool));
+          f->dump_int("metadata_pool_id", mds_map.metadata_pool);
+          f->open_array_section("data_pool_ids");
+          for (const auto &id : mds_map.data_pools) {
+            f->dump_int("data_pool_id", id);
+          }
+          f->close_section();
 
-            f->close_section();
+          f->open_array_section("data_pools");
+          for (const auto &id : mds_map.data_pools) {
+            const auto &name = mon->osdmon()->osdmap.get_pool_name(id);
+            f->dump_string("data_pool", name);
           }
           f->close_section();
         }
+        f->close_section();
       }
       f->close_section();
       f->flush(ds);
@@ -1057,8 +1051,8 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
         
         ds << "name: " << mds_map.fs_name << ", metadata pool: "
            << md_pool_name << ", data pools: [";
-        for (auto dpi : mds_map.data_pools) {
-          const string &pool_name = mon->osdmon()->osdmap.get_pool_name(dpi);
+        for (const auto &id : mds_map.data_pools) {
+          const string &pool_name = mon->osdmon()->osdmap.get_pool_name(id);
           ds << pool_name << " ";
         }
         ds << "]" << std::endl;
@@ -1082,11 +1076,9 @@ out:
     return false;
 }
 
-bool MDSMonitor::fail_mds_gid(mds_gid_t gid)
+bool MDSMonitor::fail_mds_gid(FSMap &fsmap, mds_gid_t gid)
 {
-  auto &pending = get_pending_fsmap_writeable();
-
-  const MDSMap::mds_info_t &info = pending.get_info_gid(gid);
+  const MDSMap::mds_info_t &info = fsmap.get_info_gid(gid);
   dout(10) << "fail_mds_gid " << gid << " mds." << info.name << " role " << info.rank << dendl;
 
   epoch_t blacklist_epoch = 0;
@@ -1096,7 +1088,7 @@ bool MDSMonitor::fail_mds_gid(mds_gid_t gid)
     blacklist_epoch = mon->osdmon()->blacklist(info.addr, until);
   }
 
-  pending.erase(gid, blacklist_epoch);
+  fsmap.erase(gid, blacklist_epoch);
   last_beacon.erase(gid);
   if (pending_daemon_health.count(gid)) {
     pending_daemon_health.erase(gid);
@@ -1106,14 +1098,12 @@ bool MDSMonitor::fail_mds_gid(mds_gid_t gid)
   return blacklist_epoch != 0;
 }
 
-mds_gid_t MDSMonitor::gid_from_arg(const std::string& arg, std::ostream &ss)
+mds_gid_t MDSMonitor::gid_from_arg(const FSMap &fsmap, const std::string &arg, std::ostream &ss)
 {
-  const auto &fsmap = get_working_fsmap();
-
   // Try parsing as a role
   mds_role_t role;
   std::ostringstream ignore_err;  // Don't spam 'ss' with parse_role errors
-  int r = parse_role(arg, &role, ignore_err);
+  int r = fsmap.parse_role(arg, &role, ignore_err);
   if (r == 0) {
     // See if a GID is assigned to this role
     const auto &fs = fsmap.get_filesystem(role.fscid);
@@ -1154,12 +1144,12 @@ mds_gid_t MDSMonitor::gid_from_arg(const std::string& arg, std::ostream &ss)
   return MDS_GID_NONE;
 }
 
-int MDSMonitor::fail_mds(std::ostream &ss, const std::string &arg,
-    MDSMap::mds_info_t *failed_info)
+int MDSMonitor::fail_mds(FSMap &fsmap, std::ostream &ss,
+    const std::string &arg, MDSMap::mds_info_t *failed_info)
 {
   assert(failed_info != nullptr);
 
-  mds_gid_t gid = gid_from_arg(arg, ss);
+  mds_gid_t gid = gid_from_arg(fsmap, arg, ss);
   if (gid == MDS_GID_NONE) {
     return 0;
   }
@@ -1169,9 +1159,9 @@ 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 = get_pending_fsmap().get_info_gid(gid);
+  *failed_info = fsmap.get_info_gid(gid);
 
-  fail_mds_gid(gid);
+  fail_mds_gid(fsmap, gid);
   ss << "failed mds gid " << gid;
   assert(mon->osdmon()->is_writeable());
   request_proposal(mon->osdmon());
@@ -1232,7 +1222,7 @@ bool MDSMonitor::prepare_command(MonOpRequestRef op)
     }
   }
 
-  r = filesystem_command(op, prefix, cmdmap, ss);
+  r = filesystem_command(pending, op, prefix, cmdmap, ss);
   if (r >= 0) {
     goto out;
   } else if (r == -EAGAIN) {
@@ -1268,25 +1258,8 @@ out:
   }
 }
 
-
-/**
- * Given one of the following forms:
- *   <fs name>:<rank>
- *   <fs id>:<rank>
- *   <rank>
- *
- * Parse into a mds_role_t.  The rank-only form is only valid
- * if legacy_client_ns is set.
- */
-int MDSMonitor::parse_role(
-    const std::string &role_str,
-    mds_role_t *role,
-    std::ostream &ss)
-{
-  return get_working_fsmap().parse_role(role_str, role, ss);
-}
-
 int MDSMonitor::filesystem_command(
+    FSMap &fsmap,
     MonOpRequestRef op,
     std::string const &prefix,
     const cmdmap_t& cmdmap,
@@ -1298,8 +1271,6 @@ int MDSMonitor::filesystem_command(
   string whostr;
   cmd_getval(g_ceph_context, cmdmap, "role", whostr);
 
-  auto &pending = get_pending_fsmap_writeable();
-
   if (prefix == "mds deactivate") {
     ss << "This command is deprecated because it is obsolete;"
        << " to deactivate one or more MDS, decrease max_mds appropriately"
@@ -1317,8 +1288,8 @@ int MDSMonitor::filesystem_command(
          << cmd_vartype_stringify(cmdmap.at("state")) << "'";
       return -EINVAL;
     }
-    if (pending.gid_exists(gid)) {
-      pending.modify_daemon(gid, [state](MDSMap::mds_info_t *info) {
+    if (fsmap.gid_exists(gid)) {
+      fsmap.modify_daemon(gid, [state](MDSMap::mds_info_t *info) {
         info->state = state;
       });
       ss << "set mds gid " << gid << " to state " << state << " "
@@ -1330,7 +1301,7 @@ int MDSMonitor::filesystem_command(
     cmd_getval(g_ceph_context, cmdmap, "role_or_gid", who);
 
     MDSMap::mds_info_t failed_info;
-    r = fail_mds(ss, who, &failed_info);
+    r = fail_mds(fsmap, ss, who, &failed_info);
     if (r < 0 && r == -EAGAIN) {
       mon->osdmon()->wait_for_writeable(op, new C_RetryMessage(this, op));
       return -EAGAIN; // don't propose yet; wait for message to be retried
@@ -1348,18 +1319,18 @@ int MDSMonitor::filesystem_command(
          << cmd_vartype_stringify(cmdmap.at("gid")) << "'";
       return -EINVAL;
     }
-    if (!pending.gid_exists(gid)) {
+    if (!fsmap.gid_exists(gid)) {
       ss << "mds gid " << gid << " dne";
       r = 0;
     } else {
-      const auto &info = pending.get_info_gid(gid);
+      const auto &info = fsmap.get_info_gid(gid);
       MDSMap::DaemonState state = info.state;
       if (state > 0) {
         ss << "cannot remove active mds." << info.name
            << " rank " << info.rank;
         return -EBUSY;
       } else {
-        pending.erase(gid, {});
+        fsmap.erase(gid, {});
         ss << "removed mds gid " << gid;
         return 0;
       }
@@ -1376,13 +1347,13 @@ int MDSMonitor::filesystem_command(
     std::string role_str;
     cmd_getval(g_ceph_context, cmdmap, "role", role_str);
     mds_role_t role;
-    int r = parse_role(role_str, &role, ss);
+    int r = fsmap.parse_role(role_str, &role, ss);
     if (r < 0) {
       ss << "invalid role '" << role_str << "'";
       return -EINVAL;
     }
 
-    pending.modify_filesystem(
+    fsmap.modify_filesystem(
         role.fscid,
         [role](std::shared_ptr<Filesystem> fs)
     {
@@ -1398,13 +1369,13 @@ int MDSMonitor::filesystem_command(
          << cmd_vartype_stringify(cmdmap.at("feature")) << "'";
       return -EINVAL;
     }
-    if (pending.compat.compat.contains(f)) {
+    if (fsmap.compat.compat.contains(f)) {
       ss << "removing compat feature " << f;
-      CompatSet modified = pending.compat;
+      CompatSet modified = fsmap.compat;
       modified.compat.remove(f);
-      pending.update_compat(modified);
+      fsmap.update_compat(modified);
     } else {
-      ss << "compat feature " << f << " not present in " << pending.compat;
+      ss << "compat feature " << f << " not present in " << fsmap.compat;
     }
     r = 0;
   } else if (prefix == "mds compat rm_incompat") {
@@ -1414,25 +1385,25 @@ int MDSMonitor::filesystem_command(
          << cmd_vartype_stringify(cmdmap.at("feature")) << "'";
       return -EINVAL;
     }
-    if (pending.compat.incompat.contains(f)) {
+    if (fsmap.compat.incompat.contains(f)) {
       ss << "removing incompat feature " << f;
-      CompatSet modified = pending.compat;
+      CompatSet modified = fsmap.compat;
       modified.incompat.remove(f);
-      pending.update_compat(modified);
+      fsmap.update_compat(modified);
     } else {
-      ss << "incompat feature " << f << " not present in " << pending.compat;
+      ss << "incompat feature " << f << " not present in " << fsmap.compat;
     }
     r = 0;
   } else if (prefix == "mds repaired") {
     std::string role_str;
     cmd_getval(g_ceph_context, cmdmap, "role", role_str);
     mds_role_t role;
-    r = parse_role(role_str, &role, ss);
+    r = fsmap.parse_role(role_str, &role, ss);
     if (r < 0) {
       return r;
     }
 
-    bool modified = pending.undamaged(role.fscid, role.rank);
+    bool modified = fsmap.undamaged(role.fscid, role.rank);
     if (modified) {
       dout(4) << "repaired: restoring rank " << role << dendl;
     } else {
@@ -1521,9 +1492,9 @@ void MDSMonitor::check_sub(Subscription *sub)
     if (is_mds) {
       // What (if any) namespace are you assigned to?
       auto mds_info = fsmap.get_mds_info();
-      for (const auto &i : mds_info) {
-        if (i.second.addr == sub->session->inst.addr) {
-          mds_gid = i.first;
+      for (const auto &p : mds_info) {
+        if (p.second.addr == sub->session->inst.addr) {
+          mds_gid = p.first;
           fscid = fsmap.mds_roles.at(mds_gid);
         }
       }
@@ -1621,16 +1592,15 @@ void MDSMonitor::update_metadata(mds_gid_t gid,
   paxos->trigger_propose();
 }
 
-void MDSMonitor::remove_from_metadata(MonitorDBStore::TransactionRef t)
+void MDSMonitor::remove_from_metadata(const FSMap &fsmap, MonitorDBStore::TransactionRef t)
 {
   bool update = false;
-  for (map<mds_gid_t, Metadata>::iterator i = pending_metadata.begin();
-       i != pending_metadata.end(); ) {
-    if (!get_pending_fsmap().gid_exists(i->first)) {
-      pending_metadata.erase(i++);
+  for (auto it = pending_metadata.begin(); it != pending_metadata.end(); ) {
+    if (!fsmap.gid_exists(it->first)) {
+      it = pending_metadata.erase(it);
       update = true;
     } else {
-      ++i;
+      ++it;
     }
   }
   if (!update)
@@ -1654,7 +1624,7 @@ int MDSMonitor::load_metadata(map<mds_gid_t, Metadata>& m)
   return 0;
 }
 
-void MDSMonitor::count_metadata(const string& field, map<string,int> *out)
+void MDSMonitor::count_metadata(const std::string &field, map<string,int> *out)
 {
   map<mds_gid_t,Metadata> meta;
   load_metadata(meta);
@@ -1668,7 +1638,7 @@ void MDSMonitor::count_metadata(const string& field, map<string,int> *out)
   }
 }
 
-void MDSMonitor::count_metadata(const string& field, Formatter *f)
+void MDSMonitor::count_metadata(const std::string &field, Formatter *f)
 {
   map<string,int> by_val;
   count_metadata(field, &by_val);
@@ -1679,11 +1649,12 @@ void MDSMonitor::count_metadata(const string& field, Formatter *f)
   f->close_section();
 }
 
-int MDSMonitor::dump_metadata(const std::string &who, Formatter *f, ostream& err)
+int MDSMonitor::dump_metadata(const FSMap& fsmap, const std::string &who,
+    Formatter *f, ostream& err)
 {
   assert(f);
 
-  mds_gid_t gid = gid_from_arg(who, err);
+  mds_gid_t gid = gid_from_arg(fsmap, who, err);
   if (gid == MDS_GID_NONE) {
     return -EINVAL;
   }
@@ -1708,26 +1679,27 @@ int MDSMonitor::print_nodes(Formatter *f)
 {
   assert(f);
 
+  const auto &fsmap = get_fsmap();
+
   map<mds_gid_t, Metadata> metadata;
   if (int r = load_metadata(metadata)) {
     return r;
   }
 
   map<string, list<string> > mdses; // hostname => mds
-  for (map<mds_gid_t, Metadata>::iterator it = metadata.begin();
-       it != metadata.end(); ++it) {
-    const Metadata& m = it->second;
+  for (const auto &p : metadata) {
+    const mds_gid_t& gid = p.first;
+    const Metadata& m = p.second;
     Metadata::const_iterator hostname = m.find("hostname");
     if (hostname == m.end()) {
       // not likely though
       continue;
     }
-    const mds_gid_t gid = it->first;
-    if (!get_fsmap().gid_exists(gid)) {
+    if (!fsmap.gid_exists(gid)) {
       dout(5) << __func__ << ": GID " << gid << " not existent" << dendl;
       continue;
     }
-    const MDSMap::mds_info_t& mds_info = get_fsmap().get_info_gid(gid);
+    const MDSMap::mds_info_t& mds_info = fsmap.get_info_gid(gid);
     mdses[hostname->second].push_back(mds_info.name);
   }
 
@@ -1740,16 +1712,14 @@ int MDSMonitor::print_nodes(Formatter *f)
  * attempt to find daemons to grow it. If the cluster is oversized
  * (with respect to max_mds) then shrink it by stopping its highest rank.
  */
-bool MDSMonitor::maybe_resize_cluster(fs_cluster_id_t fscid)
+bool MDSMonitor::maybe_resize_cluster(FSMap &fsmap, fs_cluster_id_t fscid)
 {
-  const auto &fsmap = get_fsmap();
-  auto &fsmap_mds_map = fsmap.get_filesystem(fscid)->mds_map;
-  auto &pending = get_pending_fsmap_writeable();
-  auto pending_fs = pending.get_filesystem(fscid);
-  auto &pending_mds_map = pending_fs->mds_map;
+  auto &current_mds_map = get_fsmap().get_filesystem(fscid)->mds_map;
+  auto fs = fsmap.get_filesystem(fscid);
+  auto &mds_map = fs->mds_map;
 
-  int in = pending_mds_map.get_num_in_mds();
-  int max = pending_mds_map.get_max_mds();
+  int in = mds_map.get_num_in_mds();
+  int max = mds_map.get_max_mds();
 
   dout(20) << __func__ << " in " << in << " max " << max << dendl;
 
@@ -1757,41 +1727,41 @@ bool MDSMonitor::maybe_resize_cluster(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 (!fsmap_mds_map.is_resizeable() ||
-      !pending_mds_map.is_resizeable()) {
+  if (!current_mds_map.is_resizeable() ||
+      !mds_map.is_resizeable()) {
     dout(5) << __func__ << " mds_map is not currently resizeable" << dendl;
     return false;
   }
 
-  if (in < max && !pending_mds_map.test_flag(CEPH_MDSMAP_NOT_JOINABLE)) {
+  if (in < max && !mds_map.test_flag(CEPH_MDSMAP_NOT_JOINABLE)) {
     mds_rank_t mds = mds_rank_t(0);
     string name;
-    while (pending_mds_map.is_in(mds)) {
+    while (mds_map.is_in(mds)) {
       mds++;
     }
-    mds_gid_t newgid = pending.find_replacement_for({fscid, mds},
+    mds_gid_t newgid = fsmap.find_replacement_for({fscid, mds},
                          name, g_conf->mon_force_standby_active);
     if (newgid == MDS_GID_NONE) {
       return false;
     }
 
-    const auto &new_info = pending.get_info_gid(newgid);
+    const auto &new_info = fsmap.get_info_gid(newgid);
     dout(1) << "assigned standby " << new_info.addr
             << " as mds." << mds << dendl;
 
     mon->clog->info() << new_info.human_name() << " assigned to "
-                         "filesystem " << pending_mds_map.fs_name << " as rank "
-                      << mds << " (now has " << pending_mds_map.get_num_in_mds() + 1
+                         "filesystem " << mds_map.fs_name << " as rank "
+                      << mds << " (now has " << mds_map.get_num_in_mds() + 1
                       << " ranks)";
-    pending.promote(newgid, pending_fs, mds);
+    fsmap.promote(newgid, fs, mds);
     return true;
   } else if (in > max) {
     mds_rank_t target = in - 1;
-    const auto &info = pending_mds_map.get_info(target);
-    if (pending_mds_map.is_active(target)) {
+    const auto &info = mds_map.get_info(target);
+    if (mds_map.is_active(target)) {
       dout(1) << "deactivating " << target << dendl;
       mon->clog->info() << "deactivating " << info.human_name();
-      pending.modify_daemon(info.global_id,
+      fsmap.modify_daemon(info.global_id,
                             [] (MDSMap::mds_info_t *info) {
                                 info->state = MDSMap::STATE_STOPPING;
                             });
@@ -1811,14 +1781,13 @@ bool MDSMonitor::maybe_resize_cluster(fs_cluster_id_t fscid)
  * is available, fail this daemon (remove from map) and pass its
  * role to another daemon.
  */
-void MDSMonitor::maybe_replace_gid(mds_gid_t gid, const MDSMap::mds_info_t& info,
-    bool *mds_propose, bool *osd_propose)
+void MDSMonitor::maybe_replace_gid(FSMap &fsmap, mds_gid_t gid,
+    const MDSMap::mds_info_t& info, bool *mds_propose, bool *osd_propose)
 {
   assert(mds_propose != nullptr);
   assert(osd_propose != nullptr);
 
-  auto &pending = get_pending_fsmap_writeable();
-  const auto fscid = pending.mds_roles.at(gid);
+  const auto fscid = fsmap.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
@@ -1839,12 +1808,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.get_filesystem(fscid)->mds_map.test_flag(CEPH_MDSMAP_NOT_JOINABLE) &&
-      (sgid = pending.find_replacement_for({fscid, info.rank}, info.name,
+      !fsmap.get_filesystem(fscid)->mds_map.test_flag(CEPH_MDSMAP_NOT_JOINABLE) &&
+      (sgid = fsmap.find_replacement_for({fscid, info.rank}, info.name,
                 g_conf->mon_force_standby_active)) != MDS_GID_NONE)
   {
     
-    MDSMap::mds_info_t si = pending.get_info_gid(sgid);
+    MDSMap::mds_info_t si = fsmap.get_info_gid(sgid);
     dout(10) << " replacing " << gid << " " << info.addr << " mds."
       << info.rank << "." << info.inc
       << " " << ceph_mds_state_name(info.state)
@@ -1856,14 +1825,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.mds_roles.at(gid);
+    const fs_cluster_id_t fscid = fsmap.mds_roles.at(gid);
 
     // Remove the old one
-    *osd_propose |= fail_mds_gid(gid);
+    *osd_propose |= fail_mds_gid(fsmap, gid);
 
     // Promote the replacement
-    auto fs = pending.filesystems.at(fscid);
-    pending.promote(sgid, fs, info.rank);
+    auto fs = fsmap.filesystems.at(fscid);
+    fsmap.promote(sgid, fs, info.rank);
 
     *mds_propose = true;
   } else if ((info.state == MDSMap::STATE_STANDBY_REPLAY ||
@@ -1873,27 +1842,25 @@ void MDSMonitor::maybe_replace_gid(mds_gid_t gid, const MDSMap::mds_info_t& info
       << dendl;
     mon->clog->info() << "Standby " << info.human_name() << " is not "
                          "responding, dropping it";
-    fail_mds_gid(gid);
+    fail_mds_gid(fsmap, gid);
     *mds_propose = true;
   } else if (!info.laggy()) {
       dout(10) << " marking " << gid << " " << info.addr << " mds." << info.rank << "." << info.inc
         << " " << ceph_mds_state_name(info.state)
         << " laggy" << dendl;
-      pending.modify_daemon(info.global_id, [](MDSMap::mds_info_t *info) {
+      fsmap.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(FSMap &fsmap, std::shared_ptr<Filesystem> &fs)
 {
   if (fs->mds_map.test_flag(CEPH_MDSMAP_NOT_JOINABLE)) {
     return false;
   }
 
-  auto &pending = get_pending_fsmap_writeable();
-
   bool do_propose = false;
 
   // have a standby take over?
@@ -1903,17 +1870,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.find_replacement_for({fs->fscid, f}, {},
+      mds_gid_t sgid = fsmap.find_replacement_for({fs->fscid, f}, {},
           g_conf->mon_force_standby_active);
       if (sgid) {
-        const MDSMap::mds_info_t si = pending.get_info_gid(sgid);
+        const MDSMap::mds_info_t si = fsmap.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.promote(sgid, fs, f);
+        fsmap.promote(sgid, fs, f);
        do_propose = true;
       }
     }
@@ -1925,12 +1892,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.standby_daemons) {
+    for (const auto &j : fsmap.standby_daemons) {
       standby_gids.push_back(j.first);
     }
 
     for (const auto &gid : standby_gids) {
-      const auto &info = pending.standby_daemons.at(gid);
+      const auto &info = fsmap.standby_daemons.at(gid);
       assert(info.state == MDSMap::STATE_STANDBY);
 
       if (!info.standby_replay) {
@@ -1949,14 +1916,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.legacy_client_fscid : info.standby_for_fscid,
+            fsmap.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.filesystem_exists(info.standby_for_fscid)) {
+            && !fsmap.filesystem_exists(info.standby_for_fscid)) {
           derr << "gid " << gid << " has invalid standby_for_fscid "
                << info.standby_for_fscid << dendl;
           continue;
@@ -1964,11 +1931,9 @@ 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) {
-          const auto &fs = pending.get_filesystem(target_role.fscid);
+          const auto &fs = fsmap.get_filesystem(target_role.fscid);
           if (fs->mds_map.is_followable(target_role.rank)) {
-            do_propose |= try_standby_replay(
-                info,
-                *fs,
+            do_propose |= try_standby_replay(fsmap, info, *fs,
                 fs->mds_map.get_info(target_role.rank));
           }
         }
@@ -1977,7 +1942,7 @@ bool MDSMonitor::maybe_promote_standby(std::shared_ptr<Filesystem> &fs)
       }
 
       // check everyone
-      for (const auto &p : pending.filesystems) {
+      for (const auto &p : fsmap.filesystems) {
        if (info.standby_for_fscid != FS_CLUSTER_ID_NONE &&
            info.standby_for_fscid != p.first)
          continue;
@@ -1993,7 +1958,7 @@ bool MDSMonitor::maybe_promote_standby(std::shared_ptr<Filesystem> &fs)
               continue;   // we're supposed to follow someone else
             }
 
-            if (try_standby_replay(info, *fs, cand_info)) {
+            if (try_standby_replay(fsmap, info, *fs, cand_info)) {
              assigned = true;
               break;
             }
@@ -2015,11 +1980,7 @@ void MDSMonitor::tick()
   // make sure mds's are still alive
   // ...if i am an active leader
 
-  if (!is_active()) return;
-
-  dout(10) << get_working_fsmap() << dendl;
-
-  if (!is_leader()) return;
+  if (!is_active() || !is_leader()) return;
 
   auto &pending = get_pending_fsmap_writeable();
 
@@ -2029,7 +1990,7 @@ void MDSMonitor::tick()
 
   // resize mds cluster (adjust @in)?
   for (auto &p : pending.filesystems) {
-    do_propose |= maybe_resize_cluster(p.second->fscid);
+    do_propose |= maybe_resize_cluster(pending, p.second->fscid);
   }
 
   mono_time now = mono_clock::now();
@@ -2064,7 +2025,7 @@ void MDSMonitor::tick()
   // check beacon timestamps
   bool propose_osdmap = false;
   bool osdmap_writeable = mon->osdmon()->is_writeable();
-  for (auto it = last_beacon.begin(); it != last_beacon.end(); it++) {
+  for (auto it = last_beacon.begin(); it != last_beacon.end(); ) {
     mds_gid_t gid = it->first;
     auto beacon_info = it->second;
     chrono::duration<double> since_last = now-beacon_info.stamp;
@@ -2085,16 +2046,18 @@ void MDSMonitor::tick()
       // If the OSDMap is writeable, we can blacklist things, so we can
       // try failing any laggy MDS daemons.  Consider each one for failure.
       if (osdmap_writeable) {
-        maybe_replace_gid(gid, info, &do_propose, &propose_osdmap);
+        maybe_replace_gid(pending, gid, info, &do_propose, &propose_osdmap);
       }
     }
+
+    ++it;
   }
   if (propose_osdmap) {
     request_proposal(mon->osdmon());
   }
 
   for (auto &p : pending.filesystems) {
-    do_propose |= maybe_promote_standby(p.second);
+    do_propose |= maybe_promote_standby(pending, p.second);
   }
 
   if (do_propose) {
@@ -2108,6 +2071,7 @@ void MDSMonitor::tick()
  * ainfo: the would-be leader
  */
 bool MDSMonitor::try_standby_replay(
+    FSMap &fsmap,
     const MDSMap::mds_info_t& finfo,
     const Filesystem &leader_fs,
     const MDSMap::mds_info_t& ainfo)
@@ -2119,7 +2083,7 @@ bool MDSMonitor::try_standby_replay(
   } else {
     // Assign the new role to the standby
     dout(10) << "  setting to follow mds rank " << ainfo.rank << dendl;
-    get_pending_fsmap_writeable().assign_standby_replay(finfo.global_id, leader_fs.fscid, ainfo.rank);
+    fsmap.assign_standby_replay(finfo.global_id, leader_fs.fscid, ainfo.rank);
     return true;
   }
 }
index 88cc9946c59915c6f13055e213d40fbae9ac6b60..2f82e8afb0ac47bd2ed54264397d163e1db1200f 100644 (file)
@@ -63,7 +63,7 @@ class MDSMonitor : public PaxosService, public PaxosFSMap {
   /**
    * Return true if a blacklist was done (i.e. OSD propose needed)
    */
-  bool fail_mds_gid(mds_gid_t gid);
+  bool fail_mds_gid(FSMap &fsmap, mds_gid_t gid);
 
   bool is_leader() const override { return mon->is_leader(); }
 
@@ -81,18 +81,15 @@ class MDSMonitor : public PaxosService, public PaxosFSMap {
   bool preprocess_offload_targets(MonOpRequestRef op);
   bool prepare_offload_targets(MonOpRequestRef op);
 
-  int fail_mds(std::ostream &ss, const std::string &arg,
+  int fail_mds(FSMap &fsmap, std::ostream &ss,
+      const std::string &arg,
       MDSMap::mds_info_t *failed_info);
 
   bool preprocess_command(MonOpRequestRef op);
   bool prepare_command(MonOpRequestRef op);
 
-  int parse_role(
-      const std::string &role_str,
-      mds_role_t *role,
-      std::ostream &ss);
-
   int filesystem_command(
+      FSMap &fsmap,
       MonOpRequestRef op,
       std::string const &prefix,
       const cmdmap_t& cmdmap,
@@ -107,37 +104,35 @@ class MDSMonitor : public PaxosService, public PaxosFSMap {
   };
   map<mds_gid_t, beacon_info_t> last_beacon;
 
-  bool try_standby_replay(
-      const MDSMap::mds_info_t& finfo,
-      const Filesystem &leader_fs,
-      const MDSMap::mds_info_t& ainfo);
+  bool try_standby_replay(FSMap &fsmap, const MDSMap::mds_info_t& finfo,
+      const Filesystem &leader_fs, const MDSMap::mds_info_t& ainfo);
 
   std::list<std::shared_ptr<FileSystemCommandHandler> > handlers;
 
-  bool maybe_promote_standby(std::shared_ptr<Filesystem> &fs);
-  bool maybe_resize_cluster(fs_cluster_id_t fscid);
-  void maybe_replace_gid(mds_gid_t gid, const MDSMap::mds_info_t& info,
-      bool *mds_propose, bool *osd_propose);
+  bool maybe_promote_standby(FSMap &fsmap, std::shared_ptr<Filesystem> &fs);
+  bool maybe_resize_cluster(FSMap &fsmap, fs_cluster_id_t fscid);
+  void maybe_replace_gid(FSMap &fsmap, mds_gid_t gid,
+      const MDSMap::mds_info_t& info, bool *mds_propose, bool *osd_propose);
   void tick() override;     // check state, take actions
 
-  int dump_metadata(const string& who, Formatter *f, ostream& err);
+  int dump_metadata(const FSMap &fsmap, const std::string &who, Formatter *f,
+      ostream& err);
 
   void update_metadata(mds_gid_t gid, const Metadata& metadata);
-  void remove_from_metadata(MonitorDBStore::TransactionRef t);
+  void remove_from_metadata(const FSMap &fsmap, MonitorDBStore::TransactionRef t);
   int load_metadata(map<mds_gid_t, Metadata>& m);
-  void count_metadata(const string& field, Formatter *f);
+  void count_metadata(const std::string& field, Formatter *f);
 public:
-  void count_metadata(const string& field, map<string,int> *out);
+  void count_metadata(const std::string& field, map<string,int> *out);
 protected:
 
   // MDS daemon GID to latest health state from that GID
   std::map<uint64_t, MDSHealth> pending_daemon_health;
   std::set<uint64_t> pending_daemon_health_rm;
 
-
   map<mds_gid_t, Metadata> pending_metadata;
 
-  mds_gid_t gid_from_arg(const std::string& arg, std::ostream& err);
+  mds_gid_t gid_from_arg(const FSMap &fsmap, const std::string &arg, std::ostream& err);
 
   // When did the mon last call into our tick() method?  Used for detecting
   // when the mon was not updating us for some period (e.g. during slow
index 8d7c8c1f1cc7dc0edf888a57d64b572d4adca103..e1ee081424de6164eb4871537c6660dd40f18c1d 100644 (file)
@@ -32,15 +32,6 @@ public:
 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;