#include "metric.hpp" #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include "arena_allocator.hpp" #include "format.hpp" // WeaselDB Metrics System Design: // // THREADING MODEL: // - Counters: Per-thread storage, single writer, atomic write/read coordination // with render thread // - Histograms: Per-thread storage, single writer, mutex protection for all // access (both observe and render) // - Gauges: Global storage with atomic CAS operations (multi-writer, no mutex // needed) // // SYNCHRONIZATION STRATEGY: // - Counters: Atomic store in Counter::inc(), atomic load in render thread // - Histograms: Mutex serializes all access - updates in observe(), reads in // render // - Gauges: Lock-free atomic operations for all updates and reads // // PRECISION STRATEGY: // - Use atomic for lock-free storage // - Store doubles using std::bit_cast to uint64_t (preserves full IEEE 754 // precision) // - Single writer for counters enables simple atomic store/load // // MEMORY MODEL: // - Thread-local metrics auto-cleanup on thread destruction // - Global metrics (gauges) persist for application lifetime // - Histogram buckets are sorted, deduplicated, sizes never change after // creation namespace metric { // ARENA OWNERSHIP AND MEMORY MANAGEMENT DOCUMENTATION // // The metrics system uses multiple arena allocators with distinct ownership // patterns: // // 1. GLOBAL ARENA (get_global_arena()): // - Lifetime: Application lifetime (never destroyed) // - Purpose: Persistent storage for metric families, interned labels, and // global state // - Owner: Static storage - automatically managed // - Content: Family names, help text, LabelsKey instances, global // accumulated values // // 2. THREAD-LOCAL ARENA (get_thread_local_arena()): // - Lifetime: Per-thread lifetime (destroyed on thread exit) // - Purpose: Storage for per-thread metric instances (Counter::State, // Histogram::State) // - Owner: thread_local ThreadInit instance // - Content: Thread-specific metric instance state // // 3. TEMPORARY ARENAS: // a) Caller-Provided Arenas (ArenaAllocator& parameters): // - Lifetime: Controlled by caller (function parameter) // - Purpose: Output formatting where caller controls result lifetime // - Owner: Caller owns arena and controls string lifetime // - Example: render(ArenaAllocator& arena) - caller manages arena // lifecycle // // b) Stack-Owned Temporary Arenas: // - Lifetime: Function/scope lifetime (automatic destruction) // - Purpose: Internal temporary allocations for lookups and processing // - Owner: Function owns arena on stack, destroyed at scope exit // - Example: intern_labels() creates ArenaAllocator lookup_arena(1024) // // CRITICAL OWNERSHIP RULES: // // - LabelsKey Arena Dependency: LabelsKey instances store string_views pointing // to arena-allocated memory. The arena MUST outlive any LabelsKey that // references its memory. LabelsKey constructor copies input strings into the // provided arena. // // - Render Function: render(arena) allocates ALL output strings in the provided // arena. Callers own the arena and control string lifetime. String_views // become invalid after arena.reset() or arena destruction. // // - Thread Cleanup: ThreadInit destructor accumulates thread-local // counter/histogram // values into global storage before thread exit. This copies VALUES (not // ownership) to prevent metric data loss when threads are destroyed. // // - Family Creation: Uses placement new in global arena without explicit // destructors. // This is acceptable because Family::State instances persist for application // lifetime and the global arena is never destroyed. // Validation helper that works in both debug and release builds static void validate_or_abort(bool condition, const char *message, std::string_view value) { if (!condition) { std::fprintf(stderr, "WeaselDB metric validation failed: %s: '%.*s'\n", message, static_cast(value.size()), value.data()); std::abort(); } } // Helper to copy a string into arena memory static std::string_view arena_copy_string(std::string_view str, ArenaAllocator &arena) { if (str.empty()) { return std::string_view{}; } char *copied = arena.allocate(str.size()); std::memcpy(copied, str.data(), str.size()); return std::string_view(copied, str.size()); } // Arena-based labels key for second level of map // Uses string_view containing labels in Prometheus text format struct LabelsKey { std::string_view prometheus_format; // Arena-owning constructor (copies strings into arena and formats as // Prometheus text) LabelsKey(std::span> l, ArenaAllocator &arena) { // Copy and validate all label keys and values, sort by key ArenaVector> labels(&arena); for (const auto &[key, value] : l) { validate_or_abort(is_valid_label_key(key), "invalid label key", key); validate_or_abort(is_valid_label_value(value), "invalid label value", value); auto key_view = arena_copy_string(key, arena); auto value_view = arena_copy_string(value, arena); labels.push_back({key_view, value_view}); } // Sort labels by key for Prometheus compatibility std::sort(labels.data(), labels.data() + labels.size(), [](const auto &a, const auto &b) { return a.first < b.first; }); // Generate Prometheus text format: {key1="value1",key2="value2"} if (labels.empty()) { prometheus_format = ""; } else { // Calculate required size for formatted string size_t required_size = 2; // {} for (const auto &[key, value] : labels) { required_size += key.length() + 3 + value.length(); // key="value" for (char c : value) { if (c == '\\' || c == '"' || c == '\n') { required_size++; } } } required_size += labels.size() - 1; // commas // Generate formatted string in arena char *buf = arena.allocate(required_size); char *p = buf; *p++ = '{'; for (size_t i = 0; i < labels.size(); ++i) { if (i > 0) *p++ = ','; std::memcpy(p, labels[i].first.data(), labels[i].first.length()); p += labels[i].first.length(); *p++ = '='; *p++ = '"'; for (char c : labels[i].second) { switch (c) { case '\\': *p++ = '\\'; *p++ = '\\'; break; case '"': *p++ = '\\'; *p++ = '"'; break; case '\n': *p++ = '\\'; *p++ = 'n'; break; default: *p++ = c; break; } } *p++ = '"'; } *p++ = '}'; prometheus_format = std::string_view(buf, p - buf); } } bool operator==(const LabelsKey &other) const { return prometheus_format == other.prometheus_format; } bool operator<(const LabelsKey &other) const { return prometheus_format < other.prometheus_format; } }; } // namespace metric namespace std { template <> struct hash { std::size_t operator()(const metric::LabelsKey &k) const { return std::hash{}(k.prometheus_format); } }; } // namespace std namespace metric { // DESIGN: Store doubles in atomic for lock-free operations // - Preserves full IEEE 754 double precision (no truncation) // - Allows atomic load/store without locks // - Use std::bit_cast for safe conversion between double and uint64_t // Family::State structures own the second-level maps (labels -> instances) template <> struct Family::State { std::string_view name; std::string_view help; struct PerThreadState { std::unordered_map instances; }; std::unordered_map per_thread_state; // Global accumulation state for destroyed threads std::unordered_map< LabelsKey, Counter::State *, std::hash, std::equal_to, ArenaStlAllocator>> global_accumulated_values; // Callback-based metrics (global, not per-thread) std::map< LabelsKey, MetricCallback, std::less, ArenaStlAllocator>>> callbacks; State(ArenaAllocator &arena) : global_accumulated_values( ArenaStlAllocator>( &arena)), callbacks( ArenaStlAllocator< std::pair>>(&arena)) {} }; template <> struct Family::State { std::string_view name; std::string_view help; std::unordered_map< LabelsKey, Gauge::State *, std::hash, std::equal_to, ArenaStlAllocator>> instances; // Callback-based metrics std::map, std::less, ArenaStlAllocator>>> callbacks; State(ArenaAllocator &arena) : instances(ArenaStlAllocator>( &arena)), callbacks(ArenaStlAllocator< std::pair>>(&arena)) {} }; template <> struct Family::State { std::string_view name; std::string_view help; ArenaVector buckets; struct PerThreadState { std::unordered_map instances; }; std::unordered_map per_thread_state; // Global accumulation state for destroyed threads std::unordered_map< LabelsKey, Histogram::State *, std::hash, std::equal_to, ArenaStlAllocator>> global_accumulated_values; State(ArenaAllocator &arena) : buckets(&arena), global_accumulated_values( ArenaStlAllocator>( &arena)) {} // Note: No callbacks map - histograms don't support callback-based metrics }; // Counter: Thread-local, monotonically increasing, single writer struct Counter::State { double value; // Single writer, atomic coordination with render thread friend struct Metric; }; // Gauge: Global, can increase/decrease, multiple writers (uses atomic CAS) struct Gauge::State { std::atomic value; // Stores double as uint64_t bits, lock-free CAS operations friend struct Metric; }; // Histogram: Thread-local buckets, single writer, mutex protection per thread, // per histogram struct Histogram::State { std::span thresholds; // Bucket boundaries (sorted, // deduplicated, sizes never change) std::span counts; // Count per bucket double sum; // Sum of observations uint64_t observations; // Total observation count std::mutex mutex; // Per-thread, per-histogram mutex for consistent reads/writes State() : sum(0.0), observations(0) {} friend struct Metric; }; struct Metric { // We use a raw pointer to these in a map, so we don't call their destructors static_assert(std::is_trivially_destructible_v); static_assert(std::is_trivially_destructible_v); static_assert(std::is_trivially_destructible_v); static std::mutex mutex; // Global arena allocator for metric families and persistent global state static ArenaAllocator &get_global_arena() { static auto *global_arena = new ArenaAllocator(64 * 1024); // 64KB initial size return *global_arena; } // Function-local statics to avoid static initialization order fiasco static auto &get_counter_families() { using FamilyMap = std::map::State>, std::less, ArenaStlAllocator< std::pair::State>>>>; static FamilyMap *counterFamilies = new FamilyMap(ArenaStlAllocator< std::pair::State>>>( &get_global_arena())); return *counterFamilies; } static auto &get_gauge_families() { using FamilyMap = std::map::State>, std::less, ArenaStlAllocator< std::pair::State>>>>; static FamilyMap *gaugeFamilies = new FamilyMap( ArenaStlAllocator::State>>>( &get_global_arena())); return *gaugeFamilies; } static auto &get_histogram_families() { using FamilyMap = std::map::State>, std::less, ArenaStlAllocator< std::pair::State>>>>; static FamilyMap *histogramFamilies = new FamilyMap(ArenaStlAllocator< std::pair::State>>>( &get_global_arena())); return *histogramFamilies; } // Global label interning set to avoid duplicate LabelsKey allocations static auto &get_interned_labels() { using InternSet = std::unordered_set, std::equal_to, ArenaStlAllocator>; static InternSet *internedLabels = new InternSet(ArenaStlAllocator(&get_global_arena())); return *internedLabels; } // Thread cleanup for per-family thread-local storage struct ThreadInit { ArenaAllocator arena; ThreadInit() {} ~ThreadInit() { // Accumulate thread-local state into global state before cleanup std::unique_lock _{mutex}; auto thread_id = std::this_thread::get_id(); // Accumulate counter families for (auto &[name, family] : Metric::get_counter_families()) { auto thread_it = family->per_thread_state.find(thread_id); if (thread_it != family->per_thread_state.end()) { for (auto &[labels_key, instance] : thread_it->second.instances) { // Get current thread-local value double current_value = instance->value; // Ensure global accumulator exists auto &global_state = family->global_accumulated_values[labels_key]; if (!global_state) { global_state = get_global_arena().construct(); global_state->value = 0.0; } // Add thread-local value to global accumulator (mutex already held) global_state->value += current_value; } family->per_thread_state.erase(thread_it); } } // Accumulate histogram families for (auto &[name, family] : Metric::get_histogram_families()) { auto thread_it = family->per_thread_state.find(thread_id); if (thread_it != family->per_thread_state.end()) { for (auto &[labels_key, instance] : thread_it->second.instances) { // Acquire lock to get consistent snapshot std::lock_guard lock(instance->mutex); // Global accumulator should have been created when we made the // histogram auto &global_state = family->global_accumulated_values[labels_key]; assert(global_state); // Accumulate bucket counts (mutex already held) for (size_t i = 0; i < instance->counts.size(); ++i) { global_state->counts[i] += instance->counts[i]; } // Accumulate sum and observations global_state->sum += instance->sum; global_state->observations += instance->observations; } family->per_thread_state.erase(thread_it); } } // Gauges are global, no per-thread cleanup needed } }; static thread_local ThreadInit thread_init; // Thread-local arena allocator for metric instances static ArenaAllocator &get_thread_local_arena() { return thread_init.arena; } // Thread cleanup now handled by ThreadInit RAII // Intern labels to avoid duplicate arena allocations static const LabelsKey &intern_labels( std::span> labels) { auto &interned_set = get_interned_labels(); // Create temporary lookup key using stack-allocated arena ArenaAllocator lookup_arena(1024); // Small arena for lookups LabelsKey lookup_key{labels, lookup_arena}; // Use standard hash set lookup auto it = interned_set.find(lookup_key); if (it != interned_set.end()) { return *it; } // Not found - create and intern new key LabelsKey new_key{labels, get_global_arena()}; auto result = interned_set.emplace(std::move(new_key)); return *result.first; } static Counter create_counter_instance( Family *family, std::span> labels) { // Force thread_local initialization (void)thread_init; std::unique_lock _{mutex}; const LabelsKey &key = intern_labels(labels); // Validate that labels aren't already registered as callback validate_or_abort( family->p->callbacks.find(key) == family->p->callbacks.end(), "labels already registered as callback", key.prometheus_format.empty() ? "(no labels)" : key.prometheus_format); // Ensure thread state exists auto thread_id = std::this_thread::get_id(); // Thread state is automatically created by operator[] auto &ptr = family->p->per_thread_state[thread_id].instances[key]; if (!ptr) { ptr = get_thread_local_arena().construct(); ptr->value = 0.0; // Ensure global accumulator exists for this label set auto &global_state = family->p->global_accumulated_values[key]; if (!global_state) { global_state = get_global_arena().construct(); global_state->value = 0.0; } } Counter result; result.p = ptr; return result; } static Gauge create_gauge_instance( Family *family, std::span> labels) { std::unique_lock _{mutex}; const LabelsKey &key = intern_labels(labels); // Validate that labels aren't already registered as callback validate_or_abort( family->p->callbacks.find(key) == family->p->callbacks.end(), "labels already registered as callback", key.prometheus_format.empty() ? "(no labels)" : key.prometheus_format); auto &ptr = family->p->instances[key]; if (!ptr) { ptr = get_global_arena().construct(); ptr->value.store(0, std::memory_order_relaxed); } Gauge result; result.p = ptr; return result; } static Histogram create_histogram_instance( Family *family, std::span> labels) { // Force thread_local initialization (void)thread_init; std::unique_lock _{mutex}; const LabelsKey &key = intern_labels(labels); // Ensure thread state exists auto thread_id = std::this_thread::get_id(); // Thread state is automatically created by operator[] auto &ptr = family->p->per_thread_state[thread_id].instances[key]; if (!ptr) { ptr = get_thread_local_arena().construct(); // DESIGN: Prometheus-compatible histogram buckets // Use buckets from family configuration size_t bucket_count = family->p->buckets.size(); double *thresholds_data = get_thread_local_arena().allocate(bucket_count); uint64_t *counts_data = get_thread_local_arena().allocate(bucket_count); // Copy thresholds and initialize counts std::memcpy(thresholds_data, family->p->buckets.data(), bucket_count * sizeof(double)); std::memset(counts_data, 0, bucket_count * sizeof(uint64_t)); ptr->thresholds = std::span(thresholds_data, bucket_count); ptr->counts = std::span(counts_data, bucket_count); // Ensure global accumulator exists for this label set auto &global_state = family->p->global_accumulated_values[key]; if (!global_state) { global_state = get_global_arena().construct(); // Allocate and copy thresholds, initialize counts double *global_thresholds_data = get_global_arena().allocate(bucket_count); uint64_t *global_counts_data = get_global_arena().allocate(bucket_count); std::memcpy(global_thresholds_data, ptr->thresholds.data(), bucket_count * sizeof(double)); std::memset(global_counts_data, 0, bucket_count * sizeof(uint64_t)); global_state->thresholds = std::span(global_thresholds_data, bucket_count); global_state->counts = std::span(global_counts_data, bucket_count); } } Histogram result; result.p = ptr; return result; } // Pre-computed data structures with resolved pointers to eliminate hash // lookups struct CounterLabelData { LabelsKey labels_key; std::vector thread_states; // Pre-resolved pointers Counter::State *global_state; // Pre-resolved global state pointer CounterLabelData(const LabelsKey &key) : labels_key(key), global_state(nullptr) {} }; struct GaugeLabelData { LabelsKey labels_key; Gauge::State *instance_state; // Direct pointer to gauge instance GaugeLabelData(const LabelsKey &key) : labels_key(key), instance_state(nullptr) {} }; struct HistogramLabelData { LabelsKey labels_key; std::vector thread_states; // Pre-resolved pointers Histogram::State *global_state; // Pre-resolved global state pointer size_t bucket_count; // Cache bucket count from family HistogramLabelData(const LabelsKey &key) : labels_key(key), global_state(nullptr), bucket_count(0) {} }; // Pre-computed data for each family type, built once and reused struct LabelSets { std::vector> counter_data; std::vector> gauge_data; std::vector> histogram_data; }; // Instruction types for the execute phase struct CallCounterCallback { const MetricCallback *callback_ptr; // Safe: callback lifetime guaranteed by family map }; struct CallGaugeCallback { const MetricCallback *callback_ptr; // Safe: callback lifetime guaranteed by family map }; struct AggregateCounter { std::vector thread_states; Counter::State *global_state; }; struct AggregateGauge { Gauge::State *instance_state; }; struct AggregateHistogram { std::vector thread_states; Histogram::State *global_state; size_t bucket_count; std::span buckets; // For bucket threshold formatting }; // Use a simpler enum-based approach to avoid variant issues enum class InstructionType { CALL_COUNTER_CALLBACK, CALL_GAUGE_CALLBACK, AGGREGATE_COUNTER, AGGREGATE_GAUGE, AGGREGATE_HISTOGRAM }; struct RenderInstruction { InstructionType type; union { CallCounterCallback counter_callback; CallGaugeCallback gauge_callback; AggregateCounter aggregate_counter; AggregateGauge aggregate_gauge; AggregateHistogram aggregate_histogram; }; // Constructors RenderInstruction(CallCounterCallback cb) : type(InstructionType::CALL_COUNTER_CALLBACK) { new (&counter_callback) CallCounterCallback(cb); } RenderInstruction(CallGaugeCallback cb) : type(InstructionType::CALL_GAUGE_CALLBACK) { new (&gauge_callback) CallGaugeCallback(cb); } RenderInstruction(AggregateCounter ac) : type(InstructionType::AGGREGATE_COUNTER) { new (&aggregate_counter) AggregateCounter(ac); } RenderInstruction(AggregateGauge ag) : type(InstructionType::AGGREGATE_GAUGE) { new (&aggregate_gauge) AggregateGauge(ag); } RenderInstruction(AggregateHistogram ah) : type(InstructionType::AGGREGATE_HISTOGRAM) { new (&aggregate_histogram) AggregateHistogram(ah); } // Destructor ~RenderInstruction() { switch (type) { case InstructionType::CALL_COUNTER_CALLBACK: counter_callback.~CallCounterCallback(); break; case InstructionType::CALL_GAUGE_CALLBACK: gauge_callback.~CallGaugeCallback(); break; case InstructionType::AGGREGATE_COUNTER: aggregate_counter.~AggregateCounter(); break; case InstructionType::AGGREGATE_GAUGE: aggregate_gauge.~AggregateGauge(); break; case InstructionType::AGGREGATE_HISTOGRAM: aggregate_histogram.~AggregateHistogram(); break; } } // Copy constructor and assignment RenderInstruction(const RenderInstruction &other) : type(other.type) { switch (type) { case InstructionType::CALL_COUNTER_CALLBACK: new (&counter_callback) CallCounterCallback(other.counter_callback); break; case InstructionType::CALL_GAUGE_CALLBACK: new (&gauge_callback) CallGaugeCallback(other.gauge_callback); break; case InstructionType::AGGREGATE_COUNTER: new (&aggregate_counter) AggregateCounter(other.aggregate_counter); break; case InstructionType::AGGREGATE_GAUGE: new (&aggregate_gauge) AggregateGauge(other.aggregate_gauge); break; case InstructionType::AGGREGATE_HISTOGRAM: new (&aggregate_histogram) AggregateHistogram(other.aggregate_histogram); break; } } RenderInstruction &operator=(const RenderInstruction &other) { if (this != &other) { // Destroy current object this->~RenderInstruction(); // Reconstruct with new type and data type = other.type; switch (type) { case InstructionType::CALL_COUNTER_CALLBACK: new (&counter_callback) CallCounterCallback(other.counter_callback); break; case InstructionType::CALL_GAUGE_CALLBACK: new (&gauge_callback) CallGaugeCallback(other.gauge_callback); break; case InstructionType::AGGREGATE_COUNTER: new (&aggregate_counter) AggregateCounter(other.aggregate_counter); break; case InstructionType::AGGREGATE_GAUGE: new (&aggregate_gauge) AggregateGauge(other.aggregate_gauge); break; case InstructionType::AGGREGATE_HISTOGRAM: new (&aggregate_histogram) AggregateHistogram(other.aggregate_histogram); break; } } return *this; } }; // Three-phase rendering system struct RenderPlan { ArenaVector static_text; ArenaVector instructions; RenderPlan(ArenaAllocator *arena) : static_text(arena), instructions(arena) {} }; // Phase 1: Compile phase - generate static text and instructions static RenderPlan compile_render_plan(ArenaAllocator &arena, const LabelSets &label_sets) { RenderPlan plan(&arena); // Helper function to append an additional label to existing Prometheus // format auto append_label_to_format = [&](std::string_view base_format, std::string_view key, std::string_view value) -> std::string_view { // Calculate size for key="value" with escaping size_t key_value_size = key.length() + 3 + value.length(); // key="value" for (char c : value) { if (c == '\\' || c == '"' || c == '\n') { key_value_size++; } } if (base_format.empty()) { // Create new format: {key="value"} size_t required_size = 2 + key_value_size; // {} char *buf = arena.allocate(required_size); char *p = buf; *p++ = '{'; std::memcpy(p, key.data(), key.length()); p += key.length(); *p++ = '='; *p++ = '"'; for (char c : value) { switch (c) { case '\\': *p++ = '\\'; *p++ = '\\'; break; case '"': *p++ = '\\'; *p++ = '"'; break; case '\n': *p++ = '\\'; *p++ = 'n'; break; default: *p++ = c; break; } } *p++ = '"'; *p++ = '}'; return std::string_view(buf, p - buf); } else { // Append to existing format: {existing,key="value"} size_t required_size = base_format.length() + 1 + key_value_size; // comma + key="value", replace } char *buf = arena.allocate(required_size); char *p = buf; // Copy everything except the closing } std::memcpy(p, base_format.data(), base_format.length() - 1); p += base_format.length() - 1; *p++ = ','; std::memcpy(p, key.data(), key.length()); p += key.length(); *p++ = '='; *p++ = '"'; for (char c : value) { switch (c) { case '\\': *p++ = '\\'; *p++ = '\\'; break; case '"': *p++ = '\\'; *p++ = '"'; break; case '\n': *p++ = '\\'; *p++ = 'n'; break; default: *p++ = c; break; } } *p++ = '"'; *p++ = '}'; return std::string_view(buf, p - buf); } }; // Track if this is the first static text entry (no leading newline) bool is_first_static = true; // Generate counters size_t counter_family_idx = 0; for (const auto &[name, family] : get_counter_families()) { // Add HELP line auto help_line = format( arena, "%s# HELP %.*s %.*s\n# TYPE %.*s counter", is_first_static ? "" : "\n", static_cast(name.length()), name.data(), static_cast(family->help.length()), family->help.data(), static_cast(name.length()), name.data()); is_first_static = false; // Callback instructions and static text for (const auto &[labels_key, callback] : family->callbacks) { plan.instructions.push_back(CallCounterCallback{&callback}); plan.static_text.push_back(arena_copy_string( format(arena, "%.*s\n%.*s%.*s ", static_cast(help_line.size()), help_line.data(), static_cast(name.length()), name.data(), static_cast(labels_key.prometheus_format.length()), labels_key.prometheus_format.data()), arena)); help_line = ""; } // Instance instructions and static text const auto &family_data = label_sets.counter_data[counter_family_idx++]; for (const auto &data : family_data) { plan.instructions.push_back( AggregateCounter{data.thread_states, data.global_state}); plan.static_text.push_back(arena_copy_string( format(arena, "%.*s\n%.*s%.*s ", static_cast(help_line.size()), help_line.data(), static_cast(name.length()), name.data(), static_cast(data.labels_key.prometheus_format.length()), data.labels_key.prometheus_format.data()), arena)); help_line = ""; } } // Generate gauges size_t gauge_family_idx = 0; for (const auto &[name, family] : get_gauge_families()) { // Add HELP line auto help_line = format( arena, "%s# HELP %.*s %.*s\n# TYPE %.*s gauge", is_first_static ? "" : "\n", static_cast(name.length()), name.data(), static_cast(family->help.length()), family->help.data(), static_cast(name.length()), name.data()); is_first_static = false; // Callback instructions and static text for (const auto &[labels_key, callback] : family->callbacks) { plan.instructions.push_back(CallCounterCallback{&callback}); plan.static_text.push_back(arena_copy_string( format(arena, "%.*s\n%.*s%.*s ", static_cast(help_line.size()), help_line.data(), static_cast(name.length()), name.data(), static_cast(labels_key.prometheus_format.length()), labels_key.prometheus_format.data()), arena)); help_line = ""; } // Instance instructions and static text const auto &family_data = label_sets.gauge_data[gauge_family_idx++]; for (const auto &data : family_data) { plan.instructions.push_back(AggregateGauge{data.instance_state}); plan.static_text.push_back(arena_copy_string( format(arena, "%.*s\n%.*s%.*s ", static_cast(help_line.size()), help_line.data(), static_cast(name.length()), name.data(), static_cast(data.labels_key.prometheus_format.length()), data.labels_key.prometheus_format.data()), arena)); help_line = ""; } } // Generate histograms size_t histogram_family_idx = 0; for (const auto &[name, family] : get_histogram_families()) { auto help_line = format( arena, "%s# HELP %.*s %.*s\n# TYPE %.*s histogram", is_first_static ? "" : "\n", static_cast(name.length()), name.data(), static_cast(family->help.length()), family->help.data(), static_cast(name.length()), name.data()); const auto &family_data = label_sets.histogram_data[histogram_family_idx++]; for (const auto &data : family_data) { plan.instructions.push_back( AggregateHistogram{data.thread_states, data.global_state, data.bucket_count, family->buckets}); // Static text for explicit buckets for (size_t i = 0; i < data.bucket_count; ++i) { auto bucket_value = static_format(arena, family->buckets[i]); auto labels = append_label_to_format( data.labels_key.prometheus_format, "le", bucket_value); plan.static_text.push_back(arena_copy_string( format(arena, "%.*s\n%.*s_bucket%.*s ", static_cast(help_line.size()), help_line.data(), static_cast(name.length()), name.data(), static_cast(labels.length()), labels.data()), arena)); help_line = ""; } // Static text for +Inf bucket auto inf_labels = append_label_to_format( data.labels_key.prometheus_format, "le", "+Inf"); plan.static_text.push_back(arena_copy_string( format(arena, "\n%.*s_bucket%.*s ", static_cast(name.length()), name.data(), static_cast(inf_labels.length()), inf_labels.data()), arena)); // Static text for sum plan.static_text.push_back(arena_copy_string( format(arena, "\n%.*s_sum%.*s ", static_cast(name.length()), name.data(), static_cast(data.labels_key.prometheus_format.length()), data.labels_key.prometheus_format.data()), arena)); // Static text for count plan.static_text.push_back(arena_copy_string( format(arena, "\n%.*s_count%.*s ", static_cast(name.length()), name.data(), static_cast(data.labels_key.prometheus_format.length()), data.labels_key.prometheus_format.data()), arena)); } } return plan; } // Phase 2: Execute phase - run instructions and generate dynamic text static ArenaVector execute_render_plan(ArenaAllocator &arena, const ArenaVector &instructions) { ArenaVector dynamic_text(&arena); for (const auto &instruction : instructions) { switch (instruction.type) { case InstructionType::CALL_COUNTER_CALLBACK: { double value = (*instruction.counter_callback.callback_ptr)(); dynamic_text.push_back(static_format(arena, value)); break; } case InstructionType::CALL_GAUGE_CALLBACK: { double value = (*instruction.gauge_callback.callback_ptr)(); dynamic_text.push_back(static_format(arena, value)); break; } case InstructionType::AGGREGATE_COUNTER: { double total_value = 0.0; // Sum thread-local values for (auto *state_ptr : instruction.aggregate_counter.thread_states) { double value; __atomic_load(&state_ptr->value, &value, __ATOMIC_RELAXED); total_value += value; } // Add global accumulated value if (instruction.aggregate_counter.global_state) { total_value += instruction.aggregate_counter.global_state->value; } dynamic_text.push_back(static_format(arena, total_value)); break; } case InstructionType::AGGREGATE_GAUGE: { double value = std::bit_cast( instruction.aggregate_gauge.instance_state->value.load( std::memory_order_relaxed)); dynamic_text.push_back(static_format(arena, value)); break; } case InstructionType::AGGREGATE_HISTOGRAM: { // Aggregate histogram data size_t bucket_count = instruction.aggregate_histogram.bucket_count; uint64_t *total_counts_data = arena.allocate(bucket_count); std::memset(total_counts_data, 0, bucket_count * sizeof(uint64_t)); std::span total_counts(total_counts_data, bucket_count); double total_sum = 0.0; uint64_t total_observations = 0; // Sum thread-local values for (auto *instance : instruction.aggregate_histogram.thread_states) { uint64_t *counts_snapshot = arena.allocate(bucket_count); double sum_snapshot; uint64_t observations_snapshot; { std::lock_guard lock(instance->mutex); for (size_t i = 0; i < instance->counts.size(); ++i) { counts_snapshot[i] = instance->counts[i]; } sum_snapshot = instance->sum; observations_snapshot = instance->observations; } for (size_t i = 0; i < bucket_count; ++i) { total_counts[i] += counts_snapshot[i]; } total_sum += sum_snapshot; total_observations += observations_snapshot; } // Add global accumulated values if (instruction.aggregate_histogram.global_state) { auto *global_state = instruction.aggregate_histogram.global_state; for (size_t i = 0; i < global_state->counts.size(); ++i) { total_counts[i] += global_state->counts[i]; } total_sum += global_state->sum; total_observations += global_state->observations; } // Format explicit bucket counts for (size_t i = 0; i < total_counts.size(); ++i) { dynamic_text.push_back(static_format(arena, total_counts[i])); } // Format +Inf bucket (total observations) dynamic_text.push_back(static_format(arena, total_observations)); // Format sum dynamic_text.push_back(static_format(arena, total_sum)); // Format count dynamic_text.push_back(static_format(arena, total_observations)); break; } } } return dynamic_text; } // Phase 3: Present phase - interleave static and dynamic text static ArenaVector present_render_output(ArenaAllocator &arena, const ArenaVector &static_text, const ArenaVector &dynamic_text) { ArenaVector output(&arena); for (size_t i = 0; i < static_text.size(); ++i) { // Copy static text into caller's arena output.push_back(arena_copy_string(static_text[i], arena)); // Add corresponding dynamic text output.push_back(dynamic_text[i]); } // Trailing newline output.push_back("\n"); return output; } // Build label sets once for reuse in both phases static LabelSets build_label_sets(ArenaAllocator &arena) { LabelSets label_sets; // Build counter data with pre-resolved pointers for (const auto &[name, family] : Metric::get_counter_families()) { // Collect all unique labels first std::set, ArenaStlAllocator> all_labels{ArenaStlAllocator(&arena)}; for (const auto &[thread_id, per_thread] : family->per_thread_state) { for (const auto &[labels_key, instance] : per_thread.instances) { all_labels.insert(labels_key); } } for (const auto &[labels_key, global_state] : family->global_accumulated_values) { if (global_state) { all_labels.insert(labels_key); } } // Pre-resolve all pointers for each label set std::vector family_data; for (const auto &labels_key : all_labels) { CounterLabelData data(labels_key); // Pre-resolve thread-local state pointers for (const auto &[thread_id, per_thread] : family->per_thread_state) { auto it = per_thread.instances.find(labels_key); if (it != per_thread.instances.end()) { data.thread_states.push_back(it->second); } } // Pre-resolve global accumulated state pointer auto global_it = family->global_accumulated_values.find(labels_key); data.global_state = (global_it != family->global_accumulated_values.end() && global_it->second) ? global_it->second : nullptr; family_data.push_back(std::move(data)); } label_sets.counter_data.push_back(std::move(family_data)); } // Build gauge data with pre-resolved pointers for (const auto &[name, family] : Metric::get_gauge_families()) { std::vector family_data; // Gauges iterate directly over instances for (const auto &[labels_key, instance] : family->instances) { GaugeLabelData data(labels_key); data.instance_state = instance; family_data.push_back(std::move(data)); } label_sets.gauge_data.push_back(std::move(family_data)); } // Build histogram data with pre-resolved pointers for (const auto &[name, family] : Metric::get_histogram_families()) { // Collect all unique labels first std::set, ArenaStlAllocator> all_labels{ArenaStlAllocator(&arena)}; for (const auto &[thread_id, per_thread] : family->per_thread_state) { for (const auto &[labels_key, instance] : per_thread.instances) { all_labels.insert(labels_key); } } for (const auto &[labels_key, global_state] : family->global_accumulated_values) { if (global_state) { all_labels.insert(labels_key); } } // Pre-resolve all pointers for each label set std::vector family_data; for (const auto &labels_key : all_labels) { HistogramLabelData data(labels_key); data.bucket_count = family->buckets.size(); // Cache bucket count // Pre-resolve thread-local state pointers for (const auto &[thread_id, per_thread] : family->per_thread_state) { auto it = per_thread.instances.find(labels_key); if (it != per_thread.instances.end()) { data.thread_states.push_back(it->second); } } // Pre-resolve global accumulated state pointer auto global_it = family->global_accumulated_values.find(labels_key); data.global_state = (global_it != family->global_accumulated_values.end() && global_it->second) ? global_it->second : nullptr; family_data.push_back(std::move(data)); } label_sets.histogram_data.push_back(std::move(family_data)); } return label_sets; } }; Counter::Counter() = default; void Counter::inc(double x) { // THREAD-SAFETY: This method mixes a non-atomic read and an atomic store // which is safe ONLY under the following conditions, which this system meets: // // 1. Single-Writer Guarantee: The underlying Counter::State is thread-local. // Only one thread will ever call inc() on a given instance. All writes // and the non-atomic read below are therefore *sequenced*, not concurrent, // preventing torn reads within this thread. // // 2. Atomic Visibility: The render thread is the only other thread that // accesses this value, and it does so via an atomic load. A concurrent // non-atomic read (writer) and atomic read (renderer) is not a data race. // // This contrasts with Gauges, whose state can be shared by multiple threads // and thus requires a fully atomic read-modify-write cycle (CAS loop). auto new_value = p->value + x; // Validate monotonic property (counter never decreases) if (new_value < p->value) [[unlikely]] { validate_or_abort(false, "counter value overflow/wraparound detected", std::to_string(new_value)); } __atomic_store(&p->value, &new_value, __ATOMIC_RELAXED); } Gauge::Gauge() = default; void Gauge::inc(double x) { // Lock-free increment using CAS loop uint64_t expected = p->value.load(std::memory_order_relaxed); uint64_t desired; do { double current_value = std::bit_cast(expected); double new_value = current_value + x; desired = std::bit_cast(new_value); } while (!p->value.compare_exchange_weak(expected, desired, std::memory_order_relaxed)); } void Gauge::dec(double x) { // Lock-free decrement using CAS loop uint64_t expected = p->value.load(std::memory_order_relaxed); uint64_t desired; do { double current_value = std::bit_cast(expected); double new_value = current_value - x; desired = std::bit_cast(new_value); } while (!p->value.compare_exchange_weak(expected, desired, std::memory_order_relaxed)); } void Gauge::set(double x) { // Simple atomic store for set operation p->value.store(std::bit_cast(x), std::memory_order_relaxed); } Histogram::Histogram() = default; // Vectorized histogram bucket updates with mutex protection for consistency // AVX-optimized implementation for high performance __attribute__((target("avx"))) static void update_histogram_buckets_simd(std::span thresholds, std::span counts, double x, size_t start_idx) { const size_t size = thresholds.size(); size_t i = start_idx; // Process 2 buckets at a time with 128-bit vectors const __m128d x_vec = _mm_set1_pd(x); for (; i + 2 <= size; i += 2) { // 128-bit vectorized comparison and arithmetic __m128d thresholds_vec = _mm_loadu_pd(&thresholds[i]); __m128d cmp_result = _mm_cmp_pd(x_vec, thresholds_vec, _CMP_LE_OQ); __m128i cmp_as_int = _mm_castpd_si128(cmp_result); __m128i ones = _mm_set1_epi64x(1); __m128i increments = _mm_and_si128(cmp_as_int, ones); // Load current counts and add increments __m128i current_counts = _mm_loadu_si128((__m128i *)&counts[i]); __m128i updated_counts = _mm_add_epi64(current_counts, increments); // Store updated counts _mm_storeu_si128((__m128i *)&counts[i], updated_counts); } // Handle remainder with scalar operations for (; i < size; ++i) { if (x <= thresholds[i]) { counts[i]++; } } } void Histogram::observe(double x) { assert(p->thresholds.size() == p->counts.size()); std::lock_guard lock(p->mutex); // Update bucket counts using SIMD update_histogram_buckets_simd(p->thresholds, p->counts, x, 0); // Update sum and observation count p->sum += x; p->observations++; } template <> Family::Family() = default; template <> Family::Family() = default; template <> Family::Family() = default; template <> Counter Family::create( std::span> labels) { return Metric::create_counter_instance(this, labels); } template <> Gauge Family::create( std::span> labels) { return Metric::create_gauge_instance(this, labels); } template <> Histogram Family::create( std::span> labels) { return Metric::create_histogram_instance(this, labels); } Family create_counter(std::string_view name, std::string_view help) { validate_or_abort(is_valid_metric_name(name), "invalid counter name", name); std::unique_lock _{Metric::mutex}; auto &global_arena = Metric::get_global_arena(); auto name_view = arena_copy_string(name, global_arena); auto &familyPtr = Metric::get_counter_families()[name_view]; if (!familyPtr) { familyPtr = global_arena.construct::State>(global_arena); familyPtr->name = name_view; familyPtr->help = arena_copy_string(help, global_arena); } else { validate_or_abort( familyPtr->help == help, "metric family already registered with different help text", name); } Family family; family.p = familyPtr.get(); return family; } Family create_gauge(std::string_view name, std::string_view help) { validate_or_abort(is_valid_metric_name(name), "invalid gauge name", name); std::unique_lock _{Metric::mutex}; auto &global_arena = Metric::get_global_arena(); auto name_view = arena_copy_string(name, global_arena); auto &familyPtr = Metric::get_gauge_families()[name_view]; if (!familyPtr) { // NOTE: Family::State instances are never destroyed - this is fine // because the number of metric families is bounded by application design familyPtr = global_arena.construct::State>(global_arena); familyPtr->name = name_view; familyPtr->help = arena_copy_string(help, global_arena); } else { validate_or_abort( familyPtr->help == help, "metric family already registered with different help text", name); } Family family; family.p = familyPtr.get(); return family; } Family create_histogram(std::string_view name, std::string_view help, std::span buckets) { validate_or_abort(is_valid_metric_name(name), "invalid histogram name", name); std::unique_lock _{Metric::mutex}; auto &global_arena = Metric::get_global_arena(); auto name_view = arena_copy_string(name, global_arena); auto &family_ptr = Metric::get_histogram_families()[name_view]; if (!family_ptr) { // NOTE: Family::State instances are never destroyed - this is fine // because the number of metric families is bounded by application design family_ptr = global_arena.construct::State>(global_arena); family_ptr->name = name_view; family_ptr->help = arena_copy_string(help, global_arena); // DESIGN: Prometheus-compatible histogram buckets // Convert to vector for sorting std::vector temp_buckets(buckets.begin(), buckets.end()); std::sort(temp_buckets.begin(), temp_buckets.end()); temp_buckets.erase(std::unique(temp_buckets.begin(), temp_buckets.end()), temp_buckets.end()); // Copy sorted buckets to arena vector for (double bucket : temp_buckets) { family_ptr->buckets.push_back(bucket); } // Note: +Inf bucket is not stored explicitly - we use total observations // count } else { validate_or_abort( family_ptr->help == help, "metric family already registered with different help text", name); std::vector new_buckets_vec(buckets.begin(), buckets.end()); std::sort(new_buckets_vec.begin(), new_buckets_vec.end()); new_buckets_vec.erase( std::unique(new_buckets_vec.begin(), new_buckets_vec.end()), new_buckets_vec.end()); // Note: +Inf bucket is not stored explicitly - we use total observations // count // Compare with existing buckets bool buckets_match = (family_ptr->buckets.size() == new_buckets_vec.size()); if (buckets_match) { for (size_t i = 0; i < family_ptr->buckets.size(); ++i) { if (family_ptr->buckets[i] != new_buckets_vec[i]) { buckets_match = false; break; } } } validate_or_abort(buckets_match, "metric family already registered with different buckets", name); } Family family; family.p = family_ptr.get(); return family; } std::vector linear_buckets(double start, double width, int count) { validate_or_abort(width > 0, "linear bucket width must be positive", std::to_string(width)); validate_or_abort(count >= 0, "linear bucket count must be non-negative", std::to_string(count)); std::vector buckets; buckets.reserve(count); for (int i = 0; i < count; ++i) { buckets.push_back(start + i * width); } return buckets; } std::vector exponential_buckets(double start, double factor, int count) { validate_or_abort(start > 0, "exponential bucket start must be positive", std::to_string(start)); validate_or_abort(factor > 1, "exponential bucket factor must be > 1", std::to_string(factor)); validate_or_abort(count >= 0, "exponential bucket count must be non-negative", std::to_string(count)); std::vector buckets; buckets.reserve(count); double current = start; for (int i = 0; i < count; ++i) { buckets.push_back(current); current *= factor; } return buckets; } // Prometheus validation functions // Metric names must match [a-zA-Z_:][a-zA-Z0-9_:]* bool is_valid_metric_name(std::string_view name) { if (name.empty()) return false; // First character must be letter, underscore, or colon char first = name[0]; if (!std::isalpha(first) && first != '_' && first != ':') { return false; } // Remaining characters must be alphanumeric, underscore, or colon for (size_t i = 1; i < name.size(); ++i) { char c = name[i]; if (!std::isalnum(c) && c != '_' && c != ':') { return false; } } return true; } // Label keys must match [a-zA-Z_][a-zA-Z0-9_]* bool is_valid_label_key(std::string_view key) { if (key.empty()) return false; // First character must be letter or underscore char first = key[0]; if (!std::isalpha(first) && first != '_') { return false; } // Remaining characters must be alphanumeric or underscore for (size_t i = 1; i < key.size(); ++i) { char c = key[i]; if (!std::isalnum(c) && c != '_') { return false; } } // Label keys starting with __ are reserved for internal use if (key.size() >= 2 && key[0] == '_' && key[1] == '_') { return false; } return true; } // Label values can contain any UTF-8 characters (no specific restrictions) bool is_valid_label_value(std::string_view value) { // Prometheus allows any UTF-8 string as label value // Validate UTF-8 encoding for correctness using simdutf return simdutf::validate_utf8(value.data(), value.size()); } union MetricValue { double as_double; uint64_t as_uint64; }; // Legacy function kept for reference - will be replaced static ArenaVector compute_metric_values_legacy(ArenaAllocator &arena, const Metric::LabelSets &label_sets) { ArenaVector values(&arena); // Compute counter values - ITERATION ORDER MUST MATCH FORMAT PHASE size_t counter_family_idx = 0; for (const auto &[name, family] : Metric::get_counter_families()) { // Callback values for (const auto &[labels_key, callback] : family->callbacks) { auto value = callback(); values.push_back({.as_double = value}); } // Use pre-computed data with resolved pointers - no hash lookups! const auto &family_data = label_sets.counter_data[counter_family_idx++]; for (const auto &data : family_data) { double total_value = 0.0; // Sum thread-local values using pre-resolved pointers for (auto *state_ptr : data.thread_states) { // Atomic read to match atomic store in Counter::inc() double value; __atomic_load(&state_ptr->value, &value, __ATOMIC_RELAXED); total_value += value; } // Add global accumulated value using pre-resolved pointer if (data.global_state) { total_value += data.global_state->value; } values.push_back({.as_double = total_value}); } } // Compute gauge values - ITERATION ORDER MUST MATCH FORMAT PHASE size_t gauge_family_idx = 0; for (const auto &[name, family] : Metric::get_gauge_families()) { // Callback values for (const auto &[labels_key, callback] : family->callbacks) { auto value = callback(); values.push_back({.as_double = value}); } // Use pre-computed data with resolved pointers - no hash lookups! const auto &family_data = label_sets.gauge_data[gauge_family_idx++]; for (const auto &data : family_data) { auto value = std::bit_cast( data.instance_state->value.load(std::memory_order_relaxed)); values.push_back({.as_double = value}); } } // Compute histogram values - ITERATION ORDER MUST MATCH FORMAT PHASE size_t histogram_family_idx = 0; for ([[maybe_unused]] const auto &[_name, _family] : Metric::get_histogram_families()) { // Use pre-computed data with resolved pointers - no hash lookups! const auto &family_data = label_sets.histogram_data[histogram_family_idx++]; for (const auto &data : family_data) { size_t bucket_count = data.bucket_count; // Use cached bucket count uint64_t *total_counts_data = arena.allocate(bucket_count); std::memset(total_counts_data, 0, bucket_count * sizeof(uint64_t)); std::span total_counts(total_counts_data, bucket_count); double total_sum = 0.0; uint64_t total_observations = 0; // Sum thread-local values using pre-resolved pointers for (auto *instance : data.thread_states) { // Extract data under lock - minimize critical section uint64_t *counts_snapshot = arena.allocate(bucket_count); double sum_snapshot; uint64_t observations_snapshot; { std::lock_guard lock(instance->mutex); for (size_t i = 0; i < instance->counts.size(); ++i) { counts_snapshot[i] = instance->counts[i]; } sum_snapshot = instance->sum; observations_snapshot = instance->observations; } // Add to totals for (size_t i = 0; i < bucket_count; ++i) { total_counts[i] += counts_snapshot[i]; } total_sum += sum_snapshot; total_observations += observations_snapshot; } // Add global accumulated value using pre-resolved pointer if (data.global_state) { auto *global_state = data.global_state; for (size_t i = 0; i < global_state->counts.size(); ++i) { total_counts[i] += global_state->counts[i]; } total_sum += global_state->sum; total_observations += global_state->observations; } // Store histogram values // Store explicit bucket counts for (size_t i = 0; i < total_counts.size(); ++i) { values.push_back({.as_uint64 = total_counts[i]}); } // Store +Inf bucket (total observations) values.push_back({.as_uint64 = total_observations}); // Store sum values.push_back({.as_double = total_sum}); // Store count values.push_back({.as_uint64 = total_observations}); } } return values; } // Forward declaration std::span render_legacy(ArenaAllocator &arena); // New three-phase render implementation std::span render(ArenaAllocator &arena) { // Hold lock throughout all phases to prevent registry changes std::unique_lock _{Metric::mutex}; // Build label sets once for all phases Metric::LabelSets label_sets = Metric::build_label_sets(arena); // Phase 1: Compile - generate static text and instructions Metric::RenderPlan plan = Metric::compile_render_plan(arena, label_sets); // Phase 2: Execute - run instructions and generate dynamic text ArenaVector dynamic_text = Metric::execute_render_plan(arena, plan.instructions); // Phase 3: Present - interleave static and dynamic text ArenaVector output = Metric::present_render_output(arena, plan.static_text, dynamic_text); return output; } // Legacy render function - kept for reference during transition std::span render_legacy(ArenaAllocator &arena) { // Hold lock throughout both phases to prevent registry changes std::unique_lock _{Metric::mutex}; // Build label sets once for both phases Metric::LabelSets label_sets = Metric::build_label_sets(arena); // Phase 1: Compute all metric values ArenaVector metric_values = compute_metric_values_legacy(arena, label_sets); const MetricValue *next_value = metric_values.data(); ArenaVector output(&arena); // Helper function to append an additional label to existing Prometheus format auto append_label_to_format = [&](std::string_view base_format, std::string_view key, std::string_view value) -> std::string_view { // Calculate size for key="value" with escaping size_t key_value_size = key.length() + 3 + value.length(); // key="value" for (char c : value) { if (c == '\\' || c == '"' || c == '\n') { key_value_size++; } } if (base_format.empty()) { // Create new format: {key="value"} size_t required_size = 2 + key_value_size; // {} char *buf = arena.allocate(required_size); char *p = buf; *p++ = '{'; std::memcpy(p, key.data(), key.length()); p += key.length(); *p++ = '='; *p++ = '"'; for (char c : value) { switch (c) { case '\\': *p++ = '\\'; *p++ = '\\'; break; case '"': *p++ = '\\'; *p++ = '"'; break; case '\n': *p++ = '\\'; *p++ = 'n'; break; default: *p++ = c; break; } } *p++ = '"'; *p++ = '}'; return std::string_view(buf, p - buf); } else { // Append to existing format: {existing,key="value"} size_t required_size = base_format.length() + 1 + key_value_size; // comma + key="value", replace } char *buf = arena.allocate(required_size); char *p = buf; // Copy everything except the closing } std::memcpy(p, base_format.data(), base_format.length() - 1); p += base_format.length() - 1; *p++ = ','; std::memcpy(p, key.data(), key.length()); p += key.length(); *p++ = '='; *p++ = '"'; for (char c : value) { switch (c) { case '\\': *p++ = '\\'; *p++ = '\\'; break; case '"': *p++ = '\\'; *p++ = '"'; break; case '\n': *p++ = '\\'; *p++ = 'n'; break; default: *p++ = c; break; } } *p++ = '"'; *p++ = '}'; return std::string_view(buf, p - buf); } }; // Format counters - ITERATION ORDER MUST MATCH COMPUTE PHASE size_t counter_family_idx = 0; for (const auto &[name, family] : Metric::get_counter_families()) { output.push_back(format(arena, "# HELP %.*s %.*s\n", static_cast(name.length()), name.data(), static_cast(family->help.length()), family->help.data())); output.push_back(format(arena, "# TYPE %.*s counter\n", static_cast(name.length()), name.data())); // Format callback values for (const auto &[labels_key, callback] : family->callbacks) { auto value = next_value++->as_double; output.push_back(format( arena, "%.*s%.*s %.17g\n", static_cast(name.length()), name.data(), static_cast(labels_key.prometheus_format.length()), labels_key.prometheus_format.data(), value)); } // Use pre-computed data (same as compute phase) const auto &family_data = label_sets.counter_data[counter_family_idx++]; // Format counter values using pre-computed values for (const auto &data : family_data) { auto total_value = next_value++->as_double; output.push_back( format(arena, "%.*s%.*s %.17g\n", static_cast(name.length()), name.data(), static_cast(data.labels_key.prometheus_format.length()), data.labels_key.prometheus_format.data(), total_value)); } } // Format gauges - ITERATION ORDER MUST MATCH COMPUTE PHASE size_t gauge_family_idx = 0; for (const auto &[name, family] : Metric::get_gauge_families()) { output.push_back(format(arena, "# HELP %.*s %.*s\n", static_cast(name.length()), name.data(), static_cast(family->help.length()), family->help.data())); output.push_back(format(arena, "# TYPE %.*s gauge\n", static_cast(name.length()), name.data())); // Format callback values for (const auto &[labels_key, callback] : family->callbacks) { auto value = next_value++->as_double; output.push_back(format( arena, "%.*s%.*s %.17g\n", static_cast(name.length()), name.data(), static_cast(labels_key.prometheus_format.length()), labels_key.prometheus_format.data(), value)); } // Use pre-computed data (same as compute phase) const auto &family_data = label_sets.gauge_data[gauge_family_idx++]; for (const auto &data : family_data) { auto value = next_value++->as_double; output.push_back( format(arena, "%.*s%.*s %.17g\n", static_cast(name.length()), name.data(), static_cast(data.labels_key.prometheus_format.length()), data.labels_key.prometheus_format.data(), value)); } } // Format histograms - ITERATION ORDER MUST MATCH COMPUTE PHASE size_t histogram_family_idx = 0; for (const auto &[name, family] : Metric::get_histogram_families()) { output.push_back(format(arena, "# HELP %.*s %.*s\n", static_cast(name.length()), name.data(), static_cast(family->help.length()), family->help.data())); output.push_back(format(arena, "# TYPE %.*s histogram\n", static_cast(name.length()), name.data())); // Use pre-computed data (same as compute phase) const auto &family_data = label_sets.histogram_data[histogram_family_idx++]; // Format histogram data using pre-computed values for (const auto &data : family_data) { // Get bucket count from pre-computed data size_t bucket_count = data.bucket_count; // Format explicit bucket counts for (size_t i = 0; i < bucket_count; ++i) { auto count = next_value++->as_uint64; auto bucket_value = static_format(arena, family->buckets[i]); auto labels = append_label_to_format(data.labels_key.prometheus_format, "le", bucket_value); output.push_back(format( arena, "%.*s_bucket%.*s %llu\n", static_cast(name.length()), name.data(), static_cast(labels.length()), labels.data(), static_cast(count))); } // Format +Inf bucket auto observations = next_value++->as_uint64; auto inf_labels = append_label_to_format( data.labels_key.prometheus_format, "le", "+Inf"); output.push_back(format( arena, "%.*s_bucket%.*s %llu\n", static_cast(name.length()), name.data(), static_cast(inf_labels.length()), inf_labels.data(), static_cast(observations))); // Format sum auto sum = next_value++->as_double; output.push_back( format(arena, "%.*s_sum%.*s %.17g\n", static_cast(name.length()), name.data(), static_cast(data.labels_key.prometheus_format.length()), data.labels_key.prometheus_format.data(), sum)); // Format count auto count = next_value++->as_uint64; output.push_back( format(arena, "%.*s_count%.*s %llu\n", static_cast(name.length()), name.data(), static_cast(data.labels_key.prometheus_format.length()), data.labels_key.prometheus_format.data(), static_cast(count))); } } return output; } // Template specialization implementations for register_callback template <> void Family::register_callback( std::span> labels, MetricCallback callback) { std::unique_lock _{Metric::mutex}; const LabelsKey &key = Metric::intern_labels(labels); // Validate that labels aren't already in use by create() calls for (const auto &[thread_id, per_thread] : p->per_thread_state) { validate_or_abort( per_thread.instances.find(key) == per_thread.instances.end(), "labels already registered as static instance", key.prometheus_format.empty() ? "(no labels)" : key.prometheus_format); } // Validate that callback isn't already registered for these labels validate_or_abort(p->callbacks.find(key) == p->callbacks.end(), "callback already registered for labels", key.prometheus_format.empty() ? "(no labels)" : key.prometheus_format); p->callbacks[std::move(key)] = std::move(callback); } template <> void Family::register_callback( std::span> labels, MetricCallback callback) { std::unique_lock _{Metric::mutex}; const LabelsKey &key = Metric::intern_labels(labels); // Validate that labels aren't already in use by create() calls validate_or_abort(p->instances.find(key) == p->instances.end(), "labels already registered as static instance", key.prometheus_format.empty() ? "(no labels)" : key.prometheus_format); // Validate that callback isn't already registered for these labels validate_or_abort(p->callbacks.find(key) == p->callbacks.end(), "callback already registered for labels", key.prometheus_format.empty() ? "(no labels)" : key.prometheus_format); p->callbacks[std::move(key)] = std::move(callback); } // Explicit template instantiations to provide member implementations // Static member definitions std::mutex Metric::mutex; thread_local Metric::ThreadInit Metric::thread_init; void reset_metrics_for_testing() { std::lock_guard _{Metric::mutex}; // WARNING: This function assumes no metric objects are in use! // Clear all family maps - this will leak the Family::State objects but // that's acceptable for testing since they were allocated in the global arena // Get references to the maps auto &counter_families = Metric::get_counter_families(); auto &gauge_families = Metric::get_gauge_families(); auto &histogram_families = Metric::get_histogram_families(); auto &interned_labels = Metric::get_interned_labels(); // Clear all family registrations counter_families.clear(); gauge_families.clear(); histogram_families.clear(); interned_labels.clear(); // Reset the global arena - this will invalidate all arena-allocated strings // but since we're clearing everything, that's OK Metric::get_global_arena().reset(); // Note: Thread-local arenas will be cleaned up by ThreadInit destructors // when threads exit naturally } } // namespace metric