Pulsar stats
Topic stats
The following table outlines the stats of a topic. For more details about how to get these stats, see Get stats.
All stats below are reset to 0 upon broker restart or topic unloading, except the stats marked with asterisks * (the values of them keep unchanged).
Stat | Description |
---|---|
msgRateIn | The sum of all local and replication publishers' publish rates (message per second). |
msgThroughputIn | The sum of all local and replication publishers' publish rates (byte per second). |
msgRateOut | The sum of all local and replication consumers' dispatch rates (message per second). |
msgThroughputOut | The sum of all local and replication consumers' dispatch rates (byte per second). |
averageMsgSize | The average size (bytes) of messages published within the last interval. |
storageSize* | The sum of the ledgers' storage size in BookKeeper and in tiered storage for a topic (in bytes). |
offloadedStorageSize* | The sum of the storage size in tiered storage for a topic (in bytes). Note: the total storage size of a topic = storageSize , includes offloadedStorageSize . |
earliestMsgPublishTimeInBacklogs* | The publish time of the earliest message in the backlog (in milliseconds). |
bytesInCounter | The total bytes published to the topic. |
msgInCounter | The total messages published to the topic. |
bytesOutCounter | The total bytes delivered to consumers. |
msgOutCounter | The total messages delivered to consumers. |
msgChunkPublished | The topics that have chunked messages published on it. |
backlogSize* | The estimated total unconsumed or backlog size (in bytes). |
oldestBacklogMessageAgeSeconds | The age of the oldest unacknowledged (i.e. backlog) message, measured by the time elapsed from its published time, in seconds. This value is recorded every backlog quota check interval, hence it represents the value seen in the last check. |
oldestBacklogMessageSubscriptionName | The name of the subscription containing the oldest unacknowledged message. This value is recorded every backlog quota check interval, hence it represents the value seen in the last check. |
backlogQuotaLimitSize | the size in bytes of the topic backlog quota. |
backlogQuotaLimitTime | the topic backlog age (time) quota, in seconds. |
waitingPublishers | The number of publishers waiting in a queue in exclusive access mode. |
deduplicationStatus | The status of message deduplication for the topic. |
topicEpoch | The topic epoch or empty if not set. |
filteredEntriesCount | The count of skipped entries for the topic. |
nonContiguousDeletedMessagesRanges | The number of non-contiguous deleted messages ranges. |
nonContiguousDeletedMessagesRangesSerializedSize | The serialized size of non-contiguous deleted messages ranges. |
ownerBroker | The broker that owns this topic. |
publishers | The list of all local publishers on the topic, ranging from zero to thousands. |
subscriptions | The list of all local subscriptions to the topic. |
replication | This section gives the stats for cross-colo replication of this topic. |
Producer stats
Stat | Description |
---|---|
producerId | The internal identifier for this producer on this topic. |
producerName | The internal identifier for this producer, generated by the client library. |
address | IP address and source port for the connection of this producer. |
connectedSince | The timestamp when this producer is created or reconnected last time. |
accessMode | The type of access to the topic that the producer requires. |
msgRateIn | The total rate of messages (message per second) published by this publisher. |
msgThroughputIn | The total throughput (byte per second) of the messages published by this publisher. |
averageMsgSize | The average message size in bytes from this publisher within the last interval. |
chunkedMessageRate | The total rate of chunked messages published by this publisher. |
clientVersion | The client library version of this producer. |
metadata | The metadata (key/value strings) associated with this publisher. |
Subscription stats
Stat | Description |
---|---|
my-subscription | The name of this subscription. It is defined by the client. |
msgRateOut | The total rate of messages (message per second) delivered on this subscription. |
msgThroughputOut | The total throughput (byte per second) delivered on this subscription. |
msgBacklog | The number of messages in the subscription backlog. |
type | This subscription type. |
msgRateExpired | The rate at which messages are discarded instead of dispatched from this subscription due to TTL. |
lastExpireTimestamp | The timestamp of the last message expiration. |
lastConsumedFlowTimestamp | The timestamp of the last flow command received. |
lastConsumedTimestamp | The latest timestamp of all the consumed timestamps of the consumers. |
lastAckedTimestamp | The latest timestamp of all the acknowledged timestamps of the consumers. |
msgRateRedeliver | The total rate of messages redelivered on this subscription (message per second). |
chunkedMessageRate | The chunked message dispatch rate. |
earliestMsgPublishTimeInBacklog* | The publish time of the earliest message in the backlog for the subscription (in milliseconds). |
msgBacklogNoDelayed | The number of messages in the subscription backlog that do not contain the delay messages. |
blockedSubscriptionOnUnackedMsgs | The flag to verify if a subscription is blocked due to reaching the threshold of unacked messages. |
msgDelayed | The number of delayed messages that are currently tracked. |
unackedMessages | The number of unacknowledged messages for the subscription, where an unacknowledged message is one that has been sent to a consumer but not yet acknowledged. This field is only meaningful when using a subscription that tracks individual message acknowledgment. |
activeConsumerName | The name of the consumer that is active for single active consumer subscriptions (such as failover or exclusive). |
totalMsgExpired | The total messages expired on this subscription. |
lastMarkDeleteAdvancedTimestamp | The last MarkDelete position advanced timestamp. |
durable | Whether the subscription is durable or ephemeral (for example, from a reader). |
replicated | Mark that the subscription state is kept in sync across different regions. |
allowOutOfOrderDelivery | Whether out-of-order delivery is allowed on the Key_Shared subscription. |
keySharedMode | The options of the Key_Shared subscription mode include AUTO_SPLIT or STICKY . |
consumersAfterMarkDeletePosition | Get recentJoinedConsumers for the Key_Shared subscription. |
filterProcessedMsgCount | The number of messages processed by EntryFilter . |
filterAcceptedMsgCount | The number of messages accepted by EntryFilter . |
filterRejectedMsgCount | The number of messages rejected by EntryFilter . |
filterRescheduledMsgCount | The number of messages rescheduled by EntryFilter . |
bytesOutCounter | The total bytes delivered to a consumer. |
msgOutCounter | The total messages delivered to a consumer. |
backlogSize* | The size of backlog for this subscription (in bytes). |
nonContiguousDeletedMessagesRanges | The number of non-contiguous deleted messages ranges. |
nonContiguousDeletedMessagesRangesSerializedSize | The serialized size of non-contiguous deleted messages ranges. |
consumers | The list of connected consumers for this subscription. |
Consumer stats
Stat | Description |
---|---|
consumerName | Internal identifier for this consumer, generated by the client library. |
msgRateOut | The total rate of messages (message per second) delivered to the consumer. |
msgThroughputOut | The total throughput (byte per second) delivered to the consumer. |
availablePermits | The number of messages that the consumer has space for in the client library's listening queue. 0 means the client library's queue is full and receive() isn't called. A non-zero value means this consumer is ready for dispatched messages. |
unackedMessages | The number of unacknowledged messages for the consumer, where an unacknowledged message has been sent to the consumer but not yet acknowledged. This field is only meaningful when using a subscription that tracks individual message acknowledgment. |
blockedConsumerOnUnackedMsgs | The flag used to verify if the consumer is blocked due to reaching the threshold of the unacknowledged messages. |
lastConsumedTimestamp (Deprecated) | The timestamp when the consumer reads a message the last time. |
lastConsumedTime | The time when the consumer reads a message the last time. |
lastAckedTimestamp (Deprecated) | The timestamp when the consumer acknowledges a message the last time. |
lastAckedTime | The time when the consumer acknowledges a message the last time. |
address | The IP address and source port for the connection of this consumer. |
connectedSince | The timestamp when this consumer is created or reconnected last time. |
clientVersion | The client library version of this consumer. |
bytesOutCounter | The total bytes delivered to a consumer. |
msgOutCounter | The total messages delivered to a consumer. |
msgRateRedeliver | The total rate of messages redelivered by this consumer (message per second). |
chunkedMessageRate | The total rate of chunked messages delivered to this consumer. |
avgMessagesPerEntry | The number of average messages per entry for the consumer consumed. |
readPositionWhenJoining | The read position of the cursor when the consumer joins. |
keyHashRanges | The hash ranges assigned to this consumer if it uses Key_Shared sub mode. |
metadata | The metadata (key/value strings) associated with this consumer. |
Replication stats
Stat | Description |
---|---|
msgRateIn | The total rate (message per second) of messages received from the remote cluster. |
msgThroughputIn | The total throughput (byte per second) received from the remote cluster. |
msgRateOut | The total rate of messages (message per second) delivered to the replication-subscriber. |
msgThroughputOut | The total throughput (byte per second) delivered to the replication-subscriber. |
msgRateExpired | The total rate of messages (message per second) expired. |
replicationBacklog | The number of messages pending to be replicated to remote cluster. |
connected | Whether the outbound replicator is connected. |
replicationDelayInSeconds | How long the oldest message has been waiting to be sent through the connection when connected. |
inboundConnection | The IP and port of the broker in the remote cluster's publisher connection to this broker. |
inboundConnectedSince | The TCP connection used to publish messages to the remote cluster. If no local publishers are connected, this connection is automatically closed after a minute. |
outboundConnection | The address of the outbound replication connection. |
outboundConnectedSince | The timestamp of establishing an outbound connection. |
Topic internal stats
The following table outlines the internal stats inside a topic. For more details about how to get these stats, see Get stats.
Stat | Description |
---|---|
entriesAddedCounter | Messages published since this broker loads this topic. |
numberOfEntries | The total number of messages tracked. |
totalSize | The total storage size in bytes of all messages. |
currentLedgerEntries | The count of messages written to the ledger that is currently open for writing. |
currentLedgerSize | The size in bytes of messages written to the ledger that is currently open for writing. |
lastLedgerCreatedTimestamp | The time when the last ledger is created. |
lastLedgerCreationFailureTimestamp | The time when the last ledger is failed. |
waitingCursorsCount | The number of cursors that are caught up and waiting for a new message to be published. |
pendingAddEntriesCount | The number of messages that have write requests (asynchronous) waiting on completion. |
lastConfirmedEntry | The ledgerid:entryid of the last message that is written successfully. If the entryid is -1 , then the ledger is opened or is currently opened but has no entries written yet. |
state | The state of the cursor ledger. The state LedgerOpened means that a ledger is open for saving published messages. |
ledgers | The ordered list of all ledgers for this topic that holds its messages. |
cursors | The list of all cursors on this topic. Each subscription in the topic stats has a cursor. |
markDeletePosition | The ack position: the last message that the subscriber acknowledges. All messages before this position are acknowledged by the subscriber. |
readPosition | The latest position of the subscriber for reading messages. |
waitingReadOp | This is true when the subscription reads the latest message that is published to the topic and waits for new messages to be published. |
pendingReadOps | The count of outstanding read requests to the BookKeepers in progress. |
messagesConsumedCounter | The number of messages this cursor has acknowledged since this broker loads this topic. |
cursorLedger | The ledger used to persistently store the current markDeletePosition . |
cursorLedgerLastEntry | The last entryid used to persistently store the current markDeletePosition . |
individuallyDeletedMessages | The range of messages acknowledged between markDeletePosition and the readPosition when acknowledges are done out of order. |
lastLedgerSwitchTimestamp | The last time when the cursor ledger is rolled over. |
schemaLedgers | The ordered list of all ledgers for this topic schema. |
compactedLedger | The ledgers holding un-acked messages after topic compaction. |
ledgerId | The ID of this ledger. |
entries | The total number of entries that belong to this ledger. |
size | The size of messages written to this ledger (in bytes). |
offloaded | Whether this ledger is offloaded. The value is false for the compacted topic ledger. |
metadata | The ledger metadata. |