Metrics

Druid generates metrics related to queries, ingestion, and coordination.

Metrics are emitted as JSON objects to a runtime log file or over HTTP (to a service such as Apache Kafka). Metric emission is disabled by default.

All Druid metrics share a common set of fields:

  • timestamp - the time the metric was created
  • metric - the name of the metric
  • service - the service name that emitted the metric
  • host - the host name that emitted the metric
  • value - some numeric value associated with the metric

Metrics may have additional dimensions beyond those listed above.

Most metric values reset each emission period. By default druid emission period is 1 minute, this can be changed by setting the property druid.monitoring.emissionPeriod.

Available Metrics

Query metrics

Broker

MetricDescriptionDimensionsNormal Value
query/timeMilliseconds taken to complete a query.Common: dataSource, type, interval, hasFilters, duration, context, remoteAddress, id. Aggregation Queries: numMetrics, numComplexMetrics. GroupBy: numDimensions. TopN: threshold, dimension.< 1s
query/bytesnumber of bytes returned in query response.Common: dataSource, type, interval, hasFilters, duration, context, remoteAddress, id. Aggregation Queries: numMetrics, numComplexMetrics. GroupBy: numDimensions. TopN: threshold, dimension.
query/node/timeMilliseconds taken to query individual historical/realtime processes.id, status, server.< 1s
query/node/bytesnumber of bytes returned from querying individual historical/realtime processes.id, status, server.
query/node/ttfbTime to first byte. Milliseconds elapsed until Broker starts receiving the response from individual historical/realtime processes.id, status, server.< 1s
query/node/backpressureMilliseconds that the channel to this process has spent suspended due to backpressure.id, status, server.
query/intervalChunk/timeOnly 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. See Query Context.id, status, chunkInterval (if interval chunking is enabled).< 1s
query/countnumber of total queriesThis metric is only available if the QueryCountStatsMonitor module is included.
query/success/countnumber of queries successfully processedThis metric is only available if the QueryCountStatsMonitor module is included.
query/failed/countnumber of failed queriesThis metric is only available if the QueryCountStatsMonitor module is included.
query/interrupted/countnumber of queries interrupted due to cancellation or timeoutThis metric is only available if the QueryCountStatsMonitor module is included.

Historical

MetricDescriptionDimensionsNormal Value
query/timeMilliseconds taken to complete a query.Common: dataSource, type, interval, hasFilters, duration, context, remoteAddress, id. Aggregation Queries: numMetrics, numComplexMetrics. GroupBy: numDimensions. TopN: threshold, dimension.< 1s
query/segment/timeMilliseconds taken to query individual segment. Includes time to page in the segment from disk.id, status, segment.several hundred milliseconds
query/wait/timeMilliseconds spent waiting for a segment to be scanned.id, segment.< several hundred milliseconds
segment/scan/pendingNumber of segments in queue waiting to be scanned.Close to 0
query/segmentAndCache/timeMilliseconds taken to query individual segment or hit the cache (if it is enabled on the Historical process).id, segment.several hundred milliseconds
query/cpu/timeMicroseconds of CPU time taken to complete a queryCommon: dataSource, type, interval, hasFilters, duration, context, remoteAddress, id. Aggregation Queries: numMetrics, numComplexMetrics. GroupBy: numDimensions. TopN: threshold, dimension.Varies
query/countnumber of total queriesThis metric is only available if the QueryCountStatsMonitor module is included.
query/success/countnumber of queries successfully processedThis metric is only available if the QueryCountStatsMonitor module is included.
query/failed/countnumber of failed queriesThis metric is only available if the QueryCountStatsMonitor module is included.
query/interrupted/countnumber of queries interrupted due to cancellation or timeoutThis metric is only available if the QueryCountStatsMonitor module is included.

Real-time

