mirror of
https://github.com/apache/druid.git
synced 2025-02-16 23:15:16 +00:00
* fix duplicate dimension (#12778)
This commit is contained in:
parent
d8f8c56f94
commit
c0380e7b0a
@ -220,31 +220,31 @@ Note: If the JVM does not support CPU time measurement for the current thread, i
|
||||
|
||||
## Indexing service
|
||||
|
||||
|Metric|Description|Dimensions|Normal Value|
|
||||
|------|-----------|----------|------------|
|
||||
|`task/run/time`|Milliseconds taken to run a task.|dataSource, taskId, taskType, taskStatus.|Varies.|
|
||||
|`task/pending/time`|Milliseconds taken for a task to wait for running.|dataSource, taskId, taskType.|Varies.|
|
||||
|`task/action/log/time`|Milliseconds taken to log a task action to the audit log.|dataSource, taskId, taskType|< 1000 (subsecond)|
|
||||
|`task/action/run/time`|Milliseconds taken to execute a task action.|dataSource, taskId, taskType|Varies from subsecond to a few seconds, based on action type.|
|
||||
|`segment/added/bytes`|Size in bytes of new segments created.|dataSource, taskId, taskType, interval.|Varies.|
|
||||
|`segment/moved/bytes`|Size in bytes of segments moved/archived via the Move Task.|dataSource, taskId, taskType, interval.|Varies.|
|
||||
|`segment/nuked/bytes`|Size in bytes of segments deleted via the Kill Task.|dataSource, taskId, taskType, interval.|Varies.|
|
||||
|`task/success/count`|Number of successful tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included.|dataSource.|Varies.|
|
||||
|`task/failed/count`|Number of failed tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included.|dataSource.|Varies.|
|
||||
|`task/running/count`|Number of current running tasks. This metric is only available if the TaskCountStatsMonitor module is included.|dataSource.|Varies.|
|
||||
|`task/pending/count`|Number of current pending tasks. This metric is only available if the TaskCountStatsMonitor module is included.|dataSource.|Varies.|
|
||||
|`task/waiting/count`|Number of current waiting tasks. This metric is only available if the TaskCountStatsMonitor module is included.|dataSource.|Varies.|
|
||||
|`taskSlot/total/count`|Number of total task slots per emission period. This metric is only available if the TaskSlotCountStatsMonitor module is included.|category.|Varies.|
|
||||
|`taskSlot/idle/count`|Number of idle task slots per emission period. This metric is only available if the TaskSlotCountStatsMonitor module is included.|category.|Varies.|
|
||||
|`taskSlot/used/count`|Number of busy task slots per emission period. This metric is only available if the TaskSlotCountStatsMonitor module is included.|category.|Varies.|
|
||||
|`taskSlot/lazy/count`|Number of total task slots in lazy marked MiddleManagers and Indexers per emission period. This metric is only available if the TaskSlotCountStatsMonitor module is included.|category.|Varies.|
|
||||
|`taskSlot/blacklisted/count`|Number of total task slots in blacklisted MiddleManagers and Indexers per emission period. This metric is only available if the TaskSlotCountStatsMonitor module is included.|category.|Varies.|
|
||||
|`task/segmentAvailability/wait/time`|The amount of milliseconds a batch indexing task waited for newly created segments to become available for querying.|dataSource, taskType, taskId, segmentAvailabilityConfirmed|Varies.|
|
||||
|`worker/task/failed/count`|Number of failed tasks run on the reporting worker per emission period. This metric is only available if the WorkerTaskCountStatsMonitor module is included, and is only supported for middleManager nodes.|category, version.|Varies.|
|
||||
|`worker/task/success/count`|Number of successful tasks run on the reporting worker per emission period. This metric is only available if the WorkerTaskCountStatsMonitor module is included, and is only supported for middleManager nodes.|category, version.|Varies.|
|
||||
|`worker/taskSlot/idle/count`|Number of idle task slots on the reporting worker per emission period. This metric is only available if the WorkerTaskCountStatsMonitor module is included, and is only supported for middleManager nodes.|category, version.|Varies.|
|
||||
|`worker/taskSlot/total/count`|Number of total task slots on the reporting worker per emission period. This metric is only available if the WorkerTaskCountStatsMonitor module is included.|category, version.|Varies.|
|
||||
|`worker/taskSlot/used/count`|Number of busy task slots on the reporting worker per emission period. This metric is only available if the WorkerTaskCountStatsMonitor module is included.|category, version.|Varies.|
|
||||
|Metric|Description| Dimensions |Normal Value|
|
||||
|------|-----------|------------------------------------------------------------|------------|
|
||||
|`task/run/time`|Milliseconds taken to run a task.| dataSource, taskId, taskType, taskStatus. |Varies.|
|
||||
|`task/pending/time`|Milliseconds taken for a task to wait for running.| dataSource, taskId, taskType. |Varies.|
|
||||
|`task/action/log/time`|Milliseconds taken to log a task action to the audit log.| dataSource, taskId, taskType |< 1000 (subsecond)|
|
||||
|`task/action/run/time`|Milliseconds taken to execute a task action.| dataSource, taskId, taskType |Varies from subsecond to a few seconds, based on action type.|
|
||||
|`segment/added/bytes`|Size in bytes of new segments created.| dataSource, taskId, taskType, interval. |Varies.|
|
||||
|`segment/moved/bytes`|Size in bytes of segments moved/archived via the Move Task.| dataSource, taskId, taskType, interval. |Varies.|
|
||||
|`segment/nuked/bytes`|Size in bytes of segments deleted via the Kill Task.| dataSource, taskId, taskType, interval. |Varies.|
|
||||
|`task/success/count`|Number of successful tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included.| dataSource. |Varies.|
|
||||
|`task/failed/count`|Number of failed tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included.| dataSource. |Varies.|
|
||||
|`task/running/count`|Number of current running tasks. This metric is only available if the TaskCountStatsMonitor module is included.| dataSource. |Varies.|
|
||||
|`task/pending/count`|Number of current pending tasks. This metric is only available if the TaskCountStatsMonitor module is included.| dataSource. |Varies.|
|
||||
|`task/waiting/count`|Number of current waiting tasks. This metric is only available if the TaskCountStatsMonitor module is included.| dataSource. |Varies.|
|
||||
|`taskSlot/total/count`|Number of total task slots per emission period. This metric is only available if the TaskSlotCountStatsMonitor module is included.| category. |Varies.|
|
||||
|`taskSlot/idle/count`|Number of idle task slots per emission period. This metric is only available if the TaskSlotCountStatsMonitor module is included.| category. |Varies.|
|
||||
|`taskSlot/used/count`|Number of busy task slots per emission period. This metric is only available if the TaskSlotCountStatsMonitor module is included.| category. |Varies.|
|
||||
|`taskSlot/lazy/count`|Number of total task slots in lazy marked MiddleManagers and Indexers per emission period. This metric is only available if the TaskSlotCountStatsMonitor module is included.| category. |Varies.|
|
||||
|`taskSlot/blacklisted/count`|Number of total task slots in blacklisted MiddleManagers and Indexers per emission period. This metric is only available if the TaskSlotCountStatsMonitor module is included.| category. |Varies.|
|
||||
|`task/segmentAvailability/wait/time`|The amount of milliseconds a batch indexing task waited for newly created segments to become available for querying.| dataSource, taskType, taskId, segmentAvailabilityConfirmed |Varies.|
|
||||
|`worker/task/failed/count`|Number of failed tasks run on the reporting worker per emission period. This metric is only available if the WorkerTaskCountStatsMonitor module is included, and is only supported for middleManager nodes.| category, workerVersion. |Varies.|
|
||||
|`worker/task/success/count`|Number of successful tasks run on the reporting worker per emission period. This metric is only available if the WorkerTaskCountStatsMonitor module is included, and is only supported for middleManager nodes.| category, workerVersion. |Varies.|
|
||||
|`worker/taskSlot/idle/count`|Number of idle task slots on the reporting worker per emission period. This metric is only available if the WorkerTaskCountStatsMonitor module is included, and is only supported for middleManager nodes.| category, workerVersion. |Varies.|
|
||||
|`worker/taskSlot/total/count`|Number of total task slots on the reporting worker per emission period. This metric is only available if the WorkerTaskCountStatsMonitor module is included.| category, workerVersion. |Varies.|
|
||||
|`worker/taskSlot/used/count`|Number of busy task slots on the reporting worker per emission period. This metric is only available if the WorkerTaskCountStatsMonitor module is included.| category, workerVersion. |Varies.|
|
||||
|
||||
|
||||
|
||||
|
@ -63,11 +63,11 @@
|
||||
"task/pending/count" : { "dimensions" : ["dataSource"], "type" : "gauge" },
|
||||
"task/waiting/count" : { "dimensions" : ["dataSource"], "type" : "gauge" },
|
||||
|
||||
"worker/task/failed/count" : { "dimensions" : ["category", "version"], "type" : "count" },
|
||||
"worker/task/success/count" : { "dimensions" : ["category", "version"], "type" : "count" },
|
||||
"worker/taskSlot/idle/count" : { "dimensions" : ["category", "version"], "type" : "gauge" },
|
||||
"worker/taskSlot/total/count" : { "dimensions" : ["category", "version"], "type" : "gauge" },
|
||||
"worker/taskSlot/used/count" : { "dimensions" : ["category", "version"], "type" : "gauge" },
|
||||
"worker/task/failed/count" : { "dimensions" : ["category", "workerVersion"], "type" : "count" },
|
||||
"worker/task/success/count" : { "dimensions" : ["category", "workerVersion"], "type" : "count" },
|
||||
"worker/taskSlot/idle/count" : { "dimensions" : ["category", "workerVersion"], "type" : "gauge" },
|
||||
"worker/taskSlot/total/count" : { "dimensions" : ["category", "workerVersion"], "type" : "gauge" },
|
||||
"worker/taskSlot/used/count" : { "dimensions" : ["category", "workerVersion"], "type" : "gauge" },
|
||||
|
||||
"taskSlot/total/count" : { "dimensions" : ["category"], "type" : "gauge" },
|
||||
"taskSlot/idle/count" : { "dimensions" : ["category"], "type" : "gauge" },
|
||||
|
@ -72,7 +72,7 @@ public class WorkerTaskCountStatsMonitor extends AbstractMonitor
|
||||
if (value != null) {
|
||||
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
|
||||
builder.setDimension("category", workerCategory);
|
||||
builder.setDimension("version", workerVersion);
|
||||
builder.setDimension("workerVersion", workerVersion);
|
||||
emitter.emit(builder.build(metricName, value));
|
||||
}
|
||||
}
|
||||
|
@ -174,23 +174,23 @@ public class WorkerTaskCountStatsMonitorTest
|
||||
Assert.assertEquals(5, emitter.getEvents().size());
|
||||
Assert.assertEquals("worker/task/failed/count", emitter.getEvents().get(0).toMap().get("metric"));
|
||||
Assert.assertEquals("workerCategory", emitter.getEvents().get(0).toMap().get("category"));
|
||||
Assert.assertEquals("workerVersion", emitter.getEvents().get(0).toMap().get("version"));
|
||||
Assert.assertEquals("workerVersion", emitter.getEvents().get(0).toMap().get("workerVersion"));
|
||||
Assert.assertEquals(4L, emitter.getEvents().get(0).toMap().get("value"));
|
||||
Assert.assertEquals("worker/task/success/count", emitter.getEvents().get(1).toMap().get("metric"));
|
||||
Assert.assertEquals("workerCategory", emitter.getEvents().get(1).toMap().get("category"));
|
||||
Assert.assertEquals("workerVersion", emitter.getEvents().get(1).toMap().get("version"));
|
||||
Assert.assertEquals("workerVersion", emitter.getEvents().get(1).toMap().get("workerVersion"));
|
||||
Assert.assertEquals(2L, emitter.getEvents().get(1).toMap().get("value"));
|
||||
Assert.assertEquals("worker/taskSlot/idle/count", emitter.getEvents().get(2).toMap().get("metric"));
|
||||
Assert.assertEquals("workerCategory", emitter.getEvents().get(2).toMap().get("category"));
|
||||
Assert.assertEquals("workerVersion", emitter.getEvents().get(2).toMap().get("version"));
|
||||
Assert.assertEquals("workerVersion", emitter.getEvents().get(2).toMap().get("workerVersion"));
|
||||
Assert.assertEquals(3L, emitter.getEvents().get(2).toMap().get("value"));
|
||||
Assert.assertEquals("worker/taskSlot/total/count", emitter.getEvents().get(3).toMap().get("metric"));
|
||||
Assert.assertEquals("workerCategory", emitter.getEvents().get(3).toMap().get("category"));
|
||||
Assert.assertEquals("workerVersion", emitter.getEvents().get(3).toMap().get("version"));
|
||||
Assert.assertEquals("workerVersion", emitter.getEvents().get(3).toMap().get("workerVersion"));
|
||||
Assert.assertEquals(5L, emitter.getEvents().get(3).toMap().get("value"));
|
||||
Assert.assertEquals("worker/taskSlot/used/count", emitter.getEvents().get(4).toMap().get("metric"));
|
||||
Assert.assertEquals("workerCategory", emitter.getEvents().get(4).toMap().get("category"));
|
||||
Assert.assertEquals("workerVersion", emitter.getEvents().get(4).toMap().get("version"));
|
||||
Assert.assertEquals("workerVersion", emitter.getEvents().get(4).toMap().get("workerVersion"));
|
||||
Assert.assertEquals(1L, emitter.getEvents().get(4).toMap().get("value"));
|
||||
}
|
||||
|
||||
|
@ -1439,6 +1439,7 @@ SysMonitor
|
||||
TaskCountStatsMonitor
|
||||
TaskSlotCountStatsMonitor
|
||||
WorkerTaskCountStatsMonitor
|
||||
workerVersion
|
||||
bufferCapacity
|
||||
bufferpoolName
|
||||
cms
|
||||
|
Loading…
x
Reference in New Issue
Block a user