]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
mon: remove legacy get_health infrastructure
authorSage Weil <sage@redhat.com>
Fri, 25 Aug 2017 22:31:54 +0000 (18:31 -0400)
committerSage Weil <sage@redhat.com>
Wed, 6 Sep 2017 14:18:02 +0000 (10:18 -0400)
Signed-off-by: Sage Weil <sage@redhat.com>
24 files changed:
src/mon/ConfigKeyService.h
src/mon/DataHealthService.cc
src/mon/DataHealthService.h
src/mon/HealthMonitor.h
src/mon/MDSMonitor.cc
src/mon/MDSMonitor.h
src/mon/MgrMonitor.cc
src/mon/MgrMonitor.h
src/mon/MgrStatMonitor.cc
src/mon/MgrStatMonitor.h
src/mon/Monitor.cc
src/mon/Monitor.h
src/mon/MonmapMonitor.cc
src/mon/MonmapMonitor.h
src/mon/OSDMonitor.cc
src/mon/OSDMonitor.h
src/mon/OldHealthMonitor.cc
src/mon/OldHealthMonitor.h
src/mon/PGMap.cc
src/mon/PGMap.h
src/mon/PGMonitor.cc
src/mon/PGMonitor.h
src/mon/PaxosService.h
src/mon/QuorumService.h

index 7dfb140c7e7c3a34f38e8a804ee21bb1718292c1..a20969939e1171c6b347bfb717220b4623fc0ea5 100644 (file)
@@ -57,8 +57,6 @@ public:
    * @{
    */
   void init() override { }
-  void get_health(list<pair<health_status_t,string> >& summary,
-                  list<pair<health_status_t,string> > *detail) override { }
   bool service_dispatch(MonOpRequestRef op) override;
 
   void start_epoch() override { }
index 4a5b42ab388897075a0499762cc64e6aa606c8d1..236851473c35f07a1ca2aa351bfa515f42c0e632 100644 (file)
@@ -64,49 +64,6 @@ void DataHealthService::start_epoch()
   last_warned_percent = 0;
 }
 
-void DataHealthService::get_health(
-    list<pair<health_status_t,string> >& summary,
-    list<pair<health_status_t,string> > *detail)
-{
-  dout(10) << __func__ << dendl;
-  for (map<entity_inst_t,DataStats>::iterator it = stats.begin();
-       it != stats.end(); ++it) {
-    string mon_name = mon->monmap->get_name(it->first.addr);
-    DataStats& stats = it->second;
-
-    health_status_t health_status = HEALTH_OK;
-    string health_detail;
-    if (stats.fs_stats.avail_percent <= g_conf->mon_data_avail_crit) {
-      health_status = HEALTH_ERR;
-      health_detail = "low disk space, shutdown imminent";
-    } else if (stats.fs_stats.avail_percent <= g_conf->mon_data_avail_warn) {
-      health_status = HEALTH_WARN;
-      health_detail = "low disk space";
-    }
-
-    if (stats.store_stats.bytes_total >= g_conf->mon_data_size_warn) {
-      if (health_status > HEALTH_WARN)
-        health_status = HEALTH_WARN;
-      if (!health_detail.empty())
-        health_detail.append("; ");
-      stringstream ss;
-      ss << "store is getting too big! "
-         << prettybyte_t(stats.store_stats.bytes_total)
-         << " >= " << prettybyte_t(g_conf->mon_data_size_warn);
-      health_detail.append(ss.str());
-    }
-
-    if (health_status != HEALTH_OK) {
-      stringstream ss;
-      ss << "mon." << mon_name << " " << health_detail;
-      summary.push_back(make_pair(health_status, ss.str()));
-      ss << " -- " <<  stats.fs_stats.avail_percent << "% avail";
-      if (detail)
-       detail->push_back(make_pair(health_status, ss.str()));
-    }
-  }
-}
-
 int DataHealthService::update_store_stats(DataStats &ours)
 {
   map<string,uint64_t> extra;
index 91caf4e32115c394cb96aff9b476a6767aad05e7..3dfcd54db075f4b072ea5a60fc3dbf396d1e4d99 100644 (file)
@@ -65,10 +65,6 @@ public:
     start_tick();
   }
 
-  void get_health(
-    list<pair<health_status_t,string> >& summary,
-    list<pair<health_status_t,string> > *detail) override;
-
   int get_type() override {
     return HealthService::SERVICE_HEALTH_DATA;
   }
index 342778dbc22312af2b1b2e3c4b411f96df79c2f3..ca9e083c8de3c28fcadb888182deac9ac756d55a 100644 (file)
@@ -39,11 +39,6 @@ public:
    */
   void init() override;
 
-  void get_health(
-    list<pair<health_status_t,string> >& summary,
-    list<pair<health_status_t,string> > *detail,
-    CephContext *cct) const override {}
-
   bool preprocess_query(MonOpRequestRef op) override;
   bool prepare_update(MonOpRequestRef op) override;
 
index f38c1e91fd9531f5585eee48d5256d9cc27174a0..b103b6b9b5e70acd52fdd13d8928160e91056cc9 100644 (file)
@@ -830,58 +830,6 @@ void MDSMonitor::on_active()
   }
 }
 
-void MDSMonitor::get_health(list<pair<health_status_t, string> >& summary,
-                           list<pair<health_status_t, string> > *detail,
-                           CephContext* cct) const
-{
-  fsmap.get_health(summary, detail);
-
-  // For each MDS GID...
-  const auto info_map = fsmap.get_mds_info();
-  for (const auto &i : info_map) {
-    const auto &gid = i.first;
-    const auto &info = i.second;
-
-    // Decode MDSHealth
-    bufferlist bl;
-    mon->store->get(MDS_HEALTH_PREFIX, stringify(gid), bl);
-    if (!bl.length()) {
-      derr << "Missing health data for MDS " << gid << dendl;
-      continue;
-    }
-    MDSHealth health;
-    bufferlist::iterator bl_i = bl.begin();
-    health.decode(bl_i);
-
-    for (const auto &metric : health.metrics) {
-      const int rank = info.rank;
-      std::ostringstream message;
-      message << "mds" << rank << ": " << metric.message;
-      summary.push_back(std::make_pair(metric.sev, message.str()));
-
-      if (detail) {
-        // There is no way for us to clealy associate detail entries with summary entries (#7192), so
-        // we duplicate the summary message in the detail string and tag the metadata on.
-        std::ostringstream detail_message;
-        detail_message << message.str();
-        if (metric.metadata.size()) {
-          detail_message << "(";
-          auto k = metric.metadata.begin();
-          while (k != metric.metadata.end()) {
-            detail_message << k->first << ": " << k->second;
-            if (boost::next(k) != metric.metadata.end()) {
-              detail_message << ", ";
-            }
-            ++k;
-          }
-          detail_message << ")";
-        }
-        detail->push_back(std::make_pair(metric.sev, detail_message.str()));
-      }
-    }
-  }
-}
-
 void MDSMonitor::dump_info(Formatter *f)
 {
   f->open_object_section("fsmap");
index c14c9603943cc25c1a3699f70516f5f6b6f8d6d2..0bf51adfe60d8f0c7fa701e485605388ee462809 100644 (file)
@@ -87,9 +87,6 @@ class MDSMonitor : public PaxosService {
   bool preprocess_offload_targets(MonOpRequestRef op);
   bool prepare_offload_targets(MonOpRequestRef op);
 
-  void get_health(list<pair<health_status_t,string> >& summary,
-                 list<pair<health_status_t,string> > *detail,
-                 CephContext *cct) const override;
   int fail_mds(std::ostream &ss, const std::string &arg,
       MDSMap::mds_info_t *failed_info);
 
index e35a43a6738be034cd57f37274ef094824898aa7..e5fe9d64bbd85ad32ac0819ef8178c72e6d07433 100644 (file)
@@ -475,36 +475,6 @@ void MgrMonitor::on_active()
   }
 }
 