MetricDescriptionDimensionsNormal Value
query/timeMilliseconds taken to complete a query.Common: dataSource, type, interval, hasFilters, duration, context, remoteAddress, id. Aggregation Queries: numMetrics, numComplexMetrics. GroupBy: numDimensions. TopN: threshold, dimension.< 1s
query/wait/timeMilliseconds spent waiting for a segment to be scanned.id, segment.several hundred milliseconds
segment/scan/pendingNumber of segments in queue waiting to be scanned.Close to 0
query/countnumber of total queriesThis metric is only available if the QueryCountStatsMonitor module is included.
query/success/countnumber of queries successfully processedThis metric is only available if the QueryCountStatsMonitor module is included.
query/failed/countnumber of failed queriesThis metric is only available if the QueryCountStatsMonitor module is included.
query/interrupted/countnumber of queries interrupted due to cancellation or timeoutThis metric is only available if the QueryCountStatsMonitor module is included.

Jetty

MetricDescriptionNormal Value
jetty/numOpenConnectionsNumber of open jetty connections.Not much higher than number of jetty threads.

Cache

MetricDescriptionNormal Value
query/cache/delta/Cache metrics since the last emission.N/A
query/cache/total/Total cache metrics.N/A
MetricDescriptionDimensionsNormal Value
/numEntriesNumber of cache entries.Varies.
/sizeBytesSize in bytes of cache entries.Varies.
/hitsNumber of cache hits.Varies.
/missesNumber of cache misses.Varies.
/evictionsNumber of cache evictions.Varies.
/hitRateCache hit rate.~40%
/averageByteAverage cache entry byte size.Varies.
/timeoutsNumber of cache timeouts.0
/errorsNumber of cache errors.0
/put/okNumber of new cache entries successfully cached.Varies, but more than zero.
/put/errorNumber of new cache entries that could not be cached due to errors.Varies, but more than zero.
/put/oversizedNumber of potential new cache entries that were skipped due to being too large (based on druid.{broker,historical,realtime}.cache.maxEntrySize properties).Varies.

Memcached only metrics

Memcached client metrics are reported as per the following. These metrics come directly from the client as opposed to from the cache retrieval layer.

MetricDescriptionDimensionsNormal Value
query/cache/memcached/totalCache metrics unique to memcached (only if druid.cache.type=memcached) as their actual valuesVariableN/A
query/cache/memcached/deltaCache metrics unique to memcached (only if druid.cache.type=memcached) as their delta from the prior event emissionVariableN/A

SQL Metrics

If SQL is enabled, the Broker will emit the following metrics for SQL.

MetricDescriptionDimensionsNormal Value
sqlQuery/timeMilliseconds taken to complete a SQL.id, nativeQueryIds, dataSource, remoteAddress, success.< 1s
sqlQuery/bytesnumber of bytes returned in SQL response.id, nativeQueryIds, dataSource, remoteAddress, success.

Ingestion Metrics (Kafka Indexing Service)

These metrics are applicable for the Kafka Indexing Service.

MetricDescriptionDimensionsNormal Value
ingest/kafka/lagTotal 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.dataSource.Greater than 0, should not be a very high number
ingest/kafka/maxLagMax 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.dataSource.Greater than 0, should not be a very high number
ingest/kafka/avgLagAverage 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.dataSource.Greater than 0, should not be a very high number

Ingestion metrics (Realtime process)

These metrics are only available if the RealtimeMetricsMonitor is included in the monitors list for the Realtime process. These metrics are deltas for each emission period.

MetricDescriptionDimensionsNormal Value
ingest/events/thrownAwayNumber of events rejected because they are outside the windowPeriod.dataSource, taskId, taskType.0
ingest/events/unparseableNumber of events rejected because the events are unparseable.dataSource, taskId, taskType.0
ingest/events/duplicateNumber of events rejected because the events are duplicated.dataSource, taskId, taskType.0
ingest/events/processedNumber of events successfully processed per emission period.dataSource, taskId, taskType.Equal to your # of events per emission period.
ingest/rows/outputNumber of Druid rows persisted.dataSource, taskId, taskType.Your # of events with rollup.
ingest/persists/countNumber of times persist occurred.dataSource, taskId, taskType.Depends on configuration.
ingest/persists/timeMilliseconds spent doing intermediate persist.dataSource, taskId, taskType.Depends on configuration. Generally a few minutes at most.
ingest/persists/cpuCpu time in Nanoseconds spent on doing intermediate persist.dataSource, taskId, taskType.Depends on configuration. Generally a few minutes at most.
ingest/persists/backPressureMilliseconds spent creating persist tasks and blocking waiting for them to finish.dataSource, taskId, taskType.0 or very low
ingest/persists/failedNumber of persists that failed.dataSource, taskId, taskType.0
ingest/handoff/failedNumber of handoffs that failed.dataSource, taskId, taskType.0
ingest/merge/timeMilliseconds spent merging intermediate segmentsdataSource, taskId, taskType.Depends on configuration. Generally a few minutes at most.
ingest/merge/cpuCpu time in Nanoseconds spent on merging intermediate segments.dataSource, taskId, taskType.Depends on configuration. Generally a few minutes at most.
ingest/handoff/countNumber of handoffs that happened.dataSource, taskId, taskType.Varies. Generally greater than 0 once every segment granular period if cluster operating normally
ingest/sink/countNumber of sinks not handoffed.dataSource, taskId, taskType.1~3
ingest/events/messageGapTime gap between the data time in event and current system time.dataSource, taskId, taskType.Greater than 0, depends on the time carried in event

