#include "MgrStatMonitor.h"
#include "mon/PGMap.h"
#include "messages/MMonMgrReport.h"
+#include "messages/MStatfs.h"
+#include "messages/MStatfsReply.h"
class MgrPGStatService : public PGStatService {
PGMapDigest& digest;
return digest.get_num_pg_by_osd(osd);
}
+ ceph_statfs get_statfs() const override {
+ return digest.get_statfs();
+ }
+
void print_summary(Formatter *f, ostream *out) const override {
digest.print_summary(f, out);
}
{
auto m = static_cast<PaxosServiceMessage*>(op->get_req());
switch (m->get_type()) {
+ case CEPH_MSG_STATFS:
+ return preprocess_statfs(op);
case MSG_MON_MGR_REPORT:
return preprocess_report(op);
default:
pending_health_detail.swap(m->health_detail);
return true;
}
+
+bool MgrStatMonitor::preprocess_statfs(MonOpRequestRef op)
+{
+ op->mark_pgmon_event(__func__);
+ auto statfs = static_cast<MStatfs*>(op->get_req());
+ auto session = statfs->get_session();
+ if (!session)
+ return true;
+ if (!session->is_capable("pg", MON_CAP_R)) {
+ dout(0) << "MStatfs received from entity with insufficient privileges "
+ << session->caps << dendl;
+ return true;
+ }
+ if (statfs->fsid != mon->monmap->fsid) {
+ dout(0) << __func__ << " on fsid " << statfs->fsid
+ << " != " << mon->monmap->fsid << dendl;
+ return true;
+ }
+ dout(10) << __func__ << " " << *statfs
+ << " from " << statfs->get_orig_source() << dendl;
+ epoch_t ver = 0;
+ if (mon->pgservice == get_pg_stat_service()) {
+ ver = get_last_committed();
+ } else {
+ ver = mon->pgmon()->get_last_committed();
+ }
+ auto reply = new MStatfsReply(statfs->fsid, statfs->get_tid(), ver);
+ reply->h.st = mon->pgservice->get_statfs();
+ mon->send_reply(op, reply);
+ return true;
+}
bool preprocess_report(MonOpRequestRef op);
bool prepare_report(MonOpRequestRef op);
+ bool preprocess_statfs(MonOpRequestRef op);
+
void check_sub(Subscription *sub);
void check_subs();
void send_digests();
#include "messages/MGenericMessage.h"
#include "messages/MMonCommand.h"
#include "messages/MMonCommandAck.h"
+#include "messages/MMonHealth.h"
#include "messages/MMonMetadata.h"
#include "messages/MMonSync.h"
#include "messages/MMonScrub.h"
#include "messages/MAuthReply.h"
#include "messages/MTimeCheck.h"
-#include "messages/MMonHealth.h"
#include "messages/MPing.h"
#include "common/strtol.h"
paxos_service[PAXOS_MGR]->dispatch(op);
break;
+ // MgrStat
case MSG_MON_MGR_REPORT:
+ case CEPH_MSG_STATFS:
paxos_service[PAXOS_MGRSTAT]->dispatch(op);
break;
// pg
- case CEPH_MSG_STATFS:
case MSG_PGSTATS:
case MSG_GETPOOLSTATS:
paxos_service[PAXOS_PGMAP]->dispatch(op);
return p->second.primary;
}
+ ceph_statfs get_statfs() const {
+ ceph_statfs statfs;
+ // these are in KB.
+ statfs.kb = osd_sum.kb;
+ statfs.kb_used = osd_sum.kb_used;
+ statfs.kb_avail = osd_sum.kb_avail;
+ statfs.num_objects = pg_sum.stats.sum.num_objects;
+ return statfs;
+ }
+
int64_t get_rule_avail(const OSDMap& osdmap, int ruleno) const;
// kill me post-luminous:
#include "messages/MGetPoolStats.h"
#include "messages/MGetPoolStatsReply.h"
-#include "messages/MStatfs.h"
-#include "messages/MStatfsReply.h"
#include "messages/MOSDPGCreate.h"
#include "messages/MMonCommand.h"
#include "messages/MOSDScrub.h"
PaxosServiceMessage *m = static_cast<PaxosServiceMessage*>(op->get_req());
dout(10) << "preprocess_query " << *m << " from " << m->get_orig_source_inst() << dendl;
switch (m->get_type()) {
- case CEPH_MSG_STATFS:
- handle_statfs(op);
- return true;
-
case MSG_GETPOOLSTATS:
return preprocess_getpoolstats(op);
}
}
-void PGMonitor::handle_statfs(MonOpRequestRef op)
-{
- op->mark_pgmon_event(__func__);
- MStatfs *statfs = static_cast<MStatfs*>(op->get_req());
- // check caps
- MonSession *session = statfs->get_session();
- if (!session)
- return;
-
- if (!session->is_capable("pg", MON_CAP_R)) {
- dout(0) << "MStatfs received from entity with insufficient privileges "
- << session->caps << dendl;
- return;
- }
-
- if (statfs->fsid != mon->monmap->fsid) {
- dout(0) << "handle_statfs on fsid " << statfs->fsid
- << " != " << mon->monmap->fsid << dendl;
- return;
- }
-
-
- dout(10) << "handle_statfs " << *statfs
- << " from " << statfs->get_orig_source() << dendl;
-
- // fill out stfs
- MStatfsReply *reply = new MStatfsReply(mon->monmap->fsid, statfs->get_tid(),
- get_last_committed());
-
- // these are in KB.
- reply->h.st.kb = pg_map.osd_sum.kb;
- reply->h.st.kb_used = pg_map.osd_sum.kb_used;
- reply->h.st.kb_avail = pg_map.osd_sum.kb_avail;
- reply->h.st.num_objects = pg_map.pg_sum.stats.sum.num_objects;
-
- // reply
- mon->send_reply(op, reply);
-}
-
bool PGMonitor::preprocess_getpoolstats(MonOpRequestRef op)
{
op->mark_pgmon_event(__func__);
size_t get_num_pg_by_osd(int osd) const override {
return pgmap.get_num_pg_by_osd(osd);
}
-
+ ceph_statfs get_statfs() const override {
+ ceph_statfs statfs;
+ statfs.kb = pgmap.osd_sum.kb;
+ statfs.kb_used = pgmap.osd_sum.kb_used;
+ statfs.kb_avail = pgmap.osd_sum.kb_avail;
+ statfs.num_objects = pgmap.pg_sum.stats.sum.num_objects;
+ return statfs;
+ }
void print_summary(Formatter *f, ostream *out) const override {
pgmap.print_summary(f, out);
}
struct C_Stats;
- void handle_statfs(MonOpRequestRef op);
bool preprocess_getpoolstats(MonOpRequestRef op);
bool preprocess_command(MonOpRequestRef op);
}
virtual size_t get_num_pg_by_osd(int osd) const = 0;
+ virtual ceph_statfs get_statfs() const = 0;
virtual void print_summary(Formatter *f, ostream *out) const = 0;
virtual void dump_info(Formatter *f) const = 0;
virtual void dump_fs_stats(stringstream *ss, Formatter *f, bool verbose) const = 0;