-void MgrMonitor::get_health(
-  list<pair<health_status_t,string> >& summary,
-  list<pair<health_status_t,string> > *detail,
-  CephContext *cct) const
-{
-  // start mgr warnings as soon as the mons and osds are all upgraded,
-  // but before the require_luminous osdmap flag is set.  this way the
-  // user gets some warning before the osd flag is set and mgr is
-  // actually *required*.
-  if (!mon->monmap->get_required_features().contains_all(
-       ceph::features::mon::FEATURE_LUMINOUS) ||
-      !HAVE_FEATURE(mon->osdmon()->osdmap.get_up_osd_features(),
-                   SERVER_LUMINOUS)) {
-    return;
-  }
-
-  if (map.active_gid == 0) {
-    auto level = HEALTH_WARN;
-    // do not escalate to ERR if they are still upgrading to jewel.
-    if (mon->osdmon()->osdmap.require_osd_release >= CEPH_RELEASE_LUMINOUS) {
-      utime_t now = ceph_clock_now();
-      if (first_seen_inactive != utime_t() &&
-         now - first_seen_inactive > g_conf->mon_mgr_inactive_grace) {
-       level = HEALTH_ERR;
-      }
-    }
-    summary.push_back(make_pair(level, "no active mgr"));
-  }
-}
-
 void MgrMonitor::tick()
 {
   if (!is_active() || !mon->is_leader())
index 65451633dbefc0aa85c71bd4d65b50e222548922..393211c204732f46306aec33cbc97f1f6e469017 100644 (file)
@@ -92,9 +92,6 @@ public:
   void on_active() override;
   void on_restart() override;
 
-  void get_health(list<pair<health_status_t,string> >& summary,
-                 list<pair<health_status_t,string> > *detail,
-                 CephContext *cct) const override;
   void tick() override;
 
   void print_summary(Formatter *f, std::ostream *ss) const;
index 3ea5f30032a80a90c7ba1bde48715515490e13f0..77dd5e854e04d9439ac12efebe7f780a76e0b784 100644 (file)
@@ -189,12 +189,6 @@ void MgrStatMonitor::on_active()
   update_logger();
 }
 
-void MgrStatMonitor::get_health(list<pair<health_status_t,string> >& summary,
-                               list<pair<health_status_t,string> > *detail,
-                               CephContext *cct) const
-{
-}
-
 void MgrStatMonitor::tick()
 {
 }
index 1dc264f86b6d06671f5634d023146b4d93eda0e3..157a415914c458778f68a7f724ee05a1d5d0c806 100644 (file)
@@ -57,9 +57,6 @@ public:
   void send_digests();
 
   void on_active() override;
-  void get_health(list<pair<health_status_t,string> >& summary,
-                 list<pair<health_status_t,string> > *detail,
-                 CephContext *cct) const override;
   void tick() override;
 
   uint64_t get_last_osd_stat_seq(int osd) {
index 5e8eb4d3c65cd06eaa79450dd1de091129ea0a07..cd5fcc81228ab5b548d05c7b42d79bf93692b424 100755 (executable)
@@ -2444,39 +2444,15 @@ void Monitor::do_health_to_clog(bool force)
 
   dout(10) << __func__ << (force ? " (force)" : "") << dendl;
 
-  if (osdmon()->osdmap.require_osd_release >= CEPH_RELEASE_LUMINOUS) {
-    string summary;
-    health_status_t level = get_health_status(false, nullptr, &summary);
-    if (!force &&
-       summary == health_status_cache.summary &&
-       level == health_status_cache.overall)
-      return;
-    clog->health(level) << "overall " << summary;
-    health_status_cache.summary = summary;
-    health_status_cache.overall = level;
-  } else {
-    // for jewel only
-    list<string> status;
-    health_status_t overall = get_health(status, NULL, NULL);
-    dout(25) << __func__
-            << (force ? " (force)" : "")
-            << dendl;
-
-    string summary = joinify(status.begin(), status.end(), string("; "));
-
-    if (!force &&
-       overall == health_status_cache.overall &&
-       !health_status_cache.summary.empty() &&
-       health_status_cache.summary == summary) {
-      // we got a dup!
-      return;
-    }
-
-    clog->info() << summary;
-
-    health_status_cache.overall = overall;
-    health_status_cache.summary = summary;
-  }
+  string summary;
+  health_status_t level = get_health_status(false, nullptr, &summary);
+  if (!force &&
+      summary == health_status_cache.summary &&
+      level == health_status_cache.overall)
+    return;
+  clog->health(level) << "overall " << summary;
+  health_status_cache.summary = summary;
+  health_status_cache.overall = level;
 }
 
 health_status_t Monitor::get_health_status(
@@ -2659,106 +2635,6 @@ void Monitor::log_health(
   }
 }
 
