]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
mds: rework FSMap to avoid shared_ptrs 52409/head
authorPatrick Donnelly <pdonnell@redhat.com>
Fri, 7 Jul 2023 01:33:06 +0000 (21:33 -0400)
committerPatrick Donnelly <pdonnell@redhat.com>
Wed, 12 Jul 2023 14:56:40 +0000 (10:56 -0400)
There's zero reason to complicate the interface with
shared_ptr<Filesystem> when a Filesystem's lifetime is tied to the
FSMap.

Signed-off-by: Patrick Donnelly <pdonnell@redhat.com>
18 files changed:
src/client/Client.cc
src/mds/FSMap.cc
src/mds/FSMap.h
src/mds/MDSMap.cc
src/mds/MDSMap.h
src/mon/AuthMonitor.cc
src/mon/FSCommands.cc
src/mon/MDSMonitor.cc
src/mon/MDSMonitor.h
src/mon/PaxosFSMap.h
src/tools/cephfs/DataScan.cc
src/tools/cephfs/Dumper.cc
src/tools/cephfs/JournalTool.cc
src/tools/cephfs/MetaTool.cc
src/tools/cephfs/Resetter.cc
src/tools/cephfs/RoleSelector.cc
src/tools/cephfs/TableTool.cc
src/tools/cephfs_mirror/ClusterWatcher.cc

