options.cc
page.cc
perf_counters.cc
+ perf_counters_cache.cc
perf_counters_collection.cc
perf_counters_key.cc
perf_histogram.cc
other form of policies that Amazon does, so if you are mirroring
policies between RGW and AWS, you may wish to set this to false.
default: true
+- name: rgw_perf_counters_cache
+ type: bool
+ level: dev
+ default: false
+ desc: enable rgw labeled perf counters cache
+ long desc: If set to true, rgw creates labeled perf counters and stores them
+ in an rgw specific labeled perf counters cache.
+ see_also:
+ - rgw_perf_counters_cache_size
services:
- rgw
+ with_legacy: true
+- name: rgw_perf_counters_cache_size
+ type: uint
+ level: advanced
+ desc: Number of labeled perf counters the rgw perf counters cache can store
+ default: 10000
+ services:
+ - rgw
+ with_legacy: true
- name: rgw_d4n_host
type: str
level: advanced
const std::string &counter) const
{
f->open_object_section("perfcounter_collection");
-
+ // close out all of counters collection immediately if collection is empty
+ if (m_loggers.empty()) {
+ f->close_section(); // all of counters collection
+ return;
+ }
+
if (dump_labeled) {
std::string prev_key_name;
for (auto l = m_loggers.begin(); l != m_loggers.end(); ++l) {
--- /dev/null
+#include "common/perf_counters_cache.h"
+#include "common/perf_counters_key.h"
+
+namespace ceph::perf_counters {
+
+void PerfCountersCache::check_key(const std::string &key) {
+ std::string_view key_name = ceph::perf_counters::key_name(key);
+ // return false for empty key name
+ assert(key_name != "");
+
+ // if there are no labels key name is not valid
+ auto key_labels = ceph::perf_counters::key_labels(key);
+ assert(key_labels.begin() != key_labels.end());
+
+ // don't accept keys where any labels have an empty label name
+ for (auto key_label : key_labels) {
+ assert(key_label.first != "");
+ assert(key_label.second != "");
+ }
+}
+
+std::shared_ptr<PerfCounters> PerfCountersCache::add(const std::string &key) {
+ check_key(key);
+
+ auto [ref, key_existed] = cache.get_or_create(key);
+ if (!key_existed) {
+ ref->counters = create_counters(key, cct);
+ assert(ref->counters);
+ ref->cct = cct;
+ }
+ return ref->counters;
+}
+
+
+std::shared_ptr<PerfCounters> PerfCountersCache::get(const std::string &key) {
+ std::lock_guard lock(m_lock);
+ return add(key);
+}
+
+void PerfCountersCache::inc(const std::string &key, int indx, uint64_t v) {
+ std::lock_guard lock(m_lock);
+ auto counters = add(key);
+ if (counters) {
+ counters->inc(indx, v);
+ }
+}
+
+void PerfCountersCache::dec(const std::string &key, int indx, uint64_t v) {
+ std::lock_guard lock(m_lock);
+ auto counters = add(key);
+ if (counters) {
+ counters->dec(indx, v);
+ }
+}
+
+void PerfCountersCache::tinc(const std::string &key, int indx, utime_t amt) {
+ std::lock_guard lock(m_lock);
+ auto counters = add(key);
+ if (counters) {
+ counters->tinc(indx, amt);
+ }
+}
+
+void PerfCountersCache::tinc(const std::string &key, int indx, ceph::timespan amt) {
+ std::lock_guard lock(m_lock);
+ auto counters = add(key);
+ if (counters) {
+ counters->tinc(indx, amt);
+ }
+}
+
+void PerfCountersCache::set_counter(const std::string &key, int indx, uint64_t val) {
+ std::lock_guard lock(m_lock);
+ auto counters = add(key);
+ if (counters) {
+ counters->set(indx, val);
+ }
+}
+
+uint64_t PerfCountersCache::get_counter(const std::string &key, int indx) {
+ std::lock_guard lock(m_lock);
+ auto counters = add(key);
+ uint64_t val = 0;
+ if (counters) {
+ val = counters->get(indx);
+ }
+ return val;
+}
+
+utime_t PerfCountersCache::tget(const std::string &key, int indx) {
+ std::lock_guard lock(m_lock);
+ auto counters = add(key);
+ utime_t val;
+ if (counters) {
+ val = counters->tget(indx);
+ return val;
+ } else {
+ return utime_t();
+ }
+}
+
+void PerfCountersCache::tset(const std::string &key, int indx, utime_t amt) {
+ std::lock_guard lock(m_lock);
+ auto counters = add(key);
+ if (counters) {
+ counters->tset(indx, amt);
+ }
+}
+
+PerfCountersCache::PerfCountersCache(CephContext *_cct, size_t _target_size,
+ std::function<std::shared_ptr<PerfCounters>(const std::string&, CephContext*)> _create_counters)
+ : cct(_cct), create_counters(_create_counters), m_lock(ceph::make_mutex("PerfCountersCache")) { cache.set_target_size(_target_size); }
+
+PerfCountersCache::~PerfCountersCache() { cache.set_target_size(0); }
+
+} // namespace ceph::perf_counters
--- /dev/null
+#pragma once
+
+#include "common/perf_counters.h"
+#include "common/ceph_context.h"
+#include "common/intrusive_lru.h"
+
+namespace ceph::perf_counters {
+
+struct perf_counters_cache_item_to_key;
+
+struct PerfCountersCacheEntry : public ceph::common::intrusive_lru_base<
+ ceph::common::intrusive_lru_config<
+ std::string, PerfCountersCacheEntry, perf_counters_cache_item_to_key>> {
+ std::string key;
+ std::shared_ptr<PerfCounters> counters;
+ CephContext *cct;
+
+ PerfCountersCacheEntry(const std::string &_key) : key(_key) {}
+
+ ~PerfCountersCacheEntry() {
+ if (counters) {
+ cct->get_perfcounters_collection()->remove(counters.get());
+ }
+ }
+};
+
+struct perf_counters_cache_item_to_key {
+ using type = std::string;
+ const type &operator()(const PerfCountersCacheEntry &entry) {
+ return entry.key;
+ }
+};
+
+class PerfCountersCache {
+private:
+ CephContext *cct;
+ std::function<std::shared_ptr<PerfCounters>(const std::string&, CephContext*)> create_counters;
+ PerfCountersCacheEntry::lru_t cache;
+ mutable ceph::mutex m_lock;
+
+ /* check to make sure key name is non-empty and non-empty labels
+ *
+ * A valid key has the the form
+ * key\0label1\0val1\0label2\0val2 ... label\0valN
+ * The following 3 properties checked for in this function
+ * 1. A non-empty key
+ * 2. At least 1 set of labels
+ * 3. Each label has a non-empty key and value
+ *
+ * See perf_counters_key.h
+ */
+ void check_key(const std::string &key);
+
+ // adds a new entry to the cache and returns its respective PerfCounter*
+ // or returns the PerfCounter* of an existing entry in the cache
+ std::shared_ptr<PerfCounters> add(const std::string &key);
+
+public:
+
+ // get() and its associated shared_ptr reference counting should be avoided
+ // unless the caller intends to modify multiple counter values at the same time.
+ // If multiple counter values will not be modified at the same time, inc/dec/etc.
+ // are recommended.
+ std::shared_ptr<PerfCounters> get(const std::string &key);
+
+ void inc(const std::string &key, int indx, uint64_t v);
+ void dec(const std::string &key, int indx, uint64_t v);
+ void tinc(const std::string &key, int indx, utime_t amt);
+ void tinc(const std::string &key, int indx, ceph::timespan amt);
+ void set_counter(const std::string &key, int indx, uint64_t val);
+ uint64_t get_counter(const std::string &key, int indx);
+ utime_t tget(const std::string &key, int indx);
+ void tset(const std::string &key, int indx, utime_t amt);
+
+ // _create_counters should be a function that returns a valid, newly created perf counters instance
+ // Ceph components utilizing the PerfCountersCache are encouraged to pass in a factory function that would
+ // create and initialize different kinds of counters based on the name returned from ceph::perfcounters::key_name(key)
+ PerfCountersCache(CephContext *_cct, size_t _target_size,
+ std::function<std::shared_ptr<PerfCounters>(const std::string&, CephContext*)> _create_counters);
+ ~PerfCountersCache();
+};
+
+} // namespace ceph::perf_counters
ceph_assert(! dlo_manifest);
ceph_assert(! slo_info);
- perfcounter->inc(l_rgw_put);
+ rgw::op_counters::global_op_counters->inc(l_rgw_op_put);
op_ret = -EINVAL;
if (state->object->empty()) {
real_time appx_t = real_clock::now();
state->obj_size = bytes_written;
- perfcounter->inc(l_rgw_put_b, state->obj_size);
+ rgw::op_counters::global_op_counters->inc(l_rgw_op_put_b, state->obj_size);
// flush data in filters
op_ret = filter->process({}, state->obj_size);
}
done:
- perfcounter->tinc(l_rgw_put_lat, state->time_elapsed());
+ rgw::op_counters::global_op_counters->tinc(l_rgw_op_put_lat, state->time_elapsed());
return op_ret;
} /* exec_finish */
#include "common/utf8.h"
#include "common/ceph_json.h"
#include "common/static_ptr.h"
+#include "common/perf_counters_key.h"
#include "rgw_tracer.h"
#include "rgw_rados.h"
return 0;
}
- perfcounter->inc(l_rgw_get_b, cur_end - cur_ofs);
+ auto labeled_counters = rgw::op_counters::get({{"Bucket", s->bucket_name}, {"User", s->user->get_id().id}});
+ rgw::op_counters::inc(labeled_counters, l_rgw_op_get_b, cur_end - cur_ofs);
filter->fixup_range(cur_ofs, cur_end);
op_ret = read_op->iterate(this, cur_ofs, cur_end, filter, s->yield);
if (op_ret >= 0)
found_end = true;
}
- perfcounter->tinc(l_rgw_get_lat,
- (ceph_clock_now() - start_time));
+ rgw::op_counters::global_op_counters->tinc(l_rgw_op_get_lat,
+ (ceph_clock_now() - start_time));
if (found_start && !handled_end) {
len_count += end_ofs - start_ofs;
found_end = true;
}
- perfcounter->tinc(l_rgw_get_lat,
- (ceph_clock_now() - start_time));
+ rgw::op_counters::global_op_counters->tinc(l_rgw_op_get_lat,
+ (ceph_clock_now() - start_time));
if (found_start) {
if (cb) {
std::unique_ptr<RGWGetObj_Filter> run_lua;
map<string, bufferlist>::iterator attr_iter;
- perfcounter->inc(l_rgw_get);
+ auto labeled_counters = rgw::op_counters::get({{"Bucket", s->bucket_name}, {"User", s->user->get_id().id}});
+ rgw::op_counters::inc(labeled_counters, l_rgw_op_get, 1);
std::unique_ptr<rgw::sal::Object::ReadOp> read_op(s->object->get_read_op());
return;
}
- perfcounter->inc(l_rgw_get_b, end - ofs);
+ rgw::op_counters::inc(labeled_counters, l_rgw_op_get_b, end-ofs);
op_ret = read_op->iterate(this, ofs_x, end_x, filter, s->yield);
if (op_ret >= 0)
op_ret = filter->flush();
- perfcounter->tinc(l_rgw_get_lat, s->time_elapsed());
+ rgw::op_counters::tinc(labeled_counters, l_rgw_op_get_lat, s->time_elapsed());
+
if (op_ret < 0) {
goto done_err;
}
const uint64_t max_buckets = s->cct->_conf->rgw_list_buckets_max_chunk;
+ auto labeled_counters = rgw::op_counters::get({{"User", s->user->get_id().id}});
+ rgw::op_counters::inc(labeled_counters, l_rgw_op_list_buckets, 1);
+
auto g = make_scope_guard([this, &started] {
if (!started) {
send_response_begin(false);
handle_listing_chunk(listing.buckets);
} while (!marker.empty() && !done);
+
+ rgw::op_counters::tinc(labeled_counters, l_rgw_op_list_buckets_lat, s->time_elapsed());
}
void RGWGetUsage::execute(optional_yield y)
objs = std::move(results.objs);
common_prefixes = std::move(results.common_prefixes);
}
+
+ auto labeled_counters = rgw::op_counters::get({{"Bucket", s->bucket_name}, {"User", s->user->get_id().id}});
+ rgw::op_counters::inc(labeled_counters, l_rgw_op_list_obj, 1);
+ rgw::op_counters::tinc(labeled_counters, l_rgw_op_list_obj_lat, s->time_elapsed());
}
int RGWGetBucketLogging::verify_permission(optional_yield y)
op_ret = 0;
}
+ auto labeled_counters = rgw::op_counters::get({{"Bucket", s->bucket_name}, {"User", s->user->get_id().id}});
+ rgw::op_counters::inc(labeled_counters, l_rgw_op_del_bucket, 1);
+ rgw::op_counters::tinc(labeled_counters, l_rgw_op_del_bucket_lat, s->time_elapsed());
+
return;
}
off_t fst;
off_t lst;
+ auto labeled_counters = rgw::op_counters::get({{"Bucket", s->bucket_name}, {"User", s->user->get_id().id}});
+
bool need_calc_md5 = (dlo_manifest == NULL) && (slo_info == NULL);
- perfcounter->inc(l_rgw_put);
+ rgw::op_counters::inc(labeled_counters, l_rgw_op_put, 1);
+
// report latency on return
auto put_lat = make_scope_guard([&] {
- perfcounter->tinc(l_rgw_put_lat, s->time_elapsed());
+ rgw::op_counters::tinc(labeled_counters, l_rgw_op_put_lat, s->time_elapsed());
});
op_ret = -EINVAL;
s->obj_size = ofs;
s->object->set_obj_size(ofs);
- perfcounter->inc(l_rgw_put_b, s->obj_size);
+ rgw::op_counters::inc(labeled_counters, l_rgw_op_put_b, s->obj_size);
op_ret = do_aws4_auth_completion();
if (op_ret < 0) {
op_ret = 0;
}
+ auto labeled_counters = rgw::op_counters::get({{"Bucket", s->bucket_name}, {"User", s->user->get_id().id}});
+ rgw::op_counters::inc(labeled_counters, l_rgw_op_del_obj, 1);
+ rgw::op_counters::inc(labeled_counters, l_rgw_op_del_obj_b, obj_size);
+ rgw::op_counters::tinc(labeled_counters, l_rgw_op_del_obj_lat, s->time_elapsed());
+
// send request to notification manager
int ret = res->publish_commit(this, obj_size, ceph::real_clock::now(), etag, version_id);
if (ret < 0) {
ldpp_dout(this, 1) << "ERROR: publishing notification failed, with error: " << ret << dendl;
// too late to rollback operation, hence op_ret is not set here
}
+
+ auto labeled_counters = rgw::op_counters::get({{"Bucket", s->bucket_name}, {"User", s->user->get_id().id}});
+ rgw::op_counters::inc(labeled_counters, l_rgw_op_copy_obj, 1);
+ rgw::op_counters::inc(labeled_counters, l_rgw_op_copy_obj_b, obj_size);
+ rgw::op_counters::tinc(labeled_counters, l_rgw_op_copy_obj_lat, s->time_elapsed());
}
int RGWGetACLs::verify_permission(optional_yield y)
#include "rgw_perf_counters.h"
#include "common/perf_counters.h"
+#include "common/perf_counters_key.h"
#include "common/ceph_context.h"
PerfCounters *perfcounter = NULL;
+ceph::perf_counters::PerfCountersCache *perf_counters_cache = NULL;
+std::string rgw_op_counters_key = "rgw_op";
-int rgw_perf_start(CephContext *cct)
-{
- PerfCountersBuilder plb(cct, "rgw", l_rgw_first, l_rgw_last);
-
+static void add_rgw_frontend_counters(PerfCountersBuilder *pcb) {
// RGW emits comparatively few metrics, so let's be generous
// and mark them all USEFUL to get transmission to ceph-mgr by default.
- plb.set_prio_default(PerfCountersBuilder::PRIO_USEFUL);
+ pcb->set_prio_default(PerfCountersBuilder::PRIO_USEFUL);
- plb.add_u64_counter(l_rgw_req, "req", "Requests");
- plb.add_u64_counter(l_rgw_failed_req, "failed_req", "Aborted requests");
+ pcb->add_u64_counter(l_rgw_req, "req", "Requests");
+ pcb->add_u64_counter(l_rgw_failed_req, "failed_req", "Aborted requests");
- plb.add_u64_counter(l_rgw_get, "get", "Gets");
- plb.add_u64_counter(l_rgw_get_b, "get_b", "Size of gets");
- plb.add_time_avg(l_rgw_get_lat, "get_initial_lat", "Get latency");
- plb.add_u64_counter(l_rgw_put, "put", "Puts");
- plb.add_u64_counter(l_rgw_put_b, "put_b", "Size of puts");
- plb.add_time_avg(l_rgw_put_lat, "put_initial_lat", "Put latency");
+ pcb->add_u64(l_rgw_qlen, "qlen", "Queue length");
+ pcb->add_u64(l_rgw_qactive, "qactive", "Active requests queue");
- plb.add_u64(l_rgw_qlen, "qlen", "Queue length");
- plb.add_u64(l_rgw_qactive, "qactive", "Active requests queue");
+ pcb->add_u64_counter(l_rgw_cache_hit, "cache_hit", "Cache hits");
+ pcb->add_u64_counter(l_rgw_cache_miss, "cache_miss", "Cache miss");
- plb.add_u64_counter(l_rgw_cache_hit, "cache_hit", "Cache hits");
- plb.add_u64_counter(l_rgw_cache_miss, "cache_miss", "Cache miss");
+ pcb->add_u64_counter(l_rgw_keystone_token_cache_hit, "keystone_token_cache_hit", "Keystone token cache hits");
+ pcb->add_u64_counter(l_rgw_keystone_token_cache_miss, "keystone_token_cache_miss", "Keystone token cache miss");
- plb.add_u64_counter(l_rgw_keystone_token_cache_hit, "keystone_token_cache_hit", "Keystone token cache hits");
- plb.add_u64_counter(l_rgw_keystone_token_cache_miss, "keystone_token_cache_miss", "Keystone token cache miss");
+ pcb->add_u64_counter(l_rgw_gc_retire, "gc_retire_object", "GC object retires");
- plb.add_u64_counter(l_rgw_gc_retire, "gc_retire_object", "GC object retires");
-
- plb.add_u64_counter(l_rgw_lc_expire_current, "lc_expire_current",
+ pcb->add_u64_counter(l_rgw_lc_expire_current, "lc_expire_current",
"Lifecycle current expiration");
- plb.add_u64_counter(l_rgw_lc_expire_noncurrent, "lc_expire_noncurrent",
+ pcb->add_u64_counter(l_rgw_lc_expire_noncurrent, "lc_expire_noncurrent",
"Lifecycle non-current expiration");
- plb.add_u64_counter(l_rgw_lc_expire_dm, "lc_expire_dm",
+ pcb->add_u64_counter(l_rgw_lc_expire_dm, "lc_expire_dm",
"Lifecycle delete-marker expiration");
- plb.add_u64_counter(l_rgw_lc_transition_current, "lc_transition_current",
+ pcb->add_u64_counter(l_rgw_lc_transition_current, "lc_transition_current",
"Lifecycle current transition");
- plb.add_u64_counter(l_rgw_lc_transition_noncurrent,
+ pcb->add_u64_counter(l_rgw_lc_transition_noncurrent,
"lc_transition_noncurrent",
"Lifecycle non-current transition");
- plb.add_u64_counter(l_rgw_lc_abort_mpu, "lc_abort_mpu",
+ pcb->add_u64_counter(l_rgw_lc_abort_mpu, "lc_abort_mpu",
"Lifecycle abort multipart upload");
- plb.add_u64_counter(l_rgw_pubsub_event_triggered, "pubsub_event_triggered", "Pubsub events with at least one topic");
- plb.add_u64_counter(l_rgw_pubsub_event_lost, "pubsub_event_lost", "Pubsub events lost");
- plb.add_u64_counter(l_rgw_pubsub_store_ok, "pubsub_store_ok", "Pubsub events successfully stored");
- plb.add_u64_counter(l_rgw_pubsub_store_fail, "pubsub_store_fail", "Pubsub events failed to be stored");
- plb.add_u64(l_rgw_pubsub_events, "pubsub_events", "Pubsub events in store");
- plb.add_u64_counter(l_rgw_pubsub_push_ok, "pubsub_push_ok", "Pubsub events pushed to an endpoint");
- plb.add_u64_counter(l_rgw_pubsub_push_failed, "pubsub_push_failed", "Pubsub events failed to be pushed to an endpoint");
- plb.add_u64(l_rgw_pubsub_push_pending, "pubsub_push_pending", "Pubsub events pending reply from endpoint");
- plb.add_u64_counter(l_rgw_pubsub_missing_conf, "pubsub_missing_conf", "Pubsub events could not be handled because of missing configuration");
-
- plb.add_u64_counter(l_rgw_lua_script_ok, "lua_script_ok", "Successfull executions of Lua scripts");
- plb.add_u64_counter(l_rgw_lua_script_fail, "lua_script_fail", "Failed executions of Lua scripts");
- plb.add_u64(l_rgw_lua_current_vms, "lua_current_vms", "Number of Lua VMs currently being executed");
+ pcb->add_u64_counter(l_rgw_pubsub_event_triggered, "pubsub_event_triggered", "Pubsub events with at least one topic");
+ pcb->add_u64_counter(l_rgw_pubsub_event_lost, "pubsub_event_lost", "Pubsub events lost");
+ pcb->add_u64_counter(l_rgw_pubsub_store_ok, "pubsub_store_ok", "Pubsub events successfully stored");
+ pcb->add_u64_counter(l_rgw_pubsub_store_fail, "pubsub_store_fail", "Pubsub events failed to be stored");
+ pcb->add_u64(l_rgw_pubsub_events, "pubsub_events", "Pubsub events in store");
+ pcb->add_u64_counter(l_rgw_pubsub_push_ok, "pubsub_push_ok", "Pubsub events pushed to an endpoint");
+ pcb->add_u64_counter(l_rgw_pubsub_push_failed, "pubsub_push_failed", "Pubsub events failed to be pushed to an endpoint");
+ pcb->add_u64(l_rgw_pubsub_push_pending, "pubsub_push_pending", "Pubsub events pending reply from endpoint");
+ pcb->add_u64_counter(l_rgw_pubsub_missing_conf, "pubsub_missing_conf", "Pubsub events could not be handled because of missing configuration");
- perfcounter = plb.create_perf_counters();
- cct->get_perfcounters_collection()->add(perfcounter);
+ pcb->add_u64_counter(l_rgw_lua_script_ok, "lua_script_ok", "Successfull executions of Lua scripts");
+ pcb->add_u64_counter(l_rgw_lua_script_fail, "lua_script_fail", "Failed executions of Lua scripts");
+ pcb->add_u64(l_rgw_lua_current_vms, "lua_current_vms", "Number of Lua VMs currently being executed");
+}
+
+static void add_rgw_op_counters(PerfCountersBuilder *lpcb) {
+ // description must match general rgw counters description above
+ lpcb->set_prio_default(PerfCountersBuilder::PRIO_USEFUL);
+
+ lpcb->add_u64_counter(l_rgw_op_put, "put_ops", "Puts");
+ lpcb->add_u64_counter(l_rgw_op_put_b, "put_b", "Size of puts");
+ lpcb->add_time_avg(l_rgw_op_put_lat, "put_initial_lat", "Put latency");
+
+ lpcb->add_u64_counter(l_rgw_op_get, "get_ops", "Gets");
+ lpcb->add_u64_counter(l_rgw_op_get_b, "get_b", "Size of gets");
+ lpcb->add_time_avg(l_rgw_op_get_lat, "get_initial_lat", "Get latency");
+
+ lpcb->add_u64_counter(l_rgw_op_del_obj, "del_obj_ops", "Delete objects");
+ lpcb->add_u64_counter(l_rgw_op_del_obj_b, "del_obj_bytes", "Size of delete objects");
+ lpcb->add_time_avg(l_rgw_op_del_obj_lat, "del_obj_lat", "Delete object latency");
+
+ lpcb->add_u64_counter(l_rgw_op_del_bucket, "del_bucket_ops", "Delete Buckets");
+ lpcb->add_time_avg(l_rgw_op_del_bucket_lat, "del_bucket_lat", "Delete bucket latency");
+
+ lpcb->add_u64_counter(l_rgw_op_copy_obj, "copy_obj_ops", "Copy objects");
+ lpcb->add_u64_counter(l_rgw_op_copy_obj_b, "copy_obj_bytes", "Size of copy objects");
+ lpcb->add_time_avg(l_rgw_op_copy_obj_lat, "copy_obj_lat", "Copy object latency");
+
+ lpcb->add_u64_counter(l_rgw_op_list_obj, "list_obj_ops", "List objects");
+ lpcb->add_time_avg(l_rgw_op_list_obj_lat, "list_obj_lat", "List objects latency");
+
+ lpcb->add_u64_counter(l_rgw_op_list_buckets, "list_buckets_ops", "List buckets");
+ lpcb->add_time_avg(l_rgw_op_list_buckets_lat, "list_buckets_lat", "List buckets latency");
+}
+
+std::shared_ptr<PerfCounters> create_rgw_counters(const std::string& name, CephContext *cct) {
+ std::string_view key = ceph::perf_counters::key_name(name);
+ if (rgw_op_counters_key.compare(key) == 0) {
+ PerfCountersBuilder pcb(cct, name, l_rgw_op_first, l_rgw_op_last);
+ add_rgw_op_counters(&pcb);
+ std::shared_ptr<PerfCounters> new_counters(pcb.create_perf_counters());
+ cct->get_perfcounters_collection()->add(new_counters.get());
+ return new_counters;
+ } else {
+ PerfCountersBuilder pcb(cct, name, l_rgw_first, l_rgw_last);
+ add_rgw_frontend_counters(&pcb);
+ std::shared_ptr<PerfCounters> new_counters(pcb.create_perf_counters());
+ cct->get_perfcounters_collection()->add(new_counters.get());
+ return new_counters;
+ }
+}
+
+void frontend_counters_init(CephContext *cct) {
+ PerfCountersBuilder pcb(cct, "rgw", l_rgw_first, l_rgw_last);
+ add_rgw_frontend_counters(&pcb);
+ PerfCounters *new_counters = pcb.create_perf_counters();
+ cct->get_perfcounters_collection()->add(new_counters);
+ perfcounter = new_counters;
+}
+
+namespace rgw::op_counters {
+
+PerfCounters *global_op_counters = NULL;
+
+void global_op_counters_init(CephContext *cct) {
+ PerfCountersBuilder pcb(cct, rgw_op_counters_key, l_rgw_op_first, l_rgw_op_last);
+ add_rgw_op_counters(&pcb);
+ PerfCounters *new_counters = pcb.create_perf_counters();
+ cct->get_perfcounters_collection()->add(new_counters);
+ global_op_counters = new_counters;
+}
+
+void inc(std::shared_ptr<PerfCounters> labeled_counters, int idx, uint64_t v) {
+ if (labeled_counters) {
+ PerfCounters *counter = labeled_counters.get();
+ counter->inc(idx, v);
+ }
+ if (global_op_counters) {
+ global_op_counters->inc(idx, v);
+ }
+}
+
+void tinc(std::shared_ptr<PerfCounters> labeled_counters, int idx, utime_t amt) {
+ if (labeled_counters) {
+ PerfCounters *counter = labeled_counters.get();
+ counter->tinc(idx, amt);
+ }
+ if (global_op_counters) {
+ global_op_counters->tinc(idx, amt);
+ }
+}
+
+void tinc(std::shared_ptr<PerfCounters> labeled_counters, int idx, ceph::timespan amt) {
+ if (labeled_counters) {
+ PerfCounters *counter = labeled_counters.get();
+ counter->tinc(idx, amt);
+ }
+ if (global_op_counters) {
+ global_op_counters->tinc(idx, amt);
+ }
+}
+
+} // namespace rgw::op_counters
+
+int rgw_perf_start(CephContext *cct)
+{
+ frontend_counters_init(cct);
+
+ bool cache_enabled = cct->_conf.get_val<bool>("rgw_perf_counters_cache");
+ if (cache_enabled) {
+ uint64_t target_size = cct->_conf.get_val<uint64_t>("rgw_perf_counters_cache_size");
+ perf_counters_cache = new ceph::perf_counters::PerfCountersCache(cct, target_size, create_rgw_counters);
+ }
+
+ rgw::op_counters::global_op_counters_init(cct);
return 0;
}
ceph_assert(perfcounter);
cct->get_perfcounters_collection()->remove(perfcounter);
delete perfcounter;
+ delete perf_counters_cache;
}
-
#pragma once
#include "include/common_fwd.h"
+#include "common/perf_counters_cache.h"
+#include "common/perf_counters_key.h"
extern PerfCounters *perfcounter;
+extern ceph::perf_counters::PerfCountersCache *perf_counters_cache;
+extern std::string rgw_op_counters_key;
extern int rgw_perf_start(CephContext *cct);
extern void rgw_perf_stop(CephContext *cct);
+extern void frontend_counters_init(CephContext *cct);
+extern std::shared_ptr<PerfCounters> create_rgw_counters(const std::string& name, CephContext *cct);
enum {
l_rgw_first = 15000,
l_rgw_req,
l_rgw_failed_req,
- l_rgw_get,
- l_rgw_get_b,
- l_rgw_get_lat,
-
- l_rgw_put,
- l_rgw_put_b,
- l_rgw_put_lat,
-
l_rgw_qlen,
l_rgw_qactive,
l_rgw_last,
};
+enum {
+ l_rgw_op_first = 16000,
+
+ l_rgw_op_put,
+ l_rgw_op_put_b,
+ l_rgw_op_put_lat,
+
+ l_rgw_op_get,
+ l_rgw_op_get_b,
+ l_rgw_op_get_lat,
+
+ l_rgw_op_del_obj,
+ l_rgw_op_del_obj_b,
+ l_rgw_op_del_obj_lat,
+
+ l_rgw_op_del_bucket,
+ l_rgw_op_del_bucket_lat,
+
+ l_rgw_op_copy_obj,
+ l_rgw_op_copy_obj_b,
+ l_rgw_op_copy_obj_lat,
+
+ l_rgw_op_list_obj,
+ l_rgw_op_list_obj_lat,
+
+ l_rgw_op_list_buckets,
+ l_rgw_op_list_buckets_lat,
+
+ l_rgw_op_last
+};
+
+namespace rgw::op_counters {
+
+extern PerfCounters *global_op_counters;
+
+void global_op_counters_init(CephContext *cct);
+
+template <std::size_t Count>
+std::shared_ptr<PerfCounters> get(ceph::perf_counters::label_pair (&&labels)[Count]) {
+ if (perf_counters_cache) {
+ std::string key = ceph::perf_counters::key_create(rgw_op_counters_key, std::move(labels));
+ return perf_counters_cache->get(key);
+ } else {
+ return std::shared_ptr<PerfCounters>(nullptr);
+ }
+}
+
+void inc(std::shared_ptr<PerfCounters> labeled_counters, int idx, uint64_t v);
+
+void tinc(std::shared_ptr<PerfCounters> labeled_counters, int idx, utime_t);
+
+void tinc(std::shared_ptr<PerfCounters> labeled_counters, int idx, ceph::timespan amt);
+
+} // namespace rgw::op_counters
add_ceph_unittest(unittest_perf_counters)
target_link_libraries(unittest_perf_counters global)
+# unittest_perf_counters_cache
+add_executable(unittest_perf_counters_cache
+ test_perf_counters_cache.cc
+ )
+add_ceph_unittest(unittest_perf_counters_cache)
+target_link_libraries(unittest_perf_counters_cache global)
+
# unittest_ceph_crypto
add_executable(unittest_ceph_crypto
ceph_crypto.cc)
--- /dev/null
+#include "common/perf_counters_cache.h"
+#include "common/perf_counters_key.h"
+#include "common/admin_socket_client.h"
+#include "global/global_context.h"
+#include "global/global_init.h"
+#include "include/msgr.h" // for CEPH_ENTITY_TYPE_CLIENT
+#include "gtest/gtest.h"
+
+using namespace ceph::perf_counters;
+
+int main(int argc, char **argv) {
+ std::map<std::string,std::string> defaults = {
+ { "admin_socket", get_rand_socket_path() }
+ };
+ std::vector<const char*> args;
+ auto cct = global_init(&defaults, args, CEPH_ENTITY_TYPE_CLIENT,
+ CODE_ENVIRONMENT_UTILITY,
+ CINIT_FLAG_NO_DEFAULT_CONFIG_FILE|
+ CINIT_FLAG_NO_CCT_PERF_COUNTERS);
+ common_init_finish(g_ceph_context);
+ ::testing::FLAGS_gtest_death_test_style = "threadsafe";
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
+
+enum {
+ TEST_PERFCOUNTERS1_ELEMENT_FIRST = 200,
+ TEST_PERFCOUNTERS_COUNTER,
+ TEST_PERFCOUNTERS_TIME,
+ TEST_PERFCOUNTERS_TIME_AVG,
+ TEST_PERFCOUNTERS1_ELEMENT_LAST,
+};
+
+std::string sd(const char *c)
+{
+ std::string ret(c);
+ std::string::size_type sz = ret.size();
+ for (std::string::size_type i = 0; i < sz; ++i) {
+ if (ret[i] == '\'') {
+ ret[i] = '\"';
+ }
+ }
+ return ret;
+}
+
+void add_test_counters(PerfCountersBuilder *pcb) {
+ pcb->add_u64(TEST_PERFCOUNTERS_COUNTER, "test_counter");
+ pcb->add_time(TEST_PERFCOUNTERS_TIME, "test_time");
+ pcb->add_time_avg(TEST_PERFCOUNTERS_TIME_AVG, "test_time_avg");
+}
+
+static std::shared_ptr<PerfCounters> create_test_counters(const std::string& name, CephContext *cct) {
+ PerfCountersBuilder pcb(cct, name, TEST_PERFCOUNTERS1_ELEMENT_FIRST, TEST_PERFCOUNTERS1_ELEMENT_LAST);
+ add_test_counters(&pcb);
+ std::shared_ptr<PerfCounters> new_counters(pcb.create_perf_counters());
+ cct->get_perfcounters_collection()->add(new_counters.get());
+ return new_counters;
+}
+
+static PerfCountersCache* setup_test_perf_counters_cache(CephContext *cct, uint64_t target_size = 100)
+{
+ return new PerfCountersCache(cct, target_size, create_test_counters);
+}
+
+
+void cleanup_test(PerfCountersCache *pcc) {
+ delete pcc;
+}
+
+TEST(PerfCountersCache, NoCacheTest) {
+ AdminSocketClient client(get_rand_socket_path());
+ std::string message;
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter dump" })", &message));
+ ASSERT_EQ("{}\n", message);
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter schema" })", &message));
+ ASSERT_EQ("{}\n", message);
+}
+
+TEST(PerfCountersCache, TestEviction) {
+ PerfCountersCache *pcc = setup_test_perf_counters_cache(g_ceph_context, 4);
+ std::string label1 = key_create("key1", {{"label1", "val1"}});
+ std::string label2 = key_create("key2", {{"label2", "val2"}});
+ std::string label3 = key_create("key3", {{"label3", "val3"}});
+ std::string label4 = key_create("key4", {{"label4", "val4"}});
+ std::string label5 = key_create("key5", {{"label5", "val5"}});
+ std::string label6 = key_create("key6", {{"label6", "val6"}});
+
+ pcc->set_counter(label1, TEST_PERFCOUNTERS_COUNTER, 0);
+ std::shared_ptr<PerfCounters> counter = pcc->get(label2);
+ counter->set(TEST_PERFCOUNTERS_COUNTER, 0);
+ pcc->set_counter(label3, TEST_PERFCOUNTERS_COUNTER, 0);
+ pcc->set_counter(label4, TEST_PERFCOUNTERS_COUNTER, 0);
+
+ AdminSocketClient client(get_rand_socket_path());
+ std::string message;
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter dump", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "key1": [
+ {
+ "labels": {
+ "label1": "val1"
+ },
+ "counters": {
+ "test_counter": 0,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ],
+ "key2": [
+ {
+ "labels": {
+ "label2": "val2"
+ },
+ "counters": {
+ "test_counter": 0,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ],
+ "key3": [
+ {
+ "labels": {
+ "label3": "val3"
+ },
+ "counters": {
+ "test_counter": 0,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ],
+ "key4": [
+ {
+ "labels": {
+ "label4": "val4"
+ },
+ "counters": {
+ "test_counter": 0,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ]
+}
+)", message);
+
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter schema", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "key1": [
+ {
+ "labels": {
+ "label1": "val1"
+ },
+ "counters": {
+ "test_counter": {
+ "type": 2,
+ "metric_type": "gauge",
+ "value_type": "integer",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time": {
+ "type": 1,
+ "metric_type": "gauge",
+ "value_type": "real",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time_avg": {
+ "type": 5,
+ "metric_type": "gauge",
+ "value_type": "real-integer-pair",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ }
+ }
+ }
+ ],
+ "key2": [
+ {
+ "labels": {
+ "label2": "val2"
+ },
+ "counters": {
+ "test_counter": {
+ "type": 2,
+ "metric_type": "gauge",
+ "value_type": "integer",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time": {
+ "type": 1,
+ "metric_type": "gauge",
+ "value_type": "real",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time_avg": {
+ "type": 5,
+ "metric_type": "gauge",
+ "value_type": "real-integer-pair",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ }
+ }
+ }
+ ],
+ "key3": [
+ {
+ "labels": {
+ "label3": "val3"
+ },
+ "counters": {
+ "test_counter": {
+ "type": 2,
+ "metric_type": "gauge",
+ "value_type": "integer",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time": {
+ "type": 1,
+ "metric_type": "gauge",
+ "value_type": "real",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time_avg": {
+ "type": 5,
+ "metric_type": "gauge",
+ "value_type": "real-integer-pair",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ }
+ }
+ }
+ ],
+ "key4": [
+ {
+ "labels": {
+ "label4": "val4"
+ },
+ "counters": {
+ "test_counter": {
+ "type": 2,
+ "metric_type": "gauge",
+ "value_type": "integer",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time": {
+ "type": 1,
+ "metric_type": "gauge",
+ "value_type": "real",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time_avg": {
+ "type": 5,
+ "metric_type": "gauge",
+ "value_type": "real-integer-pair",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ }
+ }
+ }
+ ]
+}
+)", message);
+
+ pcc->set_counter(label5, TEST_PERFCOUNTERS_COUNTER, 0);
+ pcc->set_counter(label6, TEST_PERFCOUNTERS_COUNTER, 0);
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter dump", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "key3": [
+ {
+ "labels": {
+ "label3": "val3"
+ },
+ "counters": {
+ "test_counter": 0,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ],
+ "key4": [
+ {
+ "labels": {
+ "label4": "val4"
+ },
+ "counters": {
+ "test_counter": 0,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ],
+ "key5": [
+ {
+ "labels": {
+ "label5": "val5"
+ },
+ "counters": {
+ "test_counter": 0,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ],
+ "key6": [
+ {
+ "labels": {
+ "label6": "val6"
+ },
+ "counters": {
+ "test_counter": 0,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ]
+}
+)", message);
+
+
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter schema", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "key3": [
+ {
+ "labels": {
+ "label3": "val3"
+ },
+ "counters": {
+ "test_counter": {
+ "type": 2,
+ "metric_type": "gauge",
+ "value_type": "integer",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time": {
+ "type": 1,
+ "metric_type": "gauge",
+ "value_type": "real",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time_avg": {
+ "type": 5,
+ "metric_type": "gauge",
+ "value_type": "real-integer-pair",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ }
+ }
+ }
+ ],
+ "key4": [
+ {
+ "labels": {
+ "label4": "val4"
+ },
+ "counters": {
+ "test_counter": {
+ "type": 2,
+ "metric_type": "gauge",
+ "value_type": "integer",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time": {
+ "type": 1,
+ "metric_type": "gauge",
+ "value_type": "real",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time_avg": {
+ "type": 5,
+ "metric_type": "gauge",
+ "value_type": "real-integer-pair",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ }
+ }
+ }
+ ],
+ "key5": [
+ {
+ "labels": {
+ "label5": "val5"
+ },
+ "counters": {
+ "test_counter": {
+ "type": 2,
+ "metric_type": "gauge",
+ "value_type": "integer",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time": {
+ "type": 1,
+ "metric_type": "gauge",
+ "value_type": "real",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time_avg": {
+ "type": 5,
+ "metric_type": "gauge",
+ "value_type": "real-integer-pair",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ }
+ }
+ }
+ ],
+ "key6": [
+ {
+ "labels": {
+ "label6": "val6"
+ },
+ "counters": {
+ "test_counter": {
+ "type": 2,
+ "metric_type": "gauge",
+ "value_type": "integer",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time": {
+ "type": 1,
+ "metric_type": "gauge",
+ "value_type": "real",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time_avg": {
+ "type": 5,
+ "metric_type": "gauge",
+ "value_type": "real-integer-pair",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ }
+ }
+ }
+ ]
+}
+)", message);
+ cleanup_test(pcc);
+}
+
+TEST(PerfCountersCache, TestLabeledCounters) {
+ PerfCountersCache *pcc = setup_test_perf_counters_cache(g_ceph_context);
+ std::string label1 = key_create("key1", {{"label1", "val1"}});
+ std::string label2 = key_create("key2", {{"label2", "val2"}});
+ std::string label3 = key_create("key3", {{"label3", "val3"}});
+
+ // test inc()
+ pcc->inc(label1, TEST_PERFCOUNTERS_COUNTER, 1);
+ pcc->inc(label2, TEST_PERFCOUNTERS_COUNTER, 2);
+
+ AdminSocketClient client(get_rand_socket_path());
+ std::string message;
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter dump", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "key1": [
+ {
+ "labels": {
+ "label1": "val1"
+ },
+ "counters": {
+ "test_counter": 1,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ],
+ "key2": [
+ {
+ "labels": {
+ "label2": "val2"
+ },
+ "counters": {
+ "test_counter": 2,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ]
+}
+)", message);
+
+
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter schema", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "key1": [
+ {
+ "labels": {
+ "label1": "val1"
+ },
+ "counters": {
+ "test_counter": {
+ "type": 2,
+ "metric_type": "gauge",
+ "value_type": "integer",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time": {
+ "type": 1,
+ "metric_type": "gauge",
+ "value_type": "real",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time_avg": {
+ "type": 5,
+ "metric_type": "gauge",
+ "value_type": "real-integer-pair",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ }
+ }
+ }
+ ],
+ "key2": [
+ {
+ "labels": {
+ "label2": "val2"
+ },
+ "counters": {
+ "test_counter": {
+ "type": 2,
+ "metric_type": "gauge",
+ "value_type": "integer",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time": {
+ "type": 1,
+ "metric_type": "gauge",
+ "value_type": "real",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time_avg": {
+ "type": 5,
+ "metric_type": "gauge",
+ "value_type": "real-integer-pair",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ }
+ }
+ }
+ ]
+}
+)", message);
+
+ // tests to ensure there is no interaction with normal perf counters
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "perf dump", "format": "raw" })", &message));
+ ASSERT_EQ("{}\n", message);
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "perf schema", "format": "raw" })", &message));
+ ASSERT_EQ("{}\n", message);
+
+ // test dec()
+ pcc->dec(label2, TEST_PERFCOUNTERS_COUNTER, 1);
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter dump", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "key1": [
+ {
+ "labels": {
+ "label1": "val1"
+ },
+ "counters": {
+ "test_counter": 1,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ],
+ "key2": [
+ {
+ "labels": {
+ "label2": "val2"
+ },
+ "counters": {
+ "test_counter": 1,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ]
+}
+)", message);
+
+
+ // test set_counters()
+ pcc->set_counter(label3, TEST_PERFCOUNTERS_COUNTER, 4);
+ uint64_t val = pcc->get_counter(label3, TEST_PERFCOUNTERS_COUNTER);
+ ASSERT_EQ(val, 4);
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter dump", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "key1": [
+ {
+ "labels": {
+ "label1": "val1"
+ },
+ "counters": {
+ "test_counter": 1,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ],
+ "key2": [
+ {
+ "labels": {
+ "label2": "val2"
+ },
+ "counters": {
+ "test_counter": 1,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ],
+ "key3": [
+ {
+ "labels": {
+ "label3": "val3"
+ },
+ "counters": {
+ "test_counter": 4,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ]
+}
+)", message);
+
+ cleanup_test(pcc);
+}
+
+TEST(PerfCountersCache, TestLabeledTimes) {
+ PerfCountersCache *pcc = setup_test_perf_counters_cache(g_ceph_context);
+ std::string label1 = key_create("key1", {{"label1", "val1"}});
+ std::string label2 = key_create("key2", {{"label2", "val2"}});
+ std::string label3 = key_create("key3", {{"label3", "val3"}});
+
+ // test inc()
+ pcc->tinc(label1, TEST_PERFCOUNTERS_TIME, utime_t(100,0));
+ pcc->tinc(label2, TEST_PERFCOUNTERS_TIME, utime_t(200,0));
+
+ //tinc() that takes a ceph_timespan
+ ceph::timespan ceph_timespan = std::chrono::seconds(10);
+ pcc->tinc(label1, TEST_PERFCOUNTERS_TIME, ceph_timespan);
+
+ pcc->tinc(label1, TEST_PERFCOUNTERS_TIME_AVG, utime_t(200,0));
+ pcc->tinc(label1, TEST_PERFCOUNTERS_TIME_AVG, utime_t(400,0));
+ pcc->tinc(label2, TEST_PERFCOUNTERS_TIME_AVG, utime_t(100,0));
+ pcc->tinc(label2, TEST_PERFCOUNTERS_TIME_AVG, utime_t(200,0));
+
+ AdminSocketClient client(get_rand_socket_path());
+ std::string message;
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter dump", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "key1": [
+ {
+ "labels": {
+ "label1": "val1"
+ },
+ "counters": {
+ "test_counter": 0,
+ "test_time": 110.000000000,
+ "test_time_avg": {
+ "avgcount": 2,
+ "sum": 600.000000000,
+ "avgtime": 300.000000000
+ }
+ }
+ }
+ ],
+ "key2": [
+ {
+ "labels": {
+ "label2": "val2"
+ },
+ "counters": {
+ "test_counter": 0,
+ "test_time": 200.000000000,
+ "test_time_avg": {
+ "avgcount": 2,
+ "sum": 300.000000000,
+ "avgtime": 150.000000000
+ }
+ }
+ }
+ ]
+}
+)", message);
+
+
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter schema", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "key1": [
+ {
+ "labels": {
+ "label1": "val1"
+ },
+ "counters": {
+ "test_counter": {
+ "type": 2,
+ "metric_type": "gauge",
+ "value_type": "integer",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time": {
+ "type": 1,
+ "metric_type": "gauge",
+ "value_type": "real",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time_avg": {
+ "type": 5,
+ "metric_type": "gauge",
+ "value_type": "real-integer-pair",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ }
+ }
+ }
+ ],
+ "key2": [
+ {
+ "labels": {
+ "label2": "val2"
+ },
+ "counters": {
+ "test_counter": {
+ "type": 2,
+ "metric_type": "gauge",
+ "value_type": "integer",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time": {
+ "type": 1,
+ "metric_type": "gauge",
+ "value_type": "real",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ },
+ "test_time_avg": {
+ "type": 5,
+ "metric_type": "gauge",
+ "value_type": "real-integer-pair",
+ "description": "",
+ "nick": "",
+ "priority": 0,
+ "units": "none"
+ }
+ }
+ }
+ ]
+}
+)", message);
+
+ // test tset() & tget()
+ pcc->tset(label1, TEST_PERFCOUNTERS_TIME, utime_t(500,0));
+ utime_t label1_time = pcc->tget(label1, TEST_PERFCOUNTERS_TIME);
+ ASSERT_EQ(utime_t(500,0), label1_time);
+
+ cleanup_test(pcc);
+}
+
+TEST(PerfCountersCache, TestLabelStrings) {
+ AdminSocketClient client(get_rand_socket_path());
+ std::string message;
+ PerfCountersCache *pcc = setup_test_perf_counters_cache(g_ceph_context);
+ std::string empty_key = "";
+
+ // empty string as should not create a labeled entry
+ EXPECT_DEATH(pcc->set_counter(empty_key, TEST_PERFCOUNTERS_COUNTER, 1), "");
+ EXPECT_DEATH(pcc->get(empty_key), "");
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter dump", "format": "raw" })", &message));
+ ASSERT_EQ("{}\n", message);
+
+ // key name but no labels at all should not create a labeled entry
+ std::string only_key = "only_key";
+ // run an op on an invalid key name to make sure nothing happens
+ EXPECT_DEATH(pcc->set_counter(only_key, TEST_PERFCOUNTERS_COUNTER, 4), "");
+ EXPECT_DEATH(pcc->get(only_key), "");
+
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter dump", "format": "raw" })", &message));
+ ASSERT_EQ("{}\n", message);
+
+ // test valid key name with multiple valid label pairs
+ std::string label1 = key_create("good_ctrs", {{"label3", "val3"}, {"label2", "val4"}});
+ pcc->set_counter(label1, TEST_PERFCOUNTERS_COUNTER, 8);
+
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter dump", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "good_ctrs": [
+ {
+ "labels": {
+ "label2": "val4",
+ "label3": "val3"
+ },
+ "counters": {
+ "test_counter": 8,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ]
+}
+)", message);
+
+ // test empty val in a label pair will get the label pair added into the perf counters cache but empty key will not
+ std::string label2 = key_create("bad_ctrs1", {{"label3", "val4"}, {"label1", ""}});
+ EXPECT_DEATH(pcc->set_counter(label2, TEST_PERFCOUNTERS_COUNTER, 2), "");
+
+ std::string label3 = key_create("bad_ctrs2", {{"", "val4"}, {"label1", "val1"}});
+ EXPECT_DEATH(pcc->set_counter(label3, TEST_PERFCOUNTERS_COUNTER, 2), "");
+
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter dump", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "good_ctrs": [
+ {
+ "labels": {
+ "label2": "val4",
+ "label3": "val3"
+ },
+ "counters": {
+ "test_counter": 8,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ]
+}
+)", message);
+
+ // test empty keys in each of the label pairs will not get the label added into the perf counters cache
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter dump", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "good_ctrs": [
+ {
+ "labels": {
+ "label2": "val4",
+ "label3": "val3"
+ },
+ "counters": {
+ "test_counter": 8,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ]
+}
+)", message);
+
+ // a key with a somehow odd number of entries after the the key name will omit final unfinished label pair
+ std::string label5 = "too_many_delimiters";
+ label5 += '\0';
+ label5 += "label1";
+ label5 += '\0';
+ label5 += "val1";
+ label5 += '\0';
+ label5 += "label2";
+ label5 += '\0';
+ pcc->set_counter(label5, TEST_PERFCOUNTERS_COUNTER, 0);
+
+ ASSERT_EQ("", client.do_request(R"({ "prefix": "counter dump", "format": "raw" })", &message));
+ ASSERT_EQ(R"({
+ "good_ctrs": [
+ {
+ "labels": {
+ "label2": "val4",
+ "label3": "val3"
+ },
+ "counters": {
+ "test_counter": 8,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ],
+ "too_many_delimiters": [
+ {
+ "labels": {
+ "label1": "val1"
+ },
+ "counters": {
+ "test_counter": 0,
+ "test_time": 0.000000000,
+ "test_time_avg": {
+ "avgcount": 0,
+ "sum": 0.000000000,
+ "avgtime": 0.000000000
+ }
+ }
+ }
+ ]
+}
+)", message);
+
+ cleanup_test(pcc);
+}