Samza Metrics Reference

The following table lists all the metrics emited by Samza.

Metrics are grouped by group names usually representing classes that are adding their own metrics to the registry. Those are the following:

Words highlighted like this are placeholders for your own variable names defined in configuration file or system variables defined while starting the job.

Note: Average time is calculated for the current time window (set to 300 seconds)

NameDescription
org.apache.samza.container.SamzaContainerMetrics
process-calls Number of times process method is called
process-ns Average time spent for processing messages
process-envelopes Number of message envelopes processed
process-null-envelopes Number of times no message envelope was available
choose-ns Average time spent for choosing the envelope to process
commit-calls Number of commit method calls
commit-ns Average time spent for commiting tasks states.
event-loop-utilization Number representing duty-cycle (active time in event loop divided by the total time spent in the event loop)
window-calls Number of window method calls
window-ns Average time spent for windowing tasks
block-ns Average time run loop was blocked (for multithreaded processing)
task-name-store-name-restore-time Time taken to restore task stores (per task store)
disk-usage-bytes Total disk space size used by key-value stores (in bytes)
disk-quota-bytes Disk memory usage quota for key-value stores (in bytes)
executor-work-factor Current work factor in use
physical-memory-mb The physical memory used by the Samza container process (native + on heap) (in megabytes)
container-startup-time Average time spent for the container to startup
job-coordinator
system-stream-partitionCount The difference in the partition count from the initial state for a particular system and stream (enabled if job.coordinator.monitor-partition-change configured to true)
org.apache.samza.container.TaskInstanceMetrics
- Following metrics apply to each TaskInstance
commit-calls Number of commit method calls
flush-calls Number of times underlying SystemProducers were flushed
system-stream-partition-offset Latest offsets for each SystemStreamPartition of the task instance
messages-actually-processed Number of messages actually processed by a task
process-calls Number of proccess method calls
send-calls Number of send method calls (representing number of messages that were sent to the underlying SystemProducers)
window-calls Number of times windowing the task
pending-messages Current number of pending messages in a queue (for multithreaded processing)
messages-in-flight Number of messages in processing (for multithreaded processing)
org.apache.samza.checkpoint.OffsetManagerMetrics
system-stream-partition-checkpointed-offset Latest checkpointed offsets for each SystemStreamPartition of the task instance
org.apache.samza.metrics.JvmMetrics
gc-count Number of GC calls
gc-time-millis Total time spent in GC (in milliseconds)
mem-heap-comitted-mb Size of comitted heap memory (in megabytes)
mem-heap-max-mb Size of maximum heap memory (in megabytes)
mem-heap-used-mb Size of used heap memory (in megabytes)
mem-non-heap-commited-mb Size of comitted non-heap memory (in megabytes)
mem-non-heap-max-mb Size of maximum non-heap memory (in megabytes)
mem-non-heap-used-mb Size of used non-heap memory (in megabytes)
gc-name-gc-count Number of garbage collection calls (for each garbage collector)
gc-name-gc-time-millis Total time spent in garbage collection (for each garbage collector) (in milliseconds)
threads-new Current number of threads not started yet
threads-runnable Current number of running threads
threads-timed-waiting Current number of timed-waiting threads
threads-waiting Current number of waiting threads
threads-blocked Current number of blocked threads
threads-terminated Current number of terminated threads
process-cpu-usage Current CPU usage of the JVM process as a percentage from 0 to 100. The percentage represents the proportion of executed ticks by the JVM process to the total ticks across all CPUs. A negative number indicates the value was not available from the operating system. For more detail, see the JavaDoc for com.sun.management.OperatingSystemMXBean.
system-cpu-usage Current CPU usage of the all processes in the whole system as a percentage from 0 to 100. The percentage represents the proportion of executed ticks by all processes to the total ticks across all CPUs. A negative number indicates the value was not available from the operating system. For more detail, see the JavaDoc for com.sun.management.OperatingSystemMXBean.
open-file-descriptor-count Current number of open file descriptors
org.apache.samza.system.SystemConsumersMetrics
chose-null Number of times message chooser returned null message envelope
chose-object Number of times message chooser returned an incoming message envelope
deserialization error Number of times an incoming message was not deserialized successfully
system-polls Number of times the poll method for particular system was called
system-ssp-fetches-per-poll Number representing how many partitions of a particular system were provided to the SystemConsumer to get new messages
system-messages-per-poll Number of times the SystemConsumer for underlying system was actually polled to get new messages
system-stream-partition-messages-chosen Number of messages that were chosen by the MessageChooser for particular system stream partition.
poll-ns Average time spent polling all underlying systems for new messages (in nanoseconds)
deserialization-ns Average time spent deserializing incoming messages (in nanoseconds)
ssps-needed-by-chooser Current number of empty system stream partitions
poll-timeout Current poll timeout
unprocessed-messages Current number of unprocessed messages in buffer of incoming messages that are eligible for choosing
org.apache.samza.system.SystemProducersMetrics
sends Number of send method calls. Representing total number of sent messages
flushes Number of flush method calls. Flush method flushes messages for all registered producers
source-sends Number of sent messages for particular source (task instance)
source-flushes Number of flushes for particular source (task instance)
serialization error Number of errors occured while serializing envelopes before sending
org.apache.samza.metrics.ContainerProcessManagerMetrics
running-containers Number of running containers
needed-containers Number of containers needed for the job to be declared healthy
completed-containers Number of containers that have completed their execution and exited successfully
failed-containers Number of failed containers
released-containers Number of containers released due to extra allocation returned by the RM
container-count Number of containers configured for the job
job-healthy State indicating whether the job is healthy or not
locality-matched Ratio beetwen number of container assigned to the prefered host and total number of container allocation requests
org.apache.samza.job.yarn.SamzaAppMasterMetrics (Deprecated)
- After deprecation, all the following metrics will be under new group "org.apache.samza.metrics.ContainerProcessManagerMetrics" mentioned above
running-containers Number of running containers
needed-containers Number of containers needed for the job to be declared healthy
completed-containers Number of containers that have completed their execution and exited successfully
failed-containers Number of failed containers
released-containers Number of containers released due to extra allocation returned by the RM
container-count Number of containers configured for the job
job-healthy State indicating whether the job is healthy or not
heartbeats-expired Number of heartbeat requests from containers that are invalid
org.apache.samza.system.kafka.KafkaSystemConsumerMetrics
system-topic-partition-offset-change The next offset to be read for a topic partition
system-topic-partition-bytes-read Total size of all messages read for a topic partition (payload + key size)
system-topic-partition-messages-read Number of messages read for a topic partition
system-topic-partition-high-watermark Offset of the last committed message in kafka's topic partition
system-topic-partition-messages-behind-high-watermark Number of messages that are not fetched yet (difference beetwen high watermark and next offset)
system-host-port-reconnects Number of reconnects to broker on particular host and port.
system-host-port-bytes-read Total size of all messages read from a broker on particular host and port
system-host-port-messages-read Number of times consumer fetched a broker on a particular host and port to get new messages
system-host-port-skipped-fetch-requests Number of times fetchMessage method is called but no topic/partitions needed new messages
system-host-port-topic-partitions Number of broker's topic partitions which are being consumed
poll-count Number of polling the KafkaSystemConsumer to get any available messages
no-more-messages-SystemStreamPartition [system, stream, partition] Indicates if kafka consumer is at head for particular partition
blocking-poll-count-SystemStreamPartition [system, stream, partition] Number that represents how many times blocking poll is executed (polling until we get at least one message, or until we catch up to the head of the stream) (per partition)
blocking-poll-timeout-count-SystemStreamPartition [system, stream, partition] Number that represents how many times timed blocking poll is executed (polling until we get at least one message within a timeout period) (per partition)
buffered-message-count-SystemStreamPartition [system, stream, partition] Current number of messages in queue (per partition)
buffered-message-size-SystemStreamPartition [system, stream, partition] Current size of messages in queue (if systems.system.samza.fetch.threshold.bytes is defined) (per partition)
org.apache.samza.system.kafka.KafkaSystemProducerMetrics
system-producer-sends Number of calls made to send in KafkaSystemProducer
system-producer-send-success Number of send requests that was successfully completed by the KafkaSystemProducer
system-producer-send-failed Number of send requests that was failed by the KafkaSystemProducer (due to unrecoverable errors)
system-producer-retries (Deprecated) Number of times the system producer retries a send request (due to RetriableException)
system-flushes Number of calls made to flush in KafkaSystemProducer
system-flush-ns Represents average time the flush call takes to complete (in nanoseconds)
system-flush-failed Number of times flush operation failed
org.apache.samza.storage.kv.KeyValueStorageEngineMetrics
store-name-puts Total number of updating the mapping of the specified key-value pair in the engine's key value store
store-name-put-ns Average duration of updating the mapping of the key-value pairs in the engine's key value store
store-name-gets Total number of looking for the value associated with the specified key in the engine's key value store
store-name-get-ns Average duration of looking for the value associated with the specified key in the engine's key value store
store-name-alls Total number of accessing the iterator for all entries in engine's key-value store
store-name-all-ns Average duration of accessing the iterator for all entries in engine's key-value store
store-name-ranges Total number of accessing the iterator for a sorted range of entries in engine's key-value store
store-name-range-ns Average duration of accessing the iterator for a sorted range of entries in engine's key-value store
store-name-deletes Total number of deleting the mapping for the specified key from engine's key-value store
store-name-delete-ns Average duration of deleting mappings for the specifide key from engine's key-value store
store-name-flushes Total number of flushing engine's key-value store
store-name-flush-ns Average duration of flushing engine's key-value store
store-name-messages-restored Number of restored messages (key-value pairs) from the change log
store-name-message-bytes Total amount of bytes (key and value size) restored from the change log
org.apache.samza.storage.kv.KeyValueStoreMetrics
store-name-puts Total number of updating the mapping of the specified key-value pair. Representing total number of calls to the put method
store-name-gets Total number of looking for the value associated with the specified key. Representing total number of calls to the get method
store-name-getalls Total number of looking for values associated with the specified keys. Representing total number of calls to the getAll method
store-name-alls Total number of accessing the iterator for all entries in this key-value store. Representing total number of calls to the all method
store-name-ranges Total number of accessing the iterator for a sorted range of entries. Representing total number of calls to the range method
store-name-bytes-written Total amount of bytes that are written to the key-value store
store-name-bytes-read Total amount of bytes that are read from the key-value store
store-name-flushes Total number of key-value store flushes. Representing total number of calls to the flush method
store-name-deletes Total number of deleting the mapping for the specified key from the key-value store. Representing total number of calls to the delete method
store-name-deletealls Total number of deleting the mappings for the specified keys from this key-value store. Representing total number of calls to the deleteAll method
org.apache.samza.storage.kv.LoggedStoreMetrics
store-name-puts Total number of updating the mapping of the specified key-value pair in the underlying key-value store and logging it out to the changelog
store-name-gets Total number of looking for the value associated with the specified key from the underlying key-value store
store-name-alls Total number of accessing the iterator for all entries in the underlying key-value store
store-name-ranges Total number of accessing the iterator for a sorted range of entries in the underlying key-value store
store-name-deletes Total number of deleting the mapping for the specified key from the underlying key-value store and logging it out to the changelog as a null value
store-name-flushes Total number of flushing the underlying key-value store
org.apache.samza.storage.kv.SerializedKeyValueStoreMetrics
store-name-puts Total number of updating the mapping of the specified key-value pair in the underlying key-value store
store-name-gets Total number of looking for the value associated with the specified key
store-name-alls Total number of accessing the iterator for all entries in the underlying key-value store
store-name-ranges Total number of accessing the iterator for a sorted range of entries in the underlying key-value store
store-name-bytes-serialized Total amount of bytes serialized while writing mappings to the underlying key-value store
store-name-bytes-deserialized Total amount of bytes deserialized while reading mappings from the underlying key-value store
store-name-deletes Total number of deleting the mapping for the specified key from the underlying key-value store
store-name-flushes Total number of flushing the underlying key-value store
org.apache.samza.storage.kv.CachedStoreMetrics
store-name-puts Total number of updating the mapping of the specified key-value pair in the cached store
store-name-put-all-dirty-entires-batch-size Total number of dirty mappings that have been written to the underlying store
store-name-gets Total number of looking for the value associated with the specified key in the cached storage
store-name-alls Total number of accessing the iterator for all entries in the underlying key-value store
store-name-ranges Total number of accessing the iterator for a sorted range of entries in the underlying key-value store
store-name-flushes Total number of purging dirty entries in the cache and flushing underlying key-value store
store-name-deletes Total number of deleting the mapping for the specified key from the cache storage
store-name-cache-hits Total number of times the item in the cache storage is read
store-name-cache-size The number of items currently in the cache
store-name-dirty-count The number of items currently in the dirty list
org.apache.samza.system.chooser.RoundRobinChooserMetrics
buffered-messages Size of the queue with potential messages to process
org.apache.samza.system.chooser.BatchingChooserMetrics
batch-resets The number of batch resets due to exceeding max batch size limit
batched-envelopes The number of envelopes in the current batch
org.apache.samza.system.chooser.BootstrappingChooserMetrics
lagging-batch-streams The number of bootstrapping streams that are lagging
system-stream-lagging-partitions The number of lagging partitions in the stream (for each stream marked as bootstrapping stream)
org.apache.samza.system.hdfs.HdfsSystemProducerMetrics
system-producer-sends Total number of attempts to write to the HDFS
system-send-success Total number of successful writes to the HDFS
system-send-failed Total number of failures while sending envelopes to the HDFS
system-send-ms Average time spent for writing messages to the HDFS (in milliseconds)
system-flushes Total number of attempts to flush data to the file system
system-flush-success Total number of successfully flushed all written data to the file system
system-flush-failed Total number of failures while flushing data to the file system
system-flush-ms Average time spent for flushing messages to the HDFS (in milliseconds)
org.apache.samza.system.elasticsearch.ElasticsearchSystemProducerMetrics
system-bulk-send-success Total number of successfully sent bulk requests
system-docs-inserted Total number of created documents
system-docs-updated Total number of document updates
system-version-conflicts Number of times the request could not be completed due to a conflict with the current state of the document
org.apache.samza.zk.ZkUtilsMetrics
reads Number of reads from Zookeeper
writes Number of writes to Zookeeper
subscriptions Number of subscriptions to znodes in Zookeeper
zk-connection-errors Number of Zookeeper connection errors
org.apache.samza.zk.ZkJobCoordinatorMetrics
is-leader Denotes if the processor is a leader or not
barrier-creation Number of times a barrier was created by the leader
barrier-state-change Number of times the barrier state changed
barrier-error Number of times the barrier encountered an error while attaining consensus on the job model version
single-barrier-rebalancing-time Average time taken for all the processors to get the latest version of the job model after single processor change (without the occurence of a barrier timeout)