index 83885a980e3f5aeab94406b6246ade5a035d387f..847a2fb9a3e50f6d1bc1db5f29345f5ab74d634a 100644 (file)
@@ -6134,7 +6134,8 @@ int Client::resolve_mds(
   int role_r = fsmap->parse_role(mds_spec, &role, *css);
   if (role_r == 0) {
     // We got a role, resolve it to a GID
-    auto& info = fsmap->get_filesystem(role.fscid)->mds_map.get_info(role.rank);
+    const auto& mdsmap = fsmap->get_filesystem(role.fscid).get_mds_map();
+    auto& info = mdsmap.get_info(role.rank);
     ldout(cct, 10) << __func__ << ": resolved " << mds_spec << " to role '"
       << role << "' aka " << info.human_name() << dendl;
     targets->push_back(info.global_id);
index b9ae05ac0f62246831f30ed0d7ed6c9d0df8b1ae..f8ba203077e8ff96f5f5ab485d6484d1f4d36f57 100644 (file)
@@ -150,9 +150,9 @@ void FSMap::dump(Formatter *f) const
   f->close_section();
 
   f->open_array_section("filesystems");
-  for (const auto &fs : filesystems) {
+  for ([[maybe_unused]] const auto& [fscid, fs] : filesystems) {
     f->open_object_section("filesystem");
-    fs.second->dump(f);
+    fs.dump(f);
     f->close_section();
   }
   f->close_section();
@@ -170,9 +170,8 @@ FSMap &FSMap::operator=(const FSMap &rhs)
   standby_epochs = rhs.standby_epochs;
 
   filesystems.clear();
-  for (const auto &i : rhs.filesystems) {
-    const auto &fs = i.second;
-    filesystems[fs->fscid] = std::make_shared<Filesystem>(*fs);
+  for (const auto& [fscid, fs] : rhs.filesystems) {
+    filesystems.emplace(std::piecewise_construct, std::forward_as_tuple(fscid), std::forward_as_tuple(fs));
   }
 
   return *this;
@@ -180,22 +179,21 @@ FSMap &FSMap::operator=(const FSMap &rhs)
 
 void FSMap::generate_test_instances(std::list<FSMap*>& ls)
 {
-  FSMap *m = new FSMap();
+  FSMap* fsmap = new FSMap();
 
   std::list<MDSMap*> mds_map_instances;
   MDSMap::generate_test_instances(mds_map_instances);
 
   int k = 20;
-  for (auto i : mds_map_instances) {
-    auto fs = Filesystem::create();
-    fs->fscid = k++;
-    fs->mds_map = *i;
-    delete i;
-    m->filesystems[fs->fscid] = fs;
+  for (auto& mdsmap : mds_map_instances) {
+    auto fs = Filesystem();
+    fs.fscid = k++;
+    fs.mds_map = *mdsmap;
+    fsmap->filesystems[fs.fscid] = fs;
+    delete mdsmap;
   }
-  mds_map_instances.clear();
 
-  ls.push_back(m);
+  ls.push_back(fsmap);
 }
 
 void FSMap::print(ostream& out) const
@@ -211,8 +209,8 @@ void FSMap::print(ostream& out) const
     out << "No filesystems configured" << std::endl;
   }
 
-  for (const auto& p : filesystems) {
-    p.second->print(out);
+  for ([[maybe_unused]] const auto& [fscid, fs] : filesystems) {
+    fs.print(out);
     out << " " << std::endl << " " << std::endl;  // Space out a bit
   }
 
@@ -230,11 +228,11 @@ void FSMap::print_daemon_summary(ostream& out) const
   // this appears in the "services:" section of "ceph status"
   int num_up = 0, num_in = 0, num_failed = 0;
   int num_standby_replay = 0;
-  for (auto& [fscid, fs] : filesystems) {
-    num_up += fs->mds_map.get_num_up_mds();
-    num_in += fs->mds_map.get_num_in_mds();
-    num_failed += fs->mds_map.get_num_failed_mds();
-    num_standby_replay += fs->mds_map.get_num_standby_replay_mds();
+  for ([[maybe_unused]] auto& [fscid, fs] : filesystems) {
+    num_up += fs.mds_map.get_num_up_mds();
+    num_in += fs.mds_map.get_num_in_mds();
+    num_failed += fs.mds_map.get_num_failed_mds();
+    num_standby_replay += fs.mds_map.get_num_standby_replay_mds();
   }
   int num_standby = standby_daemons.size();
   out << num_up << "/" << num_in << " daemons up";
@@ -255,15 +253,15 @@ void FSMap::print_fs_summary(ostream& out) const
   if (!filesystems.empty()) {
     int num_failed = 0, num_recovering = 0, num_stopped = 0, num_healthy = 0;
     int num_damaged = 0;
-    for (auto& [fscid, fs] : filesystems) {
-      if (fs->mds_map.is_any_damaged()) {
+    for ([[maybe_unused]] auto& [fscid, fs] : filesystems) {
+      if (fs.mds_map.is_any_damaged()) {
        ++num_damaged;
       }
-      if (fs->mds_map.is_any_failed()) {
+      if (fs.mds_map.is_any_failed()) {
        ++num_failed;
-      } else if (fs->mds_map.is_degraded()) {
+      } else if (fs.mds_map.is_degraded()) {
        ++num_recovering;
-      } else if (fs->mds_map.get_max_mds() == 0) {
+      } else if (fs.mds_map.get_max_mds() == 0) {
        ++num_stopped;
       } else {
        ++num_healthy;
@@ -291,26 +289,24 @@ void FSMap::print_summary(Formatter *f, ostream *out) const
 {
   if (f) {
     f->dump_unsigned("epoch", get_epoch());
-    for (const auto &p : filesystems) {
-      auto& fs = p.second;
-      f->dump_unsigned("id", fs->fscid);
-      f->dump_unsigned("up", fs->mds_map.up.size());
-      f->dump_unsigned("in", fs->mds_map.in.size());
-      f->dump_unsigned("max", fs->mds_map.max_mds);
+    for (const auto& [fscid, fs] : filesystems) {
+      f->dump_unsigned("id", fscid);
+      f->dump_unsigned("up", fs.mds_map.up.size());
+      f->dump_unsigned("in", fs.mds_map.in.size());
+      f->dump_unsigned("max", fs.mds_map.max_mds);
     }
   } else {
     auto count = filesystems.size();
     if (count <= 3) {
       bool first = true;
-      for (const auto& p : filesystems) {
-        const auto& fs = p.second;
+      for ([[maybe_unused]] const auto& [fscid, fs] : filesystems) {
         if (!first) {
           *out << " ";
         }
-        if (fs->mds_map.is_degraded()) {
-          *out << fs->mds_map.fs_name << ":" << fs->mds_map.up.size() << "/" << fs->mds_map.in.size();
+        if (fs.mds_map.is_degraded()) {
+          *out << fs.mds_map.fs_name << ":" << fs.mds_map.up.size() << "/" << fs.mds_map.in.size();
         } else {
-          *out << fs->mds_map.fs_name << ":" << fs->mds_map.in.size();
+          *out << fs.mds_map.fs_name << ":" << fs.mds_map.in.size();
         }
         first = false;
       }
@@ -319,12 +315,11 @@ void FSMap::print_summary(Formatter *f, ostream *out) const
       unsigned degraded = 0;
       CachedStackStringStream css;
       *css << " (degraded: ";
-      for (const auto& p : filesystems) {
-        const auto& fs = p.second;
-        if (fs->mds_map.is_degraded()) {
+      for ([[maybe_unused]] const auto& [fscid, fs] : filesystems) {
+        if (fs.mds_map.is_degraded()) {
           degraded++;
           if (degraded <= 3) {
-            *css << fs->mds_map.fs_name << ":" << fs->mds_map.up.size() << "/" << fs->mds_map.in.size();
+            *css << fs.mds_map.fs_name << ":" << fs.mds_map.up.size() << "/" << fs.mds_map.in.size();
           }
         }
       }
@@ -350,7 +345,7 @@ void FSMap::print_summary(Formatter *f, ostream *out) const
     if (fscid == FS_CLUSTER_ID_NONE)
       continue;
 
-    const auto& info = filesystems.at(fscid)->mds_map.get_info_gid(gid);
+    const auto& info = filesystems.at(fscid).mds_map.get_info_gid(gid);
     auto s = std::string(ceph_mds_state_name(info.state));
     if (info.laggy()) {
       s += "(laggy or crashed)";
@@ -378,7 +373,7 @@ void FSMap::print_summary(Formatter *f, ostream *out) const
         // Disambiguate filesystems
         std::map<std::string, std::string> pretty;
         for (const auto& [role,status] : by_rank) {
-          const auto &fs_name = filesystems.at(role.fscid)->mds_map.fs_name;
+          const auto &fs_name = filesystems.at(role.fscid).mds_map.fs_name;
           CachedStackStringStream css;
           *css << fs_name << ":" << role.rank;
           pretty.emplace(std::piecewise_construct, std::forward_as_tuple(css->strv()), std::forward_as_tuple(status.second));
@@ -412,10 +407,9 @@ void FSMap::print_summary(Formatter *f, ostream *out) const
 
   size_t failed = 0;
   size_t damaged = 0;
-  for (const auto& p : filesystems) {
-    auto& fs = p.second;
-    failed += fs->mds_map.failed.size();
-    damaged += fs->mds_map.damaged.size();
+  for ([[maybe_unused]] const auto& [fscid, fs] : filesystems) {
+    failed += fs.mds_map.failed.size();
+    damaged += fs.mds_map.damaged.size();
   }
 
   if (failed > 0) {
@@ -449,34 +443,35 @@ mds_gid_t Filesystem::get_standby_replay(mds_gid_t who) const
   return MDS_GID_NONE;
 }
 
-Filesystem::ref FSMap::create_filesystem(std::string_view name,
+const Filesystem& FSMap::create_filesystem(std::string_view name,
     int64_t metadata_pool, int64_t data_pool, uint64_t features,
     fs_cluster_id_t fscid, bool recover)
 {
-  auto fs = Filesystem::create();
-  fs->mds_map.epoch = epoch;
-  fs->mds_map.fs_name = name;
-  fs->mds_map.data_pools.push_back(data_pool);
-  fs->mds_map.metadata_pool = metadata_pool;
-  fs->mds_map.cas_pool = -1;
-  fs->mds_map.compat = default_compat;
-  fs->mds_map.created = ceph_clock_now();
-  fs->mds_map.modified = ceph_clock_now();
-  fs->mds_map.enabled = true;
-  if (fscid == FS_CLUSTER_ID_NONE) {
-    fs->fscid = next_filesystem_id++;
-  } else {
-    fs->fscid = fscid;
-    next_filesystem_id = std::max(fscid,  (fs_cluster_id_t)next_filesystem_id) + 1;
-  }
+  auto fs = Filesystem();
+  fs.mds_map.epoch = epoch;
+  fs.mds_map.fs_name = name;
+  fs.mds_map.data_pools.push_back(data_pool);
+  fs.mds_map.metadata_pool = metadata_pool;
+  fs.mds_map.cas_pool = -1;
+  fs.mds_map.compat = default_compat;
+  fs.mds_map.created = ceph_clock_now();
+  fs.mds_map.modified = ceph_clock_now();
+  fs.mds_map.enabled = true;
 
   if (recover) {
     // Populate rank 0 as existing (so don't go into CREATING)
     // but failed (so that next available MDS is assigned the rank)
-    fs->mds_map.in.insert(mds_rank_t(0));
-    fs->mds_map.failed.insert(mds_rank_t(0));
+    fs.mds_map.in.insert(mds_rank_t(0));
+    fs.mds_map.failed.insert(mds_rank_t(0));
 
-    fs->mds_map.set_flag(CEPH_MDSMAP_NOT_JOINABLE);
+    fs.mds_map.set_flag(CEPH_MDSMAP_NOT_JOINABLE);
+  }
+
+  if (fscid == FS_CLUSTER_ID_NONE) {
+    fs.fscid = next_filesystem_id++;
+  } else {
+    fs.fscid = fscid;
+    next_filesystem_id = std::max(fscid,  (fs_cluster_id_t)next_filesystem_id) + 1;
   }
 
   // File system's ID can be FS_CLUSTER_ID_ANONYMOUS if we're recovering
@@ -485,83 +480,73 @@ Filesystem::ref FSMap::create_filesystem(std::string_view name,
   if (fscid != FS_CLUSTER_ID_ANONYMOUS) {
     // ANONYMOUS is only for upgrades from legacy mdsmaps, we should
     // have initialized next_filesystem_id such that it's never used here.
-    ceph_assert(fs->fscid != FS_CLUSTER_ID_ANONYMOUS);
+    ceph_assert(fs.fscid != FS_CLUSTER_ID_ANONYMOUS);
   }
-  filesystems[fs->fscid] = fs;
 
   // Created first filesystem?  Set it as the one
   // for legacy clients to use
-  if (filesystems.size() == 1) {
-    legacy_client_fscid = fs->fscid;
+  if (filesystems.size() == 0) {
+    legacy_client_fscid = fs.fscid;
   }
 
-  return fs;
+  auto [it, inserted] = filesystems.emplace(std::piecewise_construct, std::forward_as_tuple(fs.fscid), std::forward_as_tuple(std::move(fs)));
+  ceph_assert(inserted);
+  return it->second;
 }
 
-Filesystem::const_ref FSMap::get_filesystem(std::string_view name) const
+Filesystem const* FSMap::get_filesystem(std::string_view name) const
 {
-  for (const auto& p : filesystems) {
-    if (p.second->mds_map.fs_name == name) {
-      return p.second;
+  for ([[maybe_unused]] const auto& [fscid, fs] : filesystems) {
+    if (fs.mds_map.fs_name == name) {
+      return &fs;
     }
   }
   return nullptr;
 }
 
-std::vector<Filesystem::const_ref> FSMap::get_filesystems(void) const
-{
-  std::vector<Filesystem::const_ref> ret;
-  for (const auto& p : filesystems) {
-    ret.push_back(p.second);
-  }
-  return ret;
-}
-
 void FSMap::reset_filesystem(fs_cluster_id_t fscid)
 {
   auto fs = get_filesystem(fscid);
-  auto new_fs = Filesystem::create();
+  auto new_fs = Filesystem();
 
   // Populate rank 0 as existing (so don't go into CREATING)
   // but failed (so that next available MDS is assigned the rank)
-  new_fs->mds_map.in.insert(mds_rank_t(0));
-  new_fs->mds_map.failed.insert(mds_rank_t(0));
+  new_fs.mds_map.in.insert(mds_rank_t(0));
+  new_fs.mds_map.failed.insert(mds_rank_t(0));
 
   // Carry forward what makes sense
-  new_fs->fscid = fs->fscid;
-  new_fs->mds_map.inline_data_enabled = fs->mds_map.inline_data_enabled;
-  new_fs->mds_map.data_pools = fs->mds_map.data_pools;
-  new_fs->mds_map.metadata_pool = fs->mds_map.metadata_pool;
-  new_fs->mds_map.cas_pool = fs->mds_map.cas_pool;
-  new_fs->mds_map.fs_name = fs->mds_map.fs_name;
-  new_fs->mds_map.compat = default_compat;
-  new_fs->mds_map.created = ceph_clock_now();
-  new_fs->mds_map.modified = ceph_clock_now();
-  new_fs->mds_map.standby_count_wanted = fs->mds_map.standby_count_wanted;
-  new_fs->mds_map.enabled = true;
+  new_fs.fscid = fs.fscid;
+  new_fs.mds_map.inline_data_enabled = fs.mds_map.inline_data_enabled;
+  new_fs.mds_map.data_pools = fs.mds_map.data_pools;
+  new_fs.mds_map.metadata_pool = fs.mds_map.metadata_pool;
+  new_fs.mds_map.cas_pool = fs.mds_map.cas_pool;
+  new_fs.mds_map.fs_name = fs.mds_map.fs_name;
+  new_fs.mds_map.compat = default_compat;
+  new_fs.mds_map.created = ceph_clock_now();
+  new_fs.mds_map.modified = ceph_clock_now();
+  new_fs.mds_map.standby_count_wanted = fs.mds_map.standby_count_wanted;
+  new_fs.mds_map.enabled = true;
 
   // Remember mds ranks that have ever started. (They should load old inotable
   // instead of creating new one if they start again.)
-  new_fs->mds_map.stopped.insert(fs->mds_map.in.begin(), fs->mds_map.in.end());
-  new_fs->mds_map.stopped.insert(fs->mds_map.stopped.begin(), fs->mds_map.stopped.end());
-  new_fs->mds_map.stopped.erase(mds_rank_t(0));
+  new_fs.mds_map.stopped.insert(fs.mds_map.in.begin(), fs.mds_map.in.end());
+  new_fs.mds_map.stopped.insert(fs.mds_map.stopped.begin(), fs.mds_map.stopped.end());
+  new_fs.mds_map.stopped.erase(mds_rank_t(0));
 
   // Persist the new FSMap
-  filesystems[new_fs->fscid] = new_fs;
+  filesystems[new_fs.fscid] = new_fs;
 }
 
 void FSMap::get_health(list<pair<health_status_t,string> >& summary,
                        list<pair<health_status_t,string> > *detail) const
 {
   mds_rank_t standby_count_wanted = 0;
-  for (const auto &i : filesystems) {
-    const auto &fs = i.second;
-
+  for ([[maybe_unused]] const auto& [fscid, fs] : filesystems) {
     // TODO: move get_health up into here so that we can qualify
     // all the messages with what filesystem they're talking about
-    fs->mds_map.get_health(summary, detail);
+    fs.mds_map.get_health(summary, detail);
 
-    standby_count_wanted = std::max(standby_count_wanted, fs->mds_map.get_standby_count_wanted((mds_rank_t)standby_daemons.size()));
+    standby_count_wanted = std::max(standby_count_wanted, fs.mds_map.get_standby_count_wanted((mds_rank_t)standby_daemons.size()));
   }
 
   if (standby_count_wanted) {
@@ -574,8 +559,8 @@ void FSMap::get_health(list<pair<health_status_t,string> >& summary,
 bool FSMap::check_health(void)
 {
   bool changed = false;
-  for (auto &i : filesystems) {
-    changed |= i.second->mds_map.check_health((mds_rank_t)standby_daemons.size());
+  for ([[maybe_unused]] auto& [fscid, fs] : filesystems) {
+    changed |= fs.mds_map.check_health((mds_rank_t)standby_daemons.size());
   }
   return changed;
 }
@@ -583,19 +568,18 @@ bool FSMap::check_health(void)
 void FSMap::get_health_checks(health_check_map_t *checks) const
 {
   mds_rank_t standby_count_wanted = 0;
-  for (const auto &i : filesystems) {
-    const auto &fs = i.second;
+  for ([[maybe_unused]] const auto& [fscid, fs] : filesystems) {
     health_check_map_t fschecks;
 
-    fs->mds_map.get_health_checks(&fschecks);
+    fs.mds_map.get_health_checks(&fschecks);
 
     // Some of the failed ranks might be transient (i.e. there are standbys
     // ready to replace them).  We will report only on "stuck" failed, i.e.
     // ranks which are failed and have no standby replacement available.
     std::set<mds_rank_t> stuck_failed;
 
-    for (const auto &rank : fs->mds_map.failed) {
-      auto rep_info = find_replacement_for({fs->fscid, rank});
+    for (const auto &rank : fs.mds_map.failed) {
+      auto rep_info = find_replacement_for({fs.fscid, rank});
       if (!rep_info) {
         stuck_failed.insert(rank);
       }
@@ -607,14 +591,14 @@ void FSMap::get_health_checks(health_check_map_t *checks) const
         "FS_WITH_FAILED_MDS", HEALTH_WARN,
         "%num% filesystem%plurals% %hasorhave% a failed mds daemon", 1);
       CachedStackStringStream css;
-      *css << "fs " << fs->mds_map.fs_name << " has " << stuck_failed.size()
+      *css << "fs " << fs.mds_map.fs_name << " has " << stuck_failed.size()
          << " failed mds" << (stuck_failed.size() > 1 ? "s" : "");
       fscheck.detail.push_back(css->str()); }
 
     checks->merge(fschecks);
     standby_count_wanted = std::max(
       standby_count_wanted,
-      fs->mds_map.get_standby_count_wanted((mds_rank_t)standby_daemons.size()));
+      fs.mds_map.get_standby_count_wanted((mds_rank_t)standby_daemons.size()));
   }
 
   // MDS_INSUFFICIENT_STANDBY
@@ -637,10 +621,11 @@ void FSMap::encode(bufferlist& bl, uint64_t features) const
   encode(default_compat, bl);
   encode(enable_multiple, bl);
   {
-    std::vector<Filesystem::ref> v;
-    v.reserve(filesystems.size());
-    for (auto& p : filesystems) v.emplace_back(p.second);
-    encode(v, bl, features);
+    __u32 len = filesystems.size();
+    encode(len, bl, features);
+    for ([[maybe_unused]] const auto& [fscid, fs] : filesystems) {
+      encode(fs, bl, features);
+    }
   }
   encode(mds_roles, bl);
   encode(standby_daemons, bl, features);
@@ -661,12 +646,14 @@ void FSMap::decode(bufferlist::const_iterator& p)
   decode(default_compat, p);
   decode(enable_multiple, p);
   {
-    std::vector<Filesystem::ref> v;
-    decode(v, p);
+    __u32 len;
+    decode(len, p);
     filesystems.clear();
-    for (auto& ref : v) {
-      auto em = filesystems.emplace(std::piecewise_construct, std::forward_as_tuple(ref->fscid), std::forward_as_tuple(std::move(ref)));
-      ceph_assert(em.second);
+    for (__u32 i = 0; i < len; i++) {
+      auto fs = Filesystem();
+      decode(fs, p); /* need fscid to insert into map */
+      [[maybe_unused]] auto [it, inserted] = filesystems.emplace(std::piecewise_construct, std::forward_as_tuple(fs.fscid), std::forward_as_tuple(std::move(fs)));
+      ceph_assert(inserted);
     }
   }
   decode(mds_roles, p);
@@ -680,8 +667,8 @@ void FSMap::decode(bufferlist::const_iterator& p)
 
 void FSMap::sanitize(const std::function<bool(int64_t pool)>& pool_exists)
 {
-  for (auto &fs : filesystems) {
-    fs.second->mds_map.sanitize(pool_exists);
+  for ([[maybe_unused]] auto& [fscid, fs] : filesystems) {
+    fs.mds_map.sanitize(pool_exists);
   }
 }
 
@@ -710,24 +697,21 @@ void Filesystem::decode(bufferlist::const_iterator& p)
   DECODE_FINISH(p);
 }
 
-int FSMap::parse_filesystem(
-      std::string_view ns_str,
-      Filesystem::const_ref* result
-      ) const
+int FSMap::parse_filesystem(std::string_view ns_str, Filesystem const** result) const
 {
   std::string ns_err;
   std::string s(ns_str);
   fs_cluster_id_t fscid = strict_strtol(s.c_str(), 10, &ns_err);
   if (!ns_err.empty() || filesystems.count(fscid) == 0) {
-    for (auto &fs : filesystems) {
-      if (fs.second->mds_map.fs_name == s) {
-        *result = std::const_pointer_cast<const Filesystem>(fs.second);
+    for ([[maybe_unused]] auto& [fscid, fs] : filesystems) {
+      if (fs.mds_map.fs_name == s) {
+        *result = &fs;
         return 0;
       }
     }
     return -CEPHFS_ENOENT;
   } else {
-    *result = get_filesystem(fscid);
+    *result = &get_filesystem(fscid);
     return 0;
   }
 }
@@ -744,8 +728,8 @@ void Filesystem::print(std::ostream &out) const
 
 bool FSMap::is_any_degraded() const
 {
-  for (auto& i : filesystems) {
-    if (i.second->mds_map.is_degraded()) {
+  for ([[maybe_unused]] const auto& [fscid, fs] : filesystems) {
+    if (fs.mds_map.is_degraded()) {
       return true;
     }
   }
@@ -759,8 +743,8 @@ std::map<mds_gid_t, MDSMap::mds_info_t> FSMap::get_mds_info() const
     result[i.first] = i.second;
   }
 
-  for (const auto &i : filesystems) {
-    const auto &fs_info = i.second->mds_map.get_mds_info();
+  for ([[maybe_unused]] const auto& [fscid, fs] : filesystems) {
+    const auto &fs_info = fs.mds_map.get_mds_info();
     for (const auto &j : fs_info) {
       result[j.first] = j.second;
     }
@@ -819,8 +803,8 @@ const MDSMap::mds_info_t* FSMap::find_by_name(std::string_view name) const
     }
   }
 
-  for (const auto &i : filesystems) {
-    const auto &fs_info = i.second->mds_map.get_mds_info();
+  for ([[maybe_unused]] const auto& [fscid, fs] : filesystems) {
+    const auto &fs_info = fs.mds_map.get_mds_info();
     for (const auto &j : fs_info) {
       if (j.second.name == name) {
         return &(j.second);
@@ -833,22 +817,22 @@ const MDSMap::mds_info_t* FSMap::find_by_name(std::string_view name) const
 
 const MDSMap::mds_info_t* FSMap::find_replacement_for(mds_role_t role) const
 {
-  auto&& fs = get_filesystem(role.fscid);
+  auto& fs = get_filesystem(role.fscid);
 
   // First see if we have a STANDBY_REPLAY
-  for (const auto& [gid, info] : fs->mds_map.mds_info) {
+  for (const auto& [gid, info] : fs.mds_map.mds_info) {
     if (info.rank == role.rank && info.state == MDSMap::STATE_STANDBY_REPLAY) {
       if (info.is_frozen()) {
         /* the standby-replay is frozen, do nothing! */
         return nullptr;
       } else {
-        ceph_assert(info.compat.writeable(fs->mds_map.compat));
+        ceph_assert(info.compat.writeable(fs.mds_map.compat));
         return &info;
       }
     }
   }
 
-  return get_available_standby(*fs);
+  return get_available_standby(fs);
 }
 
 void FSMap::sanity(bool pending) const
@@ -861,28 +845,28 @@ void FSMap::sanity(bool pending) const
     ceph_assert(filesystems.count(legacy_client_fscid) == 1);
   }
 
-  for (const auto& [fscid, fs] : filesystems) {
-    ceph_assert(fscid  == fs->fscid);
-    for (const auto& [gid, info] : fs->mds_map.mds_info) {
+  for ([[maybe_unused]] const auto& [fscid, fs] : filesystems) {
+    ceph_assert(fscid  == fs.fscid);
+    for (const auto& [gid, info] : fs.mds_map.mds_info) {
       ceph_assert(info.rank != MDS_RANK_NONE);
       ceph_assert(mds_roles.at(gid) == fscid);
       ceph_assert(standby_daemons.count(gid) == 0);
       ceph_assert(standby_epochs.count(gid) == 0);
       if (info.state != MDSMap::STATE_STANDBY_REPLAY) {
-        ceph_assert(fs->mds_map.up.at(info.rank) == gid);
-        ceph_assert(fs->mds_map.failed.count(info.rank) == 0);
-        ceph_assert(fs->mds_map.damaged.count(info.rank) == 0);
+        ceph_assert(fs.mds_map.up.at(info.rank) == gid);
+        ceph_assert(fs.mds_map.failed.count(info.rank) == 0);
+        ceph_assert(fs.mds_map.damaged.count(info.rank) == 0);
       } else {
-        ceph_assert(!pending || fs->mds_map.allows_standby_replay());
+        ceph_assert(!pending || fs.mds_map.allows_standby_replay());
       }
-      ceph_assert(info.compat.writeable(fs->mds_map.compat));
+      ceph_assert(info.compat.writeable(fs.mds_map.compat));
     }
 
-    for (const auto &j : fs->mds_map.up) {
+    for (const auto &j : fs.mds_map.up) {
       mds_rank_t rank = j.first;
-      ceph_assert(fs->mds_map.in.count(rank) == 1);
+      ceph_assert(fs.mds_map.in.count(rank) == 1);
       mds_gid_t gid = j.second;
-      ceph_assert(fs->mds_map.mds_info.count(gid) == 1);
+      ceph_assert(fs.mds_map.mds_info.count(gid) == 1);
     }
   }
 
@@ -904,14 +888,14 @@ void FSMap::sanity(bool pending) const
       ceph_assert(standby_daemons.count(i.first) == 1);
     } else {
       ceph_assert(filesystems.count(i.second) == 1);
-      ceph_assert(filesystems.at(i.second)->mds_map.mds_info.count(i.first) == 1);
+      ceph_assert(filesystems.at(i.second).mds_map.mds_info.count(i.first) == 1);
     }
   }
 }
 
 void FSMap::promote(
     mds_gid_t standby_gid,
-    Filesystem& filesystem,
+    fs_cluster_id_t fscid,
     mds_rank_t assigned_rank)
 {
   ceph_assert(gid_exists(standby_gid));
@@ -921,7 +905,8 @@ void FSMap::promote(
     ceph_assert(standby_daemons.at(standby_gid).state == MDSMap::STATE_STANDBY);
   }
 
-  MDSMap &mds_map = filesystem.mds_map;
+  auto& fs = filesystems.at(fscid);
+  MDSMap &mds_map = fs.mds_map;
 
   // Insert daemon state to Filesystem
   if (!is_standby_replay) {
@@ -933,9 +918,9 @@ void FSMap::promote(
   }
   auto& info = mds_map.mds_info.at(standby_gid);
 
-  if (!filesystem.mds_map.compat.writeable(info.compat)) {
-    ceph_assert(filesystem.is_upgradeable());
-    filesystem.mds_map.compat.merge(info.compat);
+  if (!fs.mds_map.compat.writeable(info.compat)) {
+    ceph_assert(fs.is_upgradeable());
+    fs.mds_map.compat.merge(info.compat);
   }
 
   if (mds_map.stopped.erase(assigned_rank)) {
@@ -951,7 +936,7 @@ void FSMap::promote(
   }
   info.rank = assigned_rank;
   info.inc = epoch;
-  mds_roles.at(standby_gid) = filesystem.fscid;
+  mds_roles.at(standby_gid) = fscid;
 
   // Update the rank state in Filesystem
   mds_map.in.insert(assigned_rank);
@@ -965,6 +950,7 @@ void FSMap::promote(
 
   // Indicate that Filesystem has been modified
   mds_map.epoch = epoch;
+  mds_map.modified = ceph_clock_now();
 }
 
 void FSMap::assign_standby_replay(
@@ -978,10 +964,10 @@ void FSMap::assign_standby_replay(
   ceph_assert(standby_daemons.count(standby_gid));
 
   // Insert to the filesystem
-  auto fs = filesystems.at(leader_ns);
-  fs->mds_map.mds_info[standby_gid] = standby_daemons.at(standby_gid);
-  fs->mds_map.mds_info[standby_gid].rank = leader_rank;
-  fs->mds_map.mds_info[standby_gid].state = MDSMap::STATE_STANDBY_REPLAY;
+  auto& fs = filesystems.at(leader_ns);
+  fs.mds_map.mds_info[standby_gid] = standby_daemons.at(standby_gid);
+  fs.mds_map.mds_info[standby_gid].rank = leader_rank;
+  fs.mds_map.mds_info[standby_gid].state = MDSMap::STATE_STANDBY_REPLAY;
   mds_roles[standby_gid] = leader_ns;
 
   // Remove from the list of standbys
@@ -989,7 +975,8 @@ void FSMap::assign_standby_replay(
   standby_epochs.erase(standby_gid);
 
   // Indicate that Filesystem has been modified
-  fs->mds_map.epoch = epoch;
+  fs.mds_map.epoch = epoch;
+  fs.mds_map.modified = ceph_clock_now();
 }
 
 void FSMap::erase(mds_gid_t who, epoch_t blocklist_epoch)
@@ -998,25 +985,26 @@ void FSMap::erase(mds_gid_t who, epoch_t blocklist_epoch)
     standby_daemons.erase(who);
     standby_epochs.erase(who);
   } else {
-    auto &fs = filesystems.at(mds_roles.at(who));
-    const auto &info = fs->mds_map.mds_info.at(who);
+    autofs = filesystems.at(mds_roles.at(who));
+    const auto &info = fs.mds_map.mds_info.at(who);
     if (info.state != MDSMap::STATE_STANDBY_REPLAY) {
       if (info.state == MDSMap::STATE_CREATING) {
         // If this gid didn't make it past CREATING, then forget
         // the rank ever existed so that next time it's handed out
         // to a gid it'll go back into CREATING.
-        fs->mds_map.in.erase(info.rank);
+        fs.mds_map.in.erase(info.rank);
       } else {
         // Put this rank into the failed list so that the next available
         // STANDBY will pick it up.
-        fs->mds_map.failed.insert(info.rank);
+        fs.mds_map.failed.insert(info.rank);
       }
-      ceph_assert(fs->mds_map.up.at(info.rank) == info.global_id);
-      fs->mds_map.up.erase(info.rank);
+      ceph_assert(fs.mds_map.up.at(info.rank) == info.global_id);
+      fs.mds_map.up.erase(info.rank);
     }
-    fs->mds_map.mds_info.erase(who);
-    fs->mds_map.last_failure_osd_epoch = blocklist_epoch;
-    fs->mds_map.epoch = epoch;
+    fs.mds_map.mds_info.erase(who);
+    fs.mds_map.last_failure_osd_epoch = blocklist_epoch;
+    fs.mds_map.epoch = epoch;
+    fs.mds_map.modified = ceph_clock_now();
   }
 
   mds_roles.erase(who);
@@ -1025,14 +1013,14 @@ void FSMap::erase(mds_gid_t who, epoch_t blocklist_epoch)
 void FSMap::damaged(mds_gid_t who, epoch_t blocklist_epoch)
 {
   ceph_assert(mds_roles.at(who) != FS_CLUSTER_ID_NONE);
-  auto fs = filesystems.at(mds_roles.at(who));
-  mds_rank_t rank = fs->mds_map.mds_info.at(who).rank;
+  auto& fs = filesystems.at(mds_roles.at(who));
+  mds_rank_t rank = fs.mds_map.mds_info.at(who).rank;
 
   erase(who, blocklist_epoch);
-  fs->mds_map.failed.erase(rank);
-  fs->mds_map.damaged.insert(rank);
+  fs.mds_map.failed.erase(rank);
+  fs.mds_map.damaged.insert(rank);
 
-  ceph_assert(fs->mds_map.epoch == epoch);
+  ceph_assert(fs.mds_map.epoch == epoch);
 }
 
 /**
@@ -1041,11 +1029,12 @@ void FSMap::damaged(mds_gid_t who, epoch_t blocklist_epoch)
  */
 bool FSMap::undamaged(const fs_cluster_id_t fscid, const mds_rank_t rank)
 {
-  auto fs = filesystems.at(fscid);
+  auto& fs = filesystems.at(fscid);
 
-  if (fs->mds_map.damaged.erase(rank)) {
-    fs->mds_map.failed.insert(rank);
-    fs->mds_map.epoch = epoch;
+  if (fs.mds_map.damaged.erase(rank)) {
+    fs.mds_map.failed.insert(rank);
+    fs.mds_map.epoch = epoch;
+    fs.mds_map.modified = ceph_clock_now();
     return true;
   } else {
     return false;
@@ -1076,15 +1065,15 @@ void FSMap::insert(const MDSMap::mds_info_t &new_info)
 std::vector<mds_gid_t> FSMap::stop(mds_gid_t who)
 {
   ceph_assert(mds_roles.at(who) != FS_CLUSTER_ID_NONE);
-  auto fs = filesystems.at(mds_roles.at(who));
-  const auto &info = fs->mds_map.mds_info.at(who);
-  fs->mds_map.up.erase(info.rank);
-  fs->mds_map.in.erase(info.rank);
-  fs->mds_map.stopped.insert(info.rank);
+  auto& fs = filesystems.at(mds_roles.at(who));
+  const auto &info = fs.mds_map.mds_info.at(who);
+  fs.mds_map.up.erase(info.rank);
+  fs.mds_map.in.erase(info.rank);
+  fs.mds_map.stopped.insert(info.rank);
 
   // Also drop any standby replays that were following this rank
   std::vector<mds_gid_t> standbys;
-  for (const auto &i : fs->mds_map.mds_info) {
+  for (const auto &i : fs.mds_map.mds_info) {
     const auto &other_gid = i.first;
     const auto &other_info = i.second;
     if (other_info.rank == info.rank
@@ -1097,10 +1086,11 @@ std::vector<mds_gid_t> FSMap::stop(mds_gid_t who)
     erase(other_gid, 0);
   }
 
-  fs->mds_map.mds_info.erase(who);
+  fs.mds_map.mds_info.erase(who);
   mds_roles.erase(who);
 
-  fs->mds_map.epoch = epoch;
+  fs.mds_map.epoch = epoch;
+  fs.mds_map.modified = ceph_clock_now();
 
   return standbys;
 }
@@ -1125,7 +1115,7 @@ int FSMap::parse_role(
   int r = parse_role(role_str, role, ss);
   if (r < 0) return r;
 
-  string_view fs_name = get_filesystem(role->fscid)->mds_map.get_fs_name();
+  string_view fs_name = get_filesystem(role->fscid).mds_map.get_fs_name();
 
   if (!filter.empty() &&
       std::find(filter.begin(), filter.end(), fs_name) == filter.end()) {
@@ -1145,13 +1135,13 @@ int FSMap::parse_role(
 {
   size_t colon_pos = role_str.find(":");
   size_t rank_pos;
-  Filesystem::const_ref fs;
+  Filesystem const* fs;
   if (colon_pos == std::string::npos) {
     if (legacy_client_fscid == FS_CLUSTER_ID_NONE) {
       ss << "No filesystem selected";
       return -CEPHFS_ENOENT;
     }
-    fs = get_filesystem(legacy_client_fscid);
+    fs = &get_filesystem(legacy_client_fscid);
     rank_pos = 0;
   } else {
     if (parse_filesystem(role_str.substr(0, colon_pos), &fs) < 0) {
@@ -1184,9 +1174,9 @@ int FSMap::parse_role(
 
 bool FSMap::pool_in_use(int64_t poolid) const
 {
-  for (auto const &i : filesystems) {
-    if (i.second->mds_map.is_data_pool(poolid)
-        || i.second->mds_map.metadata_pool == poolid) {
+  for ([[maybe_unused]] auto const& [fscid, fs] : filesystems) {
+    if (fs.mds_map.is_data_pool(poolid)
+        || fs.mds_map.metadata_pool == poolid) {
       return true;
     }
   }
@@ -1203,8 +1193,8 @@ void FSMap::erase_filesystem(fs_cluster_id_t fscid)
       });
     }
   }
-  for (auto& p : filesystems) {
-    for (auto& [gid, info] : p.second->mds_map.get_mds_info()) {
+  for ([[maybe_unused]] auto& [fscid, fs] : filesystems) {
+    for (auto& [gid, info] : fs.mds_map.get_mds_info()) {
       if (info.join_fscid == fscid) {
         modify_daemon(gid, [](auto& info) {
           info.join_fscid = FS_CLUSTER_ID_NONE;
index 5bf2f6b267f1d2ab75c5aa2c478f4fc1a4013461..3961c2c8d10b87e574c0051030965eb74dc2e6b6 100644 (file)
@@ -189,14 +189,7 @@ WRITE_CLASS_ENCODER(MirrorInfo)
 class Filesystem
 {
 public:
-  using ref = std::shared_ptr<Filesystem>;
-  using const_ref = std::shared_ptr<Filesystem const>;
-
-  template<typename... Args>
-  static ref create(Args&&... args)
-  {
-    return std::make_shared<Filesystem>(std::forward<Args>(args)...);
-  }
+  Filesystem() = default;
 
   void encode(ceph::buffer::list& bl, uint64_t features) const;
   void decode(ceph::buffer::list::const_iterator& p);
@@ -228,6 +221,32 @@ public:
     return false;
   }
 
+  const auto& get_mds_map() const
+  {
+    return mds_map;
+  }
+  auto& get_mds_map()
+  {
+    return mds_map;
+  }
+
+  const auto& get_mirror_info() const
+  {
+    return mirror_info;
+  }
+  auto& get_mirror_info()
+  {
+    return mirror_info;
+  }
+
+  auto get_fscid() const
+  {
+    return fscid;
+  }
+
+private:
+  friend class FSMap;
+
   fs_cluster_id_t fscid = FS_CLUSTER_ID_NONE;
   MDSMap mds_map;
   MirrorInfo mirror_info;
@@ -236,9 +255,10 @@ WRITE_CLASS_ENCODER_FEATURES(Filesystem)
 
 class FSMap {
 public:
-  friend class MDSMonitor;
-  friend class PaxosFSMap;
   using mds_info_t = MDSMap::mds_info_t;
+  using fsmap = typename std::map<fs_cluster_id_t, Filesystem>;
+  using const_iterator = typename fsmap::const_iterator;
+  using iterator = typename fsmap::iterator;
 
   static const version_t STRUCT_VERSION = 7;
   static const version_t STRUCT_VERSION_TRIM_TO = 7;
@@ -259,15 +279,20 @@ public:
       struct_version(rhs.struct_version)
   {
     filesystems.clear();
-    for (const auto &i : rhs.filesystems) {
-      const auto &fs = i.second;
-      filesystems[fs->fscid] = std::make_shared<Filesystem>(*fs);
-    }
+    filesystems = rhs.filesystems;
   }
 
   FSMap &operator=(const FSMap &rhs);
 
-  const CompatSet &get_default_compat() const {return default_compat;}
+  const_iterator begin() const {
+    return filesystems.begin();
+  }
+  const_iterator end() const {
+    return filesystems.end();
+  }
+
+  const CompatSet& get_default_compat() const {return default_compat;}
+  CompatSet& get_default_compat() {return default_compat;}
 
   void filter(const std::vector<std::string>& allowed)
   {
@@ -276,7 +301,7 @@ public:
     }
 
     erase_if(filesystems, [&](const auto& f) {
-      return std::find(allowed.begin(), allowed.end(), f.second->mds_map.get_fs_name()) == allowed.end();
+      return std::find(allowed.begin(), allowed.end(), f.second.mds_map.get_fs_name()) == allowed.end();
     });
 
     erase_if(mds_roles, [&](const auto& r) {
@@ -381,7 +406,7 @@ public:
    */
   void promote(
       mds_gid_t standby_gid,
-      Filesystem& filesystem,
+      fs_cluster_id_t fscid,
       mds_rank_t assigned_rank);
 
   /**
@@ -417,10 +442,9 @@ public:
    * Caller must already have validated all arguments vs. the existing
    * FSMap and OSDMap contents.
    */
-  Filesystem::ref create_filesystem(
-      std::string_view name, int64_t metadata_pool,
-      int64_t data_pool, uint64_t features,
-      fs_cluster_id_t fscid, bool recover);
+  const Filesystem& create_filesystem(
+      std::string_view name, int64_t metadata_pool, int64_t data_pool,
+      uint64_t features, fs_cluster_id_t fscid, bool recover);
 
   /**
    * Remove the filesystem (it must exist).  Caller should already
@@ -444,7 +468,8 @@ public:
   {
     auto& fs = filesystems.at(fscid);
     fn(fs);
-    fs->mds_map.epoch = epoch;
+    fs.mds_map.epoch = epoch;
+    fs.mds_map.modified = ceph_clock_now();
   }
 
   /**
@@ -462,9 +487,10 @@ public:
       standby_epochs[who] = epoch;
     } else {
       auto& fs = filesystems.at(fscid);
-      auto& info = fs->mds_map.mds_info.at(who);
+      auto& info = fs.mds_map.mds_info.at(who);
       fn(info);
-      fs->mds_map.epoch = epoch;
+      fs.mds_map.epoch = epoch;
+      fs.mds_map.modified = ceph_clock_now();
     }
   }
 
@@ -478,7 +504,7 @@ public:
     if (fscid == FS_CLUSTER_ID_NONE) {
       return standby_daemons.at(gid);
     } else {
-      return filesystems.at(fscid)->mds_map.mds_info.at(gid);
+      return filesystems.at(fscid).mds_map.mds_info.at(gid);
     }
   }
 
@@ -488,26 +514,26 @@ public:
     if (fscid == FS_CLUSTER_ID_NONE or !filesystem_exists(fscid)) {
       return std::string_view();
     } else {
-      return get_filesystem(fscid)->mds_map.get_fs_name();
+      return filesystems.at(fscid).mds_map.get_fs_name();
     }
   }
 
   bool is_standby_replay(mds_gid_t who) const
   {
-    return filesystems.at(mds_roles.at(who))->is_standby_replay(who);
+    return filesystems.at(mds_roles.at(who)).is_standby_replay(who);
   }
 
   mds_gid_t get_standby_replay(mds_gid_t who) const
   {
-    return filesystems.at(mds_roles.at(who))->get_standby_replay(who);
+    return filesystems.at(mds_roles.at(who)).get_standby_replay(who);
   }
 
-  Filesystem::const_ref get_legacy_filesystem()
+  const Filesystem* get_legacy_filesystem() const
   {
     if (legacy_client_fscid == FS_CLUSTER_ID_NONE) {
       return nullptr;
     } else {
-      return filesystems.at(legacy_client_fscid);
+      return &filesystems.at(legacy_client_fscid);
     }
   }
 
@@ -518,7 +544,7 @@ public:
   {
     auto fscid = mds_roles.at(who);
     modify_filesystem(fscid, [who, &targets](auto&& fs) {
-      fs->mds_map.mds_info.at(who).export_targets = targets;
+      fs.mds_map.mds_info.at(who).export_targets = targets;
     });
   }
 
@@ -530,25 +556,24 @@ public:
     return struct_version < STRUCT_VERSION_TRIM_TO;
   }
 
-  size_t filesystem_count() const {return filesystems.size();}
-  bool filesystem_exists(fs_cluster_id_t fscid) const {return filesystems.count(fscid) > 0;}
-  Filesystem::const_ref get_filesystem(fs_cluster_id_t fscid) const {return std::const_pointer_cast<const Filesystem>(filesystems.at(fscid));}
-  Filesystem::ref get_filesystem(fs_cluster_id_t fscid) {return filesystems.at(fscid);}
-  Filesystem::ref get_filesystem(mds_gid_t gid) {
-    return filesystems.at(mds_roles.at(gid));
+  size_t filesystem_count() const {
+    return filesystems.size();
   }
-  Filesystem::const_ref get_filesystem(void) const {return std::const_pointer_cast<const Filesystem>(filesystems.begin()->second);}
-  Filesystem::const_ref get_filesystem(std::string_view name) const;
-  Filesystem::const_ref get_filesystem(mds_gid_t gid) const {
+  bool filesystem_exists(fs_cluster_id_t fscid) const {
+    return filesystems.count(fscid) > 0;
+  }
+  const Filesystem& get_filesystem(fs_cluster_id_t fscid) const {
+    return filesystems.at(fscid);
+  }
+  const Filesystem& get_filesystem(void) const {
+    return filesystems.at(filesystems.begin()->second.fscid);
+  }
+  Filesystem const* get_filesystem(std::string_view name) const;
+  const Filesystem& get_filesystem(mds_gid_t gid) const {
     return filesystems.at(mds_roles.at(gid));
   }
 
-  std::vector<Filesystem::const_ref> get_filesystems(void) const;
-
-  int parse_filesystem(
-      std::string_view ns_str,
-      Filesystem::const_ref *result
-      ) const;
+  int parse_filesystem(std::string_view ns_str, Filesystem const** result) const;
 
   int parse_role(
       std::string_view role_str,
@@ -575,6 +600,18 @@ public:
 
   bool check_health(void);
 
+  const auto& get_mds_roles() const {
+    return mds_roles;
+  }
+
+  const auto& get_standby_daemons() const {
+    return standby_daemons;
+  }
+
+  const auto& get_standby_epochs() const {
+    return standby_epochs;
+  }
+
   /**
    * Assert that the FSMap, Filesystem, MDSMap, mds_info_t relations are
    * all self-consistent.
@@ -598,6 +635,13 @@ public:
   static void generate_test_instances(std::list<FSMap*>& ls);
 
 protected:
+  iterator begin() {
+    return filesystems.begin();
+  }
+  iterator end() {
+    return filesystems.end();
+  }
+
   epoch_t epoch = 0;
   uint64_t next_filesystem_id = FS_CLUSTER_ID_ANONYMOUS + 1;
   fs_cluster_id_t legacy_client_fscid = FS_CLUSTER_ID_NONE;
@@ -605,7 +649,7 @@ protected:
   bool enable_multiple = true;
   bool ever_enabled_multiple = true; // < the cluster had multiple FS enabled once
 
-  std::map<fs_cluster_id_t, Filesystem::ref> filesystems;
+  fsmap filesystems;
 
   // Remember which Filesystem an MDS daemon's info is stored in
   // (or in standby_daemons for FS_CLUSTER_ID_NONE)
index 6ea1de533de7ce9d42e6e69b75bd7ecd5cca3cab..6c6270364919ccff2fe2d93c3c76143d727a2bdf 100644 (file)
@@ -818,7 +818,6 @@ void MDSMap::decode(bufferlist::const_iterator& p)
 {
   std::map<mds_rank_t,int32_t> inc;  // Legacy field, parse and drop
 
-  cached_up_features = 0;
   DECODE_START_LEGACY_COMPAT_LEN_16(5, 4, 4, p);
   decode(epoch, p);
   decode(flags, p);
@@ -1099,24 +1098,20 @@ void MDSMap::get_up_mds_set(std::set<mds_rank_t>& s) const {
     s.insert(p->first);
 }
 
-uint64_t MDSMap::get_up_features() {
-  if (!cached_up_features) {
-    bool first = true;
-    for (std::map<mds_rank_t, mds_gid_t>::const_iterator p = up.begin();
-         p != up.end();
-         ++p) {
-      std::map<mds_gid_t, mds_info_t>::const_iterator q =
-        mds_info.find(p->second);
-      ceph_assert(q != mds_info.end());
-      if (first) {
-        cached_up_features = q->second.mds_features;
-        first = false;
-      } else {
-        cached_up_features &= q->second.mds_features;
-      }
+uint64_t MDSMap::get_up_features() const {
+  uint64_t features = 0;
+  bool first = true;
+  for ([[maybe_unused]] auto& [rank, gid] : up) {
+    auto it = mds_info.find(gid);
+    ceph_assert(it != mds_info.end());
+    if (first) {
+      features = it->second.mds_features;
+      first = false;
+    } else {
+      features &= it->second.mds_features;
     }
   }
-  return cached_up_features;
+  return features;
 }
 
 void MDSMap::get_recovery_mds_set(std::set<mds_rank_t>& s) const {
index 7e1814e5977a49e7b71b40537badd2af73665f62..800484a1741c319fec86162af8127ac489df160e 100644 (file)
@@ -381,7 +381,7 @@ public:
   }
 
   // features
-  uint64_t get_up_features();
+  uint64_t get_up_features() const;
 
   /**
    * Get MDS ranks which are in but not up.
@@ -668,7 +668,6 @@ protected:
 
   bool inline_data_enabled = false;
 
-  uint64_t cached_up_features = 0;
 private:
   inline static const std::map<int, std::string> flag_display = {
     {CEPH_MDSMAP_NOT_JOINABLE, "joinable"}, //inverse for user display
index 8cb6789394b36dd08cc8aa99f250f14f47cbeb1f..8dadce7f0375d2cac6682d860ee682b0b2c20713 100644 (file)
@@ -1705,15 +1705,16 @@ bool AuthMonitor::prepare_command(MonOpRequestRef op)
     string mds_cap_string, osd_cap_string;
     string osd_cap_wanted = "r";
 
-    std::shared_ptr<const Filesystem> fs;
+    const Filesystem* fs = nullptr;
     if (filesystem != "*" && filesystem != "all") {
-      fs = mon.mdsmon()->get_fsmap().get_filesystem(filesystem);
+      const auto& fsmap = mon.mdsmon()->get_fsmap();
+      fs = fsmap.get_filesystem(filesystem);
       if (fs == nullptr) {
        ss << "filesystem " << filesystem << " does not exist.";
        err = -EINVAL;
        goto done;
       } else {
-       mon_cap_string += " fsname=" + std::string(fs->mds_map.get_fs_name());
+       mon_cap_string += " fsname=" + std::string(fs->get_mds_map().get_fs_name());
       }
     }
 
@@ -1760,7 +1761,7 @@ bool AuthMonitor::prepare_command(MonOpRequestRef op)
       mds_cap_string += "allow " + cap;
 
       if (filesystem != "*" && filesystem != "all" && fs != nullptr) {
-       mds_cap_string += " fsname=" + std::string(fs->mds_map.get_fs_name());
+       mds_cap_string += " fsname=" + std::string(fs->get_mds_map().get_fs_name());
       }
 
       if (path != "/") {
index 4ea716acaf50dfdac554bcd558e047a92f25b5bc..54df36a1b8f81ac1cb436501fca520a8230a8614 100644 (file)
@@ -112,15 +112,19 @@ class FailHandler : public FileSystemCommandHandler
       return -EINVAL;
     }
 
-    auto fs = fsmap.get_filesystem(fs_name);
+    auto* fsp = fsmap.get_filesystem(fs_name);
+    if (fsp == nullptr) {
+      ss << "Not found: '" << fs_name << "'";
+      return -ENOENT;
+    }
 
-    auto f = [](auto fs) {
-      fs->mds_map.set_flag(CEPH_MDSMAP_NOT_JOINABLE);
+    auto f = [](auto&& fs) {
+      fs.get_mds_map().set_flag(CEPH_MDSMAP_NOT_JOINABLE);
     };
-    fsmap.modify_filesystem(fs->fscid, std::move(f));
+    fsmap.modify_filesystem(fsp->get_fscid(), std::move(f));
 
     std::vector<mds_gid_t> to_fail;
-    for (const auto& p : fs->mds_map.get_mds_info()) {
+    for (const auto& p : fsp->get_mds_map().get_mds_info()) {
       to_fail.push_back(p.first);
     }
 
@@ -184,10 +188,9 @@ class FsNewHandler : public FileSystemCommandHandler
         return -EINVAL;
     }
 
-    if (fsmap.get_filesystem(fs_name)) {
-      auto fs = fsmap.get_filesystem(fs_name);
-      if (*(fs->mds_map.get_data_pools().begin()) == data
-          && fs->mds_map.get_metadata_pool() == metadata) {
+    if (auto* fsp = fsmap.get_filesystem(fs_name); fsp) {
+      if (*(fsp->get_mds_map().get_data_pools().begin()) == data
+          && fsp->get_mds_map().get_metadata_pool() == metadata) {
         // Identical FS created already, this is a no-op
         ss << "filesystem '" << fs_name << "' already exists";
         return 0;
@@ -221,10 +224,10 @@ class FsNewHandler : public FileSystemCommandHandler
     bool allow_overlay = false;
     cmd_getval(cmdmap, "allow_dangerous_metadata_overlay", allow_overlay);
 
-    for (auto& fs : fsmap.get_filesystems()) {
-      const std::vector<int64_t> &data_pools = fs->mds_map.get_data_pools();
+    for (const auto& [fscid, fs] : std::as_const(fsmap)) {
+      const std::vector<int64_t> &data_pools = fs.get_mds_map().get_data_pools();
       if ((std::find(data_pools.begin(), data_pools.end(), data) != data_pools.end()
-          || fs->mds_map.get_metadata_pool() == metadata)
+          || fs.get_mds_map().get_metadata_pool() == metadata)
          && !allow_overlay) {
        ss << "Filesystem '" << fs_name
           << "' is already using one of the specified RADOS pools. This should ONLY be done in emergencies and after careful reading of the documentation. Pass --allow-dangerous-metadata-overlay to permit this.";
@@ -295,12 +298,12 @@ class FsNewHandler : public FileSystemCommandHandler
     }
 
     // assign a standby to rank 0 to avoid health warnings
-    auto info = fsmap.find_replacement_for({fs->fscid, 0});
+    auto info = fsmap.find_replacement_for({fs.get_fscid(), 0});
 
     if (info) {
       mon->clog->info() << info->human_name() << " assigned to filesystem "
           << fs_name << " as rank 0";
-      fsmap.promote(info->global_id, *fs, 0);
+      fsmap.promote(info->global_id, fs.get_fscid(), 0);
     }
 
     return 0;
@@ -330,7 +333,7 @@ public:
       return -EINVAL;
     }
 
-    auto fs = fsmap.get_filesystem(fs_name);
+    auto* fsp = fsmap.get_filesystem(fs_name);
     string var;
     if (!cmd_getval(cmdmap, "var", var) || var.empty()) {
       ss << "Invalid variable";
@@ -356,9 +359,9 @@ public:
         return -EINVAL;
       }
 
-      if (n > 1 && n > fs->mds_map.get_max_mds()) {
-       if (fs->mds_map.was_snaps_ever_allowed() &&
-           !fs->mds_map.allows_multimds_snaps()) {
+      if (n > 1 && n > fsp->get_mds_map().get_max_mds()) {
+       if (fsp->get_mds_map().was_snaps_ever_allowed() &&
+           !fsp->get_mds_map().allows_multimds_snaps()) {
          ss << "multi-active MDS is not allowed while there are snapshots possibly created by pre-mimic MDS";
          return -EINVAL;
        }
@@ -369,11 +372,11 @@ public:
       }
 
       fsmap.modify_filesystem(
-          fs->fscid,
-          [n](std::shared_ptr<Filesystem> fs)
+          fsp->get_fscid(),
+          [n](auto&& fs)
       {
-       fs->mds_map.clear_flag(CEPH_MDSMAP_NOT_JOINABLE);
-        fs->mds_map.set_max_mds(n);
+       fs.get_mds_map().clear_flag(CEPH_MDSMAP_NOT_JOINABLE);
+        fs.get_mds_map().set_max_mds(n);
       });
     } else if (var == "inline_data") {
       bool enable_inline = false;
@@ -393,18 +396,18 @@ public:
        ss << "inline data enabled";
 
         fsmap.modify_filesystem(
-            fs->fscid,
-            [](std::shared_ptr<Filesystem> fs)
+            fsp->get_fscid(),
+            [](auto&& fs)
         {
-          fs->mds_map.set_inline_data_enabled(true);
+          fs.get_mds_map().set_inline_data_enabled(true);
         });
       } else {
        ss << "inline data disabled";
         fsmap.modify_filesystem(
-            fs->fscid,
-            [](std::shared_ptr<Filesystem> fs)
+            fsp->get_fscid(),
+            [](auto&& fs)
         {
-          fs->mds_map.set_inline_data_enabled(false);
+          fs.get_mds_map().set_inline_data_enabled(false);
         });
       }
     } else if (var == "balancer") {
@@ -414,10 +417,10 @@ public:
         ss << "setting the metadata load balancer to " << val;
       }
       fsmap.modify_filesystem(
-       fs->fscid,
-       [val](std::shared_ptr<Filesystem> fs)
+       fsp->get_fscid(),
+       [val](auto&& fs)
         {
-          fs->mds_map.set_balancer(val);
+          fs.get_mds_map().set_balancer(val);
         });
       return true;
     } else if (var == "bal_rank_mask") {
@@ -426,9 +429,9 @@ public:
        return -EINVAL;
       }
 
-      if (fs->mds_map.check_special_bal_rank_mask(val, MDSMap::BAL_RANK_MASK_TYPE_ANY) == false) {
+      if (fsp->get_mds_map().check_special_bal_rank_mask(val, MDSMap::BAL_RANK_MASK_TYPE_ANY) == false) {
        std::string bin_string;
-       int r = fs->mds_map.hex2bin(val, bin_string, MAX_MDS, ss);
+       int r = fsp->get_mds_map().hex2bin(val, bin_string, MAX_MDS, ss);
        if (r != 0) {
          return r;
        }
@@ -436,10 +439,10 @@ public:
       ss << "setting the metadata balancer rank mask to " << val;
 
       fsmap.modify_filesystem(
-       fs->fscid,
-       [val](std::shared_ptr<Filesystem> fs)
+       fsp->get_fscid(),
+       [val](auto&& fs)
         {
-          fs->mds_map.set_bal_rank_mask(val);
+          fs.get_mds_map().set_bal_rank_mask(val);
         });
       return true;
     } else if (var == "max_file_size") {
@@ -452,10 +455,10 @@ public:
        return -ERANGE;
       }
       fsmap.modify_filesystem(
-          fs->fscid,
-          [n](std::shared_ptr<Filesystem> fs)
+          fsp->get_fscid(),
+          [n](auto&& fs)
       {
-        fs->mds_map.set_max_filesize(n);
+        fs.get_mds_map().set_max_filesize(n);
       });
     } else if (var == "max_xattr_size") {
       if (interr.length()) {
@@ -463,10 +466,10 @@ public:
        return -EINVAL;
       }
       fsmap.modify_filesystem(
-          fs->fscid,
-          [n](std::shared_ptr<Filesystem> fs)
+          fsp->get_fscid(),
+          [n](auto&& fs)
       {
-        fs->mds_map.set_max_xattr_size(n);
+        fs.get_mds_map().set_max_xattr_size(n);
       });
     } else if (var == "allow_new_snaps") {
       bool enable_snaps = false;
@@ -477,18 +480,18 @@ public:
 
       if (!enable_snaps) {
         fsmap.modify_filesystem(
-            fs->fscid,
-            [](std::shared_ptr<Filesystem> fs)
+            fsp->get_fscid(),
+            [](auto&& fs)
         {
-          fs->mds_map.clear_snaps_allowed();
+          fs.get_mds_map().clear_snaps_allowed();
         });
        ss << "disabled new snapshots";
       } else {
         fsmap.modify_filesystem(
-            fs->fscid,
-            [](std::shared_ptr<Filesystem> fs)
+            fsp->get_fscid(),
+            [](auto&& fs)
         {
-          fs->mds_map.set_snaps_allowed();
+          fs.get_mds_map().set_snaps_allowed();
         });
        ss << "enabled new snapshots";
       }
@@ -514,18 +517,18 @@ public:
       if (enable) {
        ss << "enabled multimds with snapshot";
         fsmap.modify_filesystem(
-            fs->fscid,
-            [](std::shared_ptr<Filesystem> fs)
+            fsp->get_fscid(),
+            [](auto&& fs)
         {
-         fs->mds_map.set_multimds_snaps_allowed();
+         fs.get_mds_map().set_multimds_snaps_allowed();
         });
       } else {
        ss << "disabled multimds with snapshot";
         fsmap.modify_filesystem(
-            fs->fscid,
-            [](std::shared_ptr<Filesystem> fs)
+            fsp->get_fscid(),
+            [](auto&& fs)
         {
-         fs->mds_map.clear_multimds_snaps_allowed();
+         fs.get_mds_map().clear_multimds_snaps_allowed();
         });
       }
     } else if (var == "allow_dirfrags") {
@@ -538,27 +541,27 @@ public:
         return r;
       }
 
-      ss << fs->mds_map.get_fs_name();
+      ss << fsp->get_mds_map().get_fs_name();
 
       fsmap.modify_filesystem(
-          fs->fscid,
-          [is_down](std::shared_ptr<Filesystem> fs)
+          fsp->get_fscid(),
+          [is_down](auto&& fs)
       {
        if (is_down) {
-          if (fs->mds_map.get_max_mds() > 0) {
-           fs->mds_map.set_old_max_mds();
-           fs->mds_map.set_max_mds(0);
+          if (fs.get_mds_map().get_max_mds() > 0) {
+           fs.get_mds_map().set_old_max_mds();
+           fs.get_mds_map().set_max_mds(0);
           } /* else already down! */
        } else {
-         mds_rank_t oldmax = fs->mds_map.get_old_max_mds();
-         fs->mds_map.set_max_mds(oldmax ? oldmax : 1);
+         mds_rank_t oldmax = fs.get_mds_map().get_old_max_mds();
+         fs.get_mds_map().set_max_mds(oldmax ? oldmax : 1);
        }
       });
 
       if (is_down) {
        ss << " marked down. ";
       } else {
-       ss << " marked up, max_mds = " << fs->mds_map.get_max_mds();
+       ss << " marked up, max_mds = " << fsp->get_mds_map().get_max_mds();
       }
     } else if (var == "cluster_down" || var == "joinable") {
       bool joinable = true;
@@ -570,16 +573,16 @@ public:
         joinable = !joinable;
       }
 
-      ss << fs->mds_map.get_fs_name();
+      ss << fsp->get_mds_map().get_fs_name();
 
       fsmap.modify_filesystem(
-          fs->fscid,
-          [joinable](std::shared_ptr<Filesystem> fs)
+          fsp->get_fscid(),
+          [joinable](auto&& fs)
       {
        if (joinable) {
-         fs->mds_map.clear_flag(CEPH_MDSMAP_NOT_JOINABLE);
+         fs.get_mds_map().clear_flag(CEPH_MDSMAP_NOT_JOINABLE);
        } else {
-         fs->mds_map.set_flag(CEPH_MDSMAP_NOT_JOINABLE);
+         fs.get_mds_map().set_flag(CEPH_MDSMAP_NOT_JOINABLE);
        }
       });
 
@@ -603,10 +606,10 @@ public:
        return -ERANGE;
       }
       fsmap.modify_filesystem(
-          fs->fscid,
-          [n](std::shared_ptr<Filesystem> fs)
+          fsp->get_fscid(),
+          [n](auto&& fs)
       {
-        fs->mds_map.set_standby_count_wanted(n);
+        fs.get_mds_map().set_standby_count_wanted(n);
       });
     } else if (var == "session_timeout") {
       if (interr.length()) {
@@ -618,10 +621,10 @@ public:
        return -ERANGE;
       }
       fsmap.modify_filesystem(
-          fs->fscid,
-          [n](std::shared_ptr<Filesystem> fs)
+          fsp->get_fscid(),
+          [n](auto&& fs)
       {
-        fs->mds_map.set_session_timeout((uint32_t)n);
+        fs.get_mds_map().set_session_timeout((uint32_t)n);
       });
     } else if (var == "session_autoclose") {
       if (interr.length()) {
@@ -633,10 +636,10 @@ public:
        return -ERANGE;
       }
       fsmap.modify_filesystem(
-          fs->fscid,
-          [n](std::shared_ptr<Filesystem> fs)
+          fsp->get_fscid(),
+          [n](auto&& fs)
       {
-        fs->mds_map.set_session_autoclose((uint32_t)n);
+        fs.get_mds_map().set_session_autoclose((uint32_t)n);
       });
     } else if (var == "allow_standby_replay") {
       bool allow = false;
@@ -652,7 +655,7 @@ public:
           return -EAGAIN;
         }
         std::vector<mds_gid_t> to_fail;
-        for (const auto& [gid, info]: fs->mds_map.get_mds_info()) {
+        for (const auto& [gid, info]: fsp->get_mds_map().get_mds_info()) {
           if (info.state == MDSMap::STATE_STANDBY_REPLAY) {
             to_fail.push_back(gid);
           }
@@ -666,14 +669,14 @@ public:
         }
       }
 
-      auto f = [allow](auto& fs) {
+      auto f = [allow](auto&& fs) {
         if (allow) {
-          fs->mds_map.set_standby_replay_allowed();
+          fs.get_mds_map().set_standby_replay_allowed();
         } else {
-          fs->mds_map.clear_standby_replay_allowed();
+          fs.get_mds_map().clear_standby_replay_allowed();
         }
       };
-      fsmap.modify_filesystem(fs->fscid, std::move(f));
+      fsmap.modify_filesystem(fsp->get_fscid(), std::move(f));
     } else if (var == "min_compat_client") {
       auto vno = ceph_release_from_name(val.c_str());
       if (!vno) {
@@ -685,9 +688,9 @@ public:
             "The oldest release to set is octopus.\n"
             "Please migrate to `ceph fs required_client_features ...`.";
       auto f = [vno](auto&& fs) {
-        fs->mds_map.set_min_compat_client(vno);
+        fs.get_mds_map().set_min_compat_client(vno);
       };
-      fsmap.modify_filesystem(fs->fscid, std::move(f));
+      fsmap.modify_filesystem(fsp->get_fscid(), std::move(f));
     } else if (var == "refuse_client_session") {
       bool refuse_session = false;
       int r = parse_bool(val, &refuse_session, ss);
@@ -696,24 +699,24 @@ public:
       }
 
       if (refuse_session) {
-        if (!(fs->mds_map.test_flag(CEPH_MDSMAP_REFUSE_CLIENT_SESSION))) {
+        if (!(fsp->get_mds_map().test_flag(CEPH_MDSMAP_REFUSE_CLIENT_SESSION))) {
           fsmap.modify_filesystem(
-            fs->fscid,
-            [](std::shared_ptr<Filesystem> fs)
+            fsp->get_fscid(),
+            [](auto&& fs)
           {
-            fs->mds_map.set_flag(CEPH_MDSMAP_REFUSE_CLIENT_SESSION);
+            fs.get_mds_map().set_flag(CEPH_MDSMAP_REFUSE_CLIENT_SESSION);
           });
           ss << "client(s) blocked from establishing new session(s)"; 
         } else {
           ss << "client(s) already blocked from establishing new session(s)";
         }     
       } else {
-          if (fs->mds_map.test_flag(CEPH_MDSMAP_REFUSE_CLIENT_SESSION)) {
+          if (fsp->get_mds_map().test_flag(CEPH_MDSMAP_REFUSE_CLIENT_SESSION)) {
             fsmap.modify_filesystem(
-              fs->fscid,
-              [](std::shared_ptr<Filesystem> fs)
+              fsp->get_fscid(),
+              [](auto&& fs)
             {
-              fs->mds_map.clear_flag(CEPH_MDSMAP_REFUSE_CLIENT_SESSION);
+              fs.get_mds_map().clear_flag(CEPH_MDSMAP_REFUSE_CLIENT_SESSION);
             });
             ss << "client(s) allowed to establish new session(s)"; 
           } else {
@@ -751,8 +754,8 @@ class CompatSetHandler : public FileSystemCommandHandler
        ss << "Missing filesystem name";
        return -EINVAL;
       }
-      auto fs = fsmap.get_filesystem(fs_name);
-      if (fs == nullptr) {
+      auto* fsp = fsmap.get_filesystem(fs_name);
+      if (fsp == nullptr) {
        ss << "Not found: '" << fs_name << "'";
        return -ENOENT;
       }
@@ -769,12 +772,12 @@ class CompatSetHandler : public FileSystemCommandHandler
         return -EINVAL;
       }
 
-      if (fs->mds_map.get_num_up_mds() > 0) {
+      if (fsp->get_mds_map().get_num_up_mds() > 0) {
         ss << "file system must be failed or down; use `ceph fs fail` to bring down";
         return -EBUSY;
       }
 
-      CompatSet cs = fs->mds_map.compat;
+      CompatSet cs = fsp->get_mds_map().compat;
       if (subop == "rm_compat") {
         if (cs.compat.contains(feature)) {
           ss << "removed compat feature " << feature;
@@ -826,10 +829,10 @@ class CompatSetHandler : public FileSystemCommandHandler
       } else ceph_assert(0);
 
       auto modifyf = [cs = std::move(cs)](auto&& fs) {
-        fs->mds_map.compat = cs;
+        fs.get_mds_map().compat = cs;
       };
 
-      fsmap.modify_filesystem(fs->fscid, std::move(modifyf));
+      fsmap.modify_filesystem(fsp->get_fscid(), std::move(modifyf));
       return 0;
     }
 };
@@ -854,8 +857,8 @@ class RequiredClientFeaturesHandler : public FileSystemCommandHandler
        ss << "Missing filesystem name";
        return -EINVAL;
       }
-      auto fs = fsmap.get_filesystem(fs_name);
-      if (fs == nullptr) {
+      auto* fsp = fsmap.get_filesystem(fs_name);
+      if (fsp == nullptr) {
        ss << "Not found: '" << fs_name << "'";
        return -ENOENT;
       }
@@ -888,12 +891,12 @@ class RequiredClientFeaturesHandler : public FileSystemCommandHandler
       if (subop == "add") {
        bool ret = false;
        fsmap.modify_filesystem(
-           fs->fscid,
+           fsp->get_fscid(),
            [feature, &ret](auto&& fs)
        {
-         if (fs->mds_map.get_required_client_features().test(feature))
+         if (fs.get_mds_map().get_required_client_features().test(feature))
            return;
-         fs->mds_map.add_required_client_feature(feature);
+         fs.get_mds_map().add_required_client_feature(feature);
          ret = true;
        });
        if (ret) {
@@ -904,12 +907,12 @@ class RequiredClientFeaturesHandler : public FileSystemCommandHandler
       } else {
        bool ret = false;
        fsmap.modify_filesystem(
-           fs->fscid,
+           fsp->get_fscid(),
            [feature, &ret](auto&& fs)
        {
-          if (!fs->mds_map.get_required_client_features().test(feature))
+          if (!fs.get_mds_map().get_required_client_features().test(feature))
             return;
-          fs->mds_map.remove_required_client_feature(feature);
+          fs.get_mds_map().remove_required_client_feature(feature);
           ret = true;
        });
        if (ret) {
@@ -964,9 +967,14 @@ class AddDataPoolHandler : public FileSystemCommandHandler
       return r;
     }
 
-    auto fs = fsmap.get_filesystem(fs_name);
+    auto* fsp = fsmap.get_filesystem(fs_name);
+    if (fsp == nullptr) {
+        ss << "filesystem '" << fs_name << "' does not exist";
+        return -ENOENT;
+    }
+
     // no-op when the data_pool already on fs
-    if (fs->mds_map.is_data_pool(poolid)) {
+    if (fsp->get_mds_map().is_data_pool(poolid)) {
       ss << "data pool " << poolid << " is already on fs " << fs_name;
       return 0;
     }
@@ -982,10 +990,10 @@ class AddDataPoolHandler : public FileSystemCommandHandler
     mon->osdmon()->propose_pending();
 
     fsmap.modify_filesystem(
-        fs->fscid,
-        [poolid](std::shared_ptr<Filesystem> fs)
+        fsp->get_fscid(),
+        [poolid](auto&&  fs)
     {
-      fs->mds_map.add_data_pool(poolid);
+      fs.get_mds_map().add_data_pool(poolid);
     });
 
     ss << "added data pool " << poolid << " to fsmap";
@@ -1013,13 +1021,13 @@ class SetDefaultHandler : public FileSystemCommandHandler
   {
     std::string fs_name;
     cmd_getval(cmdmap, "fs_name", fs_name);
-    auto fs = fsmap.get_filesystem(fs_name);
-    if (fs == nullptr) {
+    auto* fsp = fsmap.get_filesystem(fs_name);
+    if (fsp == nullptr) {
         ss << "filesystem '" << fs_name << "' does not exist";
         return -ENOENT;
     }
 
-    fsmap.set_legacy_client_fscid(fs->fscid);
+    fsmap.set_legacy_client_fscid(fsp->get_fscid());
     return 0;
   }
 };
@@ -1050,15 +1058,15 @@ class RemoveFilesystemHandler : public FileSystemCommandHandler
     //  syntax should apply to multi-FS future)
     string fs_name;
     cmd_getval(cmdmap, "fs_name", fs_name);
-    auto fs = fsmap.get_filesystem(fs_name);
-    if (fs == nullptr) {
+    auto* fsp = fsmap.get_filesystem(fs_name);
+    if (fsp == nullptr) {
         // Consider absence success to make deletes idempotent
         ss << "filesystem '" << fs_name << "' does not exist";
         return 0;
     }
 
     // Check that no MDS daemons are active
-    if (fs->mds_map.get_num_up_mds() > 0) {
+    if (fsp->get_mds_map().get_num_up_mds() > 0) {
       ss << "all MDS daemons must be inactive/failed before removing filesystem. See `ceph fs fail`.";
       return -EINVAL;
     }
@@ -1072,13 +1080,13 @@ class RemoveFilesystemHandler : public FileSystemCommandHandler
       return -EPERM;
     }
 
-    if (fsmap.get_legacy_client_fscid() == fs->fscid) {
+    if (fsmap.get_legacy_client_fscid() == fsp->get_fscid()) {
       fsmap.set_legacy_client_fscid(FS_CLUSTER_ID_NONE);
     }
 
     std::vector<mds_gid_t> to_fail;
     // There may be standby_replay daemons left here
-    for (const auto &i : fs->mds_map.get_mds_info()) {
+    for (const auto &i : fsp->get_mds_map().get_mds_info()) {
       ceph_assert(i.second.state == MDSMap::STATE_STANDBY_REPLAY);
       to_fail.push_back(i.first);
     }
@@ -1092,7 +1100,7 @@ class RemoveFilesystemHandler : public FileSystemCommandHandler
       mon->osdmon()->propose_pending(); /* maybe new blocklists */
     }
 
-    fsmap.erase_filesystem(fs->fscid);
+    fsmap.erase_filesystem(fsp->get_fscid());
 
     return 0;
   }
@@ -1114,15 +1122,15 @@ class ResetFilesystemHandler : public FileSystemCommandHandler
   {
     string fs_name;
     cmd_getval(cmdmap, "fs_name", fs_name);
-    auto fs = fsmap.get_filesystem(fs_name);
-    if (fs == nullptr) {
+    auto* fsp = fsmap.get_filesystem(fs_name);
+    if (fsp == nullptr) {
         ss << "filesystem '" << fs_name << "' does not exist";
         // Unlike fs rm, we consider this case an error
         return -ENOENT;
     }
 
     // Check that no MDS daemons are active
-    if (fs->mds_map.get_num_up_mds() > 0) {
+    if (fsp->get_mds_map().get_num_up_mds() > 0) {
       ss << "all MDS daemons must be inactive before resetting filesystem: set the cluster_down flag"
             " and use `ceph mds fail` to make this so";
       return -EINVAL;
@@ -1137,7 +1145,7 @@ class ResetFilesystemHandler : public FileSystemCommandHandler
       return -EPERM;
     }
 
-    fsmap.reset_filesystem(fs->fscid);
+    fsmap.reset_filesystem(fsp->get_fscid());
 
     return 0;
   }
@@ -1162,14 +1170,14 @@ class RenameFilesystemHandler : public FileSystemCommandHandler
 
     string fs_name;
     cmd_getval(cmdmap, "fs_name", fs_name);
-    auto fs = fsmap.get_filesystem(fs_name);
+    auto* fsp = fsmap.get_filesystem(fs_name);
 
     string new_fs_name;
     cmd_getval(cmdmap, "new_fs_name", new_fs_name);
-    auto new_fs = fsmap.get_filesystem(new_fs_name);
+    auto* new_fsp = fsmap.get_filesystem(new_fs_name);
 
-    if (fs == nullptr) {
-        if (new_fs) {
+    if (fsp == nullptr) {
+        if (new_fsp) {
           // make 'fs rename' idempotent
          ss << "File system may already have been renamed. Desired file system '"
             << new_fs_name << "' exists.";
@@ -1180,12 +1188,12 @@ class RenameFilesystemHandler : public FileSystemCommandHandler
        }
     }
 
-    if (new_fs) {
+    if (new_fsp) {
       ss << "Desired file system name '" << new_fs_name << "' already in use";
       return -EINVAL;
     }
 
-    if (fs->mirror_info.mirrored) {
+    if (fsp->get_mirror_info().mirrored) {
       ss << "Mirroring is enabled on file system '"<< fs_name << "'. Disable mirroring on the "
         "file system after ensuring it's OK to do so, and then retry to rename.";
       return -EPERM;
@@ -1206,21 +1214,21 @@ class RenameFilesystemHandler : public FileSystemCommandHandler
       mon->osdmon()->wait_for_writeable(op, new PaxosService::C_RetryMessage(mon->mdsmon(), op));
       return -EAGAIN;
     }
-    for (const auto p : fs->mds_map.get_data_pools()) {
+    for (const auto p : fsp->get_mds_map().get_data_pools()) {
       mon->osdmon()->do_application_enable(p,
                                           pg_pool_t::APPLICATION_NAME_CEPHFS,
                                           "data", new_fs_name, true);
     }
 
-    mon->osdmon()->do_application_enable(fs->mds_map.get_metadata_pool(),
+    mon->osdmon()->do_application_enable(fsp->get_mds_map().get_metadata_pool(),
                                         pg_pool_t::APPLICATION_NAME_CEPHFS,
                                         "metadata", new_fs_name, true);
     mon->osdmon()->propose_pending();
 
-    auto f = [new_fs_name](auto fs) {
-                    fs->mds_map.set_fs_name(new_fs_name);
+    auto f = [new_fs_name](auto&& fs) {
+                    fs.get_mds_map().set_fs_name(new_fs_name);
              };
-    fsmap.modify_filesystem(fs->fscid, std::move(f));
+    fsmap.modify_filesystem(fsp->get_fscid(), std::move(f));
 
     ss << "File system is renamed. cephx credentials authorized to "
           "old file system name need to be reauthorized to new file "
@@ -1272,17 +1280,22 @@ class RemoveDataPoolHandler : public FileSystemCommandHandler
 
     ceph_assert(poolid >= 0);  // Checked by parsing code above
 
-    auto fs = fsmap.get_filesystem(fs_name);
-    if (fs->mds_map.get_first_data_pool() == poolid) {
+    auto* fsp = fsmap.get_filesystem(fs_name);
+    if (fsp == nullptr) {
+        ss << "filesystem '" << fs_name << "' does not exist";
+        return -ENOENT;
+    }
+
+    if (fsp->get_mds_map().get_first_data_pool() == poolid) {
       ss << "cannot remove default data pool";
       return -EINVAL;
     }
 
     int r = 0;
-    fsmap.modify_filesystem(fs->fscid,
-        [&r, poolid](std::shared_ptr<Filesystem> fs)
+    fsmap.modify_filesystem(fsp->get_fscid(),
+        [&r, poolid](auto&& fs)
     {
-      r = fs->mds_map.remove_data_pool(poolid);
+      r = fs.get_mds_map().remove_data_pool(poolid);
     });
     if (r == -ENOENT) {
       // It was already removed, succeed in silence
@@ -1342,20 +1355,20 @@ public:
       return -EINVAL;
     }
 
-    auto fs = fsmap.get_filesystem(fs_name);
-    if (fs == nullptr) {
+    auto* fsp = fsmap.get_filesystem(fs_name);
+    if (fsp == nullptr) {
       ss << "Filesystem '" << fs_name << "' not found";
       return -ENOENT;
     }
 
-    if (fs->mirror_info.is_mirrored()) {
+    if (fsp->get_mirror_info().is_mirrored()) {
       return 0;
     }
 
-    auto f = [](auto &&fs) {
-               fs->mirror_info.enable_mirroring();
+    auto f = [](auto&& fs) {
+      fs.get_mirror_info().enable_mirroring();
     };
-    fsmap.modify_filesystem(fs->fscid, std::move(f));
+    fsmap.modify_filesystem(fsp->get_fscid(), std::move(f));
 
     return 0;
   }
@@ -1377,20 +1390,20 @@ public:
       return -EINVAL;
     }
 
-    auto fs = fsmap.get_filesystem(fs_name);
-    if (fs == nullptr) {
+    auto* fsp = fsmap.get_filesystem(fs_name);
+    if (fsp == nullptr) {
       ss << "Filesystem '" << fs_name << "' not found";
       return -ENOENT;
     }
 
-    if (!fs->mirror_info.is_mirrored()) {
+    if (!fsp->get_mirror_info().is_mirrored()) {
       return 0;
     }
 
-    auto f = [](auto &&fs) {
-      fs->mirror_info.disable_mirroring();
+    auto f = [](auto&& fs) {
+      fs.get_mirror_info().disable_mirroring();
     };
-    fsmap.modify_filesystem(fs->fscid, std::move(f));
+    fsmap.modify_filesystem(fsp->get_fscid(), std::move(f));
 
     return 0;
   }
@@ -1416,7 +1429,7 @@ public:
     return std::make_pair(client, cluster);
   }
 
-  bool peer_add(FSMap &fsmap, Filesystem::const_ref &&fs,
+  bool peer_add(FSMap &fsmap, const Filesystem& fs,
                 const cmdmap_t &cmdmap, std::ostream &ss) {
     string peer_uuid;
     string remote_spec;
@@ -1432,21 +1445,21 @@ public:
       return false;
     }
 
-    if (fs->mirror_info.has_peer(peer_uuid)) {
+    if (fs.get_mirror_info().has_peer(peer_uuid)) {
       ss << "peer already exists";
       return true;
     }
-    if (fs->mirror_info.has_peer((*remote_conf).first, (*remote_conf).second,
+    if (fs.get_mirror_info().has_peer((*remote_conf).first, (*remote_conf).second,
                                  remote_fs_name)) {
       ss << "peer already exists";
       return true;
     }
 
-    auto f = [peer_uuid, remote_conf, remote_fs_name](auto &&fs) {
-               fs->mirror_info.peer_add(peer_uuid, (*remote_conf).first,
+    auto f = [peer_uuid, remote_conf, remote_fs_name](auto&& fs) {
+               fs.get_mirror_info().peer_add(peer_uuid, (*remote_conf).first,
                                         (*remote_conf).second, remote_fs_name);
              };
-    fsmap.modify_filesystem(fs->fscid, std::move(f));
+    fsmap.modify_filesystem(fs.get_fscid(), std::move(f));
     return true;
   }
 
@@ -1459,18 +1472,18 @@ public:
       return -EINVAL;
     }
 
-    auto fs = fsmap.get_filesystem(fs_name);
-    if (fs == nullptr) {
+    auto* fsp = fsmap.get_filesystem(fs_name);
+    if (fsp == nullptr) {
       ss << "Filesystem '" << fs_name << "' not found";
       return -ENOENT;
     }
 
-    if (!fs->mirror_info.is_mirrored()) {
+    if (!fsp->get_mirror_info().is_mirrored()) {
       ss << "Mirroring not enabled for filesystem '" << fs_name << "'";
       return -EINVAL;
     }
 
-    auto res = peer_add(fsmap, std::move(fs), cmdmap, ss);
+    auto res = peer_add(fsmap, *fsp, cmdmap, ss);
     if (!res) {
       return -EINVAL;
     }
@@ -1486,20 +1499,20 @@ public:
     : FileSystemCommandHandler("fs mirror peer_remove")
   {}
 
-  bool peer_remove(FSMap &fsmap, Filesystem::const_ref &&fs,
+  bool peer_remove(FSMap &fsmap, const Filesystem& fs,
                    const cmdmap_t &cmdmap, std::ostream &ss) {
     string peer_uuid;
     cmd_getval(cmdmap, "uuid", peer_uuid);
 
-    if (!fs->mirror_info.has_peer(peer_uuid)) {
+    if (!fs.get_mirror_info().has_peer(peer_uuid)) {
       ss << "cannot find peer with uuid: " << peer_uuid;
       return true;
     }
 
-    auto f = [peer_uuid](auto &&fs) {
-               fs->mirror_info.peer_remove(peer_uuid);
+    auto f = [peer_uuid](auto&& fs) {
+               fs.get_mirror_info().peer_remove(peer_uuid);
              };
-    fsmap.modify_filesystem(fs->fscid, std::move(f));
+    fsmap.modify_filesystem(fs.get_fscid(), std::move(f));
     return true;
   }
 
@@ -1512,18 +1525,18 @@ public:
       return -EINVAL;
     }
 
-    auto fs = fsmap.get_filesystem(fs_name);
-    if (fs == nullptr) {
+    auto* fsp = fsmap.get_filesystem(fs_name);
+    if (fsp == nullptr) {
       ss << "Filesystem '" << fs_name << "' not found";
       return -ENOENT;
     }
 
-    if (!fs->mirror_info.is_mirrored()) {
+    if (!fsp->get_mirror_info().is_mirrored()) {
       ss << "Mirroring not enabled for filesystem '" << fs_name << "'";
       return -EINVAL;
     }
 
-    auto res = peer_remove(fsmap, std::move(fs), cmdmap, ss);
+    auto res = peer_remove(fsmap, *fsp, cmdmap, ss);
     if (!res) {
       return -EINVAL;
     }
@@ -1674,8 +1687,8 @@ int FileSystemCommandHandler::is_op_allowed(
     FSMap fsmap_copy = fsmap;
     fsmap_copy.filter(op->get_session()->get_allowed_fs_names());
 
-    auto fs = fsmap_copy.get_filesystem(fs_name);
-    if (fs == nullptr) {
+    auto* fsp = fsmap_copy.get_filesystem(fs_name);
+    if (fsp == nullptr) {
       auto prefix = get_prefix();
       /* let "fs rm" and "fs rename" handle idempotent cases where file systems do not exist */
       if (!(prefix == "fs rm" || prefix == "fs rename") && fsmap.get_filesystem(fs_name) == nullptr) {
index defd8ecd85f8e5dee2d6c375c91bfdbfce58cf17..efb71043ef1291221b56e7265c53cd1f09df780e 100644 (file)
@@ -129,12 +129,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 == get_fsmap().epoch)
+  if (version == get_fsmap().get_epoch())
     return;
 
   dout(10) << __func__ << " version " << version
-          << ", my e " << get_fsmap().epoch << dendl;
-  ceph_assert(version > get_fsmap().epoch);
+          << ", my e " << get_fsmap().get_epoch() << dendl;
+  ceph_assert(version > get_fsmap().get_epoch());
 
   load_health();
 
@@ -177,13 +177,13 @@ void MDSMonitor::create_pending()
     fsmap.sanitize([&osdmap](int64_t pool){return osdmap.have_pg_pool(pool);});
   }
 
-  dout(10) << "create_pending e" << fsmap.epoch << dendl;
+  dout(10) << "create_pending e" << fsmap.get_epoch() << dendl;
 }
 
 void MDSMonitor::encode_pending(MonitorDBStore::TransactionRef t)
 {
   auto &pending = get_pending_fsmap_writeable();
-  auto &epoch = pending.epoch;
+  auto epoch = pending.get_epoch();
 
   dout(10) << "encode_pending e" << epoch << dendl;
 
@@ -193,21 +193,14 @@ void MDSMonitor::encode_pending(MonitorDBStore::TransactionRef t)
     pending.sanity(true);
   }
 
-  // Set 'modified' on maps modified this epoch
-  for (auto &p : pending.filesystems) {
-    if (p.second->mds_map.epoch == epoch) {
-      p.second->mds_map.modified = ceph_clock_now();
-    }
-  }
-
   // apply to paxos
-  ceph_assert(get_last_committed() + 1 == pending.epoch);
+  ceph_assert(get_last_committed() + 1 == pending.get_epoch());
   bufferlist pending_bl;
   pending.encode(pending_bl, mon.get_quorum_con_features());
 
   /* put everything in the transaction */
-  put_version(t, pending.epoch, pending_bl);
-  put_last_committed(t, pending.epoch);
+  put_version(t, pending.get_epoch(), pending_bl);
+  put_last_committed(t, pending.get_epoch());
 
   // Encode MDSHealth data
   for (std::map<uint64_t, MDSHealth>::iterator i = pending_daemon_health.begin();
@@ -438,11 +431,11 @@ bool MDSMonitor::preprocess_beacon(MonOpRequestRef op)
 
   // Work out the latest epoch that this daemon should have seen
   {
-    fs_cluster_id_t fscid = fsmap.mds_roles.at(gid);
+    fs_cluster_id_t fscid = fsmap.fscid_from_gid(gid);
     if (fscid == FS_CLUSTER_ID_NONE) {
-      effective_epoch = fsmap.standby_epochs.at(gid);
+      effective_epoch = fsmap.get_standby_epochs().at(gid);
     } else {
-      effective_epoch = fsmap.get_filesystem(fscid)->mds_map.epoch;
+      effective_epoch = fsmap.get_filesystem(fscid).get_mds_map().get_epoch();
     }
     if (effective_epoch != m->get_last_epoch_seen()) {
       dout(10) << "mds_beacon " << *m
@@ -463,9 +456,9 @@ bool MDSMonitor::preprocess_beacon(MonOpRequestRef op)
   // did the join_fscid change
   if (m->get_fs().size()) {
     fs_cluster_id_t fscid = FS_CLUSTER_ID_NONE;
-    auto f = fsmap.get_filesystem(m->get_fs());
-    if (f) {
-      fscid = f->fscid;
+    auto* fsp = fsmap.get_filesystem(m->get_fs());
+    if (fsp) {
+      fscid = fsp->get_fscid();
     }
     if (info.join_fscid != fscid) {
       dout(10) << __func__ << " standby mds_join_fs changed to " << fscid
@@ -659,7 +652,7 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
     }
 
     // Add this daemon to the map
-    if (pending.mds_roles.count(gid) == 0) {
+    if (!pending.gid_exists(gid)) {
       MDSMap::mds_info_t new_info;
       new_info.global_id = gid;
       new_info.name = m->get_name();
@@ -670,9 +663,9 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
       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());
-       if (f) {
-         fscid = f->fscid;
+       auto* fsp = pending.get_filesystem(m->get_fs());
+       if (fsp) {
+         fscid = fsp->get_fscid();
        }
         new_info.join_fscid = fscid;
       }
@@ -754,9 +747,9 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
 
     fs_cluster_id_t fscid = FS_CLUSTER_ID_NONE;
     if (m->get_fs().size()) {
-      auto f = pending.get_filesystem(m->get_fs());
-      if (f) {
-        fscid = f->fscid;
+      auto* fsp = pending.get_filesystem(m->get_fs());
+      if (fsp) {
+        fscid = fsp->get_fscid();
       }
     }
     pending.modify_daemon(gid, [fscid](auto& info) {
@@ -764,13 +757,13 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
     });
 
     if (state == MDSMap::STATE_STOPPED) {
-      const auto fscid = pending.mds_roles.at(gid);
-      const auto &fs = pending.get_filesystem(fscid);
+      const auto fscid = pending.fscid_from_gid(gid);
+      const autofs = pending.get_filesystem(fscid);
 
       mon.clog->info() << info.human_name() << " finished "
                         << "stopping rank " << info.rank << " in filesystem "
-                        << fs->mds_map.fs_name << " (now has "
-                        << fs->mds_map.get_num_in_mds() - 1 << " ranks)";
+                        << fs.get_mds_map().fs_name << " (now has "
+                        << fs.get_mds_map().get_num_in_mds() - 1 << " ranks)";
 
       auto erased = pending.stop(gid);
       erased.push_back(gid);
@@ -796,10 +789,10 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
       // won't try to run it.
       dout(0) << __func__ << ": marking rank " << rank << " damaged" << dendl;
 
-      auto fs = pending.get_filesystem(gid);
-      auto rankgid = fs->mds_map.get_gid(rank);
+      auto& fs = pending.get_filesystem(gid);
+      auto rankgid = fs.get_mds_map().get_gid(rank);
       auto rankinfo = pending.get_info_gid(rankgid);
-      auto followergid = fs->mds_map.get_standby_replay(rank);
+      auto followergid = fs.get_mds_map().get_standby_replay(rank);
 
       ceph_assert(gid == rankgid || gid == followergid);
 
@@ -818,10 +811,10 @@ bool MDSMonitor::prepare_beacon(MonOpRequestRef op)
       /* MDS expects beacon reply back */
     } else {
       if (info.state != MDSMap::STATE_ACTIVE && state == MDSMap::STATE_ACTIVE) {
-        const auto &fscid = pending.mds_roles.at(gid);
-        const auto &fs = pending.get_filesystem(fscid);
+        const auto& fscid = pending.fscid_from_gid(gid);
+        const autofs = pending.get_filesystem(fscid);
         mon.clog->info() << info.human_name() << " is now active in "
-                          << "filesystem " << fs->mds_map.fs_name << " as rank "
+                          << "filesystem " << fs.get_mds_map().fs_name << " as rank "
                           << info.rank;
       }
 
@@ -1128,8 +1121,8 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
   } 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) {
+    auto* fsp = fsmap.get_filesystem(fs_name);
+    if (fsp == nullptr) {
       ss << "filesystem '" << fs_name << "' not found";
       r = -ENOENT;
       goto out;
@@ -1137,50 +1130,49 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
 
     if (f) {
       f->open_object_section("mds_compat");
-      fs->mds_map.compat.dump(f.get());
+      fsp->get_mds_map().compat.dump(f.get());
       f->close_section();
       f->flush(ds);
     } else {
-      ds << fs->mds_map.compat;
+      ds << fsp->get_mds_map().compat;
     }
     r = 0;
   } else if (prefix == "mds compat show") {
       if (f) {
        f->open_object_section("mds_compat");
-       fsmap.default_compat.dump(f.get());
+       fsmap.get_default_compat().dump(f.get());
        f->close_section();
        f->flush(ds);
       } else {
-       ds << fsmap.default_compat;
+       ds << fsmap.get_default_compat();
       }
       r = 0;
   } else if (prefix == "fs get") {
     string fs_name;
     cmd_getval(cmdmap, "fs_name", fs_name);
-    const auto &fs = fsmap.get_filesystem(fs_name);
-    if (fs == nullptr) {
+    auto* fsp = fsmap.get_filesystem(fs_name);
+    if (fsp == nullptr) {
       ss << "filesystem '" << fs_name << "' not found";
       r = -ENOENT;
     } else {
       if (f != nullptr) {
         f->open_object_section("filesystem");
-        fs->dump(f.get());
+        fsp->dump(f.get());
         f->close_section();
         f->flush(ds);
         r = 0;
       } else {
-        fs->print(ds);
+        fsp->print(ds);
         r = 0;
       }
     }
   } else if (prefix == "fs ls") {
     if (f) {
       f->open_array_section("filesystems");
-      for (const auto &p : fsmap.filesystems) {
-        const auto &fs = p.second;
+      for (const auto& [fscid, fs] : fsmap) {
         f->open_object_section("filesystem");
         {
-          const MDSMap &mds_map = fs->mds_map;
+          const MDSMap &mds_map = fs.get_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 */
@@ -1205,9 +1197,8 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
       f->close_section();
       f->flush(ds);
     } else {
-      for (const auto &p : fsmap.filesystems) {
-        const auto &fs = p.second;
-        const MDSMap &mds_map = fs->mds_map;
+      for (const auto& [fscid, fs] : fsmap) {
+        const MDSMap &mds_map = fs.get_mds_map();
         const string &md_pool_name = mon.osdmon()->osdmap.get_pool_name(
             mds_map.metadata_pool);
         
@@ -1220,7 +1211,7 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
         ds << "]" << std::endl;
       }
 
-      if (fsmap.filesystems.empty()) {
+      if (fsmap.filesystem_count() == 0) {
         ds << "No filesystems enabled" << std::endl;
       }
     }
@@ -1245,12 +1236,12 @@ bool MDSMonitor::preprocess_command(MonOpRequestRef op)
   } 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) {
+    auto* fsp = fsmap.get_filesystem(fs_name);
+    if (!fsp) {
       ss << "filesystem '" << fs_name << "' not found";
       r = -ENOENT;
     } else {
-      const MDSMap &mds_map = fs->mds_map;
+      auto& mds_map = fsp->get_mds_map();
       if (f) {
         mds_map.dump_flags_state(f.get());
         f->flush(ds);
@@ -1307,12 +1298,12 @@ mds_gid_t MDSMonitor::gid_from_arg(const FSMap &fsmap, const std::string &arg, s
   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);
-    ceph_assert(fs != nullptr);  // parse_role ensures it exists
-    if (fs->mds_map.is_up(role.rank)) {
+    auto& fs = fsmap.get_filesystem(role.fscid);
+    //ceph_assert(fs != nullptr);  // parse_role ensures it exists FIXME ???
+    if (fs.get_mds_map().is_up(role.rank)) {
       dout(10) << __func__ << ": validated rank/GID " << role
                << " as a rank" << dendl;
-      return fs->mds_map.get_mds_info(role.rank).global_id;
+      return fs.get_mds_map().get_mds_info(role.rank).global_id;
     }
   }
 
@@ -1567,7 +1558,7 @@ int MDSMonitor::filesystem_command(
       ss << "invalid role '" << role_str << "'";
       return -EINVAL;
     }
-    string_view fs_name = fsmap.get_filesystem(role.fscid)->mds_map.get_fs_name();
+    string_view fs_name = fsmap.get_filesystem(role.fscid).get_mds_map().get_fs_name();
     if (!op->get_session()->fs_name_capable(fs_name, MON_CAP_W)) {
       ss << "Permission denied.";
       return -EPERM;
@@ -1575,9 +1566,9 @@ int MDSMonitor::filesystem_command(
 
     fsmap.modify_filesystem(
         role.fscid,
-        [role](std::shared_ptr<Filesystem> fs)
+        [role](auto&& fs)
     {
-      fs->mds_map.failed.erase(role.rank);
+      fs.get_mds_map().failed.erase(role.rank);
     });
 
     ss << "removed failed mds." << role;
@@ -1590,11 +1581,12 @@ int MDSMonitor::filesystem_command(
          << cmd_vartype_stringify(cmdmap.at("feature")) << "'";
       return -EINVAL;
     }
-    if (fsmap.default_compat.compat.contains(f)) {
+    auto& default_compat = fsmap.get_default_compat();
+    if (default_compat.compat.contains(f)) {
       ss << "removing compat feature " << f;
-      fsmap.default_compat.compat.remove(f);
+      default_compat.compat.remove(f);
     } else {
-      ss << "compat feature " << f << " not present in " << fsmap.default_compat;
+      ss << "compat feature " << f << " not present in " << default_compat;
     }
     r = 0;
   } else if (prefix == "mds compat rm_incompat") {
@@ -1604,11 +1596,12 @@ int MDSMonitor::filesystem_command(
          << cmd_vartype_stringify(cmdmap.at("feature")) << "'";
       return -EINVAL;
     }
-    if (fsmap.default_compat.incompat.contains(f)) {
+    auto& default_compat = fsmap.get_default_compat();
+    if (default_compat.incompat.contains(f)) {
       ss << "removing incompat feature " << f;
-      fsmap.default_compat.incompat.remove(f);
+      default_compat.incompat.remove(f);
     } else {
-      ss << "incompat feature " << f << " not present in " << fsmap.default_compat;
+      ss << "incompat feature " << f << " not present in " << default_compat;
     }
     r = 0;
   } else if (prefix == "mds repaired") {
@@ -1620,7 +1613,7 @@ int MDSMonitor::filesystem_command(
     if (r < 0) {
       return r;
     }
-    string_view fs_name = fsmap.get_filesystem(role.fscid)->mds_map.get_fs_name();
+    string_view fs_name = fsmap.get_filesystem(role.fscid).get_mds_map().get_fs_name();
     if (!op->get_session()->fs_name_capable(fs_name, MON_CAP_W)) {
       ss << "Permission denied.";
       return -EPERM;
@@ -1688,8 +1681,7 @@ void MDSMonitor::check_subs()
     "mdsmap",
   };
 
-  for (const auto &p : get_fsmap().filesystems) {
-    const auto &fscid = p.first;
+  for (const auto& [fscid, fs] : get_fsmap()) {
     CachedStackStringStream cos;
     *cos << "mdsmap." << fscid;
     types.push_back(std::string(cos->strv()));
@@ -1732,11 +1724,11 @@ void MDSMonitor::check_sub(Subscription *sub)
   } else if (sub->type == "fsmap.user") {
     FSMapUser fsmap_u;
     fsmap_u.epoch = fsmap.get_epoch();
-    fsmap_u.legacy_client_fscid = fsmap.legacy_client_fscid;
-    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;
+    fsmap_u.legacy_client_fscid = fsmap.get_legacy_client_fscid();
+    for (const auto& [fscid, fs] : fsmap) {
+      FSMapUser::fs_info_t& fs_info = fsmap_u.filesystems[fscid];
+      fs_info.cid = fscid;
+      fs_info.name = fs.get_mds_map().fs_name;
     }
     sub->session->con->send_message(new MFSMapUser(mon.monmap->fsid, fsmap_u));
     if (sub->onetime) {
@@ -1754,7 +1746,7 @@ void MDSMonitor::check_sub(Subscription *sub)
       for (const auto &p : mds_info) {
         if (p.second.addrs == sub->session->addrs) {
           mds_gid = p.first;
-          fscid = fsmap.mds_roles.at(mds_gid);
+          fscid = fsmap.fscid_from_gid(mds_gid);
         }
       }
     } else {
@@ -1774,8 +1766,9 @@ void MDSMonitor::check_sub(Subscription *sub)
       } else {
         // Unqualified request for "mdsmap": give it the one marked
         // for use by legacy clients.
-        if (fsmap.legacy_client_fscid != FS_CLUSTER_ID_NONE) {
-          fscid = fsmap.legacy_client_fscid;
+        auto legacy_client_fscid = fsmap.get_legacy_client_fscid();
+        if (legacy_client_fscid != FS_CLUSTER_ID_NONE) {
+          fscid = legacy_client_fscid;
         } else {
           dout(1) << "Client subscribed for legacy filesystem but "
                      "none is configured" << dendl;
@@ -1801,18 +1794,19 @@ void MDSMonitor::check_sub(Subscription *sub)
       // For a client, we should have already dropped out
       ceph_assert(is_mds);
 
-      auto it = fsmap.standby_daemons.find(mds_gid);
-      if (it != fsmap.standby_daemons.end()) {
+      auto& standby_daemons = fsmap.get_standby_daemons();
+      auto it = standby_daemons.find(mds_gid);
+      if (it != standby_daemons.end()) {
         // For an MDS, we need to feed it an MDSMap with its own state in
         null_map.mds_info[mds_gid] = it->second;
-        null_map.epoch = fsmap.standby_epochs.at(mds_gid);
+        null_map.epoch = fsmap.get_standby_epochs().at(mds_gid);
       } else {
-        null_map.epoch = fsmap.epoch;
+        null_map.epoch = fsmap.get_epoch();
       }
       mds_map = &null_map;
     } else {
       // Check the effective epoch 
-      mds_map = &fsmap.get_filesystem(fscid)->mds_map;
+      mds_map = &fsmap.get_filesystem(fscid).get_mds_map();
     }
 
     ceph_assert(mds_map != nullptr);
@@ -1985,10 +1979,10 @@ 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(FSMap &fsmap, fs_cluster_id_t fscid)
+bool MDSMonitor::maybe_resize_cluster(FSMap &fsmap, const Filesystem& fs)
 {
-  auto&& fs = fsmap.get_filesystem(fscid);
-  auto &mds_map = fs->mds_map;
+  auto fscid = fs.get_fscid();
+  auto& mds_map = fs.get_mds_map();
 
   int in = mds_map.get_num_in_mds();
   int max = mds_map.get_max_mds();
@@ -2000,7 +1994,7 @@ bool MDSMonitor::maybe_resize_cluster(FSMap &fsmap, fs_cluster_id_t fscid)
    * becoming active in the next epoch.
    */
   if (!get_fsmap().filesystem_exists(fscid) ||
-      !get_fsmap().get_filesystem(fscid)->mds_map.is_resizeable() ||
+      !get_fsmap().get_filesystem(fscid).get_mds_map().is_resizeable() ||
       !mds_map.is_resizeable()) {
     dout(5) << __func__ << " mds_map is not currently resizeable" << dendl;
     return false;
@@ -2022,7 +2016,7 @@ bool MDSMonitor::maybe_resize_cluster(FSMap &fsmap, fs_cluster_id_t fscid)
                          "filesystem " << mds_map.fs_name << " as rank "
                       << mds << " (now has " << mds_map.get_num_in_mds() + 1
                       << " ranks)";
-    fsmap.promote(info->global_id, *fs, mds);
+    fsmap.promote(info->global_id, fscid, mds);
     return true;
   } else if (in > max) {
     mds_rank_t target = in - 1;
@@ -2052,7 +2046,7 @@ bool MDSMonitor::drop_mds(FSMap &fsmap, mds_gid_t gid, const mds_info_t* rep_inf
 {
   ceph_assert(osd_propose != nullptr);
 
-  const auto fscid = fsmap.mds_roles.at(gid);
+  const auto fscid = fsmap.fscid_from_gid(gid);
   const auto& info = fsmap.get_info_gid(gid);
   const auto rank = info.rank;
   const auto state = info.state;
@@ -2068,8 +2062,8 @@ bool MDSMonitor::drop_mds(FSMap &fsmap, mds_gid_t gid, const mds_info_t* rep_inf
     *osd_propose |= fail_mds_gid(fsmap, gid);
     return true;
   } else if (rank >= 0 && rep_info) {
-    auto fs = fsmap.filesystems.at(fscid);
-    if (fs->mds_map.test_flag(CEPH_MDSMAP_NOT_JOINABLE)) {
+    auto& fs = fsmap.get_filesystem(fscid);
+    if (fs.get_mds_map().test_flag(CEPH_MDSMAP_NOT_JOINABLE)) {
       return false;
     }
     // are we in?
@@ -2088,7 +2082,7 @@ bool MDSMonitor::drop_mds(FSMap &fsmap, mds_gid_t gid, const mds_info_t* rep_inf
     *osd_propose |= fail_mds_gid(fsmap, gid);
 
     // Promote the replacement
-    fsmap.promote(rep_info->global_id, *fs, rank);
+    fsmap.promote(rep_info->global_id, fscid, rank);
 
     return true;
   }
@@ -2123,8 +2117,7 @@ bool MDSMonitor::check_health(FSMap& fsmap, bool* propose_osdmap)
   }
 
   // make sure last_beacon is fully populated
-  for (auto& p : fsmap.mds_roles) {
-    auto& gid = p.first;
+  for ([[maybe_unused]] const auto& [gid, fscid] : fsmap.get_mds_roles()) {
     last_beacon.emplace(std::piecewise_construct,
         std::forward_as_tuple(gid),
         std::forward_as_tuple(now, 0));
@@ -2208,12 +2201,12 @@ bool MDSMonitor::check_health(FSMap& fsmap, bool* propose_osdmap)
   }
 
   if (osdmap_writeable) {
-    for (auto& [fscid, fs] : fsmap.filesystems) {
-      if (!fs->mds_map.test_flag(CEPH_MDSMAP_NOT_JOINABLE) &&
-          fs->mds_map.is_resizeable()) {
+    for (auto& [fscid, fs] : std::as_const(fsmap)) {
+      if (!fs.get_mds_map().test_flag(CEPH_MDSMAP_NOT_JOINABLE) &&
+          fs.get_mds_map().is_resizeable()) {
         // Check if a rank or standby-replay should be replaced with a stronger
         // affinity standby. This looks at ranks and standby-replay:
-        for (const auto& [gid, info] : fs->mds_map.get_mds_info()) {
+        for (const auto& [gid, info] : fs.get_mds_map().get_mds_info()) {
           const auto join_fscid = info.join_fscid;
           if (join_fscid == fscid)
             continue;
@@ -2221,7 +2214,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(*fs);
+            rep_info = fsmap.get_available_standby(fs);
           } else if (state == MDSMap::STATE_ACTIVE) {
             rep_info = fsmap.find_replacement_for({fscid, rank});
           } else {
@@ -2260,9 +2253,10 @@ bool MDSMonitor::check_health(FSMap& fsmap, bool* propose_osdmap)
   return do_propose;
 }
 
-bool MDSMonitor::maybe_promote_standby(FSMap &fsmap, Filesystem& fs)
+bool MDSMonitor::maybe_promote_standby(FSMap &fsmap, const Filesystem& fs)
 {
-  if (fs.mds_map.test_flag(CEPH_MDSMAP_NOT_JOINABLE)) {
+  auto& mds_map = fs.get_mds_map();
+  if (mds_map.test_flag(CEPH_MDSMAP_NOT_JOINABLE)) {
     return false;
   }
 
@@ -2270,22 +2264,22 @@ bool MDSMonitor::maybe_promote_standby(FSMap &fsmap, Filesystem& fs)
 
   // have a standby take over?
   set<mds_rank_t> failed;
-  fs.mds_map.get_failed_mds_set(failed);
+  mds_map.get_failed_mds_set(failed);
   for (const auto& rank : failed) {
-    auto info = fsmap.find_replacement_for({fs.fscid, rank});
+    auto info = fsmap.find_replacement_for({fs.get_fscid(), rank});
     if (info) {
       dout(1) << " taking over failed mds." << rank << " with " << info->global_id
               << "/" << info->name << " " << info->addrs << dendl;
       mon.clog->info() << "Standby " << info->human_name()
-                        << " assigned to filesystem " << fs.mds_map.fs_name
+                        << " assigned to filesystem " << mds_map.fs_name
                         << " as rank " << rank;
 
-      fsmap.promote(info->global_id, fs, rank);
+      fsmap.promote(info->global_id, fs.get_fscid(), rank);
       do_propose = true;
     }
   }
 
-  if (fs.mds_map.is_resizeable() && fs.mds_map.allows_standby_replay()) {
+  if (mds_map.is_resizeable() && mds_map.allows_standby_replay()) {
     // There were no failures to replace, so try using any available standbys
     // 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.
@@ -2294,12 +2288,12 @@ bool MDSMonitor::maybe_promote_standby(FSMap &fsmap, Filesystem& fs)
       if (!info) break;
       dout(20) << "standby available mds." << info->global_id << dendl;
       bool changed = false;
-      for (const auto& rank : fs.mds_map.in) {
+      for (const auto& rank : mds_map.in) {
         dout(20) << "examining " << rank << dendl;
-        if (fs.mds_map.is_followable(rank)) {
+        if (mds_map.is_followable(rank)) {
           dout(1) << "  setting mds." << info->global_id
                   << " to follow mds rank " << rank << dendl;
-          fsmap.assign_standby_replay(info->global_id, fs.fscid, rank);
+          fsmap.assign_standby_replay(info->global_id, fs.get_fscid(), rank);
           do_propose = true;
           changed = true;
           break;
@@ -2363,13 +2357,13 @@ void MDSMonitor::tick()
   do_propose |= check_health(pending, &propose_osdmap);
 
   /* Resize the cluster according to max_mds. */
-  for (auto& p : pending.filesystems) {
-    do_propose |= maybe_resize_cluster(pending, p.second->fscid);
+  for ([[maybe_unused]] const auto& [fscid, fs] : std::as_const(pending)) {
+    do_propose |= maybe_resize_cluster(pending, fs);
   }
 
   /* Replace any failed ranks. */
-  for (auto& p : pending.filesystems) {
-    do_propose |= maybe_promote_standby(pending, *p.second);
+  for ([[maybe_unused]] auto& [fscid, fs] : std::as_const(pending)) {
+    do_propose |= maybe_promote_standby(pending, fs);
   }
 
   if (propose_osdmap) {
index c41246992d5c83c7402258ff91868ea2e077d44c..36d53fe4e48322baf866ba27e235758cf3082621 100644 (file)
@@ -115,8 +115,8 @@ class MDSMonitor : public PaxosService, public PaxosFSMap, protected CommandHand
 
   std::list<std::shared_ptr<FileSystemCommandHandler> > handlers;
 
-  bool maybe_promote_standby(FSMap& fsmap, Filesystem& fs);
-  bool maybe_resize_cluster(FSMap &fsmap, fs_cluster_id_t fscid);
+  bool maybe_promote_standby(FSMap& fsmap, const Filesystem& fs);
+  bool maybe_resize_cluster(FSMap &fsmap, const Filesystem& fs);
   bool drop_mds(FSMap &fsmap, mds_gid_t gid, const mds_info_t* rep_info, bool* osd_propose);
   bool check_health(FSMap &fsmap, bool* osd_propose);
   void tick() override;     // check state, take actions
index e32c44e0bc222a2dfad49392d1814eb557f5a758..7299988316118fe6a6111d494e001be45d579643 100644 (file)
@@ -35,7 +35,7 @@ protected:
   FSMap &create_pending() {
     ceph_assert(is_leader());
     pending_fsmap = fsmap;
-    pending_fsmap.epoch++;
+    pending_fsmap.inc_epoch();
     return pending_fsmap;
   }
 
index 0ba56c5154ec0c4d564ebcbb5d7a6f47639d0128..c0de302bc8d58cdb4138d6addadcf44c599a6046 100644 (file)
@@ -103,13 +103,13 @@ bool DataScan::parse_kwarg(
     dout(10) << "Applying tag filter: '" << filter_tag << "'" << dendl;
     return true;
   } else if (arg == std::string("--filesystem")) {
-    std::shared_ptr<const Filesystem> fs;
+    Filesystem const* fs;
     *r = fsmap->parse_filesystem(val, &fs);
     if (*r != 0) {
       std::cerr << "Invalid filesystem '" << val << "'" << std::endl;
       return false;
     }
-    fscid = fs->fscid;
+    fscid = fs->get_fscid();
     return true;
   } else if (arg == std::string("--alternate-pool")) {
     metadata_pool_name = val;
@@ -223,14 +223,13 @@ int DataScan::main(const std::vector<const char*> &args)
   // one if only one exists
   if (fscid == FS_CLUSTER_ID_NONE) {
     if (fsmap->filesystem_count() == 1) {
-      fscid = fsmap->get_filesystem()->fscid;
+      fscid = fsmap->get_filesystem().get_fscid();
     } else {
       std::cerr << "Specify a filesystem with --filesystem" << std::endl;
       return -EINVAL;
     }
   }
-  auto fs =  fsmap->get_filesystem(fscid);
-  ceph_assert(fs != nullptr);
+  auto& fs = fsmap->get_filesystem(fscid);
 
   // Default to output to metadata pool
   if (driver == NULL) {
@@ -265,7 +264,7 @@ int DataScan::main(const std::vector<const char*> &args)
   if (command == "scan_inodes" ||
       command == "scan_extents" ||
       command == "cleanup") {
-    data_pool_id = fs->mds_map.get_first_data_pool();
+    data_pool_id = fs.get_mds_map().get_first_data_pool();
 
     std::string pool_name;
     r = rados.pool_reverse_lookup(data_pool_id, &pool_name);
@@ -309,7 +308,7 @@ int DataScan::main(const std::vector<const char*> &args)
     if (autodetect_data_pools) {
       ceph_assert(extra_data_pool_names.empty());
 
-      for (auto &pool_id : fs->mds_map.get_data_pools()) {
+      for (auto &pool_id : fs.get_mds_map().get_data_pools()) {
        if (pool_id == data_pool_id) {
          continue;
        }
@@ -335,7 +334,7 @@ int DataScan::main(const std::vector<const char*> &args)
                << dendl;
       }
 
-      if (!fs->mds_map.is_data_pool(pool_id)) {
+      if (!fs.get_mds_map().is_data_pool(pool_id)) {
        std::cerr << "Warning: pool '" << data_pool_name << "' is not a "
          "CephFS data pool!" << std::endl;
        if (!force_pool) {
@@ -355,12 +354,8 @@ int DataScan::main(const std::vector<const char*> &args)
 
   // Initialize metadata_io from MDSMap for scan_frags
   if (command == "scan_frags" || command == "scan_links") {
-    const auto fs = fsmap->get_filesystem(fscid);
-    if (fs == nullptr) {
-      std::cerr << "Filesystem id " << fscid << " does not exist" << std::endl;
-      return -ENOENT;
-    }
-    int64_t const metadata_pool_id = fs->mds_map.get_metadata_pool();
+    auto& fs = fsmap->get_filesystem(fscid);
+    int64_t const metadata_pool_id = fs.get_mds_map().get_metadata_pool();
 
     dout(4) << "resolving metadata pool " << metadata_pool_id << dendl;
     int r = rados.pool_reverse_lookup(metadata_pool_id, &metadata_pool_name);
@@ -375,7 +370,7 @@ int DataScan::main(const std::vector<const char*> &args)
       return r;
     }
 
-    data_pools = fs->mds_map.get_data_pools();
+    data_pools = fs.get_mds_map().get_data_pools();
   }
 
   // Finally, dispatch command
@@ -390,7 +385,7 @@ int DataScan::main(const std::vector<const char*> &args)
   } else if (command == "cleanup") {
     return cleanup();
   } else if (command == "init") {
-    return driver->init_roots(fs->mds_map.get_first_data_pool());
+    return driver->init_roots(fs.get_mds_map().get_first_data_pool());
   } else {
     std::cerr << "Unknown command '" << command << "'" << std::endl;
     return -EINVAL;
@@ -864,8 +859,8 @@ int DataScan::scan_inodes()
         // ID, so if the pool from loaded_layout is not found in the list of
         // the data pools, we'll force the injected layout to point to the
         // pool we read from.
-       if (!fsmap->get_filesystem(fscid)->mds_map.is_data_pool(
-             guessed_layout.pool_id)) {
+        auto& fs = fsmap->get_filesystem(fscid);
+       if (!fs.get_mds_map().is_data_pool(guessed_layout.pool_id)) {
          dout(20) << "overwriting layout pool_id " << data_pool_id << dendl;
          guessed_layout.pool_id = data_pool_id;
        }
@@ -2185,9 +2180,8 @@ int MetadataDriver::init(
   fs_cluster_id_t fscid)
 {
   if (metadata_pool_name.empty()) {
-    auto fs =  fsmap->get_filesystem(fscid);
-    ceph_assert(fs != nullptr);
-    int64_t const metadata_pool_id = fs->mds_map.get_metadata_pool();
+    auto& fs =  fsmap->get_filesystem(fscid);
+    int64_t const metadata_pool_id = fs.get_mds_map().get_metadata_pool();
 
     dout(4) << "resolving metadata pool " << metadata_pool_id << dendl;
     int r = rados.pool_reverse_lookup(metadata_pool_id, &metadata_pool_name);
index 68a190182dd03cc11892e0640117f4cb24a8fd60..2253ebc6cb7c75705065547cc843f6dbb2770076 100644 (file)
@@ -45,11 +45,10 @@ int Dumper::init(mds_role_t role_, const std::string &type)
     return r;
   }
 
-  auto fs =  fsmap->get_filesystem(role.fscid);
-  ceph_assert(fs != nullptr);
+  auto& fs =  fsmap->get_filesystem(role.fscid);
 
   if (type == "mdlog") {
-    JournalPointer jp(role.rank, fs->mds_map.get_metadata_pool());
+    JournalPointer jp(role.rank, fs.get_mds_map().get_metadata_pool());
     int jp_load_result = jp.load(objecter);
     if (jp_load_result != 0) {
       std::cerr << "Error loading journal: " << cpp_strerror(jp_load_result) << std::endl;
@@ -88,10 +87,9 @@ int Dumper::dump(const char *dump_file)
 {
   int r = 0;
 
-  auto fs =  fsmap->get_filesystem(role.fscid);
-  ceph_assert(fs != nullptr);
+  auto& fs = fsmap->get_filesystem(role.fscid);
 
-  Journaler journaler("dumper", ino, fs->mds_map.get_metadata_pool(),
+  Journaler journaler("dumper", ino, fs.get_mds_map().get_metadata_pool(),
                       CEPH_FS_ONDISK_MAGIC, objecter, 0, 0,
                       &finisher);
   r = recover_journal(&journaler);
@@ -202,12 +200,11 @@ int Dumper::undump(const char *dump_file, bool force)
 {
   cout << "undump " << dump_file << std::endl;
   
-  auto fs =  fsmap->get_filesystem(role.fscid);
-  ceph_assert(fs != nullptr);
+  auto& fs = fsmap->get_filesystem(role.fscid);
 
   int r = 0;
   // try get layout info from cluster
-  Journaler journaler("umdumper", ino, fs->mds_map.get_metadata_pool(),
+  Journaler journaler("umdumper", ino, fs.get_mds_map().get_metadata_pool(),
                       CEPH_FS_ONDISK_MAGIC, objecter, 0, 0,
                       &finisher);
   int recovered = recover_journal(&journaler);
@@ -330,13 +327,13 @@ int Dumper::undump(const char *dump_file, bool force)
   h.layout.stripe_unit = stripe_unit;
   h.layout.stripe_count = stripe_count;
   h.layout.object_size = object_size;
-  h.layout.pool_id = fs->mds_map.get_metadata_pool();
+  h.layout.pool_id = fs.get_mds_map().get_metadata_pool();
   
   bufferlist hbl;
   encode(h, hbl);
 
   object_t oid = file_object_t(ino, 0);
-  object_locator_t oloc(fs->mds_map.get_metadata_pool());
+  object_locator_t oloc(fs.get_mds_map().get_metadata_pool());
   SnapContext snapc;
 
   cout << "writing header " << oid << std::endl;
index 6bca9bb08167b960111efe2b2f727cffef15f950..fed266123551ff2783c8d63fe97158f2c1e9f6dd 100644 (file)
@@ -138,9 +138,8 @@ int JournalTool::main(std::vector<const char*> &argv)
     return r;
   }
  
-  auto fs = fsmap->get_filesystem(role_selector.get_ns());
-  ceph_assert(fs != nullptr);
-  int64_t const pool_id = fs->mds_map.get_metadata_pool();
+  auto& fs = fsmap->get_filesystem(role_selector.get_ns());
+  int64_t const pool_id = fs.get_mds_map().get_metadata_pool();
   dout(4) << "JournalTool: resolving pool " << pool_id << dendl;
   std::string pool_name;
   r = rados.pool_reverse_lookup(pool_id, &pool_name);
@@ -1201,9 +1200,9 @@ int JournalTool::consume_inos(const std::set<inodeno_t> &inos)
   int r = 0;
 
   // InoTable is a per-MDS structure, so iterate over assigned ranks
-  auto fs = fsmap->get_filesystem(role_selector.get_ns());
+  auto& fs = fsmap->get_filesystem(role_selector.get_ns());
   std::set<mds_rank_t> in_ranks;
-  fs->mds_map.get_mds_set(in_ranks);
+  fs.get_mds_map().get_mds_set(in_ranks);
 
   for (std::set<mds_rank_t>::iterator rank_i = in_ranks.begin();
       rank_i != in_ranks.end(); ++rank_i)
index baa0d498a2aa1a2e0e8fb119ff93f1c9d0b9db30..f80baa2f83bae772565cdd04905706d427a72501 100644 (file)
@@ -125,12 +125,12 @@ int MetaTool::main(string& mode,
       return r;
     }
 
-    auto fs = fsmap->get_filesystem(role_selector.get_ns());
-    assert(fs != nullptr);
+    auto& fs = fsmap->get_filesystem(role_selector.get_ns());
+    auto& mds_map = fs.get_mds_map();
 
     // prepare io for meta pool
-    int64_t const pool_id = fs->mds_map.get_metadata_pool();
-    features = fs->mds_map.get_up_features();
+    int64_t const pool_id = mds_map.get_metadata_pool();
+    features = mds_map.get_up_features();
     if (features == 0)
       features = CEPH_FEATURES_SUPPORTED_DEFAULT;
     else if (features != CEPH_FEATURES_SUPPORTED_DEFAULT) {
@@ -152,7 +152,7 @@ int MetaTool::main(string& mode,
     output.dup(io_meta);
 
     // prepare io for data pool
-    for (const auto p : fs->mds_map.get_data_pools()) {
+    for (const auto p : mds_map.get_data_pools()) {
       r = rados.pool_reverse_lookup(p, &pool_name);
       if (r < 0) {
         cerr << "Pool " << pool_id << " named in MDS map not found in RADOS!" << std::endl;
index 7c0aa30ab6a88acf92c1c83a620034eaf76dcd34..151a2e22e44128a8e1507a9547048b5af339774a 100644 (file)
@@ -36,12 +36,11 @@ int Resetter::init(mds_role_t role_, const std::string &type, bool hard)
     return r;
   }
 
-  auto fs = fsmap->get_filesystem(role.fscid);
-  ceph_assert(nullptr != fs);
+  auto& fs = fsmap->get_filesystem(role.fscid);
 
   is_mdlog = false;
   if (type == "mdlog") {
-    JournalPointer jp(role.rank, fs->mds_map.get_metadata_pool());
+    JournalPointer jp(role.rank, fs.get_mds_map().get_metadata_pool());
     int rt = 0;
     if (hard) {
       jp.front = role.rank + MDS_INO_LOG_OFFSET;
@@ -78,11 +77,10 @@ int Resetter::reset()
   bool done;
   int r;
 
-  auto fs =  fsmap->get_filesystem(role.fscid);
-  ceph_assert(fs != nullptr);
+  auto& fs = fsmap->get_filesystem(role.fscid);
 
   Journaler journaler("resetter", ino,
-      fs->mds_map.get_metadata_pool(),
+      fs.get_mds_map().get_metadata_pool(),
       CEPH_FS_ONDISK_MAGIC,
       objecter, 0, 0, &finisher);
   {
@@ -147,16 +145,16 @@ int Resetter::reset()
 
 int Resetter::reset_hard()
 {
-  auto fs =  fsmap->get_filesystem(role.fscid);
+  auto& fs = fsmap->get_filesystem(role.fscid);
+  auto& mds_map = fs.get_mds_map();
   
   Journaler journaler("resetter", ino,
-    fs->mds_map.get_metadata_pool(),
+    mds_map.get_metadata_pool(),
     CEPH_FS_ONDISK_MAGIC,
     objecter, 0, 0, &finisher);
   journaler.set_writeable();
 
-  file_layout_t default_log_layout = MDCache::gen_default_log_layout(
-      fsmap->get_filesystem(role.fscid)->mds_map);
+  file_layout_t default_log_layout = MDCache::gen_default_log_layout(mds_map);
   journaler.create(&default_log_layout, g_conf()->mds_journal_format);
 
   C_SaferCond cond;
index e2d53b86ea790b21f06a8b85dde2ba5412aade66..e8f9d7f12d7aee5330210bbe9a1bcf59e33d227d 100644 (file)
@@ -5,9 +5,9 @@ int MDSRoleSelector::parse_rank(
     const FSMap &fsmap,
     std::string const &str)
 {
+  auto& mds_map = fsmap.get_filesystem(fscid).get_mds_map();
   if (str == "all" || str == "*") {
     std::set<mds_rank_t> in;
-    const MDSMap &mds_map = fsmap.get_filesystem(fscid)->mds_map;
     mds_map.get_mds_set(in);
 
     for (auto rank : in) {
@@ -21,7 +21,7 @@ int MDSRoleSelector::parse_rank(
     if (!rank_err.empty()) {
       return -EINVAL;
     }
-    if (fsmap.get_filesystem(fscid)->mds_map.is_dne(rank)) {
+    if (mds_map.is_dne(rank)) {
       return -ENOENT;
     }
     roles.push_back(mds_role_t(fscid, rank));
@@ -37,7 +37,7 @@ int MDSRoleSelector::parse(const FSMap &fsmap, std::string const &str,
     // An unqualified rank.  Only valid if there is only one
     // namespace.
     if (fsmap.filesystem_count() == 1 && allow_unqualified_rank) {
-      fscid = fsmap.get_filesystem()->fscid;
+      fscid = fsmap.get_filesystem().get_fscid();
       return parse_rank(fsmap, str);
     } else {
       return -EINVAL;
@@ -47,12 +47,12 @@ int MDSRoleSelector::parse(const FSMap &fsmap, std::string const &str,
   } else {
     const std::string ns_str = str.substr(0, colon_pos);
     const std::string rank_str = str.substr(colon_pos + 1);
-    std::shared_ptr<const Filesystem> fs_ptr;
+    Filesystem const* fs_ptr;
     int r = fsmap.parse_filesystem(ns_str, &fs_ptr);
     if (r != 0) {
       return r;
     }
-    fscid = fs_ptr->fscid;
+    fscid = fs_ptr->get_fscid();
     return parse_rank(fsmap, rank_str);
   }
 }
index dcd35a6241b7e5bcda4da685e2e229679881d2ad..78fc5965823c57dd6a7fb55322884407a3f2401e 100644 (file)
@@ -335,9 +335,8 @@ int TableTool::main(std::vector<const char*> &argv)
     return r;
   }
 
-  auto fs =  fsmap->get_filesystem(role_selector.get_ns());
-  ceph_assert(fs != nullptr);
-  int64_t const pool_id = fs->mds_map.get_metadata_pool();
+  auto& fs = fsmap->get_filesystem(role_selector.get_ns());
+  int64_t const pool_id = fs.get_mds_map().get_metadata_pool();
   dout(4) << "resolving pool " << pool_id << dendl;
   std::string pool_name;
   r = rados.pool_reverse_lookup(pool_id, &pool_name);
index b5f6f81d7f772eabae3297e8108e6cf869d532cc..84c34d4683ad8ab4d73900a8a0e69f4bf1048421 100644 (file)
@@ -76,7 +76,6 @@ void ClusterWatcher::handle_fsmap(const cref_t<MFSMap> &m) {
   dout(20) << dendl;
 
   auto fsmap = m->get_fsmap();
-  auto filesystems = fsmap.get_filesystems();
 
   std::vector<Filesystem> mirroring_enabled;
   std::vector<Filesystem> mirroring_disabled;
@@ -99,11 +98,11 @@ void ClusterWatcher::handle_fsmap(const cref_t<MFSMap> &m) {
       ++it;
     }
 
-    for (auto &filesystem : filesystems) {
-      auto fs = Filesystem{filesystem->fscid,
-                           std::string(filesystem->mds_map.get_fs_name())};
-      auto pool_id = filesystem->mds_map.get_metadata_pool();
-      auto &mirror_info = filesystem->mirror_info;
+    for (auto& [fscid, _fs] : std::as_const(fsmap)) {
+      auto& mds_map = _fs.get_mds_map();
+      auto fs = Filesystem{fscid, std::string(mds_map.get_fs_name())};
+      auto pool_id = mds_map.get_metadata_pool();
+      auto& mirror_info = _fs.get_mirror_info();
 
       if (!mirror_info.is_mirrored()) {
         auto it = m_filesystem_peers.find(fs);