List of Metrics

Slack Docker Pulls GitHub edit source

There are two types of metrics in Alluxio, cluster-wide aggregated metrics, and per-process detailed metrics.

  • Cluster metrics are collected and calculated by the leading master and displayed in the metrics tab of the web UI. These metrics are designed to provide a snapshot of the cluster state and the overall amount of data and metadata served by Alluxio.

  • Process metrics are collected by each Alluxio process and exposed in a machine-readable format through any configured sinks. Process metrics are highly detailed and are intended to be consumed by third-party monitoring tools. Users can then view fine-grained dashboards with time-series graphs of each metric, such as data transferred or the number of RPC invocations.

Metrics in Alluxio have the following format for master node metrics:

Master.[metricName].[tag1].[tag2]...

Metrics in Alluxio have the following format for non-master node metrics:

[processType].[metricName].[tag1].[tag2]...[hostName]

There is generally an Alluxio metric for every RPC invocation, to Alluxio or to the under store.

Tags are additional pieces of metadata for the metric such as user name or under storage location. Tags can be used to further filter or aggregate on various characteristics.

Cluster Metrics

Workers and clients send metrics data to the Alluxio master through heartbeats. The interval is defined by property alluxio.master.worker.heartbeat.interval and alluxio.user.metrics.heartbeat.interval respectively.

Bytes metrics are aggregated value from workers or clients. Bytes throughput metrics are calculated on the leading master. The values of bytes throughput metrics equal to bytes metrics counter value divided by the metrics record time and shown as bytes per minute.

