Use the correct metric types & capture perspective when naming

Some metrics were of type gauge while they should have been of type
counter. Thanks @brian-brazil for making the distinction clear. This is
now captured as a comment above the metric definitions.

Because all metrics are from RabbitMQ's perspective, cached for up to 5
seconds by default (configurable), we prepend `rabbitmq_` to all metrics
emitted by this collector.  While Some metrics are for Erlang (erlang_),
Mnesia (schema_db_) or the System (io_), they are all observed & cached
by RabbitMQ, hence the prefix.

This is the last PR which started in the context of prometheus/docs#1414

[#167846096]
This commit is contained in:
Gerhard Lazu 2019-09-04 09:55:53 +01:00
parent a1055a0199
commit 5781130b61
2 changed files with 106 additions and 92 deletions

View File

@ -30,71 +30,85 @@
-behaviour(prometheus_collector).
% Because some metrics need prefixing with rabbitmq_, others with erlang_ or process_,
% we don't set a global metric name prefix.
-define(METRIC_NAME_PREFIX, "").
%% Because all metrics are from RabbitMQ's perspective,
%% cached for up to 5 seconds by default (configurable),
%% we prepend rabbitmq_ to all metrics emitted by this collector.
%% Some metrics are for Erlang (erlang_), Mnesia (schema_db_) or the System (io_),
%% as observed by RabbitMQ.
-define(METRIC_NAME_PREFIX, "rabbitmq_").
% The source of these metrics are in the rabbit_core_metrics module
% The relevant files are:
% * rabbit_common/src/rabbit_core_metrics.erl
% * rabbit_common/include/rabbit_core_metrics.hrl
%% ==The source of these metrics can be found in the rabbit_core_metrics module==
%% The relevant files are:
%% * rabbit_common/src/rabbit_core_metrics.erl
%% * rabbit_common/include/rabbit_core_metrics.hrl
%%
%% ==How to determine if a metric should be of type GAUGE or COUNTER?==
%%
%% * GAUGE if you care about its value rather than rate of change
%% - value can decrease as well as decrease
%% * COUNTER if you care about the rate of change
%% - value can only increase
%%
%% To put it differently, if the metric is used with rate(), it's a COUNTER, otherwise it's a GAUGE.
%%
%% More info: https://prometheus.io/docs/practices/instrumentation/#counter-vs-gauge-summary-vs-histogram
-define(METRICS_RAW, [
{channel_metrics, [
{2, rabbitmq_channel_consumers, gauge, "Consumers on a channel", consumer_count},
{2, rabbitmq_channel_messages_unacked, gauge, "Delivered but not yet acknowledged messages", messages_unacknowledged},
{2, rabbitmq_channel_messages_unconfirmed, gauge, "Published but not yet confirmed messages", messages_unconfirmed},
{2, rabbitmq_channel_messages_uncommitted, gauge, "Messages received in a transaction but not yet committed", messages_uncommitted},
{2, rabbitmq_channel_acks_uncommitted, gauge, "Message acknowledgements in a transaction not yet committed", acks_uncommitted},
{2, rabbitmq_consumer_prefetch, gauge, "Limit of unacknowledged messages for each consumer", prefetch_count},
{2, rabbitmq_channel_prefetch, gauge, "Total limit of unacknowledged messages for all consumers on a channel", global_prefetch_count}
{2, channel_consumers, gauge, "Consumers on a channel", consumer_count},
{2, channel_messages_unacked, gauge, "Delivered but not yet acknowledged messages", messages_unacknowledged},
{2, channel_messages_unconfirmed, gauge, "Published but not yet confirmed messages", messages_unconfirmed},
{2, channel_messages_uncommitted, gauge, "Messages received in a transaction but not yet committed", messages_uncommitted},
{2, channel_acks_uncommitted, gauge, "Message acknowledgements in a transaction not yet committed", acks_uncommitted},
{2, consumer_prefetch, gauge, "Limit of unacknowledged messages for each consumer", prefetch_count},
{2, channel_prefetch, gauge, "Total limit of unacknowledged messages for all consumers on a channel", global_prefetch_count}
]},
{channel_exchange_metrics, [
{2, rabbitmq_channel_messages_published, gauge, "Messages published into an exchange on a channel"},
{3, rabbitmq_channel_messages_confirmed, gauge, "Messages published into an exchange and confirmed on the channel"},
{4, rabbitmq_channel_messages_unroutable_returned, gauge, "Messages published as mandatory into an exchange and returned to the publisher as unroutable"},
{5, rabbitmq_channel_messages_unroutable_dropped, gauge, "Messages published as non-mandatory into an exchange and dropped as unroutable"}
{2, channel_messages_published_total, counter, "Total number of messages published into an exchange on a channel"},
{3, channel_messages_confirmed_total, counter, "Total number of messages published into an exchange and confirmed on the channel"},
{4, channel_messages_unroutable_returned_total, counter, "Total number of messages published as mandatory into an exchange and returned to the publisher as unroutable"},
{5, channel_messages_unroutable_dropped_total, counter, "Total number of messages published as non-mandatory into an exchange and dropped as unroutable"}
]},
{channel_process_metrics, [
{2, rabbitmq_channel_process_reductions_total, counter, "Total number of channel process reductions"}
{2, channel_process_reductions_total, counter, "Total number of channel process reductions"}
]},
{channel_queue_metrics, [
{2, rabbitmq_channel_get_ack_total, counter, "Total number of messages fetched with basic.get in manual acknowledgement mode"},
{3, rabbitmq_channel_get_total, counter, "Total number of messages fetched with basic.get in automatic acknowledgement mode"},
{4, rabbitmq_channel_messages_delivered_ack_total, counter, "Total number of messages delivered to consumers in manual acknowledgement mode"},
{5, rabbitmq_channel_messages_delivered_total, counter, "Total number of messages delivered to consumers in automatic acknowledgement mode"},
{6, rabbitmq_channel_messages_redelivered_total, counter, "Total number of messages redelivered to consumers"},
{7, rabbitmq_channel_messages_acked_total, counter, "Total number of messages acknowledged by consumers"},
{8, rabbitmq_channel_get_empty_total, counter, "Total number of times basic.get operations fetched no message"}
{2, channel_get_ack_total, counter, "Total number of messages fetched with basic.get in manual acknowledgement mode"},
{3, channel_get_total, counter, "Total number of messages fetched with basic.get in automatic acknowledgement mode"},
{4, channel_messages_delivered_ack_total, counter, "Total number of messages delivered to consumers in manual acknowledgement mode"},
{5, channel_messages_delivered_total, counter, "Total number of messages delivered to consumers in automatic acknowledgement mode"},
{6, channel_messages_redelivered_total, counter, "Total number of messages redelivered to consumers"},
{7, channel_messages_acked_total, counter, "Total number of messages acknowledged by consumers"},
{8, channel_get_empty_total, counter, "Total number of times basic.get operations fetched no message"}
]},
{connection_churn_metrics, [
{2, rabbitmq_connections_opened_total, counter, "Total number of connections opened"},
{3, rabbitmq_connections_closed_total, counter, "Total number of connections closed or terminated"},
{4, rabbitmq_channels_opened_total, counter, "Total number of channels opened"},
{5, rabbitmq_channels_closed_total, counter, "Total number of channels closed"},
{6, rabbitmq_queues_declared_total, counter, "Total number of queues declared"},
{7, rabbitmq_queues_created_total, counter, "Total number of queues created"},
{8, rabbitmq_queues_deleted_total, counter, "Total number of queues deleted"}
{2, connections_opened_total, counter, "Total number of connections opened"},
{3, connections_closed_total, counter, "Total number of connections closed or terminated"},
{4, channels_opened_total, counter, "Total number of channels opened"},
{5, channels_closed_total, counter, "Total number of channels closed"},
{6, queues_declared_total, counter, "Total number of queues declared"},
{7, queues_created_total, counter, "Total number of queues created"},
{8, queues_deleted_total, counter, "Total number of queues deleted"}
]},
{connection_coarse_metrics, [
{2, rabbitmq_connection_incoming_bytes_total, counter, "Total number of bytes received on a connection"},
{3, rabbitmq_connection_outgoing_bytes_total, counter, "Total number of bytes sent on a connection"},
{4, rabbitmq_connection_process_reductions_total, counter, "Total number of connection process reductions"}
{2, connection_incoming_bytes_total, counter, "Total number of bytes received on a connection"},
{3, connection_outgoing_bytes_total, counter, "Total number of bytes sent on a connection"},
{4, connection_process_reductions_total, counter, "Total number of connection process reductions"}
]},
{connection_metrics, [
{2, rabbitmq_connection_incoming_packets_total, counter, "Total number of packets received on a connection", recv_cnt},
{2, rabbitmq_connection_outgoing_packets_total, counter, "Total number of packets sent on a connection", send_cnt},
{2, rabbitmq_connection_pending_packets, gauge, "Number of packets waiting to be sent on a connection", send_pend},
{2, rabbitmq_connection_channels, gauge, "Channels on a connection", channels}
{2, connection_incoming_packets_total, counter, "Total number of packets received on a connection", recv_cnt},
{2, connection_outgoing_packets_total, counter, "Total number of packets sent on a connection", send_cnt},
{2, connection_pending_packets, gauge, "Number of packets waiting to be sent on a connection", send_pend},
{2, connection_channels, gauge, "Channels on a connection", channels}
]},
{channel_queue_exchange_metrics, [
{2, rabbitmq_queue_messages_published_total, counter, "Total number of messages published to queues"}
{2, queue_messages_published_total, counter, "Total number of messages published to queues"}
]},
{node_coarse_metrics, [
@ -111,7 +125,7 @@
{node_metrics, [
{2, process_max_fds, gauge, "Open file descriptors limit", fd_total},
{2, process_max_sockets, gauge, "Open TCP sockets limit", sockets_total},
{2, rabbitmq_resident_memory_limit_bytes, gauge, "Memory high watermark in bytes", mem_limit},
{2, resident_memory_limit_bytes, gauge, "Memory high watermark in bytes", mem_limit},
{2, disk_space_available_limit_bytes, gauge, "Free disk space low watermark in bytes", disk_free_limit},
{2, erlang_processes_limit, gauge, "Erlang processes limit", proc_total},
{2, erlang_scheduler_run_queue, gauge, "Erlang scheduler run queue", run_queue},
@ -119,54 +133,54 @@
]},
{node_persister_metrics, [
{2, rabbitmq_io_read_ops_total, counter, "Total number of I/O read operations", io_read_count},
{2, rabbitmq_io_read_bytes_total, counter, "Total number of I/O bytes read", io_read_bytes},
{2, rabbitmq_io_write_ops_total, counter, "Total number of I/O write operations", io_write_count},
{2, rabbitmq_io_write_bytes_total, counter, "Total number of I/O bytes written", io_write_bytes},
{2, rabbitmq_io_sync_ops_total, counter, "Total number of I/O sync operations", io_sync_count},
{2, rabbitmq_io_seek_ops_total, counter, "Total number of I/O seek operations", io_seek_count},
{2, rabbitmq_io_open_attempt_ops_total, counter, "Total number of file open attempts", io_file_handle_open_attempt_count},
{2, rabbitmq_io_reopen_ops_total, counter, "Total number of times files have been reopened", io_reopen_count},
{2, rabbitmq_schema_db_ram_tx_total, counter, "Total number of Schema DB memory transactions", mnesia_ram_tx_count},
{2, rabbitmq_schema_db_disk_tx_total, counter, "Total number of Schema DB disk transactions", mnesia_disk_tx_count},
{2, rabbitmq_msg_store_read_total, counter, "Total number of Message Store read operations", msg_store_read_count},
{2, rabbitmq_msg_store_write_total, counter, "Total number of Message Store write operations", msg_store_write_count},
{2, rabbitmq_queue_index_read_ops_total, counter, "Total number of Queue Index read operations", queue_index_read_count},
{2, rabbitmq_queue_index_write_ops_total, counter, "Total number of Queue Index write operations", queue_index_write_count},
{2, rabbitmq_queue_index_journal_write_ops_total, counter, "Total number of Queue Index Journal write operations", queue_index_journal_write_count}
{2, io_read_ops_total, counter, "Total number of I/O read operations", io_read_count},
{2, io_read_bytes_total, counter, "Total number of I/O bytes read", io_read_bytes},
{2, io_write_ops_total, counter, "Total number of I/O write operations", io_write_count},
{2, io_write_bytes_total, counter, "Total number of I/O bytes written", io_write_bytes},
{2, io_sync_ops_total, counter, "Total number of I/O sync operations", io_sync_count},
{2, io_seek_ops_total, counter, "Total number of I/O seek operations", io_seek_count},
{2, io_open_attempt_ops_total, counter, "Total number of file open attempts", io_file_handle_open_attempt_count},
{2, io_reopen_ops_total, counter, "Total number of times files have been reopened", io_reopen_count},
{2, schema_db_ram_tx_total, counter, "Total number of Schema DB memory transactions", mnesia_ram_tx_count},
{2, schema_db_disk_tx_total, counter, "Total number of Schema DB disk transactions", mnesia_disk_tx_count},
{2, msg_store_read_total, counter, "Total number of Message Store read operations", msg_store_read_count},
{2, msg_store_write_total, counter, "Total number of Message Store write operations", msg_store_write_count},
{2, queue_index_read_ops_total, counter, "Total number of Queue Index read operations", queue_index_read_count},
{2, queue_index_write_ops_total, counter, "Total number of Queue Index write operations", queue_index_write_count},
{2, queue_index_journal_write_ops_total, counter, "Total number of Queue Index Journal write operations", queue_index_journal_write_count}
]},
{ra_metrics, [
{2, rabbitmq_raft_term, gauge, "Raft member term"},
{3, rabbitmq_raft_log_snapshot_index, gauge, "Raft log snapshot index"},
{4, rabbitmq_raft_log_last_applied_index, gauge, "Raft log last applied index"},
{5, rabbitmq_raft_log_commit_index, gauge, "Raft log commit index"},
{6, rabbitmq_raft_log_last_written_index, gauge, "Raft log last written index"},
{7, rabbitmq_raft_entry_commit_latency, gauge, "Time taken for an entry to be committed"}
{2, raft_term_total, counter, "Current Raft term number"},
{3, raft_log_snapshot_index, gauge, "Raft log snapshot index"},
{4, raft_log_last_applied_index, gauge, "Raft log last applied index"},
{5, raft_log_commit_index, gauge, "Raft log commit index"},
{6, raft_log_last_written_index, gauge, "Raft log last written index"},
{7, raft_entry_commit_latency, gauge, "Time taken for an entry to be committed"}
]},
{queue_coarse_metrics, [
{2, rabbitmq_queue_messages_ready, gauge, "Messages ready to be delivered to consumers"},
{3, rabbitmq_queue_messages_unacked, gauge, "Messages delivered to consumers but not yet acknowledged"},
{4, rabbitmq_queue_messages, gauge, "Sum of ready and unacknowledged messages - total queue depth"},
{5, rabbitmq_queue_process_reductions_total, counter, "Total number of queue process reductions"}
{2, queue_messages_ready, gauge, "Messages ready to be delivered to consumers"},
{3, queue_messages_unacked, gauge, "Messages delivered to consumers but not yet acknowledged"},
{4, queue_messages, gauge, "Sum of ready and unacknowledged messages - total queue depth"},
{5, queue_process_reductions_total, counter, "Total number of queue process reductions"}
]},
{queue_metrics, [
{2, rabbitmq_queue_consumers, gauge, "Consumers on a queue", queue_consumers},
{2, rabbitmq_queue_process_memory_bytes, gauge, "Memory in bytes used by the Erlang queue process", queue_memory},
{2, rabbitmq_queue_messages_bytes, gauge, "Size in bytes of ready and unacknowledged messages", queue_messages_bytes},
{2, rabbitmq_queue_messages_ram, gauge, "Ready and unacknowledged messages stored in memory", queue_messages_ram},
{2, rabbitmq_queue_messages_ready_ram, gauge, "Ready messages stored in memory", queue_messages_ready_ram},
{2, rabbitmq_queue_messages_ready_bytes, gauge, "Size in bytes of ready messages", queue_messages_bytes_ready},
{2, rabbitmq_queue_messages_unacked_ram, gauge, "Unacknowledged messages stored in memory", queue_messages_unacknowledged_ram},
{2, rabbitmq_queue_messages_unacked_bytes, gauge, "Size in bytes of all unacknowledged messages", queue_messages_bytes_unacknowledged},
{2, rabbitmq_queue_messages_persistent, gauge, "Persistent messages", queue_messages_persistent},
{2, rabbitmq_queue_messages_persistent_bytes, gauge, "Size in bytes of persistent messages", queue_messages_bytes_persistent},
{2, rabbitmq_queue_messages_paged_out, gauge, "Messages paged out to disk", queue_messages_paged_out},
{2, rabbitmq_queue_messages_paged_out_bytes, gauge, "Size in bytes of messages paged out to disk", queue_messages_bytes_paged_out},
{2, rabbitmq_queue_disk_reads_total, counter, "Total number of times queue read messages from disk", disk_reads},
{2, rabbitmq_queue_disk_writes_total, counter, "Total number of times queue wrote messages to disk", disk_writes}
{2, queue_consumers, gauge, "Consumers on a queue", queue_consumers},
{2, queue_process_memory_bytes, gauge, "Memory in bytes used by the Erlang queue process", queue_memory},
{2, queue_messages_bytes, gauge, "Size in bytes of ready and unacknowledged messages", queue_messages_bytes},
{2, queue_messages_ram, gauge, "Ready and unacknowledged messages stored in memory", queue_messages_ram},
{2, queue_messages_ready_ram, gauge, "Ready messages stored in memory", queue_messages_ready_ram},
{2, queue_messages_ready_bytes, gauge, "Size in bytes of ready messages", queue_messages_bytes_ready},
{2, queue_messages_unacked_ram, gauge, "Unacknowledged messages stored in memory", queue_messages_unacknowledged_ram},
{2, queue_messages_unacked_bytes, gauge, "Size in bytes of all unacknowledged messages", queue_messages_bytes_unacknowledged},
{2, queue_messages_persistent, gauge, "Persistent messages", queue_messages_persistent},
{2, queue_messages_persistent_bytes, gauge, "Size in bytes of persistent messages", queue_messages_bytes_persistent},
{2, queue_messages_paged_out, gauge, "Messages paged out to disk", queue_messages_paged_out},
{2, queue_messages_paged_out_bytes, gauge, "Size in bytes of messages paged out to disk", queue_messages_bytes_paged_out},
{2, queue_disk_reads_total, counter, "Total number of times queue read messages from disk", disk_reads},
{2, queue_disk_writes_total, counter, "Total number of times queue wrote messages to disk", disk_writes}
]}
]).
@ -179,11 +193,11 @@
]},
{node_persister_metrics, [
{2, 1000000, rabbitmq_io_read_time_seconds_total, counter, "Total I/O read time", io_read_time},
{2, 1000000, rabbitmq_io_write_time_seconds_total, counter, "Total I/O write time", io_write_time},
{2, 1000000, rabbitmq_io_sync_time_seconds_total, counter, "Total I/O sync time", io_sync_time},
{2, 1000000, rabbitmq_io_seek_time_seconds_total, counter, "Total I/O seek time", io_seek_time},
{2, 1000000, rabbitmq_io_open_attempt_time_seconds_total, counter, "Total file open attempts time", io_file_handle_open_attempt_time}
{2, 1000000, io_read_time_seconds_total, counter, "Total I/O read time", io_read_time},
{2, 1000000, io_write_time_seconds_total, counter, "Total I/O write time", io_write_time},
{2, 1000000, io_sync_time_seconds_total, counter, "Total I/O sync time", io_sync_time},
{2, 1000000, io_seek_time_seconds_total, counter, "Total I/O seek time", io_seek_time},
{2, 1000000, io_open_attempt_time_seconds_total, counter, "Total file open attempts time", io_file_handle_open_attempt_time}
]}
]).
@ -191,10 +205,10 @@
-define(TOTALS, [
%% ordering differs from metrics above, refer to list comprehension
{connection_created, rabbitmq_connections, gauge, "Connections currently open"},
{channel_created, rabbitmq_channels, gauge, "Channels currently open"},
{consumer_created, rabbitmq_consumers, gauge, "Consumers currently connected"},
{queue_metrics, rabbitmq_queues, gauge, "Queues available"}
{connection_created, connections, gauge, "Connections currently open"},
{channel_created, channels, gauge, "Channels currently open"},
{consumer_created, consumers, gauge, "Consumers currently connected"},
{queue_metrics, queues, gauge, "Queues available"}
]).
%%====================================================================

