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)
Name | Description |
---|---|
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) |
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 |
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 |