NameTypeDescription
Cluster.ActiveRpcReadCount COUNTER The number of active read-RPCs managed by workers
Cluster.ActiveRpcWriteCount COUNTER The number of active write-RPCs managed by workers
Cluster.BytesReadDirect COUNTER Total number of bytes read from Alluxio storage managed by workers and underlying UFS if data cannot be found in the Alluxio storage without external RPC involved. This records data read by worker internal calls (e.g. clients embedded in workers).
Cluster.BytesReadDirectThroughput GAUGE Total number of bytes read from Alluxio storage managed by workers and underlying UFS if data cannot be found in the Alluxio storage without external RPC involved. This records data read by worker internal calls (e.g. clients embedded in workers).
Cluster.BytesReadDomain COUNTER Total number of bytes read from Alluxio storage via domain socket reported by all workers
Cluster.BytesReadDomainThroughput GAUGE Bytes read per minute throughput from Alluxio storage via domain socket by all workers
Cluster.BytesReadLocal COUNTER Total number of bytes short-circuit read from local storage by all clients
Cluster.BytesReadLocalThroughput GAUGE Bytes per minute throughput short-circuit read from local storage by all clients
Cluster.BytesReadPerUfs COUNTER Total number of bytes read from a specific UFS by all workers
Cluster.BytesReadRemote COUNTER Total number of bytes read from Alluxio storage or underlying UFS if data does not exist in Alluxio storage reported by all workers. This does not include short-circuit local reads and domain socket reads
Cluster.BytesReadRemoteThroughput GAUGE Bytes read per minute throughput from Alluxio storage or underlying UFS if data does not exist in Alluxio storage reported by all workers. This does not include short-circuit local reads and domain socket reads
Cluster.BytesReadUfsAll COUNTER Total number of bytes read from all Alluxio UFSes by all workers
Cluster.BytesReadUfsThroughput GAUGE Bytes read per minute throughput from all Alluxio UFSes by all workers
Cluster.BytesWrittenDomain COUNTER Total number of bytes written to Alluxio storage via domain socket by all workers
Cluster.BytesWrittenDomainThroughput GAUGE Throughput of bytes written per minute to Alluxio storage via domain socket by all workers
Cluster.BytesWrittenLocal COUNTER Total number of bytes short-circuit written to local storage by all clients
Cluster.BytesWrittenLocalThroughput GAUGE Bytes per minute throughput written to local storage by all clients
Cluster.BytesWrittenPerUfs COUNTER Total number of bytes written to a specific Alluxio UFS by all workers
Cluster.BytesWrittenRemote COUNTER Total number of bytes written to Alluxio storage in all workers or the underlying UFS. This does not include short-circuit local writes and domain socket writes.
Cluster.BytesWrittenRemoteThroughput GAUGE Bytes write per minute throughput to Alluxio storage in all workers or the underlying UFS. This does not include short-circuit local writes and domain socket writes.
Cluster.BytesWrittenUfsAll COUNTER Total number of bytes written to all Alluxio UFSes by all workers
Cluster.BytesWrittenUfsThroughput GAUGE Bytes write per minute throughput to all Alluxio UFSes by all workers
Cluster.CacheHitRate GAUGE Cache hit rate: (# bytes read from cache) / (# bytes requested)
Cluster.CapacityFree GAUGE Total free bytes on all tiers, on all workers of Alluxio
Cluster.CapacityTotal GAUGE Total capacity (in bytes) on all tiers, on all workers of Alluxio
Cluster.CapacityUsed GAUGE Total used bytes on all tiers, on all workers of Alluxio
Cluster.LeaderId GAUGE Display current leader id
Cluster.LeaderIndex GAUGE Index of current leader
Cluster.LostWorkers GAUGE Total number of lost workers inside the cluster
Cluster.RootUfsCapacityFree GAUGE Free capacity of the Alluxio root UFS in bytes
Cluster.RootUfsCapacityTotal GAUGE Total capacity of the Alluxio root UFS in bytes
Cluster.RootUfsCapacityUsed GAUGE Used capacity of the Alluxio root UFS in bytes
Cluster.Workers GAUGE Total number of active workers inside the cluster

Server Metrics

Metrics shared by the Alluxio server processes.

NameTypeDescription
Server.JvmPauseMonitorInfoTimeExceeded GAUGE The total number of times that JVM slept and the sleep period is larger than the info level threshold defined by alluxio.jvm.monitor.info.threshold
Server.JvmPauseMonitorTotalExtraTime GAUGE The total time that JVM slept and didn't do GC
Server.JvmPauseMonitorWarnTimeExceeded GAUGE The total number of times that JVM slept and the sleep period is larger than the warn level threshold defined by alluxio.jvm.monitor.warn.threshold

Master Metrics

Default master metrics:

NameTypeDescription
Master.AbsentCacheHits GAUGE Number of cache hits on the absent cache
Master.AbsentCacheMisses GAUGE Number of cache misses on the absent cache
Master.AbsentCacheSize GAUGE Size of the absent cache
Master.AsyncPersistCancel COUNTER The number of cancelled AsyncPersist operations
Master.AsyncPersistFail COUNTER The number of failed AsyncPersist operations
Master.AsyncPersistFileCount COUNTER The number of files created by AsyncPersist operations
Master.AsyncPersistFileSize COUNTER The total size of files created by AsyncPersist operations
Master.AsyncPersistSuccess COUNTER The number of successful AsyncPersist operations
Master.AuditLogEntriesSize GAUGE The size of the audit log entries blocking queue
Master.BlockHeapSize GAUGE An estimate of the blocks heap size
Master.BlockReplicaCount GAUGE Total number of block replicas in Alluxio
Master.CompleteFileOps COUNTER Total number of the CompleteFile operations
Master.CompletedOperationRetryCount COUNTER Total number of completed operations that has been retried by client.
Master.CreateDirectoryOps COUNTER Total number of the CreateDirectory operations
Master.CreateFileOps COUNTER Total number of the CreateFile operations
Master.DeletePathOps COUNTER Total number of the Delete operations
Master.DirectoriesCreated COUNTER Total number of the succeed CreateDirectory operations
Master.EdgeCacheEvictions GAUGE Total number of edges (inode metadata) that was evicted from cache. The edge cache is responsible for managing the mapping from (parentId, childName) to childId.
Master.EdgeCacheHits GAUGE Total number of hits in the edge (inode metadata) cache. The edge cache is responsible for managing the mapping from (parentId, childName) to childId.
Master.EdgeCacheLoadTimes GAUGE Total load times in the edge (inode metadata) cache that resulted from a cache miss. The edge cache is responsible for managing the mapping from (parentId, childName) to childId.
Master.EdgeCacheMisses GAUGE Total number of misses in the edge (inode metadata) cache. The edge cache is responsible for managing the mapping from (parentId, childName) to childId.
Master.EdgeCacheSize GAUGE Total number of edges (inode metadata) cached. The edge cache is responsible for managing the mapping from (parentId, childName) to childId.
Master.EdgeLockPoolSize GAUGE The size of master edge lock pool
Master.EmbeddedJournalSnapshotDownloadGenerate TIMER Describes the amount of time taken to download journal snapshots from other masters in the cluster. Only valid when using the embedded journal. Use this metric to determine if there are potential communication bottlenecks between Alluxio masters.
Master.EmbeddedJournalSnapshotGenerateTimer TIMER Describes the amount of time taken to generate local journal snapshots on this master. Only valid when using the embedded journal. Use this metric to measure the performance of Alluxio's snapshot generation.
Master.EmbeddedJournalSnapshotInstallTimer TIMER Describes the amount of time taken to install a downloaded journal snapshot from another master. Only valid only when using the embedded journal. Use this metric to determine the performance of Alluxio when installing snapshots from the leader. Higher numbers may indicate a slow disk or CPU contention.
Master.EmbeddedJournalSnapshotLastIndex GAUGE Represents the latest journal index that was recorded by this master in the most recent local snapshot or from a snapshot downloaded from another master in the cluster. Only valid when using the embedded journal.
Master.EmbeddedJournalSnapshotReplayTimer TIMER Describes the amount of time taken to replay a journal snapshot onto the master's state machine. Only valid only when using the embedded journal. Use this metric to determine the performance of Alluxio when replaying journal snapshot file. Higher numbers may indicate a slow disk or CPU contention
Master.FileBlockInfosGot COUNTER Total number of succeed GetFileBlockInfo operations
Master.FileInfosGot COUNTER Total number of the succeed GetFileInfo operations
Master.FileSize GAUGE File size distribution
Master.FilesCompleted COUNTER Total number of the succeed CompleteFile operations
Master.FilesCreated COUNTER Total number of the succeed CreateFile operations
Master.FilesFreed COUNTER Total number of succeed FreeFile operations
Master.FilesPersisted COUNTER Total number of successfully persisted files
Master.FilesPinned GAUGE Total number of currently pinned files
Master.FilesToBePersisted GAUGE Total number of currently to be persisted files
Master.FreeFileOps COUNTER Total number of FreeFile operations
Master.GetFileBlockInfoOps COUNTER Total number of GetFileBlockInfo operations
Master.GetFileInfoOps COUNTER Total number of the GetFileInfo operations
Master.GetNewBlockOps COUNTER Total number of the GetNewBlock operations
Master.InodeCacheEvictions GAUGE Total number of inodes that was evicted from the cache.
Master.InodeCacheHitRatio GAUGE Inode Cache hit ratio
Master.InodeCacheHits GAUGE Total number of hits in the inodes (inode metadata) cache.
Master.InodeCacheLoadTimes GAUGE Total load times in the inodes (inode metadata) cache that resulted from a cache miss.
Master.InodeCacheMisses GAUGE Total number of misses in the inodes (inode metadata) cache.
Master.InodeCacheSize GAUGE Total number of inodes (inode metadata) cached.
Master.InodeHeapSize GAUGE An estimate of the inode heap size
Master.InodeLockPoolSize GAUGE The size of master inode lock pool
Master.JobCanceled COUNTER The number of canceled status job
Master.JobCompleted COUNTER The number of completed status job
Master.JobCount GAUGE The number of all status job
Master.JobCreated COUNTER The number of created status job
Master.JobDistributedLoadCancel COUNTER The number of cancelled DistributedLoad operations
Master.JobDistributedLoadFail COUNTER The number of failed DistributedLoad operations
Master.JobDistributedLoadFileCount COUNTER The number of files by DistributedLoad operations
Master.JobDistributedLoadFileSizes COUNTER The total file size by DistributedLoad operations
Master.JobDistributedLoadRate METER The average DistributedLoad loading rate
Master.JobDistributedLoadSuccess COUNTER The number of successful DistributedLoad operations
Master.JobFailed COUNTER The number of failed status job
Master.JobRunning COUNTER The number of running status job
Master.JournalCheckpointWarn GAUGE If the raft log index exceeds alluxio.master.journal.checkpoint.period.entries, and the last checkpoint exceeds alluxio.master.journal.checkpoint.warning.threshold.time, it returns 1 to indicate that a warning is required, otherwise it returns 0
Master.JournalEntriesSinceCheckPoint GAUGE Journal entries since last checkpoint
Master.JournalFlushFailure COUNTER Total number of failed journal flush
Master.JournalFlushTimer TIMER The timer statistics of journal flush
Master.JournalFreeBytes GAUGE Bytes left on the journal disk(s) for an Alluxio master. This metric is only valid on Linux and when embedded journal is used. Use this metric to monitor whether your journal is running out of disk space.
Master.JournalFreePercent GAUGE Percentage of free space left on the journal disk(s) for an Alluxio master.This metric is only valid on Linux and when embedded journal is used. Use this metric to monitor whether your journal is running out of disk space.
Master.JournalGainPrimacyTimer TIMER The timer statistics of journal gain primacy
Master.JournalLastAppliedCommitIndex GAUGE The last raft log index which was applied to the state machine
Master.JournalLastCheckPointTime GAUGE Last Journal Checkpoint Time
Master.JournalSequenceNumber GAUGE Current journal sequence number
Master.LastBackupEntriesCount GAUGE The total number of entries written in the last leading master metadata backup
Master.LastBackupRestoreCount GAUGE The total number of entries restored from backup when a leading master initializes its metadata
Master.LastBackupRestoreTimeMs GAUGE The process time of the last restore from backup
Master.LastBackupTimeMs GAUGE The process time of the last backup
Master.ListingCacheEvictions COUNTER The total number of evictions in master listing cache
Master.ListingCacheHits COUNTER The total number of hits in master listing cache
Master.ListingCacheLoadTimes COUNTER The total load time (in nanoseconds) in master listing cache that resulted from a cache miss.
Master.ListingCacheMisses COUNTER The total number of misses in master listing cache
Master.ListingCacheSize GAUGE The size of master listing cache
Master.LostBlockCount GAUGE Count of lost unique blocks
Master.LostFileCount GAUGE Count of lost files. This number is cached and may not be in sync with Master.LostBlockCount
Master.MigrateJobCancel COUNTER The number of cancelled MigrateJob operations
Master.MigrateJobFail COUNTER The number of failed MigrateJob operations
Master.MigrateJobFileCount COUNTER The number of MigrateJob files
Master.MigrateJobFileSize COUNTER The total size of MigrateJob files
Master.MigrateJobSuccess COUNTER The number of successful MigrateJob operations
Master.MountOps COUNTER Total number of Mount operations
Master.NewBlocksGot COUNTER Total number of the succeed GetNewBlock operations
Master.PathsDeleted COUNTER Total number of the succeed Delete operations
Master.PathsMounted COUNTER Total number of succeed Mount operations
Master.PathsRenamed COUNTER Total number of succeed Rename operations
Master.PathsUnmounted COUNTER Total number of succeed Unmount operations
Master.RenamePathOps COUNTER Total number of Rename operations
Master.ReplicaMgmtActiveJobSize GAUGE Number of active block replication/eviction jobs. These jobs are created by the master to maintain the block replica factor. The value is an estimate with lag.
Master.RoleId GAUGE Display master role id
Master.RpcQueueLength GAUGE Length of the master rpc queue. Use this metric to monitor the RPC pressure on master.
Master.SetAclOps COUNTER Total number of SetAcl operations
Master.SetAttributeOps COUNTER Total number of SetAttribute operations
Master.ToRemoveBlockCount GAUGE Count of block replicas to be removed from the workers. If 1 block is to be removed from 2 workers, 2 will be counted here.
Master.TotalPaths GAUGE Total number of files and directory in Alluxio namespace
Master.TotalRpcs TIMER Throughput of master RPC calls. This metrics indicates how busy the master is serving client and worker requests
Master.UfsJournalCatchupTimer TIMER The timer statistics of journal catchupOnly valid when ufs journal is used. This provides a summary of how long a standby master takes to catch up with primary master, and should be monitored if master transition takes too long
Master.UfsJournalFailureRecoverTimer TIMER The timer statistics of ufs journal failure recover
Master.UfsJournalInitialReplayTimeMs GAUGE The process time of the ufs journal initial replay.Only valid when ufs journal is used. It records the time it took for the very first journal replay. Use this metric to monitor when your master boot-up time is high。
Master.UniqueBlocks GAUGE Total number of unique blocks in Alluxio
Master.UnmountOps COUNTER Total number of Unmount operations

Dynamically generated master metrics:

Metric Name Description
Master.CapacityTotalTier{TIER_NAME} Total capacity in tier {TIER_NAME} of the Alluxio file system in bytes
Master.CapacityUsedTier{TIER_NAME} Used capacity in tier {TIER_NAME} of the Alluxio file system in bytes
Master.CapacityFreeTier{TIER_NAME} Free capacity in tier {TIER_NAME} of the Alluxio file system in bytes
Master.UfsSessionCount-Ufs:{UFS_ADDRESS} The total number of currently opened UFS sessions to connect to the given {UFS_ADDRESS}
Master.{UFS_RPC_NAME}.UFS:{UFS_ADDRESS}.UFS_TYPE:{UFS_TYPE}.User:{USER} The details UFS rpc operation done by the current master
Master.PerUfsOp{UFS_RPC_NAME}.UFS:{UFS_ADDRESS} The aggregated number of UFS operation {UFS_RPC_NAME} ran on UFS {UFS_ADDRESS} by leading master
Master.{LEADING_MASTER_RPC_NAME} The duration statistics of RPC calls exposed on leading master

Worker Metrics

Default worker metrics:

NameTypeDescription
Worker.ActiveClients COUNTER The number of clients actively reading from or writing to this worker
Worker.ActiveRpcReadCount COUNTER The number of active read-RPCs managed by this worker
Worker.ActiveRpcWriteCount COUNTER The number of active write-RPCs managed by this worker
Worker.BlockReaderCompleteTaskCount GAUGE The approximate total number of block read tasks that have completed execution
Worker.BlockReaderThreadActiveCount GAUGE The approximate number of block read threads that are actively executing tasks in reader thread pool
Worker.BlockReaderThreadCurrentCount GAUGE The current number of read threads in the reader thread pool
Worker.BlockReaderThreadMaxCount GAUGE The maximum allowed number of block read thread in the reader thread pool
Worker.BlockRemoverBlocksRemovedCount COUNTER The total number of blocks successfully removed from this worker by asynchronous block remover.
Worker.BlockRemoverRemovingBlocksSize GAUGE The size of blocks is being removed from this worker at a moment by asynchronous block remover.
Worker.BlockRemoverTryRemoveBlocksSize GAUGE The number of blocks to be removed from this worker at a moment by asynchronous block remover.
Worker.BlockRemoverTryRemoveCount COUNTER The total number of blocks this worker attempted to remove with asynchronous block remover.
Worker.BlockWriterCompleteTaskCount GAUGE The approximate total number of block write tasks that have completed execution
Worker.BlockWriterThreadActiveCount GAUGE The approximate number of block write threads that are actively executing tasks in writer thread pool
Worker.BlockWriterThreadCurrentCount GAUGE The current number of write threads in the writer thread pool
Worker.BlockWriterThreadMaxCount GAUGE The maximum allowed number of block write thread in the writer thread pool
Worker.BlocksAccessed COUNTER Total number of times any one of the blocks in this worker is accessed.
Worker.BlocksCached GAUGE Total number of blocks used for caching data in an Alluxio worker
Worker.BlocksCancelled COUNTER Total number of aborted temporary blocks in this worker.
Worker.BlocksDeleted COUNTER Total number of deleted blocks in this worker by external requests.
Worker.BlocksEvicted COUNTER Total number of evicted blocks in this worker.
Worker.BlocksEvictionRate METER Block eviction rate in this worker.
Worker.BlocksLost COUNTER Total number of lost blocks in this worker.
Worker.BlocksPromoted COUNTER Total number of times any one of the blocks in this worker moved to a new tier.
Worker.BlocksReadLocal COUNTER Total number of local blocks read by this worker.
Worker.BlocksReadRemote COUNTER Total number of a remote blocks read by this worker.
Worker.BlocksReadUfs COUNTER Total number of a UFS blocks read by this worker.
Worker.BytesReadDirect COUNTER Total number of bytes read from Alluxio storage managed by this worker and underlying UFS if data cannot be found in the Alluxio storage without external RPC involved. This records data read by worker internal calls (e.g. a client embedded in this worker).
Worker.BytesReadDirectThroughput METER Total number of bytes read from Alluxio storage managed by this worker and underlying UFS if data cannot be found in the Alluxio storage without external RPC involved. This records data read by worker internal calls (e.g. a client embedded in this worker).
Worker.BytesReadDomain COUNTER Total number of bytes read from Alluxio storage via domain socket by this worker
Worker.BytesReadDomainThroughput METER Bytes read throughput from Alluxio storage via domain socket by this worker
Worker.BytesReadPerUfs COUNTER Total number of bytes read from a specific Alluxio UFS by this worker
Worker.BytesReadRemote COUNTER Total number of bytes read from Alluxio storage managed by this worker and underlying UFS if data cannot be found in the Alluxio storage via external RPC channel. This does not include short-circuit local reads and domain socket reads.
Worker.BytesReadRemoteThroughput METER Total number of bytes read from Alluxio storage managed by this worker and underlying UFS if data cannot be found in the Alluxio storage via external RPC channel. This does not include short-circuit local reads and domain socket reads.
Worker.BytesReadUfsThroughput METER Bytes read throughput from all Alluxio UFSes by this worker
Worker.BytesWrittenDirect COUNTER Total number of bytes written to Alluxio storage managed by this worker without external RPC involved. This records data written by worker internal calls (e.g. a client embedded in this worker).
Worker.BytesWrittenDirectThroughput METER Total number of bytes written to Alluxio storage managed by this worker without external RPC involved. This records data written by worker internal calls (e.g. a client embedded in this worker).
Worker.BytesWrittenDomain COUNTER Total number of bytes written to Alluxio storage via domain socket by this worker
Worker.BytesWrittenDomainThroughput METER Throughput of bytes written to Alluxio storage via domain socket by this worker
Worker.BytesWrittenPerUfs COUNTER Total number of bytes written to a specific Alluxio UFS by this worker
Worker.BytesWrittenRemote COUNTER Total number of bytes written to Alluxio storage or the underlying UFS by this worker. This does not include short-circuit local writes and domain socket writes.
Worker.BytesWrittenRemoteThroughput METER Bytes write throughput to Alluxio storage or the underlying UFS by this workerThis does not include short-circuit local writes and domain socket writes.
Worker.BytesWrittenUfsThroughput METER Bytes write throughput to all Alluxio UFSes by this worker
Worker.CacheBlocksSize COUNTER Total number of bytes that being cached through cache requests
Worker.CacheFailedBlocks COUNTER Total number of failed cache blocks in this worker
Worker.CacheRemoteBlocks COUNTER Total number of blocks that need to be cached from remote source
Worker.CacheRequests COUNTER Total number of cache request received by this worker
Worker.CacheRequestsAsync COUNTER Total number of async cache request received by this worker
Worker.CacheRequestsSync COUNTER Total number of sync cache request received by this worker
Worker.CacheSucceededBlocks COUNTER Total number of cache succeeded blocks in this worker
Worker.CacheUfsBlocks COUNTER Total number of blocks that need to be cached from local source
Worker.CapacityFree GAUGE Total free bytes on all tiers of a specific Alluxio worker
Worker.CapacityTotal GAUGE Total capacity (in bytes) on all tiers of a specific Alluxio worker
Worker.CapacityUsed GAUGE Total used bytes on all tiers of a specific Alluxio worker
Worker.RpcQueueLength GAUGE Length of the worker rpc queue. Use this metric to monitor the RPC pressure on worker.

Dynamically generated worker metrics:

Metric Name Description
Worker.UfsSessionCount-Ufs:{UFS_ADDRESS} The total number of currently opened UFS sessions to connect to the given {UFS_ADDRESS}
Worker.{RPC_NAME} The duration statistics of RPC calls exposed on workers

Client Metrics

Each client metric will be recorded with its local hostname or alluxio.user.app.id is configured. If alluxio.user.app.id is configured, multiple clients can be combined into a logical application.

NameTypeDescription
Client.BlockReadChunkRemote TIMER The timer statistics of reading block data in chunks from remote Alluxio workers via RPC framework. This metrics will only be recorded when alluxio.user.block.read.metrics.enabled is set to true
Client.BytesReadLocal COUNTER Total number of bytes short-circuit read from local storage by this client
Client.BytesReadLocalThroughput METER Bytes throughput short-circuit read from local storage by this client
Client.BytesWrittenLocal COUNTER Total number of bytes short-circuit written to local storage by this client
Client.BytesWrittenLocalThroughput METER Bytes throughput short-circuit written to local storage by this client
Client.BytesWrittenUfs COUNTER Total number of bytes write to Alluxio UFS by this client
Client.CacheBytesEvicted METER Total number of bytes evicted from the client cache.
Client.CacheBytesReadCache METER Total number of bytes read from the client cache.
Client.CacheBytesReadExternal METER Total number of bytes read from external storage due to a cache miss on the client cache.
Client.CacheBytesRequestedExternal METER Total number of bytes the user requested to read which resulted in a cache miss. This number may be smaller than Client.CacheBytesReadExternal due to chunk reads.
Client.CacheBytesWrittenCache METER Total number of bytes written to the client cache.
Client.CacheCleanErrors COUNTER Number of failures when cleaning out the existing cache directory to initialize a new cache.
Client.CacheCleanupGetErrors COUNTER Number of failures when cleaning up a failed cache read.
Client.CacheCleanupPutErrors COUNTER Number of failures when cleaning up a failed cache write.
Client.CacheCreateErrors COUNTER Number of failures when creating a cache in the client cache.
Client.CacheDeleteErrors COUNTER Number of failures when deleting cached data in the client cache.
Client.CacheDeleteFromStoreErrors COUNTER Number of failures when deleting pages from page stores.
Client.CacheDeleteNonExistingPageErrors COUNTER Number of failures when deleting pages due to absence.
Client.CacheDeleteNotReadyErrors COUNTER Number of failures when cache is not ready to delete pages.
Client.CacheGetErrors COUNTER Number of failures when getting cached data in the client cache.
Client.CacheGetNotReadyErrors COUNTER Number of failures when cache is not ready to get pages.
Client.CacheGetStoreReadErrors COUNTER Number of failures when getting cached data in the client cache due to failed read from page stores.
Client.CacheHitRate GAUGE Cache hit rate: (# bytes read from cache) / (# bytes requested).
Client.CachePageReadCacheTimeNanos METER Time in nanoseconds taken to read a page from the client cache when the cache hits.
Client.CachePageReadExternalTimeNanos METER Time in nanoseconds taken to read a page from external source when the cache misses.
Client.CachePages COUNTER Total number of pages in the client cache.
Client.CachePagesEvicted METER Total number of pages evicted from the client cache.
Client.CachePutAsyncRejectionErrors COUNTER Number of failures when putting cached data in the client cache due to failed injection to async write queue.
Client.CachePutBenignRacingErrors COUNTER Number of failures when adding pages due to racing eviction. This error is benign.
Client.CachePutErrors COUNTER Number of failures when putting cached data in the client cache.
Client.CachePutEvictionErrors COUNTER Number of failures when putting cached data in the client cache due to failed eviction.
Client.CachePutInsufficientSpaceErrors COUNTER Number of failures when putting cached data in the client cache due to insufficient space made after eviction.
Client.CachePutNotReadyErrors COUNTER Number of failures when cache is not ready to add pages.
Client.CachePutStoreDeleteErrors COUNTER Number of failures when putting cached data in the client cache due to failed deletes in page store.
Client.CachePutStoreWriteErrors COUNTER Number of failures when putting cached data in the client cache due to failed writes to page store.
Client.CachePutStoreWriteNoSpaceErrors COUNTER Number of failures when putting cached data in the client cache but getting disk is full while cache capacity is not achieved. This can happen if the storage overhead ratio to write data is underestimated.
Client.CacheShadowCacheBytes COUNTER Amount of bytes in the client shadow cache.
Client.CacheShadowCacheBytesHit COUNTER Total number of bytes hit the client shadow cache.
Client.CacheShadowCacheBytesRead COUNTER Total number of bytes read from the client shadow cache.
Client.CacheShadowCacheFalsePositiveRatio COUNTER Probability that the working set bloom filter makes an error. The value is 0-100. If too high, need to allocate more space
Client.CacheShadowCachePages COUNTER Amount of pages in the client shadow cache.
Client.CacheShadowCachePagesHit COUNTER Total number of pages hit the client shadow cache.
Client.CacheShadowCachePagesRead COUNTER Total number of pages read from the client shadow cache.
Client.CacheSpaceAvailable GAUGE Amount of bytes available in the client cache.
Client.CacheSpaceUsed GAUGE Amount of bytes used by the client cache.
Client.CacheSpaceUsedCount COUNTER Amount of bytes used by the client cache as a counter.
Client.CacheState COUNTER State of the cache: 0 (NOT_IN_USE), 1 (READ_ONLY) and 2 (READ_WRITE)
Client.CacheStoreDeleteTimeout COUNTER Number of timeouts when deleting pages from page store.
Client.CacheStoreGetTimeout COUNTER Number of timeouts when reading pages from page store.
Client.CacheStorePutTimeout COUNTER Number of timeouts when writing new pages to page store.
Client.CacheStoreThreadsRejected COUNTER Number of rejection of I/O threads on submitting tasks to thread pool, likely due to unresponsive local file system.
Client.MetadataCacheSize GAUGE The total number of files and directories whose metadata is cached on the client-side. Only valid if the filesystem is alluxio.client.file.MetadataCachingBaseFileSystem.
Client.TotalRPCClients COUNTER The total number of RPC clients exist that is using to or can be used to connect to master or worker for operations. The sum of the sizes of FileSystemMasterClientPool, BlockMasterClientPool, and BlockWorkerClientPool.

Fuse Metrics

Fuse is a long-running Alluxio client. Depending on the launching ways, Fuse metrics show as

  • client metrics when Fuse client is launching in a standalone AlluxioFuse process.
  • worker metrics when Fuse client is embedded in the AlluxioWorker process.

Fuse metrics includes:

NameTypeDescription
Fuse.BytesRead COUNTER Total number of bytes read through Fuse.read() operations.
Fuse.BytesToRead COUNTER Total number of bytes requested by Fuse.read() operations.
Fuse.CachedPathCount GAUGE Total number of FUSE-to-Alluxio path mappings being cached. This value will be smaller or equal to alluxio.fuse.cached.paths.max
Fuse.ReadingFileCount GAUGE Total number of files being read concurrently.
Fuse.TotalCalls TIMER Throughput of JNI FUSE operation calls. This metrics indicates how busy the Alluxio Fuse application is serving requests
Fuse.WritingFileCount GAUGE Total number of files being written concurrently.

Fuse reading/writing file count can be used as the indicators for Fuse application pressure. If a large amount of concurrent read/write occur in a short period of time, each of the read/write operations may take longer time to finish.

When a user or an application runs a filesystem command under Fuse mount point, this command will be processed and translated by operating system which will trigger the related Fuse operations exposed in AlluxioFuse. The count of how many times each operation is called, and the duration of each call will be recorded with metrics name Fuse.<FUSE_OPERATION_NAME> dynamically.

The important Fuse metrics include:

Metric Name Description
Fuse.readdir The duration metrics of listing a directory
Fuse.getattr The duration metrics of getting the metadata of a file
Fuse.open The duration metrics of opening a file for read or overwrite
Fuse.read The duration metrics of reading a part of a file
Fuse.create The duration metrics of creating a file for write
Fuse.write The duration metrics of writing a file
Fuse.release The duration metrics of closing a file after read or write. Note that release is async so fuse threads will not wait for release to finish
Fuse.mkdir The duration metrics of creating a directory
Fuse.unlink The duration metrics of removing a file or a directory
Fuse.rename The duration metrics of renaming a file or a directory
Fuse.chmod The duration metrics of modifying the mode of a file or a directory
Fuse.chown The duration metrics of modifying the user and/or group ownership of a file or a directory

Fuse related metrics include:

  • Client.TotalRPCClientsshows the total number of RPC clients exist that is using to or can be used to connect to master or worker for operations.
  • Worker metrics with Direct keyword. When Fuse is embedded in worker process, it can go through worker internal API to read from / write to this worker. The related metrics are ended with Direct. For example, Worker.BytesReadDirect shows how many bytes are served by this worker to its embedded Fuse client for read.
  • If alluxio.user.block.read.metrics.enabled=true is configured, Client.BlockReadChunkRemote will be recorded. This metric shows the duration statistics of reading data from remote workers via gRPC.

Client.TotalRPCClients and Fuse.TotalCalls metrics are good indicator of the current load of the Fuse applications. If applications (e.g. Tensorflow) are running on top of Alluxio Fuse but these two metrics show a much lower value than before, the training job may be stuck with Alluxio.

Process Common Metrics

The following metrics are collected on each instance (Master, Worker or Client).

JVM Attributes

Metric Name Description
name The name of the JVM
uptime The uptime of the JVM
vendor The current JVM vendor

Garbage Collector Statistics

Metric Name Description
PS-MarkSweep.count Total number of mark and sweep
PS-MarkSweep.time The time used to mark and sweep
PS-Scavenge.count Total number of scavenge
PS-Scavenge.time The time used to scavenge

Memory Usage

Alluxio provides overall and detailed memory usage information. Detailed memory usage information of code cache, compressed class space, metaspace, PS Eden space, PS old gen, and PS survivor space is collected in each process.

A subset of the memory usage metrics are listed as following:

Metric Name Description
total.committed The amount of memory in bytes that is guaranteed to be available for use by the JVM
total.init The amount of the memory in bytes that is available for use by the JVM
total.max The maximum amount of memory in bytes that is available for use by the JVM
total.used The amount of memory currently used in bytes
heap.committed The amount of memory from heap area guaranteed to be available
heap.init The amount of memory from heap area available at initialization
heap.max The maximum amount of memory from heap area that is available
heap.usage The amount of memory from heap area currently used in GB
heap.used The amount of memory from heap area that has been used
pools.Code-Cache.used Used memory of collection usage from the pool from which memory is used for compilation and storage of native code
pools.Compressed-Class-Space.used Used memory of collection usage from the pool from which memory is use for class metadata
pools.PS-Eden-Space.used Used memory of collection usage from the pool from which memory is initially allocated for most objects
pools.PS-Survivor-Space.used Used memory of collection usage from the pool containing objects that have survived the garbage collection of the Eden space

ClassLoading Statistics

Metric Name Description
loaded The total number of classes loaded
unloaded The total number of unloaded classes

Thread Statistics

Metric Name Description
count The current number of live threads
daemon.count The current number of live daemon threads
peak.count The peak live thread count
total_started.count The total number of threads started
deadlock.count The number of deadlocked threads
deadlock The call stack of each thread related deadlock
new.count The number of threads with new state
blocked.count The number of threads with blocked state
runnable.count The number of threads with runnable state
terminated.count The number of threads with terminated state
timed_waiting.count The number of threads with timed_waiting state