Note: If the JVM does not support CPU time measurement for the current thread, ingest/merge/cpu and ingest/persists/cpu will be 0.

Indexing service

MetricDescriptionDimensionsNormal Value
task/run/timeMilliseconds taken to run a task.dataSource, taskId, taskType, taskStatus.Varies.
task/action/log/timeMilliseconds taken to log a task action to the audit log.dataSource, taskId, taskType< 1000 (subsecond)
task/action/run/timeMilliseconds taken to execute a task action.dataSource, taskId, taskTypeVaries from subsecond to a few seconds, based on action type.
segment/added/bytesSize in bytes of new segments created.dataSource, taskId, taskType, interval.Varies.
segment/moved/bytesSize in bytes of segments moved/archived via the Move Task.dataSource, taskId, taskType, interval.Varies.
segment/nuked/bytesSize in bytes of segments deleted via the Kill Task.dataSource, taskId, taskType, interval.Varies.
task/success/countNumber of successful tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included.dataSource.Varies.
task/failed/countNumber of failed tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included.dataSource.Varies.
task/running/countNumber of current running tasks. This metric is only available if the TaskCountStatsMonitor module is included.dataSource.Varies.
task/pending/countNumber of current pending tasks. This metric is only available if the TaskCountStatsMonitor module is included.dataSource.Varies.
task/waiting/countNumber of current waiting tasks. This metric is only available if the TaskCountStatsMonitor module is included.dataSource.Varies.

Coordination

These metrics are for the Druid Coordinator and are reset each time the Coordinator runs the coordination logic.

MetricDescriptionDimensionsNormal Value
segment/assigned/countNumber of segments assigned to be loaded in the cluster.tier.Varies.
segment/moved/countNumber of segments moved in the cluster.tier.Varies.
segment/dropped/countNumber of segments dropped due to being overshadowed.tier.Varies.
segment/deleted/countNumber of segments dropped due to rules.tier.Varies.
segment/unneeded/countNumber of segments dropped due to being marked as unused.tier.Varies.
segment/cost/rawUsed in cost balancing. The raw cost of hosting segments.tier.Varies.
segment/cost/normalizationUsed in cost balancing. The normalization of hosting segments.tier.Varies.
segment/cost/normalizedUsed in cost balancing. The normalized cost of hosting segments.tier.Varies.
segment/loadQueue/sizeSize in bytes of segments to load.server.Varies.
segment/loadQueue/failedNumber of segments that failed to load.server.0
segment/loadQueue/countNumber of segments to load.server.Varies.
segment/dropQueue/countNumber of segments to drop.server.Varies.
segment/sizeSize in bytes of available segments.dataSource.Varies.
segment/countNumber of available segments.dataSource.< max
segment/overShadowed/countNumber of overShadowed segments.Varies.
segment/unavailable/countNumber of segments (not including replicas) left to load until segments that should be loaded in the cluster are available for queries.datasource.0
segment/underReplicated/countNumber of segments (including replicas) left to load until segments that should be loaded in the cluster are available for queries.tier, datasource.0
tier/historical/countNumber of available historical nodes in each tier.tier.Varies.
tier/replication/factorConfigured maximum replication factor in each tier.tier.Varies.
tier/required/capacityTotal capacity in bytes required in each tier.tier.Varies.
tier/total/capacityTotal capacity in bytes available in each tier.tier.Varies.