-health_status_t Monitor::get_health(list<string>& status,
-                                    bufferlist *detailbl,
-                                    Formatter *f)
-{
-  list<pair<health_status_t,string> > summary;
-  list<pair<health_status_t,string> > detail;
-
-  if (f)
-    f->open_object_section("health");
-
-  for (vector<PaxosService*>::iterator p = paxos_service.begin();
-       p != paxos_service.end();
-       ++p) {
-    PaxosService *s = *p;
-    s->get_health(summary, detailbl ? &detail : NULL, cct);
-  }
-
-  health_monitor->get_health(summary, (detailbl ? &detail : NULL));
-
-  health_status_t overall = HEALTH_OK;
-  if (!timecheck_skews.empty()) {
-    list<string> warns;
-    for (map<entity_inst_t,double>::iterator i = timecheck_skews.begin();
-         i != timecheck_skews.end(); ++i) {
-      entity_inst_t inst = i->first;
-      double skew = i->second;
-      double latency = timecheck_latencies[inst];
-      string name = monmap->get_name(inst.addr);
-      ostringstream tcss;
-      health_status_t tcstatus = timecheck_status(tcss, skew, latency);
-      if (tcstatus != HEALTH_OK) {
-        if (overall > tcstatus)
-          overall = tcstatus;
-        warns.push_back(name);
-        ostringstream tmp_ss;
-        tmp_ss << "mon." << name
-               << " addr " << inst.addr << " " << tcss.str()
-              << " (latency " << latency << "s)";
-        detail.push_back(make_pair(tcstatus, tmp_ss.str()));
-      }
-    }
-    if (!warns.empty()) {
-      ostringstream ss;
-      ss << "clock skew detected on";
-      while (!warns.empty()) {
-        ss << " mon." << warns.front();
-        warns.pop_front();
-        if (!warns.empty())
-          ss << ",";
-      }
-      status.push_back(ss.str());
-      summary.push_back(make_pair(HEALTH_WARN, "Monitor clock skew detected "));
-    }
-  }
-
-  if (f)
-    f->open_array_section("summary");
-  if (!summary.empty()) {
-    while (!summary.empty()) {
-      if (overall > summary.front().first)
-       overall = summary.front().first;
-      status.push_back(summary.front().second);
-      if (f) {
-        f->open_object_section("item");
-        f->dump_stream("severity") <<  summary.front().first;
-        f->dump_string("summary", summary.front().second);
-        f->close_section();
-      }
-      summary.pop_front();
-    }
-  }
-  if (f)
-    f->close_section();
-
-  stringstream fss;
-  fss << overall;
-  status.push_front(fss.str());
-  if (f)
-    f->dump_stream("overall_status") << overall;
-
-  if (f)
-    f->open_array_section("detail");
-  while (!detail.empty()) {
-    if (f)
-      f->dump_string("item", detail.front().second);
-    else if (detailbl != NULL) {
-      detailbl->append(detail.front().second);
-      detailbl->append('\n');
-    }
-    detail.pop_front();
-  }
-  if (f)
-    f->close_section();
-
-  if (f)
-    f->close_section();
-
-  return overall;
-}
-
 void Monitor::get_cluster_status(stringstream &ss, Formatter *f)
 {
   if (f)
@@ -2766,12 +2642,7 @@ void Monitor::get_cluster_status(stringstream &ss, Formatter *f)
 
   if (f) {
     f->dump_stream("fsid") << monmap->get_fsid();
-    if (osdmon()->osdmap.require_osd_release >= CEPH_RELEASE_LUMINOUS) {
-      get_health_status(false, f, nullptr);
-    } else {
-      list<string> health_str;
-      get_health(health_str, nullptr, f);
-    }
+    get_health_status(false, f, nullptr);
     f->dump_unsigned("election_epoch", get_epoch());
     {
       f->open_array_section("quorum");
@@ -2806,15 +2677,8 @@ void Monitor::get_cluster_status(stringstream &ss, Formatter *f)
     ss << "    id:     " << monmap->get_fsid() << "\n";
 
     string health;
-    if (osdmon()->osdmap.require_osd_release >= CEPH_RELEASE_LUMINOUS) {
-      get_health_status(false, nullptr, &health,
-                       "\n            ", "\n            ");
-    } else {
-      list<string> ls;
-      get_health(ls, NULL, f);
-      health = joinify(ls.begin(), ls.end(),
-                      string("\n            "));
-    }
+    get_health_status(false, nullptr, &health,
+                     "\n            ", "\n            ");
     ss << "    health: " << health << "\n";
 
     ss << "\n \n  services:\n";
@@ -3351,34 +3215,12 @@ void Monitor::handle_command(MonOpRequestRef op)
       }
       rdata.append(ds);
     } else if (prefix == "health") {
-      if (osdmon()->osdmap.require_osd_release >= CEPH_RELEASE_LUMINOUS) {
-       string plain;
-       get_health_status(detail == "detail", f.get(), f ? nullptr : &plain);
-       if (f) {
-         f->flush(rdata);
-       } else {
-         rdata.append(plain);
-       }
+      string plain;
+      get_health_status(detail == "detail", f.get(), f ? nullptr : &plain);
+      if (f) {
+       f->flush(rdata);
       } else {
-       list<string> health_str;
-       get_health(health_str, detail == "detail" ? &rdata : NULL, f.get());
-       if (f) {
-         f->flush(ds);
-         ds << '\n';
-       } else {
-         assert(!health_str.empty());
-         ds << health_str.front();
-         health_str.pop_front();
-         if (!health_str.empty()) {
-           ds << ' ';
-           ds << joinify(health_str.begin(), health_str.end(), string("; "));
-         }
-       }
-       bufferlist comb;
-       comb.append(ds);
-       if (detail == "detail")
-         comb.append(rdata);
-       rdata = comb;
+       rdata.append(plain);
       }
     } else if (prefix == "df") {
       bool verbose = (detail == "detail");
@@ -3420,12 +3262,7 @@ void Monitor::handle_command(MonOpRequestRef op)
       tagstr = tagstr.substr(0, tagstr.find_last_of(' '));
     f->dump_string("tag", tagstr);
 
-    if (osdmon()->osdmap.require_osd_release >= CEPH_RELEASE_LUMINOUS) {
-      get_health_status(true, f.get(), nullptr);
-    } else {
-      list<string> health_str;
-      get_health(health_str, nullptr, f.get());
-    }
+    get_health_status(true, f.get(), nullptr);
 
     monmon()->dump_info(f.get());
     osdmon()->dump_info(f.get());
@@ -4460,13 +4297,7 @@ void Monitor::handle_ping(MonOpRequestRef op)
   boost::scoped_ptr<Formatter> f(new JSONFormatter(true));
   f->open_object_section("pong");
 
-  if (osdmon()->osdmap.require_osd_release >= CEPH_RELEASE_LUMINOUS) {
-    get_health_status(false, f.get(), nullptr);
-  } else {
-    list<string> health_str;
-    get_health(health_str, nullptr, f.get());
-  }
-
+  get_health_status(false, f.get(), nullptr);
   {
     stringstream ss;
     get_mon_status(f.get(), ss);
index c6b4a365e32a6dd290d85c0b3e529050d60987e4..548953104b3bd217af9c2fe9706a3d441c2e3fcd 100644 (file)
@@ -735,16 +735,6 @@ public:
   void do_health_to_clog_interval();
   void do_health_to_clog(bool force = false);
 
-  /**
-   * Generate health report
-   *
-   * @param status one-line status summary
-   * @param detailbl optional bufferlist* to fill with a detailed report
-   * @returns health status
-   */
-  health_status_t get_health(list<string>& status, bufferlist *detailbl,
-                             Formatter *f);
-
   health_status_t get_health_status(
     bool want_detail,
     Formatter *f,
index 7a0fb684de7e93cc1077b8d3354ed95d1e09594c..f49edccab1bbbb7532d17fb3a4a79673679876b1 100644 (file)
@@ -727,31 +727,6 @@ bool MonmapMonitor::should_propose(double& delay)
   return true;
 }
 
-void MonmapMonitor::get_health(list<pair<health_status_t, string> >& summary,
-                              list<pair<health_status_t, string> > *detail,
-                              CephContext *cct) const
-{
-  int max = mon->monmap->size();
-  int actual = mon->get_quorum().size();
-  if (actual < max) {
-    ostringstream ss;
-    ss << (max-actual) << " mons down, quorum " << mon->get_quorum() << " " << mon->get_quorum_names();
-    summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-    if (detail) {
-      set<int> q = mon->get_quorum();
-      for (int i=0; i<max; i++) {
-       if (q.count(i) == 0) {
-         ostringstream ss;
-         ss << "mon." << mon->monmap->get_name(i) << " (rank " << i
-            << ") addr " << mon->monmap->get_addr(i)
-            << " is down (out of quorum)";
-         detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-       }
-      }
-    }
-  }
-}
-
 int MonmapMonitor::get_monmap(bufferlist &bl)
 {
   version_t latest_ver = get_last_committed();
index 99a2d913060f5f7df6c069d45f0a39effcbc0bff..e3e01173b6ee58b9190aa7c8c61b533e6ab6fc68 100644 (file)
@@ -68,10 +68,6 @@ class MonmapMonitor : public PaxosService {
   bool preprocess_command(MonOpRequestRef op);
   bool prepare_command(MonOpRequestRef op);
 
-  void get_health(list<pair<health_status_t,string> >& summary,
-                 list<pair<health_status_t,string> > *detail,
-                 CephContext *cct) const override;
-
   int get_monmap(bufferlist &bl);
 
   /*
index fdf004bfb65c75ce0c1c0489ecfcb935fc8ac892..0d0b4c085931c291036f1153b703444d7720963b 100644 (file)
@@ -3515,325 +3515,6 @@ bool OSDMonitor::handle_osd_timeouts(const utime_t &now,
   return new_down;
 }
 
-void OSDMonitor::get_health(list<pair<health_status_t,string> >& summary,
-                           list<pair<health_status_t,string> > *detail,
-                           CephContext *cct) const
-{
-  int num_osds = osdmap.get_num_osds();
-
-  if (num_osds == 0) {
-    summary.push_back(make_pair(HEALTH_ERR, "no osds"));
-  } else {
-    int num_in_osds = 0;
-    int num_down_in_osds = 0;
-    set<int> osds;
-    set<int> down_in_osds;
-    set<int> up_in_osds;
-    set<int> subtree_up;
-    unordered_map<int, set<int> > subtree_type_down;
-    unordered_map<int, int> num_osds_subtree;
-    int max_type = osdmap.crush->get_max_type_id();
-
-    for (int i = 0; i < osdmap.get_max_osd(); i++) {
-      if (!osdmap.exists(i)) {
-        if (osdmap.crush->item_exists(i)) {
-          osds.insert(i);
-        }
-       continue;
-      }
-      if (osdmap.is_out(i))
-        continue;
-      ++num_in_osds;
-      if (down_in_osds.count(i) || up_in_osds.count(i))
-       continue;
-      if (!osdmap.is_up(i)) {
-       down_in_osds.insert(i);
-       int parent_id = 0;
-       int current = i;
-       for (int type = 0; type <= max_type; type++) {
-         if (!osdmap.crush->get_type_name(type))
-           continue;
-         int r = osdmap.crush->get_immediate_parent_id(current, &parent_id);
-         if (r == -ENOENT)
-           break;
-         // break early if this parent is already marked as up
-         if (subtree_up.count(parent_id))
-           break;
-         type = osdmap.crush->get_bucket_type(parent_id);
-         if (!osdmap.subtree_type_is_down(
-               g_ceph_context, parent_id, type,
-               &down_in_osds, &up_in_osds, &subtree_up, &subtree_type_down))
-           break;
-         current = parent_id;
-       }
-      }
-    }
-
-    // calculate the number of down osds in each down subtree and
-    // store it in num_osds_subtree
-    for (int type = 1; type <= max_type; type++) {
-      if (!osdmap.crush->get_type_name(type))
-       continue;
-      for (auto j = subtree_type_down[type].begin();
-          j != subtree_type_down[type].end();
-          ++j) {
-       if (type == 1) {
-          list<int> children;
-          int num = osdmap.crush->get_children(*j, &children);
-          num_osds_subtree[*j] = num;
-        } else {
-          list<int> children;
-          int num = 0;
-          int num_children = osdmap.crush->get_children(*j, &children);
-          if (num_children == 0)
-           continue;
-          for (auto l = children.begin(); l != children.end(); ++l) {
-            if (num_osds_subtree[*l] > 0) {
-              num = num + num_osds_subtree[*l];
-            }
-          }
-          num_osds_subtree[*j] = num;
-       }
-      }
-    }
-    num_down_in_osds = down_in_osds.size();
-    assert(num_down_in_osds <= num_in_osds);
-    if (num_down_in_osds > 0) {
-      // summary of down subtree types and osds
-      for (int type = max_type; type > 0; type--) {
-       if (!osdmap.crush->get_type_name(type))
-         continue;
-       if (subtree_type_down[type].size() > 0) {
-         ostringstream ss;
-         ss << subtree_type_down[type].size() << " "
-            << osdmap.crush->get_type_name(type);
-         if (subtree_type_down[type].size() > 1) {
-           ss << "s";
-         }
-         int sum_down_osds = 0;
-         for (auto j = subtree_type_down[type].begin();
-              j != subtree_type_down[type].end();
-              ++j) {
-           sum_down_osds = sum_down_osds + num_osds_subtree[*j];
-         }
-          ss << " (" << sum_down_osds << " osds) down";
-         summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-       }
-      }
-      ostringstream ss;
-      ss << down_in_osds.size() << " osds down";
-      summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-
-      if (detail) {
-       // details of down subtree types
-       for (int type = max_type; type > 0; type--) {
-         if (!osdmap.crush->get_type_name(type))
-           continue;
-         for (auto j = subtree_type_down[type].rbegin();
-              j != subtree_type_down[type].rend();
-              ++j) {
-           ostringstream ss;
-           ss << osdmap.crush->get_type_name(type);
-           ss << " ";
-           ss << osdmap.crush->get_item_name(*j);
-           // at the top level, do not print location
-           if (type != max_type) {
-              ss << " (";
-              ss << osdmap.crush->get_full_location_ordered_string(*j);
-              ss << ")";
-           }
-           int num = num_osds_subtree[*j];
-           ss << " (" << num << " osds)";
-           ss << " is down";
-           detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-         }
-        }
-       // details of down osds
-       for (auto it = down_in_osds.begin(); it != down_in_osds.end(); ++it) {
-         ostringstream ss;
-         ss << "osd." << *it << " (";
-         ss << osdmap.crush->get_full_location_ordered_string(*it);
-          ss << ") is down";
-         detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-       }
-      }
-    }
-
-    if (!osds.empty()) {
-      ostringstream ss;
-      ss << osds.size() << " osds exist in the crush map but not in the osdmap";
-      summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-      if (detail) {
-        ss << " (osds: " << osds << ")";
-        detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-      }
-    }
-
-    // note: we leave it to ceph-mgr to generate details health warnings
-    // with actual osd utilizations
-
-    // warn about flags
-    uint64_t warn_flags =
-      CEPH_OSDMAP_FULL |
-      CEPH_OSDMAP_PAUSERD |
-      CEPH_OSDMAP_PAUSEWR |
-      CEPH_OSDMAP_PAUSEREC |
-      CEPH_OSDMAP_NOUP |
-      CEPH_OSDMAP_NODOWN |
-      CEPH_OSDMAP_NOIN |
-      CEPH_OSDMAP_NOOUT |
-      CEPH_OSDMAP_NOBACKFILL |
-      CEPH_OSDMAP_NORECOVER |
-      CEPH_OSDMAP_NOSCRUB |
-      CEPH_OSDMAP_NODEEP_SCRUB |
-      CEPH_OSDMAP_NOTIERAGENT |
-      CEPH_OSDMAP_NOREBALANCE;
-    if (osdmap.test_flag(warn_flags)) {
-      ostringstream ss;
-      ss << osdmap.get_flag_string(osdmap.get_flags() & warn_flags)
-        << " flag(s) set";
-      summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-      if (detail)
-       detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-    }
-
-    // old crush tunables?
-    if (g_conf->mon_warn_on_legacy_crush_tunables) {
-      string min = osdmap.crush->get_min_required_version();
-      if (min < g_conf->mon_crush_min_required_version) {
-       ostringstream ss;
-       ss << "crush map has legacy tunables (require " << min
-          << ", min is " << g_conf->mon_crush_min_required_version << ")";
-       summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-       if (detail) {
-         ss << "; see http://docs.ceph.com/docs/master/rados/operations/crush-map/#tunables";
-         detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-       }
-      }
-    }
-    if (g_conf->mon_warn_on_crush_straw_calc_version_zero) {
-      if (osdmap.crush->get_straw_calc_version() == 0) {
-       ostringstream ss;
-       ss << "crush map has straw_calc_version=0";
-       summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-       if (detail) {
-         ss << "; see http://docs.ceph.com/docs/master/rados/operations/crush-map/#tunables";
-         detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-       }
-      }
-    }
-
-    // hit_set-less cache_mode?
-    if (g_conf->mon_warn_on_cache_pools_without_hit_sets) {
-      int problem_cache_pools = 0;
-      for (map<int64_t, pg_pool_t>::const_iterator p = osdmap.pools.begin();
-          p != osdmap.pools.end();
-          ++p) {
-       const pg_pool_t& info = p->second;
-       if (info.cache_mode_requires_hit_set() &&
-           info.hit_set_params.get_type() == HitSet::TYPE_NONE) {
-         ++problem_cache_pools;
-         if (detail) {
-           ostringstream ss;
-           ss << "pool '" << osdmap.get_pool_name(p->first)
-              << "' with cache_mode " << info.get_cache_mode_name()
-              << " needs hit_set_type to be set but it is not";
-           detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-         }
-       }
-      }
-      if (problem_cache_pools) {
-       ostringstream ss;
-       ss << problem_cache_pools << " cache pools are missing hit_sets";
-       summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-      }
-    }
-
-    if (osdmap.crush->has_multirule_rulesets()) {
-      ostringstream ss;
-      ss << "CRUSH map contains multirule rulesets";
-      summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-      if (detail) {
-       ss << "; please manually fix the map";
-       detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-      }
-    }
-
-    // Not using 'sortbitwise' and should be?
-    if (!osdmap.test_flag(CEPH_OSDMAP_SORTBITWISE) &&
-        (osdmap.get_up_osd_features() &
-        CEPH_FEATURE_OSD_BITWISE_HOBJ_SORT)) {
-      ostringstream ss;
-      ss << "no legacy OSD present but 'sortbitwise' flag is not set";
-      summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-    }
-
-    // Warn if 'mon_osd_down_out_interval' is set to zero.
-    // Having this option set to zero on the leader acts much like the
-    // 'noout' flag.  It's hard to figure out what's going wrong with clusters
-    // without the 'noout' flag set but acting like that just the same, so
-    // we report a HEALTH_WARN in case this option is set to zero.
-    // This is an ugly hack to get the warning out, but until we find a way
-    // to spread global options throughout the mon cluster and have all mons
-    // using a base set of the same options, we need to work around this sort
-    // of things.
-    // There's also the obvious drawback that if this is set on a single
-    // monitor on a 3-monitor cluster, this warning will only be shown every
-    // third monitor connection.
-    if (g_conf->mon_warn_on_osd_down_out_interval_zero &&
-        g_conf->mon_osd_down_out_interval == 0) {
-      ostringstream ss;
-      ss << "mon." << mon->name << " has mon_osd_down_out_interval set to 0";
-      summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-      if (detail) {
-        ss << "; this has the same effect as the 'noout' flag";
-        detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-      }
-    }
-
-    // warn about upgrade flags that can be set but are not.
-    if (g_conf->mon_debug_no_require_luminous) {
-      // ignore these checks
-    } else if (HAVE_FEATURE(osdmap.get_up_osd_features(), SERVER_LUMINOUS) &&
-              osdmap.require_osd_release < CEPH_RELEASE_LUMINOUS) {
-      string msg = "all OSDs are running luminous or later but"
-       " require_osd_release < luminous";
-      summary.push_back(make_pair(HEALTH_WARN, msg));
-      if (detail) {
-       detail->push_back(make_pair(HEALTH_WARN, msg));
-      }
-    } else if (HAVE_FEATURE(osdmap.get_up_osd_features(), SERVER_KRAKEN) &&
-              osdmap.require_osd_release < CEPH_RELEASE_KRAKEN) {
-      string msg = "all OSDs are running kraken or later but"
-       " require_osd_release < kraken";
-      summary.push_back(make_pair(HEALTH_WARN, msg));
-      if (detail) {
-       detail->push_back(make_pair(HEALTH_WARN, msg));
-      }
-    } else if (HAVE_FEATURE(osdmap.get_up_osd_features(), SERVER_JEWEL) &&
-              osdmap.require_osd_release < CEPH_RELEASE_JEWEL) {
-      string msg = "all OSDs are running jewel or later but"
-       " require_osd_release < jewel";
-      summary.push_back(make_pair(HEALTH_WARN, msg));
-      if (detail) {
-       detail->push_back(make_pair(HEALTH_WARN, msg));
-      }
-    }
-
-    for (auto it : osdmap.get_pools()) {
-      const pg_pool_t &pool = it.second;
-      if (pool.has_flag(pg_pool_t::FLAG_FULL)) {
-       const string& pool_name = osdmap.get_pool_name(it.first);
-       stringstream ss;
-       ss << "pool '" << pool_name << "' is full";
-       summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-       if (detail)
-         detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-      }
-    }
-  }
-}
-
 void OSDMonitor::dump_info(Formatter *f)
 {
   f->open_object_section("osdmap");
index baee6a894d1d92a58fe366454995a2de37786cfd..e4b1eb7f953f3e0079b8f76ca59842e09150cffb 100644 (file)
@@ -473,9 +473,6 @@ public:
 
   void tick() override;  // check state, take actions
 
-  void get_health(list<pair<health_status_t,string> >& summary,
-                 list<pair<health_status_t,string> > *detail,
-                 CephContext *cct) const override;
   bool preprocess_command(MonOpRequestRef op);
   bool prepare_command(MonOpRequestRef op);
   bool prepare_command_impl(MonOpRequestRef op, map<string,cmd_vartype>& cmdmap);
index d7264a7ee26bd00a597b3e12a94427c5dfd5cfe9..dac93f8ebee91f011c6a7ccdfc50444c8686eed7 100644 (file)
@@ -95,13 +95,3 @@ void OldHealthMonitor::service_shutdown()
   services.clear();
 }
 
-void OldHealthMonitor::get_health(
-  list<pair<health_status_t,string> >& summary,
-  list<pair<health_status_t,string> > *detail)
-{
-  for (map<int,HealthService*>::iterator it = services.begin();
-       it != services.end();
-       ++it) {
-    it->second->get_health(summary, detail);
-  }
-}
index f295693611b25885699ba6b6015c89b46a9b9f88..b1e6eea72b0e977997870394f71b41420604ad3b 100644 (file)
@@ -39,8 +39,6 @@ public:
    * @{
    */
   void init() override;
-  void get_health(list<pair<health_status_t,string> >& summary,
-                 list<pair<health_status_t,string> > *detail) override;
   bool service_dispatch(MonOpRequestRef op) override;
 
   void start_epoch() override;
index 0268bc78267b82b0e6ac1c9fa44637d2ea7fdc97..1cecfd9853c7bfe54d290add0d749045afb241d7 100644 (file)
@@ -3294,434 +3294,6 @@ void PGMap::get_health_checks(
   }
 }
 
-void PGMap::get_health(
-  CephContext *cct,
-  const OSDMap& osdmap,
-  list<pair<health_status_t,string> >& summary,
-  list<pair<health_status_t,string> > *detail) const
-{
-  map<string,int> note;
-  auto p = num_pg_by_state.begin();
-  auto p_end = num_pg_by_state.end();
-  for (; p != p_end; ++p) {
-    if (p->first & PG_STATE_STALE)
-      note["stale"] += p->second;
-    if (p->first & PG_STATE_DOWN)
-      note["down"] += p->second;
-    if (p->first & PG_STATE_UNDERSIZED)
-      note["undersized"] += p->second;
-    if (p->first & PG_STATE_DEGRADED)
-      note["degraded"] += p->second;
-    if (p->first & PG_STATE_INCONSISTENT)
-      note["inconsistent"] += p->second;
-    if (p->first & PG_STATE_PEERING)
-      note["peering"] += p->second;
-    if (p->first & PG_STATE_REPAIR)
-      note["repair"] += p->second;
-    if (p->first & PG_STATE_RECOVERING)
-      note["recovering"] += p->second;
-    if (p->first & PG_STATE_RECOVERY_WAIT)
-      note["recovery_wait"] += p->second;
-    if (p->first & PG_STATE_INCOMPLETE)
-      note["incomplete"] += p->second;
-    if (p->first & PG_STATE_BACKFILL_WAIT)
-      note["backfill_wait"] += p->second;
-    if (p->first & PG_STATE_BACKFILL)
-      note["backfilling"] += p->second;
-    if (p->first & PG_STATE_BACKFILL_TOOFULL)
-      note["backfill_toofull"] += p->second;
-    if (p->first & PG_STATE_RECOVERY_TOOFULL)
-      note["recovery_toofull"] += p->second;
-    if (p->first & PG_STATE_SNAPTRIM_ERROR)
-      note["snaptrim_error"] += p->second;
-  }
-
-  mempool::pgmap::unordered_map<pg_t, pg_stat_t> stuck_pgs;
-  utime_t now(ceph_clock_now());
-  utime_t cutoff = now - utime_t(cct->_conf->mon_pg_stuck_threshold, 0);
-  uint64_t num_inactive_pgs = 0;
-
-  if (detail) {
-    // we need to collect details of stuck pgs, first do a quick check
-    // whether this will yield any results
-    if (get_stuck_counts(cutoff, note)) {
-
-      // there are stuck pgs. gather details for specified statuses
-      // only if we know that there are pgs stuck in that status
-
-      if (note.find("stuck inactive") != note.end()) {
-        get_stuck_stats(PGMap::STUCK_INACTIVE, cutoff, stuck_pgs);
-        note["stuck inactive"] = stuck_pgs.size();
-        num_inactive_pgs += stuck_pgs.size();
-        note_stuck_detail(PGMap::STUCK_INACTIVE, stuck_pgs,
-                         cct->_conf->mon_health_max_detail, detail);
-        stuck_pgs.clear();
-      }
-
-      if (note.find("stuck unclean") != note.end()) {
-        get_stuck_stats(PGMap::STUCK_UNCLEAN, cutoff, stuck_pgs);
-        note["stuck unclean"] = stuck_pgs.size();
-        note_stuck_detail(PGMap::STUCK_UNCLEAN, stuck_pgs,
-                         cct->_conf->mon_health_max_detail,  detail);
-        stuck_pgs.clear();
-      }
-
-      if (note.find("stuck undersized") != note.end()) {
-        get_stuck_stats(PGMap::STUCK_UNDERSIZED, cutoff, stuck_pgs);
-        note["stuck undersized"] = stuck_pgs.size();
-        note_stuck_detail(PGMap::STUCK_UNDERSIZED, stuck_pgs,
-                         cct->_conf->mon_health_max_detail,  detail);
-        stuck_pgs.clear();
-      }
-
-      if (note.find("stuck degraded") != note.end()) {
-        get_stuck_stats(PGMap::STUCK_DEGRADED, cutoff, stuck_pgs);
-        note["stuck degraded"] = stuck_pgs.size();
-        note_stuck_detail(PGMap::STUCK_DEGRADED, stuck_pgs,
-                         cct->_conf->mon_health_max_detail,  detail);
-        stuck_pgs.clear();
-      }
-
-      if (note.find("stuck stale") != note.end()) {
-        get_stuck_stats(PGMap::STUCK_STALE, cutoff, stuck_pgs);
-        note["stuck stale"] = stuck_pgs.size();
-        num_inactive_pgs += stuck_pgs.size();
-        note_stuck_detail(PGMap::STUCK_STALE, stuck_pgs,
-                         cct->_conf->mon_health_max_detail,  detail);
-      }
-    }
-  } else {
-    get_stuck_counts(cutoff, note);
-    auto p = note.find("stuck inactive");
-    if (p != note.end())
-      num_inactive_pgs += p->second;
-    p = note.find("stuck stale");
-    if (p != note.end())
-      num_inactive_pgs += p->second;
-  }
-
-  if (cct->_conf->mon_pg_min_inactive > 0 &&
-      num_inactive_pgs >= cct->_conf->mon_pg_min_inactive) {
-    ostringstream ss;
-    ss << num_inactive_pgs << " pgs are stuck inactive for more than " << cct->_conf->mon_pg_stuck_threshold << " seconds";
-    summary.push_back(make_pair(HEALTH_ERR, ss.str()));
-  }
-
-  if (!note.empty()) {
-    for (auto p = note.begin(); p != note.end(); ++p) {
-      ostringstream ss;
-      ss << p->second << " pgs " << p->first;
-      summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-    }
-    if (detail) {
-      int n = 0, more = 0;
-      int max = cct->_conf->mon_health_max_detail;
-      for (auto p = pg_stat.begin();
-           p != pg_stat.end();
-           ++p) {
-       if ((p->second.state & (PG_STATE_STALE |
-                               PG_STATE_DOWN |
-                               PG_STATE_UNDERSIZED |
-                               PG_STATE_DEGRADED |
-                               PG_STATE_INCONSISTENT |
-                               PG_STATE_PEERING |
-                               PG_STATE_REPAIR |
-                               PG_STATE_RECOVERING |
-                               PG_STATE_RECOVERY_WAIT |
-                               PG_STATE_RECOVERY_TOOFULL |
-                               PG_STATE_INCOMPLETE |
-                               PG_STATE_BACKFILL_WAIT |
-                               PG_STATE_BACKFILL |
-                               PG_STATE_BACKFILL_TOOFULL)) &&
-           stuck_pgs.count(p->first) == 0) {
-         if (max > 0) {
-           --max;
-         } else {
-           ++more;
-           continue;
-         }
-         ++n;
-         ostringstream ss;
-         ss << "pg " << p->first << " is " << pg_state_string(p->second.state);
-         ss << ", acting " << p->second.acting;
-         if (p->second.stats.sum.num_objects_unfound)
-           ss << ", " << p->second.stats.sum.num_objects_unfound << " unfound";
-         if (p->second.state & PG_STATE_INCOMPLETE) {
-           const pg_pool_t *pi = osdmap.get_pg_pool(p->first.pool());
-           if (pi && pi->min_size > 1) {
-             ss << " (reducing pool " << osdmap.get_pool_name(p->first.pool())
-                << " min_size from " << (int)pi->min_size
-                << " may help; search ceph.com/docs for 'incomplete')";
-           }
-         }
-         detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-       }
-      }
-      if (more) {
-       ostringstream ss;
-       ss << more << " more pgs are also unhealthy";
-       detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-      }
-    }
-  }
-
-  // slow requests
-  if (cct->_conf->mon_osd_warn_op_age > 0 &&
-      osd_sum.op_queue_age_hist.upper_bound() / 1000.0  >
-      cct->_conf->mon_osd_warn_op_age) {
-    auto sum = _warn_slow_request_histogram(
-      cct, osd_sum.op_queue_age_hist, "", summary, NULL);
-    if (sum.first > 0 || sum.second > 0) {
-      if (sum.first > 0) {
-       ostringstream ss;
-       ss << sum.first << " requests are blocked > "
-          << cct->_conf->mon_osd_warn_op_age
-          << " sec";
-       summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-      }
-      if (sum.second > 0) {
-       ostringstream ss;
-       ss << sum.second << " requests are blocked > "
-          << (cct->_conf->mon_osd_warn_op_age *
-              cct->_conf->mon_osd_err_op_age_ratio)
-          << " sec";
-       summary.push_back(make_pair(HEALTH_ERR, ss.str()));
-      }
-
-      if (detail) {
-       unsigned num_warn = 0, num_err = 0;
-       // do per-osd warnings
-       for (auto p = osd_stat.begin();
-            p != osd_stat.end();
-            ++p) {
-         auto sum = _warn_slow_request_histogram(
-               cct,
-               p->second.op_queue_age_hist,
-               string(" on osd.") + stringify(p->first),
-               summary, detail);
-         if (sum.second)
-           ++num_err;
-         else if (sum.first)
-           ++num_warn;
-       }
-       if (num_err) {
-         ostringstream ss2;
-         ss2 << num_err << " osds have very slow requests";
-         summary.push_back(make_pair(HEALTH_ERR, ss2.str()));
-         detail->push_back(make_pair(HEALTH_ERR, ss2.str()));
-       }
-       if (num_warn) {
-         ostringstream ss2;
-         ss2 << num_warn << " osds have slow requests";
-         summary.push_back(make_pair(HEALTH_WARN, ss2.str()));
-         detail->push_back(make_pair(HEALTH_WARN, ss2.str()));
-       }
-      }
-    }
-  }
-
-  // recovery
-  list<string> sl;
-  overall_recovery_summary(NULL, &sl);
-  for (auto p = sl.begin(); p != sl.end(); ++p) {
-    summary.push_back(make_pair(HEALTH_WARN, "recovery " + *p));
-    if (detail)
-      detail->push_back(make_pair(HEALTH_WARN, "recovery " + *p));
-  }
-
-  // near-target max pools
-  auto& pools = osdmap.get_pools();
-  for (auto p = pools.begin();
-       p != pools.end(); ++p) {
-    if ((!p->second.target_max_objects && !p->second.target_max_bytes) ||
-        !pg_pool_sum.count(p->first))
-      continue;
-    bool nearfull = false;
-    const string& name = osdmap.get_pool_name(p->first);
-    const pool_stat_t& st = get_pg_pool_sum_stat(p->first);
-    uint64_t ratio = p->second.cache_target_full_ratio_micro +
-                     ((1000000 - p->second.cache_target_full_ratio_micro) *
-                      cct->_conf->mon_cache_target_full_warn_ratio);
-    if (p->second.target_max_objects &&
-       (uint64_t)(st.stats.sum.num_objects -
-                  st.stats.sum.num_objects_hit_set_archive) >
-        p->second.target_max_objects * (ratio / 1000000.0)) {
-      nearfull = true;
-      if (detail) {
-       ostringstream ss;
-       ss << "cache pool '" << name << "' with "
-          << si_t(st.stats.sum.num_objects)
-          << " objects at/near target max "
-          << si_t(p->second.target_max_objects) << " objects";
-       detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-      }
-    }
-    if (p->second.target_max_bytes &&
-       (uint64_t)(st.stats.sum.num_bytes -
-                  st.stats.sum.num_bytes_hit_set_archive) >
-        p->second.target_max_bytes * (ratio / 1000000.0)) {
-      nearfull = true;
-      if (detail) {
-       ostringstream ss;
-       ss << "cache pool '" << name
-          << "' with " << si_t(st.stats.sum.num_bytes)
-          << "B at/near target max "
-          << si_t(p->second.target_max_bytes) << "B";
-       detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-      }
-    }
-    if (nearfull) {
-      ostringstream ss;
-      ss << "'" << name << "' at/near target max";
-      summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-    }
-  }
-
-  // scrub
-  if (pg_sum.stats.sum.num_scrub_errors) {
-    ostringstream ss;
-    ss << pg_sum.stats.sum.num_scrub_errors << " scrub errors";
-    summary.push_back(make_pair(HEALTH_ERR, ss.str()));
-    if (detail) {
-      detail->push_back(make_pair(HEALTH_ERR, ss.str()));
-    }
-  }
-
-  // pg skew
-  int num_in = osdmap.get_num_in_osds();
-  int sum_pg_up = MAX(pg_sum.up, static_cast<int32_t>(pg_stat.size()));
-  int sum_objects = pg_sum.stats.sum.num_objects;
-  if (sum_objects < cct->_conf->mon_pg_warn_min_objects) {
-    return;
-  }
-  if (num_in && cct->_conf->mon_pg_warn_min_per_osd > 0) {
-    int per = sum_pg_up / num_in;
-    if (per < cct->_conf->mon_pg_warn_min_per_osd && per) {
-      ostringstream ss;
-      ss << "too few PGs per OSD (" << per << " < min " << cct->_conf->mon_pg_warn_min_per_osd << ")";
-      summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-      if (detail)
-       detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-    }
-  }
-  if (num_in && cct->_conf->mon_pg_warn_max_per_osd > 0) {
-    int per = sum_pg_up / num_in;
-    if (per > cct->_conf->mon_pg_warn_max_per_osd) {
-      ostringstream ss;
-      ss << "too many PGs per OSD (" << per << " > max " << cct->_conf->mon_pg_warn_max_per_osd << ")";
-      summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-      if (detail)
-       detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-    }
-  }
-  if (!pg_stat.empty()) {
-    for (auto p = pg_pool_sum.begin();
-         p != pg_pool_sum.end();
-         ++p) {
-      const pg_pool_t *pi = osdmap.get_pg_pool(p->first);
-      if (!pi)
-       continue;   // in case osdmap changes haven't propagated to PGMap yet
-      const string& name = osdmap.get_pool_name(p->first);
-      if (pi->get_pg_num() > pi->get_pgp_num() &&
-         !(name.find(".DELETED") != string::npos &&
-           cct->_conf->mon_fake_pool_delete)) {
-       ostringstream ss;
-       ss << "pool " << name << " pg_num "
-          << pi->get_pg_num() << " > pgp_num " << pi->get_pgp_num();
-       summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-       if (detail)
-         detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-      }
-      int average_objects_per_pg = pg_sum.stats.sum.num_objects / pg_stat.size();
-      if (average_objects_per_pg > 0 &&
-          pg_sum.stats.sum.num_objects >= cct->_conf->mon_pg_warn_min_objects &&
-          p->second.stats.sum.num_objects >= cct->_conf->mon_pg_warn_min_pool_objects) {
-       int objects_per_pg = p->second.stats.sum.num_objects / pi->get_pg_num();
-       float ratio = (float)objects_per_pg / (float)average_objects_per_pg;
-       if (cct->_conf->mon_pg_warn_max_object_skew > 0 &&
-           ratio > cct->_conf->mon_pg_warn_max_object_skew) {
-         ostringstream ss;
-         ss << "pool " << name << " has many more objects per pg than average (too few pgs?)";
-         summary.push_back(make_pair(HEALTH_WARN, ss.str()));
-         if (detail) {
-           ostringstream ss;
-           ss << "pool " << name << " objects per pg ("
-              << objects_per_pg << ") is more than " << ratio << " times cluster average ("
-              << average_objects_per_pg << ")";
-           detail->push_back(make_pair(HEALTH_WARN, ss.str()));
-         }
-       }
-      }
-    }
-  }
-
-  for (auto it : pools) {
-    auto it2 = pg_pool_sum.find(it.first);
-    if (it2 == pg_pool_sum.end()) {
-      continue;
-    }
-    const pool_stat_t *pstat = &it2->second;
-    const object_stat_sum_t& sum = pstat->stats.sum;
-    const string& pool_name = osdmap.get_pool_name(it.first);
-    const pg_pool_t &pool = it.second;
-
-    float warn_threshold = (float)g_conf->mon_pool_quota_warn_threshold/100;
-    float crit_threshold = (float)g_conf->mon_pool_quota_crit_threshold/100;
-
-    if (pool.quota_max_objects > 0) {
-      stringstream ss;
-      health_status_t status = HEALTH_OK;
-      if ((uint64_t)sum.num_objects >= pool.quota_max_objects) {
-      } else if (crit_threshold > 0 &&
-                sum.num_objects >= pool.quota_max_objects*crit_threshold) {
-        ss << "pool '" << pool_name
-           << "' has " << sum.num_objects << " objects"
-           << " (max " << pool.quota_max_objects << ")";
-        status = HEALTH_ERR;
-      } else if (warn_threshold > 0 &&
-                sum.num_objects >= pool.quota_max_objects*warn_threshold) {
-        ss << "pool '" << pool_name
-           << "' has " << sum.num_objects << " objects"
-           << " (max " << pool.quota_max_objects << ")";
-        status = HEALTH_WARN;
-      }
-      if (status != HEALTH_OK) {
-        pair<health_status_t,string> s(status, ss.str());
-        summary.push_back(s);
-        if (detail)
-          detail->push_back(s);
-      }
-    }
-
-    if (pool.quota_max_bytes > 0) {
-      health_status_t status = HEALTH_OK;
-      stringstream ss;
-      if ((uint64_t)sum.num_bytes >= pool.quota_max_bytes) {
-      } else if (crit_threshold > 0 &&
-                sum.num_bytes >= pool.quota_max_bytes*crit_threshold) {
-        ss << "pool '" << pool_name
-           << "' has " << si_t(sum.num_bytes) << " bytes"
-           << " (max " << si_t(pool.quota_max_bytes) << ")";
-        status = HEALTH_ERR;
-      } else if (warn_threshold > 0 &&
-                sum.num_bytes >= pool.quota_max_bytes*warn_threshold) {
-        ss << "pool '" << pool_name
-           << "' has " << si_t(sum.num_bytes) << " bytes"
-           << " (max " << si_t(pool.quota_max_bytes) << ")";
-        status = HEALTH_WARN;
-      }
-      if (status != HEALTH_OK) {
-        pair<health_status_t,string> s(status, ss.str());
-        summary.push_back(s);
-        if (detail)
-          detail->push_back(s);
-      }
-    }
-  }
-
-  print_unscrubbed_pgs(pg_stat, summary, detail, cct);
-}
-
 int process_pg_map_command(
   const string& orig_prefix,
   const map<string,cmd_vartype>& orig_cmdmap,
index 257a9c75aaf839c1d3488553b52d682c0f4834e9..32282273834200e2f46deafddc1033eaa44a6ca6 100644 (file)
@@ -500,11 +500,6 @@ public:
     return .95;
   }
 
-  void get_health(CephContext *cct,
-                 const OSDMap& osdmap,
-                 list<pair<health_status_t,string> >& summary,
-                 list<pair<health_status_t,string> > *detail) const;
-
   void get_health_checks(
     CephContext *cct,
     const OSDMap& osdmap,
index c4091ed08a1ffdb7b7cbb22aa5d9d26eb00761d7..645095f3f6adf08e417b0c2be105c1dd935957e0 100644 (file)
@@ -1113,20 +1113,6 @@ update:
   return true;
 }
 
-void PGMonitor::get_health(list<pair<health_status_t,string> >& summary,
-                          list<pair<health_status_t,string> > *detail,
-                          CephContext *cct) const
-{
-  // legacy pre-luminous full/nearfull
-  if (mon->osdmon()->osdmap.require_osd_release < CEPH_RELEASE_LUMINOUS) {
-    check_full_osd_health(summary, detail, pg_map.full_osds, "full",
-                         HEALTH_ERR);
-    check_full_osd_health(summary, detail, pg_map.nearfull_osds, "near full",
-                         HEALTH_WARN);
-    pg_map.get_health(cct, mon->osdmon()->osdmap, summary, detail);
-  }
-}
-
 void PGMonitor::check_full_osd_health(list<pair<health_status_t,string> >& summary,
                                       list<pair<health_status_t,string> > *detail,
                                       const mempool::pgmap::set<int>& s, const char *desc,
index 5395c5bb37f89fb3d70dae00f82510fddd4d2498..5e4d9558eb1d7e3173030ac55deeb23306880fd9 100644 (file)
@@ -120,9 +120,6 @@ public:
                                   list<pair<health_status_t,string> >& summary,
                                   list<pair<health_status_t,string> > *detail) const;
 
-  void get_health(list<pair<health_status_t,string> >& summary,
-                 list<pair<health_status_t,string> > *detail,
-                 CephContext *cct) const override;
   void check_full_osd_health(
     list<pair<health_status_t,string> >& summary,
     list<pair<health_status_t,string> > *detail,
index 54e4ea3815e9fffcdf9be9183b187a38ef9ca732..fb383423991c605adffc42b51705d8500b20a191 100644 (file)
@@ -425,16 +425,6 @@ public:
    */
   virtual void tick() {}
 
-  /**
-   * Get health information
-   *
-   * @param summary list of summary strings and associated severity
-   * @param detail optional list of detailed problem reports; may be NULL
-   */
-  virtual void get_health(list<pair<health_status_t,string> >& summary,
-                         list<pair<health_status_t,string> > *detail,
-                         CephContext *cct) const { }
-
   void encode_health(const health_check_map_t& next,
                     MonitorDBStore::TransactionRef t) {
     bufferlist bl;
index 626ce659e573c238e891611adf6f09f96c1b2aae..fddc2babc408d5b43fe0c07409fdbbb61a6ee2e2 100644 (file)
@@ -117,8 +117,6 @@ public:
 
   virtual void init() { }
 
-  virtual void get_health(list<pair<health_status_t,string> >& summary,
-                          list<pair<health_status_t,string> > *detail) = 0;
   virtual int get_type() = 0;
   virtual string get_name() const = 0;