diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json index 59c9de66adb7..efb482745045 100644 --- a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json +++ b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json @@ -2,24 +2,42 @@ "query/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to complete a query."}, "query/bytes" : { "dimensions" : ["dataSource", "type"], "type" : "count", "help": "Number of bytes returned in query response."}, "query/node/time" : { "dimensions" : ["server"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to query individual historical/realtime processes."}, - "query/node/ttfb" : { "dimensions" : ["server"], "type" : "timer", "conversionFactor": 1000.0, "help": "Time to first byte. Seconds elapsed until Broker starts receiving the response from individual historical/realtime processes."}, "query/node/bytes" : { "dimensions" : ["server"], "type" : "count", "help": "Number of bytes returned from querying individual historical/realtime processes."}, - "query/node/backpressure": { "dimensions" : ["server"], "type" : "timer", "help": "Seconds that the channel to this process has spent suspended due to backpressure."}, - "query/intervalChunk/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Only emitted if interval chunking is enabled. Milliseconds required to query an interval chunk. This metric is deprecated and will be removed in the future because interval chunking is deprecated."}, - - "query/segment/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to query individual segment. Includes time to page in the segment from disk."}, - "query/wait/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds spent waiting for a segment to be scanned."}, - "segment/scan/pending" : { "dimensions" : [], "type" : "gauge", "help": "Number of segments in queue waiting to be scanned."}, - "segment/scan/active" : { "dimensions" : [], "type" : "gauge", "help": "Number of segments currently scanned."}, - "query/segmentAndCache/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to query individual segment or hit the cache (if it is enabled on the Historical process)."}, - "query/cpu/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer", "conversionFactor": "1000000", "help": "Seconds of CPU time taken to complete a query"}, - + "query/node/ttfb" : { "dimensions" : ["server"], "type" : "timer", "conversionFactor": 1000.0, "help": "Time to first byte. Seconds elapsed until Broker starts receiving the response from individual historical/realtime processes."}, + "query/node/backpressure": { "dimensions" : ["server"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds that the channel to this process has spent suspended due to backpressure."}, "query/count" : { "dimensions" : [], "type" : "count", "help": "Number of total queries" }, "query/success/count" : { "dimensions" : [], "type" : "count", "help": "Number of queries successfully processed"}, "query/failed/count" : { "dimensions" : [], "type" : "count", "help": "Number of failed queries"}, "query/interrupted/count" : { "dimensions" : [], "type" : "count", "help": "Number of queries interrupted due to cancellation."}, "query/timeout/count" : { "dimensions" : [], "type" : "count", "help": "Number of timed out queries."}, "mergeBuffer/pendingRequests" : { "dimensions" : [], "type" : "count", "help": "Number of requests waiting to acquire a batch of buffers from the merge buffer pool."}, + "sqlQuery/time" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to complete a SQL query."}, + "sqlQuery/planningTimeMs" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to plan a SQL to native query"}, + "sqlQuery/bytes" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of bytes returned in the SQL query response."}, + "serverview/init/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Time taken to initialize the server view."}, + "metadatacache/init/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Time taken to initialize the segment metadata cache."}, + "metadatacache/refresh/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of segments to refresh in broker segment metadata cache."}, + "metadatacache/refresh/time" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000.0, "help": "Time taken to refresh segments in broker segment metadata cache."}, + "metadatacache/schemaPoll/count" : { "dimensions" : [], "type" : "count", "help": "Number of coordinator polls to fetch datasource schema."}, + "metadatacache/schemaPoll/failed" : { "dimensions" : [], "type" : "count", "help": "Number of failed coordinator polls to fetch datasource schema."}, + "metadatacache/schemaPoll/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Time taken for coordinator polls to fetch datasource schema."}, + "serverview/sync/healthy" : { "dimensions" : ["server"], "type" : "gauge", "help": "Sync status of the Broker with a segment-loading server such as a Historical or Peon."}, + "serverview/sync/unstableTime" : { "dimensions" : ["server"], "type" : "timer", "conversionFactor": 1000.0, "help": "Time in seconds for which the Broker has been failing to sync with a segment-loading server."}, + "query/segment/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to query individual segment. Includes time to page in the segment from disk."}, + "query/wait/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds spent waiting for a segment to be scanned."}, + "segment/scan/pending" : { "dimensions" : [], "type" : "gauge", "help": "Number of segments in queue waiting to be scanned."}, + "segment/scan/active" : { "dimensions" : [], "type" : "gauge", "help": "Number of segments currently scanned."}, + "query/segmentAndCache/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to query individual segment or hit the cache (if it is enabled on the Historical process)."}, + "query/cpu/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer", "conversionFactor": 1000000.0, "help": "Seconds of CPU time taken to complete a query"}, + + "jetty/numOpenConnections" : { "dimensions" : [], "type" : "gauge", "help": "Number of open jetty connections."}, + "jetty/threadPool/total" : { "dimensions" : [], "type" : "gauge", "help": "Number of total workable threads allocated."}, + "jetty/threadPool/idle" : { "dimensions" : [], "type" : "gauge", "help": "Number of idle threads."}, + "jetty/threadPool/busy" : { "dimensions" : [], "type" : "gauge", "help": "Number of busy threads that has work to do from the worker queue."}, + "jetty/threadPool/isLowOnThreads" : { "dimensions" : [], "type" : "gauge", "help": "A rough indicator of whether number of total workable threads allocated is enough to handle the works in the work queue."}, + "jetty/threadPool/min" : { "dimensions" : [], "type" : "gauge", "help": "Number of minimum threads allocatable."}, + "jetty/threadPool/max" : { "dimensions" : [], "type" : "gauge", "help": "Number of maximum threads allocatable."}, + "jetty/threadPool/queueSize" : { "dimensions" : [], "type" : "gauge", "help": "Size of the worker queue."}, "query/cache/delta/numEntries" : { "dimensions" : [], "type" : "gauge", "help": "Number of entries in cache"}, "query/cache/delta/sizeBytes" : { "dimensions" : [], "type" : "gauge", "help": "Size of cache in bytes."}, @@ -30,6 +48,9 @@ "query/cache/delta/averageBytes" : { "dimensions" : [], "type" : "gauge", "help": "Average size of record in bytes"}, "query/cache/delta/timeouts" : { "dimensions" : [], "type" : "gauge", "help": "Number of cache timeouts"}, "query/cache/delta/errors" : { "dimensions" : [], "type" : "gauge", "help": "Number of cache errors."}, + "query/cache/delta/put/ok" : { "dimensions" : [], "type" : "gauge", "help": "Number of new cache entries successfully cached."}, + "query/cache/delta/put/error" : { "dimensions" : [], "type" : "gauge", "help": "Number of new cache entries that could not be cached due to errors."}, + "query/cache/delta/put/oversized" : { "dimensions" : [], "type" : "gauge", "help": "Number of potential new cache entries that were skipped due to being too large (based on druid.{broker,historical,realtime}.cache.maxEntrySize properties)."}, "query/cache/total/numEntries" : { "dimensions" : [], "type" : "gauge","help": "Total number of entries in cache" }, "query/cache/total/sizeBytes" : { "dimensions" : [], "type" : "gauge", "help": "Total size of cache in bytes."}, @@ -40,14 +61,27 @@ "query/cache/total/averageBytes" : { "dimensions" : [], "type" : "gauge", "help": "Total average record size in bytes"}, "query/cache/total/timeouts" : { "dimensions" : [], "type" : "gauge", "help": "Total number of cache timeouts"}, "query/cache/total/errors" : { "dimensions" : [], "type" : "gauge", "help": "Total number of cache errors" }, + "query/cache/total/put/ok" : { "dimensions" : [], "type" : "gauge", "help": "Total number of new cache entries successfully cached."}, + "query/cache/total/put/error" : { "dimensions" : [], "type" : "gauge", "help": "Total number of new cache entries that could not be cached due to errors."}, + "query/cache/total/put/oversized" : { "dimensions" : [], "type" : "gauge", "help": "Total number of potential new cache entries that were skipped due to being too large (based on druid.{broker,historical,realtime}.cache.maxEntrySize properties)."}, - "ingest/events/thrownAway" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because they are outside the windowPeriod."}, + "ingest/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Count of 1 every time an ingestion job runs (includes compaction jobs). Aggregate using dimensions." }, + "ingest/segments/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Count of final segments created by job (includes tombstones)." }, + "ingest/tombstones/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Count of final segments created by job (includes tombstones)." }, + + "ingest/kafka/lag" : { "dimensions" : ["dataSource", "stream"], "type" : "gauge", "help": "Total lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute."}, + "ingest/kafka/maxLag" : { "dimensions" : ["dataSource", "stream"], "type" : "gauge", "help": "Max lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute."}, + "ingest/kafka/avgLag" : { "dimensions" : ["dataSource", "stream"], "type" : "gauge", "help": "Average lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute."}, + "ingest/kafka/partitionLag" : { "dimensions" : ["dataSource", "stream", "partition"], "type" : "gauge", "help": "Partition-wise lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers. Minimum emission period for this metric is a minute."}, + + "compact/segmentAnalyzer/fetchAndProcessMillis" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000.0, "help": "Time taken to fetch and process segments to infer the schema for the compaction task to run." }, + + "ingest/events/processed" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events successfully processed per emission period." }, + "ingest/events/processedWithError" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events processed with some partial errors per emission period." }, "ingest/events/unparseable" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because the events are unparseable." }, + "ingest/events/thrownAway" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because they are outside the windowPeriod."}, "ingest/events/duplicate" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because the events are duplicated."}, - "ingest/events/processed" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events successfully processed per emission period." }, - "ingest/events/processedWithError" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events processed with some partial errors per emission period" }, - "ingest/events/messageGap" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Time gap in milliseconds between the latest ingested event timestamp and the current system timestamp of metrics emission."}, - "ingest/notices/queueSize" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Number of pending notices to be processed by the coordinator."}, + "ingest/input/bytes" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of bytes read from input sources, after decompression but prior to parsing." }, "ingest/rows/output" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of Druid rows persisted."}, "ingest/persists/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of times persist occurred." }, "ingest/persists/time" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds spent doing intermediate persist."}, @@ -57,40 +91,84 @@ "ingest/handoff/failed" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of handoffs that failed." }, "ingest/merge/time" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds spent merging intermediate segments" }, "ingest/merge/cpu" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000000000.0, "help": "Cpu time in Seconds spent on merging intermediate segments."}, + "ingest/handoff/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of handoffs that happened." }, + "ingest/sink/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of sinks not handed off." }, + "ingest/events/messageGap" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Time gap in milliseconds between the latest ingested event timestamp and the current system timestamp of metrics emission."}, + "ingest/notices/queueSize" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Number of pending notices to be processed by the coordinator."}, + "ingest/notices/time" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to process a notice by the supervisor." }, + "ingest/pause/time" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds spent by a task in a paused state without ingesting." }, + "ingest/handoff/time" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000.0, "help": "Total number of seconds taken to handoff a set of segments." }, - "ingest/kafka/lag" : { "dimensions" : ["dataSource", "stream"], "type" : "gauge", "help": "Total lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute."}, - "ingest/kafka/maxLag" : { "dimensions" : ["dataSource", "stream"], "type" : "gauge", "help": "Max lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute."}, - "ingest/kafka/avgLag" : { "dimensions" : ["dataSource", "stream"], "type" : "gauge", "help": "Average lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute."}, - "ingest/kafka/partitionLag" : { "dimensions" : ["dataSource", "stream", "partition"], "type" : "gauge", "help": "Partition-wise lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers. Minimum emission period for this metric is a minute."}, - - "task/success/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of successful tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included."}, - "task/failed/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of failed tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included."}, - "task/running/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of current running tasks. This metric is only available if the TaskCountStatsMonitor module is included."}, - "task/pending/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of current pending tasks. This metric is only available if the TaskCountStatsMonitor module is included."}, - "task/waiting/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of current waiting tasks. This metric is only available if the TaskCountStatsMonitor module is included."}, - "task/run/time" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer", "conversionFactor": 1000.0, "help": "Milliseconds taken to run a task."}, - + "task/run/time" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to run a task."}, + "task/pending/time" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken for a task to wait for running."}, + "task/action/log/time" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to log a task action to the audit log."}, + "task/action/run/time" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to execute a task action."}, + "task/action/success/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of task actions that were executed successfully during the emission period."}, + "task/action/failed/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of task actions that failed during the emission period."}, + "task/action/batch/queueTime" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds spent by a batch of task actions in queue."}, + "task/action/batch/runTime" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to execute a batch of task actions."}, + "task/action/batch/size" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of task actions in a batch that was executed during the emission period."}, + "task/action/batch/attempts" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of execution attempts for a single batch of task actions."}, + "task/segmentAvailability/wait/time" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer", "conversionFactor": 1000.0, "help": "The amount of seconds a batch indexing task waited for newly created segments to become available for querying."}, "segment/added/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count", "help": "Size in bytes of new segments created." }, "segment/moved/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count", "help": "Size in bytes of segments moved/archived via the Move Task." }, "segment/nuked/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count", "help": "Size in bytes of segments deleted via the Kill Task." }, + "task/success/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of successful tasks per emission period."}, + "task/failed/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of failed tasks per emission period."}, + "task/running/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of current running tasks."}, + "task/pending/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of current pending tasks."}, + "task/waiting/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of current waiting tasks."}, "segment/assigned/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments assigned to be loaded in the cluster."}, "segment/moved/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments moved in the cluster." }, "segment/dropped/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments dropped due to being overshadowed." }, "segment/deleted/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments dropped due to rules."}, "segment/unneeded/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments dropped due to being marked as unused."}, - "segment/unavailable/count" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Number of segments (not including replicas) left to load until segments that should be loaded in the cluster are available for queries."}, - "segment/underReplicated/count" : { "dimensions" : ["dataSource", "tier"], "type" : "gauge", "help": "Number of segments (including replicas) left to load until segments that should be loaded in the cluster are available for queries."}, - "segment/cost/raw" : { "dimensions" : ["tier"], "type" : "count", "help": "Used in cost balancing. The raw cost of hosting segments."}, - "segment/cost/normalization" : { "dimensions" : ["tier"], "type" : "count", "help": "Used in cost balancing. The normalization of hosting segments."}, - "segment/cost/normalized" : { "dimensions" : ["tier"], "type" : "count", "help": "Used in cost balancing. The normalized cost of hosting segments."}, + "segment/assignSkipped/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments that could not be assigned to any server for loading."}, + "segment/moveSkipped/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments that were chosen for balancing but could not be moved."}, + "segment/dropSkipped/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments that could not be dropped from any server."}, "segment/loadQueue/size" : { "dimensions" : ["server"], "type" : "gauge", "help": "Size in bytes of segments to load."}, - "segment/loadQueue/failed" : { "dimensions" : ["server"], "type" : "gauge", "help": "Number of segments that failed to load."}, "segment/loadQueue/count" : { "dimensions" : ["server"], "type" : "gauge", "help": "Number of segments to load."}, "segment/dropQueue/count" : { "dimensions" : ["server"], "type" : "gauge", "help": "Number of segments to drop."}, + "segment/loadQueue/assigned" : { "dimensions" : ["server"], "type" : "gauge", "help": "Number of segments assigned for load or drop to the load queue of a server."}, + "segment/loadQueue/success" : { "dimensions" : ["server"], "type" : "gauge", "help": "Number of segment assignments that completed successfully."}, + "segment/loadQueue/failed" : { "dimensions" : ["server"], "type" : "gauge", "help": "Number of segment assignments that failed to complete."}, + "segment/loadQueue/cancelled" : { "dimensions" : ["server"], "type" : "gauge", "help": "Number of segment assignments that were canceled before completion."}, "segment/size" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Size in bytes of available segments."}, "segment/count" : { "dimensions" : ["dataSource"], "type" : "gauge", "help" : "Number of used segments belonging to a data source." }, "segment/overShadowed/count" : { "dimensions" : [], "type" : "gauge", "help": "Number of overShadowed segments."}, + "segment/unneededEternityTombstone/count" : { "dimensions" : [], "type" : "gauge", "help": "Number of non-overshadowed eternity tombstones marked as unused."}, + "segment/unavailable/count" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Number of segments (not including replicas) left to load until segments that should be loaded in the cluster are available for queries."}, + "segment/underReplicated/count" : { "dimensions" : ["dataSource", "tier"], "type" : "gauge", "help": "Number of segments (including replicas) left to load until segments that should be loaded in the cluster are available for queries."}, + "tier/historical/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of available historical nodes in each tier."}, + "tier/replication/factor" : { "dimensions" : ["tier"], "type" : "count", "help": "Configured maximum replication factor in each tier."}, + "tier/required/capacity" : { "dimensions" : ["tier"], "type" : "count", "help": "Total capacity in bytes required in each tier."}, + "tier/total/capacity" : { "dimensions" : ["tier"], "type" : "count", "help": "Total capacity in bytes available in each tier."}, + "compact/task/count" : { "dimensions" : [], "type" : "gauge", "help": "Number of tasks issued in the auto compaction run."}, + "compactTask/maxSlot/count" : { "dimensions" : [], "type" : "gauge", "help": "Maximum number of task slots available for auto compaction tasks in the auto compaction run."}, + "compactTask/availableSlot/count" : { "dimensions" : [], "type" : "gauge", "help": "Number of available task slots that can be used for auto compaction tasks in the auto compaction run."}, + "killTask/availableSlot/count" : { "dimensions" : [], "type" : "gauge", "help": "Number of available task slots that can be used for auto kill tasks in the auto kill run."}, + "killTask/maxSlot/count" : { "dimensions" : [], "type" : "gauge", "help": "Maximum number of task slots available for auto kill tasks in the auto kill run."}, + "kill/task/count" : { "dimensions" : [], "type" : "gauge", "help": "Number of tasks issued in the auto kill run."}, + "kill/pendingSegments/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of stale pending segments deleted from the metadata store."}, + "segment/waitCompact/bytes" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Total bytes of this datasource waiting to be compacted by the auto compaction (only consider intervals/segments that are eligible for auto compaction)."}, + "segment/waitCompact/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Total number of segments of this datasource waiting to be compacted by the auto compaction (only consider intervals/segments that are eligible for auto compaction)."}, + "interval/waitCompact/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Total number of intervals of this datasource waiting to be compacted by the auto compaction (only consider intervals/segments that are eligible for auto compaction)."}, + "segment/compacted/bytes" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Total bytes of this datasource that are already compacted with the spec set in the auto compaction config."}, + "segment/compacted/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Total number of segments of this datasource that are already compacted with the spec set in the auto compaction config."}, + "interval/compacted/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Total number of intervals of this datasource that are already compacted with the spec set in the auto compaction config."}, + "segment/skipCompact/bytes" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Total bytes of this datasource that are skipped (not eligible for auto compaction) by the auto compaction."}, + "segment/skipCompact/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Total number of segments of this datasource that are skipped (not eligible for auto compaction) by the auto compaction."}, + "interval/skipCompact/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Total number of intervals of this datasource that are skipped (not eligible for auto compaction) by the auto compaction."}, + "coordinator/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Approximate Coordinator duty runtime in seconds." }, + "coordinator/global/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Approximate runtime of a full coordination cycle in seconds." }, + "metadata/kill/supervisor/count" : { "dimensions" : [], "type" : "count", "help": "Total number of terminated supervisors that were automatically deleted from metadata store per each Coordinator kill supervisor duty run."}, + "metadata/kill/audit/count" : { "dimensions" : [], "type" : "count", "help": "Total number of audit logs that were automatically deleted from metadata store per each Coordinator kill audit duty run."}, + "metadata/kill/compaction/count" : { "dimensions" : [], "type" : "count", "help": "Total number of compaction configurations that were automatically deleted from metadata store per each Coordinator kill compaction configuration duty run."}, + "metadata/kill/rule/count" : { "dimensions" : [], "type" : "count", "help": "Total number of rules that were automatically deleted from metadata store per each Coordinator kill rule duty run."}, + "metadata/kill/datasource/count" : { "dimensions" : [], "type" : "count", "help": "Total number of datasource metadata that were automatically deleted from metadata store per each Coordinator kill datasource duty run."}, + + "service/heartbeat" : { "dimensions" : ["dataSource", "type"], "type" : "gauge", "help": "Metric indicating the service is up. This metric is only available if the ServiceStatusMonitor module is included."}, "segment/max" : { "dimensions" : [], "type" : "gauge", "help": "Maximum byte limit available for segments."}, "segment/used" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge", "help": "Bytes used for served segments."}, @@ -111,28 +189,50 @@ "jvm/gc/count" : { "dimensions" : ["gcName"], "type" : "count", "help": "Garbage collection count"}, "jvm/gc/cpu" : { "dimensions" : ["gcName"], "type" : "count", "help": "Count of CPU time in Nanoseconds spent on garbage collection. Note: `jvm/gc/cpu` represents the total time over multiple GC cycles; divide by `jvm/gc/count` to get the mean GC time per cycle."}, - "ingest/events/buffered" : { "dimensions" : ["serviceName", "bufferCapacity"], "type" : "gauge", "help": "Number of events queued in the EventReceiverFirehose's buffer"}, + "zk/connected" : { "dimensions" : [], "type" : "gauge", "help": "Indicator of connection status to zookeeper."}, + "zk/reconnect/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Amount of time, in seconds, that a server was disconnected from ZooKeeper before reconnecting." }, + + "ingest/events/buffered" : { "dimensions" : ["serviceName", "dataSource", "bufferCapacity"], "type" : "gauge", "help": "Number of events queued in the EventReceiverFirehose's buffer"}, + "ingest/bytes/received" : { "dimensions" : ["serviceName", "dataSource"], "type" : "gauge", "help": "Number of bytes received by the EventReceiverFirehose."}, "sys/swap/free" : { "dimensions" : [], "type" : "gauge", "help": "Free swap"}, "sys/swap/max" : { "dimensions" : [], "type" : "gauge", "help": "Max swap"}, "sys/swap/pageIn" : { "dimensions" : [], "type" : "gauge", "help": "Paged in swap"}, "sys/swap/pageOut" : { "dimensions" : [], "type" : "gauge", "help": "Paged out swap"}, - "sys/disk/write/count" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Writes to disk."}, - "sys/disk/read/count" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Reads from disk."}, - "sys/disk/write/size" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Bytes written to disk. Can we used to determine how much paging is occurring with regards to segments."}, - "sys/disk/read/size" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Bytes read from disk. Can we used to determine how much paging is occurring with regards to segments."}, + "sys/disk/write/count" : { "dimensions" : ["diskName"], "type" : "count", "help": "Writes to disk."}, + "sys/disk/read/count" : { "dimensions" : ["diskName"], "type" : "count", "help": "Reads from disk."}, + "sys/disk/write/size" : { "dimensions" : ["diskName"], "type" : "count", "help": "Bytes written to disk. Can we used to determine how much paging is occurring with regards to segments."}, + "sys/disk/read/size" : { "dimensions" : ["diskName"], "type" : "count", "help": "Bytes read from disk. Can we used to determine how much paging is occurring with regards to segments."}, + "sys/disk/queue" : { "dimensions" : ["diskName"], "type" : "gauge", "help": "Disk queue length."}, + "sys/disk/transferTime" : { "dimensions" : ["diskName"], "type" : "timer", "help": "Transfer time to read from or write to disk"}, "sys/net/write/size" : { "dimensions" : [], "type" : "count", "help": "Bytes written to the network."}, "sys/net/read/size" : { "dimensions" : [], "type" : "count", "help": "Bytes read from the network."}, - "sys/fs/used" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge", "help": "Filesystem bytes used."}, - "sys/fs/max" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge", "help": "Filesystesm bytes max."}, + "sys/net/read/packets" : { "dimensions" : [], "type" : "count", "help": "Total packets read from the network"}, + "sys/net/write/packets" : { "dimensions" : [], "type" : "count", "help": "Total packets written to the network"}, + "sys/net/read/errors" : { "dimensions" : [], "type" : "gauge", "help": "Total network read errors"}, + "sys/net/write/errors" : { "dimensions" : [], "type" : "gauge", "help": "Total network write errors"}, + "sys/net/read/dropped" : { "dimensions" : [], "type" : "gauge", "help": "Total packets dropped coming from network"}, + "sys/net/write/collisions" : { "dimensions" : [], "type" : "gauge", "help": "Total network write collisions"}, + "sys/fs/used" : { "dimensions" : ["fsDevName", "fsDirName"], "type" : "gauge", "help": "Filesystem bytes used."}, + "sys/fs/max" : { "dimensions" : ["fsDevName", "fsDirName"], "type" : "gauge", "help": "Filesystem bytes max."}, + "sys/fs/files/count" : { "dimensions" : ["fsDevName", "fsDirName"], "type" : "gauge", "help": "Filesystem total IO nodes"}, + "sys/fs/files/free" : { "dimensions" : ["fsDevName", "fsDirName"], "type" : "gauge", "help": "Filesystem free IO nodes"}, "sys/mem/used" : { "dimensions" : [], "type" : "gauge", "help": "Memory used."}, "sys/mem/max" : { "dimensions" : [], "type" : "gauge", "help": "Memory max"}, + "sys/mem/free" : { "dimensions" : [], "type" : "gauge", "help": "Memory free"}, "sys/storage/used" : { "dimensions" : ["fsDirName"], "type" : "gauge", "help": "Disk space used."}, "sys/cpu" : { "dimensions" : ["cpuName", "cpuTime"], "type" : "gauge", "help": "CPU used"}, + "sys/uptime" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Total system uptime"}, + "sys/tcpv4/activeOpens" : { "dimensions" : [], "type" : "gauge", "help": "Total TCP active open connections"}, + "sys/tcpv4/passiveOpens" : { "dimensions" : [], "type" : "gauge", "help": "Total TCP passive open connections"}, + "sys/tcpv4/attemptFails" : { "dimensions" : [], "type" : "gauge", "help": "Total TCP active connection failures"}, + "sys/tcpv4/estabResets" : { "dimensions" : [], "type" : "gauge", "help": "Total TCP connection resets"}, + "sys/tcpv4/in/segs" : { "dimensions" : [], "type" : "gauge", "help": "Total segments received in connection"}, + "sys/tcpv4/in/errs" : { "dimensions" : [], "type" : "gauge", "help": "Errors while reading segments"}, + "sys/tcpv4/out/segs" : { "dimensions" : [], "type" : "gauge", "help": "Total segments sent"}, + "sys/tcpv4/out/rsts" : { "dimensions" : [], "type" : "gauge", "help": "Total out reset packets sent to reset the connection"}, + "sys/tcpv4/retrans/segs" : { "dimensions" : [], "type" : "gauge", "help": "Total segments re-transmitted"}, "coordinator_segment/count" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Number of served segments."}, - "historical_segment/count" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge", "help": "Number of served segments."}, - - "service/heartbeat" : { "dimensions" : ["dataSource", "type"], "type" : "gauge", "help": "Metric indicating the service is up. This metric is only available if the ServiceStatusMonitor module is included."}, - "zk/connected" : { "dimensions" : [], "type" : "gauge", "help": "Indicator of connection status to zookeeper."} + "historical_segment/count" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge", "help": "Number of served segments."} }