Metrics

    All Druid metrics share a common set of fields:

    • - 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

    Metrics may have additional dimensions beyond those listed above.

    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, vectorized.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/countTotal number of 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.This metric is only available if the QueryCountStatsMonitor module is included.
    query/timeout/countNumber of timed out queries.This 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.This metric is only available if the QueryCountStatsMonitor module is included.
    query/timeout/countNumber of timed out queries.This metric is only available if the QueryCountStatsMonitor module is included.
    MetricDescriptionNormal Value
    jetty/numOpenConnectionsNumber of open jetty connections.Not much higher than number of jetty threads.
    jetty/threadPool/totalNumber of total workable threads allocated.The number should equal to threadPoolNumIdleThreads + threadPoolNumBusyThreads.
    jetty/threadPool/idleNumber of idle threads.Less than or equal to threadPoolNumTotalThreads. Non zero number means there is less work to do than configured capacity.
    jetty/threadPool/busyNumber of busy threads that has work to do from the worker queue.Less than or equal to threadPoolNumTotalThreads.
    jetty/threadPool/isLowOnThreadsA rough indicator of whether number of total workable threads allocated is enough to handle the works in the work queue.0
    jetty/threadPool/minNumber of minimum threads allocatable.druid.server.http.numThreads plus a small fixed number of threads allocated for Jetty acceptors and selectors.
    jetty/threadPool/maxNumber of maximum threads allocatable.druid.server.http.numThreads plus a small fixed number of threads allocated for Jetty acceptors and selectors.
    jetty/threadPool/queueSizeSize of the worker queue.Not much higher than druid.server.http.queueSize

    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.

    Ingestion metrics

    MetricDescriptionDimensionsNormal Value
    ingest/countCount of 1 every time an ingestion job runs (includes compaction jobs). Aggregate using dimensions.dataSource, taskId, taskType, taskIngestionModeAlways 1.
    ingest/segments/countCount of final segments created by job (includes tombstones).dataSource, taskId, taskType, taskIngestionModeAt least 1.
    ingest/tombstones/countCount of tombstones created by jobdataSource, taskId, taskType, taskIngestionModeZero or more for replace. Always zero for non-replace tasks (always zero for legacy replace, see below).

    The mode is decided using the values of the and isDropExisting flags in the task’s IOConfig as follows:

    isAppendToExistingisDropExistingmode
    truefalseAPPEND
    truetrueInvalid combination, exception thrown.
    falsefalseREPLACE_LEGACY (this is the default for native batch ingestion).
    falsetrueREPLACE

    Ingestion metrics for Kafka

    These metrics apply to the .

    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

    These metrics apply to the .

    MetricDescriptionDimensionsNormal Value
    ingest/kinesis/lag/timeTotal lag time in milliseconds between the current message sequence number consumed by the Kinesis indexing tasks and latest sequence number in Kinesis across all shards. Minimum emission period for this metric is a minute.dataSource.Greater than 0, up to max Kinesis retention period in milliseconds
    ingest/kinesis/maxLag/timeMax lag time in milliseconds between the current message sequence number consumed by the Kinesis indexing tasks and latest sequence number in Kinesis across all shards. Minimum emission period for this metric is a minute.dataSource.Greater than 0, up to max Kinesis retention period in milliseconds
    ingest/kinesis/avgLag/timeAverage lag time in milliseconds between the current message sequence number consumed by the Kinesis indexing tasks and latest sequence number in Kinesis across all shards. Minimum emission period for this metric is a minute.dataSource.Greater than 0, up to max Kinesis retention period in milliseconds

    Other ingestion metrics

    Streaming ingestion tasks and certain types of batch ingestion emit the following metrics. These metrics are deltas for each emission period.

    MetricDescriptionDimensionsNormal Value
    ingest/events/thrownAwayNumber of events rejected because they are either null, or filtered by the transform spec, or 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 in milliseconds between the latest ingested event timestamp and the current system timestamp of metrics emission.dataSource, taskId, taskType.Greater than 0, depends on the time carried in event
    ingest/notices/queueSizeNumber of pending notices to be processed by the coordinatordataSource.Typically 0 and occasionally in lower single digits. Should not be a very high number.
    ingest/notices/timeMilliseconds taken to process a notice by the supervisordataSource, noticeType.< 1s.

    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/pending/timeMilliseconds taken for a task to wait for running.dataSource, taskId, taskType.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.
    taskSlot/total/countNumber of total task slots per emission period. This metric is only available if the TaskSlotCountStatsMonitor module is included.category.Varies.
    taskSlot/idle/countNumber of idle task slots per emission period. This metric is only available if the TaskSlotCountStatsMonitor module is included.category.Varies.
    taskSlot/used/countNumber of busy task slots per emission period. This metric is only available if the TaskSlotCountStatsMonitor module is included.category.Varies.
    taskSlot/lazy/countNumber 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/countNumber 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/timeThe amount of milliseconds a batch indexing task waited for newly created segments to become available for querying.dataSource, taskType, taskId, segmentAvailabilityConfirmedVaries.
    worker/task/failed/countNumber 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/countNumber 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/countNumber 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/countNumber 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/countNumber 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.

    Shuffle metrics (Native parallel task)

    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/sizeTotal size of used segments in a data source. Emitted only for data sources to which at least one used segment belongs.dataSource.Varies.
    segment/countNumber of used segments belonging to a data source. Emitted only for data sources to which at least one used segment belongs.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.
    compact/task/countNumber of tasks issued in the auto compaction run.Varies.
    compactTask/maxSlot/countMax number of task slots that can be used for auto compaction tasks in the auto compaction run.Varies.
    compactTask/availableSlot/countNumber of available task slots that can be used for auto compaction tasks in the auto compaction run. (this is max slot minus any currently running compaction task)Varies.
    segment/waitCompact/bytesTotal bytes of this datasource waiting to be compacted by the auto compaction (only consider intervals/segments that are eligible for auto compaction).datasource.Varies.
    segment/waitCompact/countTotal number of segments of this datasource waiting to be compacted by the auto compaction (only consider intervals/segments that are eligible for auto compaction).datasource.Varies.
    interval/waitCompact/countTotal number of intervals of this datasource waiting to be compacted by the auto compaction (only consider intervals/segments that are eligible for auto compaction).datasource.Varies.
    segment/compacted/bytesTotal bytes of this datasource that are already compacted with the spec set in the auto compaction config.datasource.Varies.
    segment/compacted/countTotal number of segments of this datasource that are already compacted with the spec set in the auto compaction config.datasource.Varies.
    interval/compacted/countTotal number of intervals of this datasource that are already compacted with the spec set in the auto compaction config.datasource.Varies.
    segment/skipCompact/bytesTotal bytes of this datasource that are skipped (not eligible for auto compaction) by the auto compaction.datasource.Varies.
    Total number of segments of this datasource that are skipped (not eligible for auto compaction) by the auto compaction.datasource.Varies.
    interval/skipCompact/countTotal number of intervals of this datasource that are skipped (not eligible for auto compaction) by the auto compaction.datasource.Varies.
    coordinator/timeApproximate Coordinator duty runtime in milliseconds. The duty dimension is the string alias of the Duty that is being run.duty.Varies.
    coordinator/global/timeApproximate runtime of a full coordination cycle in milliseconds. The dutyGroup dimension indicates what type of coordination this run was. i.e. Historical Management vs IndexingdutyGroupVaries.
    metadata/kill/supervisor/countTotal number of terminated supervisors that were automatically deleted from metadata store per each Coordinator kill supervisor duty run. This metric can help adjust druid.coordinator.kill.supervisor.durationToRetain configuration based on whether more or less terminated supervisors need to be deleted per cycle. Note that this metric is only emitted when druid.coordinator.kill.supervisor.on is set to true.Varies.
    metadata/kill/audit/countTotal number of audit logs that were automatically deleted from metadata store per each Coordinator kill audit duty run. This metric can help adjust druid.coordinator.kill.audit.durationToRetain configuration based on whether more or less audit logs need to be deleted per cycle. Note that this metric is only emitted when druid.coordinator.kill.audit.on is set to true.Varies.
    metadata/kill/compaction/countTotal number of compaction configurations that were automatically deleted from metadata store per each Coordinator kill compaction configuration duty run. Note that this metric is only emitted when druid.coordinator.kill.compaction.on is set to true.Varies.
    metadata/kill/rule/countTotal number of rules that were automatically deleted from metadata store per each Coordinator kill rule duty run. This metric can help adjust druid.coordinator.kill.rule.durationToRetain configuration based on whether more or less rules need to be deleted per cycle. Note that this metric is only emitted when druid.coordinator.kill.rule.on is set to true.Varies.
    metadata/kill/datasource/countTotal number of datasource metadata that were automatically deleted from metadata store per each Coordinator kill datasource duty run (Note: datasource metadata only exists for datasource created from supervisor). This metric can help adjust druid.coordinator.kill.datasource.durationToRetain configuration based on whether more or less datasource metadata need to be deleted per cycle. Note that this metric is only emitted when druid.coordinator.kill.datasource.on is set to true.Varies.

    If emitBalancingStats is set to true in the Coordinator dynamic configuration, then for class org.apache.druid.server.coordinator.duty.EmitClusterStatsAndMetrics 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.
    segment/rowCount/avgThe average number of rows per segment on a historical. SegmentStatsMonitor must be enabled.dataSource, tier, priority.Varies. See segment optimization for guidance on optimal segment sizes.
    segment/rowCount/range/countThe number of segments in a bucket. SegmentStatsMonitor must be enabled.dataSource, tier, priority, range.Varies.

    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. One indicator of the amount of paging occurring for segments.fsDevName,fsDirName,fsTypeName, fsSysTypeName, fsOptions.Varies.
    sys/disk/read/sizeBytes read from disk. One indicator of the amount of paging occurring for 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/maxFilesystem 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.
    MetricDescriptionDimensionsNormal Value
    cgroup/cpu/sharesRelative value of CPU time available to this process. Read from cpu.shares.Varies.
    cgroup/cpu/cores_quotaNumber of cores available to this process. Derived from cpu.cfs_quota_us/cpu.cfs_period_us.Varies. A value of -1 indicates there is no explicit quota set.
    cgroup/memory/Memory stats for this process (e.g. cache, total_swap, etc.). Each stat produces a separate metric. Read from memory.stat.Varies.
    cgroup/memory_numa//pagesMemory stats, per NUMA node, for this process (e.g. total, unevictable, etc.). Each stat produces a separate metric. Read from memory.num_stat.numaZoneVaries.
    cgroup/cpuset/cpu_countTotal number of CPUs available to the process. Derived from cpuset.cpus.Varies.
    cgroup/cpuset/effective_cpu_countTotal number of active CPUs available to the process. Derived from cpuset.effective_cpus.Varies.
    cgroup/cpuset/mems_countTotal number of memory nodes available to the process. Derived from cpuset.mems.Varies.
    Total number of active memory nodes available to the process. Derived from cpuset.effective_mems.Varies.