#define CEPH_MMGRCONFIGURE_H_
#include "msg/Message.h"
-#include "mgr/OSDPerfMetricQuery.h"
+#include "mgr/OSDPerfMetricTypes.h"
/**
* This message is sent from ceph-mgr to MgrClient, instructing it
#include <boost/optional.hpp>
#include "msg/Message.h"
-#include "mgr/OSDPerfMetricReport.h"
+#include "mgr/OSDPerfMetricTypes.h"
#include "common/perf_counters.h"
#include "mgr/DaemonHealthMetric.h"
// encode map<string,map<int32_t,string>> of current config
bufferlist config_bl;
- std::list<OSDPerfMetricReport> osd_perf_metric_reports;
+ std::map<OSDPerfMetricQuery, OSDPerfMetricReport> osd_perf_metric_reports;
void decode_payload() override
{
#include "DaemonState.h"
#include "ClusterState.h"
-#include "OSDPerfMetricQuery.h"
+#include "OSDPerfMetricTypes.h"
class health_check_map_t;
class DaemonServer;
-struct OSDPerfMetricQuery;
class ActivePyModules
{
Gil.cc
Mgr.cc
MgrStandby.cc
- OSDPerfMetricQuery.cc
+ OSDPerfMetricTypes.cc
OSDPerfMetricCollector.cc
PyFormatter.cc
PyModule.cc
py_modules.notify_all("perf_schema_update", oss.str());
}
+ if (m->get_connection()->peer_is_osd()) {
+ osd_perf_metric_collector.process_reports(m->osd_perf_metric_reports);
+ }
+
m->put();
return true;
}
&last_config_bl_version);
if (get_perf_report_cb) {
- //get_perf_report_cb(&report->perf_report)
+ get_perf_report_cb(&report->osd_perf_metric_reports);
}
session->con->send_message(report);
stats_threshold = m->stats_threshold;
}
+ if (set_perf_queries_cb) {
+ set_perf_queries_cb(m->osd_perf_metric_queries);
+ }
+
bool starting = (stats_period == 0) && (m->stats_period != 0);
stats_period = m->stats_period;
if (starting) {
_send_stats();
}
- if (set_perf_queries_cb) {
- set_perf_queries_cb(m->osd_perf_metric_queries);
- }
-
m->put();
return true;
}
#include "mgr/DaemonHealthMetric.h"
#include "messages/MMgrReport.h"
-#include "mgr/OSDPerfMetricQuery.h"
+#include "mgr/OSDPerfMetricTypes.h"
#include "common/perf_counters.h"
#include "common/Timer.h"
#include "common/CommandTable.h"
-typedef int OSDPerfMetricReport; //Temporary
-
class MMgrMap;
class MMgrConfigure;
class MMgrClose;
// our reports (hook for use by OSD)
std::function<MPGStats*()> pgstats_cb;
std::function<void(const std::list<OSDPerfMetricQuery> &)> set_perf_queries_cb;
- std::function<void(OSDPerfMetricReport *)> get_perf_report_cb;
+ std::function<void(std::map<OSDPerfMetricQuery,
+ OSDPerfMetricReport> *)> get_perf_report_cb;
// for service registration and beacon
bool service_daemon = false;
void set_perf_metric_query_cb(
std::function<void(const std::list<OSDPerfMetricQuery> &)> cb_set,
- std::function<void(OSDPerfMetricReport *)> cb_get)
-
+ std::function<void(std::map<OSDPerfMetricQuery,
+ OSDPerfMetricReport> *)> cb_get)
{
std::lock_guard l(lock);
set_perf_queries_cb = cb_set;
get_perf_report_cb = cb_get;
}
-
void send_pgstats();
void set_pgstats_cb(std::function<MPGStats*()>&& cb_)
{
listener.handle_query_updated();
}
}
+
+void OSDPerfMetricCollector::process_reports(
+ const std::map<OSDPerfMetricQuery, OSDPerfMetricReport> &reports) {
+
+ if (reports.empty()) {
+ return;
+ }
+
+ std::lock_guard locker(lock);
+
+ for (auto &it : reports) {
+ auto &report = it.second;
+ dout(10) << "report for " << it.first << " query: "
+ << report.group_packed_performance_counters.size() << " records"
+ << dendl;
+ for (auto &it : report.group_packed_performance_counters) {
+ auto &key = it.first;
+ auto bl_it = it.second.cbegin();
+ for (auto &d : report.performance_counter_descriptors) {
+ PerformanceCounter c;
+ d.unpack_counter(bl_it, &c);
+ dout(20) << "counter " << key << " " << d << ": " << c << dendl;
+ }
+ }
+ }
+}
#include "common/Mutex.h"
-#include "mgr/OSDPerfMetricQuery.h"
+#include "mgr/OSDPerfMetricTypes.h"
#include <list>
#include <set>
int remove_query(OSDPerfMetricQueryID query_id);
void remove_all_queries();
+ void process_reports(
+ const std::map<OSDPerfMetricQuery, OSDPerfMetricReport> &reports);
+
private:
typedef std::map<OSDPerfMetricQuery, std::set<OSDPerfMetricQueryID>> Queries;
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#include "OSDPerfMetricQuery.h"
-
-std::ostream& operator<<(std::ostream& os, const OSDPerfMetricQuery &query) {
- return os << "simple";
-}
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#ifndef OSD_PERF_METRIC_QUERY_H_
-#define OSD_PERF_METRIC_QUERY_H_
-
-#include "include/denc.h"
-
-typedef int OSDPerfMetricQueryID;
-
-struct OSDPerfMetricQuery
-{
- bool operator<(const OSDPerfMetricQuery &other) const {
- return false;
- }
-
- DENC(OSDPerfMetricQuery, v, p) {
- DENC_START(1, 1, p);
- DENC_FINISH(p);
- }
-};
-WRITE_CLASS_DENC(OSDPerfMetricQuery)
-
-std::ostream& operator<<(std::ostream& os, const OSDPerfMetricQuery &query);
-
-#endif // OSD_PERF_METRIC_QUERY_H_
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-#ifndef OSD_PERF_METRIC_REPORT_H_
-#define OSD_PERF_METRIC_REPORT_H_
-#include "include/denc.h"
-
-#include "common/perf_counters.h"
-
-struct PerformanceCounterDescriptor {
- std::string name;
- perfcounter_type_d type;
-};
-
-
-struct OSDPerfMetricReport {
- std::vector<PerformanceCounterDescriptor> performance_counter_descriptors;
- std::map<std::string, bufferlist> group_packed_performance_counters;
-
- DENC(OSDPerfMetricReport, v, p) {
- DENC_START(1, 1, p);
-// denc(v.performance_counter_descriptors, p);
- DENC_FINISH(p);
- }
-};
-WRITE_CLASS_DENC(OSDPerfMetricReport)
-
-#endif // OSD_PERF_METRIC_REPORT_H_
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "mgr/OSDPerfMetricTypes.h"
+#include "messages/MOSDOp.h"
+#include "osd/OpRequest.h"
+
+void PerformanceCounterDescriptor::update_counter(
+ const OpRequest& op, uint64_t inb, uint64_t outb, const utime_t &now,
+ PerformanceCounter *c) const {
+ switch(type) {
+ case PerformanceCounterType::WRITE_OPS:
+ if (op.may_write() || op.may_cache()) {
+ c->first++;
+ }
+ return;
+ case PerformanceCounterType::READ_OPS:
+ if (op.may_read()) {
+ c->first++;
+ }
+ return;
+ case PerformanceCounterType::WRITE_BYTES:
+ if (op.may_write() || op.may_cache()) {
+ c->first += inb;
+ c->second++;
+ }
+ return;
+ case PerformanceCounterType::READ_BYTES:
+ if (op.may_read()) {
+ c->first += outb;
+ c->second++;
+ }
+ return;
+ case PerformanceCounterType::WRITE_LATENCY:
+ if (op.may_write() || op.may_cache()) {
+ const MOSDOp* const m = static_cast<const MOSDOp*>(op.get_req());
+ c->first += (now - m->get_recv_stamp()).to_nsec();
+ c->second++;
+ }
+ return;
+ case PerformanceCounterType::READ_LATENCY:
+ if (op.may_read()) {
+ const MOSDOp* const m = static_cast<const MOSDOp*>(op.get_req());
+ c->first += (now - m->get_recv_stamp()).to_nsec();
+ c->second++;
+ }
+ return;
+ default:
+ ceph_abort_msg("unknown counter type");
+ }
+}
+
+void PerformanceCounterDescriptor::pack_counter(const PerformanceCounter &c,
+ bufferlist *bl) const {
+ using ceph::encode;
+ encode(c.first, *bl);
+ switch(type) {
+ case PerformanceCounterType::WRITE_OPS:
+ case PerformanceCounterType::READ_OPS:
+ break;
+ case PerformanceCounterType::WRITE_BYTES:
+ case PerformanceCounterType::READ_BYTES:
+ case PerformanceCounterType::WRITE_LATENCY:
+ case PerformanceCounterType::READ_LATENCY:
+ encode(c.second, *bl);
+ break;
+ default:
+ ceph_abort_msg("unknown counter type");
+ }
+}
+
+void PerformanceCounterDescriptor::unpack_counter(
+ bufferlist::const_iterator& bl, PerformanceCounter *c) const {
+ using ceph::decode;
+ decode(c->first, bl);
+ switch(type) {
+ case PerformanceCounterType::WRITE_OPS:
+ case PerformanceCounterType::READ_OPS:
+ break;
+ case PerformanceCounterType::WRITE_BYTES:
+ case PerformanceCounterType::READ_BYTES:
+ case PerformanceCounterType::WRITE_LATENCY:
+ case PerformanceCounterType::READ_LATENCY:
+ decode(c->second, bl);
+ break;
+ default:
+ ceph_abort_msg("unknown counter type");
+ }
+}
+
+std::ostream& operator<<(std::ostream& os,
+ const PerformanceCounterDescriptor &d) {
+ switch(d.type) {
+ case PerformanceCounterType::WRITE_OPS:
+ return os << "write ops";
+ case PerformanceCounterType::READ_OPS:
+ return os << "read ops";
+ case PerformanceCounterType::WRITE_BYTES:
+ return os << "write bytes";
+ case PerformanceCounterType::READ_BYTES:
+ return os << "read bytes";
+ case PerformanceCounterType::WRITE_LATENCY:
+ return os << "write latency";
+ case PerformanceCounterType::READ_LATENCY:
+ return os << "read latency";
+ default:
+ return os << "unknown (" << d.type << ")";
+ }
+}
+
+bool OSDPerfMetricQuery::get_key(const OpRequest& op, std::string *key) const {
+ const MOSDOp* const m = static_cast<const MOSDOp*>(op.get_req());
+
+ *key = stringify(m->get_reqid().name);
+ return true;
+}
+
+void OSDPerfMetricQuery::update_counters(const OpRequest& op, uint64_t inb,
+ uint64_t outb, const utime_t &now,
+ PerformanceCounters *counters) const {
+ auto it = counters->begin();
+ for (auto &descriptor : performance_counter_descriptors) {
+ // TODO: optimize
+ if (it == counters->end()) {
+ counters->push_back(PerformanceCounter());
+ it = std::prev(counters->end());
+ }
+ descriptor.update_counter(op, inb, outb, now, &(*it));
+ it++;
+ }
+}
+
+void OSDPerfMetricQuery::pack_counters(const PerformanceCounters &counters,
+ bufferlist *bl) const {
+ auto it = counters.begin();
+ for (auto &descriptor : performance_counter_descriptors) {
+ if (it == counters.end()) {
+ descriptor.pack_counter(PerformanceCounter(), bl);
+ } else {
+ descriptor.pack_counter(*it, bl);
+ it++;
+ }
+ }
+}
+
+std::ostream& operator<<(std::ostream& os, const OSDPerfMetricQuery &query) {
+ return os << "simple";
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef OSD_PERF_METRIC_H_
+#define OSD_PERF_METRIC_H_
+
+#include "include/denc.h"
+#include "include/stringify.h"
+
+#include <ostream>
+
+class OpRequest;
+class utime_t;
+
+typedef std::pair<uint64_t,uint64_t> PerformanceCounter;
+typedef std::vector<PerformanceCounter> PerformanceCounters;
+
+enum class PerformanceCounterType : uint8_t {
+ WRITE_OPS = 0,
+ READ_OPS = 1,
+ WRITE_BYTES = 2,
+ READ_BYTES = 3,
+ WRITE_LATENCY = 4,
+ READ_LATENCY = 5,
+};
+
+struct PerformanceCounterDescriptor {
+ PerformanceCounterType type = static_cast<PerformanceCounterType>(-1);
+
+ PerformanceCounterDescriptor() {
+ }
+
+ PerformanceCounterDescriptor(PerformanceCounterType type) : type(type) {
+ }
+
+ DENC(PerformanceCounterDescriptor, v, p) {
+ DENC_START(1, 1, p);
+ denc(v.type, p);
+ DENC_FINISH(p);
+ }
+
+ void update_counter(const OpRequest& op, uint64_t inb, uint64_t outb,
+ const utime_t &now, PerformanceCounter *c) const;
+ void pack_counter(const PerformanceCounter &c, bufferlist *bl) const;
+ void unpack_counter(bufferlist::const_iterator& bl,
+ PerformanceCounter *c) const;
+};
+WRITE_CLASS_DENC(PerformanceCounterDescriptor)
+
+std::ostream& operator<<(std::ostream& os,
+ const PerformanceCounterDescriptor &d);
+
+typedef int OSDPerfMetricQueryID;
+
+struct OSDPerfMetricQuery {
+ bool operator<(const OSDPerfMetricQuery &other) const {
+ return false;
+ }
+
+ bool get_key(const OpRequest& op, std::string *key) const;
+
+ DENC(OSDPerfMetricQuery, v, p) {
+ DENC_START(1, 1, p);
+ denc(v.performance_counter_descriptors, p);
+ DENC_FINISH(p);
+ }
+
+ void get_performance_counter_descriptors(
+ std::vector<PerformanceCounterDescriptor> *descriptors) const {
+ *descriptors = performance_counter_descriptors;
+ }
+
+ void update_counters(const OpRequest& op, uint64_t inb, uint64_t outb,
+ const utime_t &now, PerformanceCounters *counters) const;
+ void pack_counters(const PerformanceCounters &counters, bufferlist *bl) const;
+
+ std::vector<PerformanceCounterDescriptor> performance_counter_descriptors = {
+ {PerformanceCounterType::WRITE_OPS},
+ {PerformanceCounterType::READ_OPS},
+ {PerformanceCounterType::WRITE_BYTES},
+ {PerformanceCounterType::READ_BYTES},
+ {PerformanceCounterType::WRITE_LATENCY},
+ {PerformanceCounterType::READ_LATENCY},
+ };
+};
+WRITE_CLASS_DENC(OSDPerfMetricQuery)
+
+std::ostream& operator<<(std::ostream& os, const OSDPerfMetricQuery &query);
+
+struct OSDPerfMetricReport {
+ std::vector<PerformanceCounterDescriptor> performance_counter_descriptors;
+ std::map<std::string, bufferlist> group_packed_performance_counters;
+
+ DENC(OSDPerfMetricReport, v, p) {
+ DENC_START(1, 1, p);
+ denc(v.performance_counter_descriptors, p);
+ denc(v.group_packed_performance_counters, p);
+ DENC_FINISH(p);
+ }
+};
+WRITE_CLASS_DENC(OSDPerfMetricReport)
+
+#endif // OSD_PERF_METRIC_H_
+
OpQueueItem.cc
${CMAKE_SOURCE_DIR}/src/common/TrackedOp.cc
${CMAKE_SOURCE_DIR}/src/objclass/class_api.cc
+ ${CMAKE_SOURCE_DIR}/src/mgr/OSDPerfMetricTypes.cc
${osd_cyg_functions_src}
${osdc_osd_srcs})
if(HAS_VTA)
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef DYNAMIC_PERF_STATS_H
+#define DYNAMIC_PERF_STATS_H
+
+#include "mgr/OSDPerfMetricTypes.h"
+
+class DynamicPerfStats {
+public:
+ DynamicPerfStats() {
+ }
+
+ DynamicPerfStats(const std::list<OSDPerfMetricQuery> &queries) {
+ for (auto &query : queries) {
+ data[query];
+ }
+ }
+
+ void set_queries(const std::list<OSDPerfMetricQuery> &queries) {
+ std::map<OSDPerfMetricQuery,
+ std::map<std::string, PerformanceCounters>> new_data;
+ for (auto &query : queries) {
+ std::swap(new_data[query], data[query]);
+ }
+ std::swap(data, new_data);
+ }
+
+ bool is_enabled() {
+ return !data.empty();
+ }
+
+ void add(const OpRequest& op, uint64_t inb, uint64_t outb,
+ const utime_t &now) {
+ for (auto &it : data) {
+ auto &query = it.first;
+ std::string key;
+ if (query.get_key(op, &key)) {
+ query.update_counters(op, inb, outb, now, &it.second[key]);
+ }
+ }
+ }
+
+ void add_to_reports(
+ std::map<OSDPerfMetricQuery, OSDPerfMetricReport> *reports) {
+ for (auto &it : data) {
+ auto &query = it.first;
+ auto &report = (*reports)[query];
+
+ query.get_performance_counter_descriptors(
+ &report.performance_counter_descriptors);
+
+ for (auto &it_counters : it.second) {
+ auto &bl = report.group_packed_performance_counters[it_counters.first];
+ query.pack_counters(it_counters.second, &bl);
+ }
+ }
+ }
+
+private:
+ std::map<OSDPerfMetricQuery, std::map<std::string, PerformanceCounters>> data;
+};
+
+#endif // DYNAMIC_PERF_STATS_H
mgrc.set_pgstats_cb([this](){ return collect_pg_stats(); });
mgrc.set_perf_metric_query_cb(
- [this](const std::list<OSDPerfMetricQuery> &queries){ set_perf_queries(queries);},
- [this](OSDPerfMetricReport *report){ get_perf_report(report);
- });
+ [this](const std::list<OSDPerfMetricQuery> &queries) {
+ set_perf_queries(queries);
+ },
+ [this](std::map<OSDPerfMetricQuery, OSDPerfMetricReport> *reports) {
+ get_perf_reports(reports);
+ });
mgrc.init();
client_messenger->add_dispatcher_head(&mgrc);
false,
rctx.transaction);
+ if (pg->is_primary()) {
+ Mutex::Locker locker(m_perf_queries_lock);
+ pg->set_dynamic_perf_stats_queries(m_perf_queries);
+ }
+
pg->handle_initialize(&rctx);
pg->handle_activate_map(&rctx);
}
void OSD::set_perf_queries(const std::list<OSDPerfMetricQuery> &queries) {
+ dout(10) << "setting " << queries.size() << " queries" << dendl;
+
+ {
+ Mutex::Locker locker(m_perf_queries_lock);
+ m_perf_queries = queries;
+ }
+
+ std::vector<PGRef> pgs;
+ _get_pgs(&pgs);
+ for (auto& pg : pgs) {
+ if (pg->is_primary()) {
+ pg->lock();
+ pg->set_dynamic_perf_stats_queries(queries);
+ pg->unlock();
+ }
+ }
}
-void OSD::get_perf_report(OSDPerfMetricReport *report) {
+void OSD::get_perf_reports(
+ std::map<OSDPerfMetricQuery, OSDPerfMetricReport> *reports) {
+ std::vector<PGRef> pgs;
+ _get_pgs(&pgs);
+ DynamicPerfStats dps(m_perf_queries);
+ for (auto& pg : pgs) {
+ if (pg->is_primary()) {
+ pg->lock();
+ pg->get_dynamic_perf_stats(&dps);
+ pg->unlock();
+ }
+ }
+ dps.add_to_reports(reports);
+ dout(20) << "reports for " << reports->size() << " queries" << dendl;
}
// =============================================================
private:
void set_perf_queries(const std::list<OSDPerfMetricQuery> &queries);
- void get_perf_report(OSDPerfMetricReport *report);
+ void get_perf_reports(
+ std::map<OSDPerfMetricQuery, OSDPerfMetricReport> *reports);
+
+ Mutex m_perf_queries_lock = {"OSD::m_perf_queries_lock"};
+ std::list<OSDPerfMetricQuery> m_perf_queries;
};
#include "PGBackend.h"
#include "PGPeeringEvent.h"
+#include "mgr/OSDPerfMetricTypes.h"
+
#include <atomic>
#include <list>
#include <memory>
typedef OpRequest::Ref OpRequestRef;
class MOSDPGLog;
class CephContext;
+class DynamicPerfStats;
namespace Scrub {
class Store;
void _delete_some(ObjectStore::Transaction *t);
+ virtual void set_dynamic_perf_stats_queries(
+ const std::list<OSDPerfMetricQuery> &queries) {
+ }
+ virtual void get_dynamic_perf_stats(DynamicPerfStats *stats) {
+ }
+
// reference counting
#ifdef PG_DEBUG_REFS
uint64_t get_with_id();
<< " inb " << inb
<< " outb " << outb
<< " lat " << latency << dendl;
+
+ if (m_dynamic_perf_stats.is_enabled()) {
+ m_dynamic_perf_stats.add(op, inb, outb, latency);
+ }
+}
+
+void PrimaryLogPG::set_dynamic_perf_stats_queries(
+ const std::list<OSDPerfMetricQuery> &queries)
+{
+ m_dynamic_perf_stats.set_queries(queries);
+}
+
+void PrimaryLogPG::get_dynamic_perf_stats(DynamicPerfStats *stats)
+{
+ std::swap(m_dynamic_perf_stats, *stats);
}
void PrimaryLogPG::do_scan(
#include <boost/tuple/tuple.hpp>
#include "include/ceph_assert.h"
+#include "DynamicPerfStats.h"
#include "OSD.h"
#include "PG.h"
#include "Watch.h"
int getattrs_maybe_cache(
ObjectContextRef obc,
map<string, bufferlist> *out);
+
+public:
+ void set_dynamic_perf_stats_queries(
+ const std::list<OSDPerfMetricQuery> &queries) override;
+ void get_dynamic_perf_stats(DynamicPerfStats *stats) override;
+
+private:
+ DynamicPerfStats m_dynamic_perf_stats;
};
inline ostream& operator<<(ostream& out, const PrimaryLogPG::RepGather& repop)