View File

@ -157,7 +157,7 @@ metrics_test(Config) ->
?assertEqual(match, re:run(Body, ?config(queue_name, Config), [{capture, none}])),
%% Checking the first metric from each ETS table owned by rabbitmq_metrics
?assertEqual(match, re:run(Body, "rabbitmq_channel_consumers ", [{capture, none}])),
?assertEqual(match, re:run(Body, "rabbitmq_channel_messages_published ", [{capture, none}])),
?assertEqual(match, re:run(Body, "rabbitmq_channel_messages_published_total ", [{capture, none}])),
?assertEqual(match, re:run(Body, "rabbitmq_channel_process_reductions_total ", [{capture, none}])),
?assertEqual(match, re:run(Body, "rabbitmq_channel_get_ack_total ", [{capture, none}])),
?assertEqual(match, re:run(Body, "rabbitmq_connections_opened_total ", [{capture, none}])),
@ -167,7 +167,7 @@ metrics_test(Config) ->
?assertEqual(match, re:run(Body, "process_open_fds ", [{capture, none}])),
?assertEqual(match, re:run(Body, "process_max_fds ", [{capture, none}])),
?assertEqual(match, re:run(Body, "rabbitmq_io_read_ops_total ", [{capture, none}])),
?assertEqual(match, re:run(Body, "rabbitmq_raft_term ", [{capture, none}])),
?assertEqual(match, re:run(Body, "rabbitmq_raft_term_total ", [{capture, none}])),
?assertEqual(match, re:run(Body, "rabbitmq_queue_messages_ready ", [{capture, none}])),
?assertEqual(match, re:run(Body, "rabbitmq_queue_consumers ", [{capture, none}])),
%% Checking the first metric in each ETS table that requires converting