If emitBalancingStats is set to true in the Coordinator dynamic configuration, then log entries for class org.apache.druid.server.coordinator.helper.DruidCoordinatorLogger will have extra information on balancing decisions.

General Health

Historical

MetricDescriptionDimensionsNormal Value
segment/maxMaximum byte limit available for segments.Varies.
segment/usedBytes used for served segments.dataSource, tier, priority.< max
segment/usedPercentPercentage of space used by served segments.dataSource, tier, priority.< 100%
segment/countNumber of served segments.dataSource, tier, priority.Varies.
segment/pendingDeleteOn-disk size in bytes of segments that are waiting to be cleared outVaries.

JVM

These metrics are only available if the JVMMonitor module is included.

MetricDescriptionDimensionsNormal Value
jvm/pool/committedCommitted pool.poolKind, poolName.close to max pool
jvm/pool/initInitial pool.poolKind, poolName.Varies.
jvm/pool/maxMax pool.poolKind, poolName.Varies.
jvm/pool/usedPool used.poolKind, poolName.< max pool
jvm/bufferpool/countBufferpool count.bufferPoolName.Varies.
jvm/bufferpool/usedBufferpool used.bufferPoolName.close to capacity
jvm/bufferpool/capacityBufferpool capacity.bufferPoolName.Varies.
jvm/mem/initInitial memory.memKind.Varies.
jvm/mem/maxMax memory.memKind.Varies.
jvm/mem/usedUsed memory.memKind.< max memory
jvm/mem/committedCommitted memory.memKind.close to max memory
jvm/gc/countGarbage collection count.gcName (cms/g1/parallel/etc.), gcGen (old/young)Varies.
jvm/gc/cpuCount 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 cyclegcName, gcGenSum of jvm/gc/cpu should be within 10-30% of sum of jvm/cpu/total, depending on the GC algorithm used (reported by JvmCpuMonitor)

EventReceiverFirehose

The following metric is only available if the EventReceiverFirehoseMonitor module is included.

MetricDescriptionDimensionsNormal Value
ingest/events/bufferedNumber of events queued in the EventReceiverFirehose’s bufferserviceName, dataSource, taskId, taskType, bufferCapacity.Equal to current # of events in the buffer queue.
ingest/bytes/receivedNumber of bytes received by the EventReceiverFirehose.serviceName, dataSource, taskId, taskType.Varies.

Sys

These metrics are only available if the SysMonitor module is included.

MetricDescriptionDimensionsNormal Value
sys/swap/freeFree swap.Varies.
sys/swap/maxMax swap.Varies.
sys/swap/pageInPaged in swap.Varies.
sys/swap/pageOutPaged out swap.Varies.
sys/disk/write/countWrites to disk.fsDevName, fsDirName, fsTypeName, fsSysTypeName, fsOptions.Varies.
sys/disk/read/countReads from disk.fsDevName, fsDirName, fsTypeName, fsSysTypeName, fsOptions.Varies.
sys/disk/write/sizeBytes written to disk. Can we used to determine how much paging is occurring with regards to segments.fsDevName, fsDirName, fsTypeName, fsSysTypeName, fsOptions.Varies.
sys/disk/read/sizeBytes read from disk. Can we used to determine how much paging is occurring with regards to segments.fsDevName, fsDirName, fsTypeName, fsSysTypeName, fsOptions.Varies.
sys/net/write/sizeBytes written to the network.netName, netAddress, netHwaddrVaries.
sys/net/read/sizeBytes read from the network.netName, netAddress, netHwaddrVaries.
sys/fs/usedFilesystem bytes used.fsDevName, fsDirName, fsTypeName, fsSysTypeName, fsOptions.< max
sys/fs/maxFilesystesm bytes max.fsDevName, fsDirName, fsTypeName, fsSysTypeName, fsOptions.Varies.
sys/mem/usedMemory used.< max
sys/mem/maxMemory max.Varies.
sys/storage/usedDisk space used.fsDirName.Varies.
sys/cpuCPU used.cpuName, cpuTime.Varies.