mirror of https://github.com/apache/kafka.git
MINOR: improve Kafka Streams metrics documentation (#17900)
Reviewers: Bruno Cadonna <bruno@confluent.io>, Guozhang Wang <guozhang.wang.us@gmail.com>
This commit is contained in:
parent
21563380f3
commit
a0a501952b
120
docs/ops.html
120
docs/ops.html
|
@ -2753,7 +2753,7 @@ These metrics are reported on both Controllers and Brokers in a KRaft Cluster
|
|||
<pre><code>metrics.recording.level="info"</code></pre>
|
||||
|
||||
<h5 class="anchor-heading"><a id="kafka_streams_client_monitoring" class="anchor-link"></a><a href="#kafka_streams_client_monitoring">Client Metrics</a></h5>
|
||||
All of the following metrics have a recording level of <code>info</code>:
|
||||
All the following metrics have a recording level of <code>info</code>:
|
||||
<table class="data-table">
|
||||
<tbody>
|
||||
<tr>
|
||||
|
@ -2800,7 +2800,7 @@ All of the following metrics have a recording level of <code>info</code>:
|
|||
</table>
|
||||
|
||||
<h5 class="anchor-heading"><a id="kafka_streams_thread_monitoring" class="anchor-link"></a><a href="#kafka_streams_thread_monitoring">Thread Metrics</a></h5>
|
||||
All of the following metrics have a recording level of <code>info</code>:
|
||||
All the following metrics have a recording level of <code>info</code>:
|
||||
<table class="data-table">
|
||||
<tbody>
|
||||
<tr>
|
||||
|
@ -2850,7 +2850,7 @@ All of the following metrics have a recording level of <code>info</code>:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>commit-rate</td>
|
||||
<td>The average number of commits per second.</td>
|
||||
<td>The average number of commits per sec.</td>
|
||||
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -2860,7 +2860,7 @@ All of the following metrics have a recording level of <code>info</code>:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>poll-rate</td>
|
||||
<td>The average number of consumer poll calls per second.</td>
|
||||
<td>The average number of consumer poll calls per sec.</td>
|
||||
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -2870,7 +2870,7 @@ All of the following metrics have a recording level of <code>info</code>:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>process-rate</td>
|
||||
<td>The average number of processed records per second.</td>
|
||||
<td>The average number of processed records per sec.</td>
|
||||
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -2880,7 +2880,7 @@ All of the following metrics have a recording level of <code>info</code>:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>punctuate-rate</td>
|
||||
<td>The average number of punctuate calls per second.</td>
|
||||
<td>The average number of punctuate calls per sec.</td>
|
||||
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -2890,7 +2890,7 @@ All of the following metrics have a recording level of <code>info</code>:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>task-created-rate</td>
|
||||
<td>The average number of tasks created per second.</td>
|
||||
<td>The average number of tasks created per sec.</td>
|
||||
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -2900,7 +2900,7 @@ All of the following metrics have a recording level of <code>info</code>:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>task-closed-rate</td>
|
||||
<td>The average number of tasks closed per second.</td>
|
||||
<td>The average number of tasks closed per sec.</td>
|
||||
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -2910,19 +2910,19 @@ All of the following metrics have a recording level of <code>info</code>:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>blocked-time-ns-total</td>
|
||||
<td>The total time the thread spent blocked on kafka.</td>
|
||||
<td>The total time in ns the thread spent blocked on Kafka brokers.</td>
|
||||
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>thread-start-time</td>
|
||||
<td>The time that the thread was started.</td>
|
||||
<td>The system timestamp in ms that the thread was started.</td>
|
||||
<td>kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
</tbody>
|
||||
</table>
|
||||
|
||||
<h5 class="anchor-heading"><a id="kafka_streams_task_monitoring" class="anchor-link"></a><a href="#kafka_streams_task_monitoring">Task Metrics</a></h5>
|
||||
All of the following metrics have a recording level of <code>debug</code>, except for the dropped-records-* and
|
||||
All the following metrics have a recording level of <code>debug</code>, except for the dropped-records-* and
|
||||
active-process-ratio metrics which have a recording level of <code>info</code>:
|
||||
<table class="data-table">
|
||||
<tbody>
|
||||
|
@ -2943,7 +2943,7 @@ active-process-ratio metrics which have a recording level of <code>info</code>:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>process-rate</td>
|
||||
<td>The average number of processed records per second across all source processor nodes of this task.</td>
|
||||
<td>The average number of processed records per sec across all source processor nodes of this task.</td>
|
||||
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -2953,17 +2953,17 @@ active-process-ratio metrics which have a recording level of <code>info</code>:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>record-lateness-avg</td>
|
||||
<td>The average observed lateness of records (stream time - record timestamp).</td>
|
||||
<td>The average observed lateness in ms of records (stream time - record timestamp).</td>
|
||||
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>record-lateness-max</td>
|
||||
<td>The max observed lateness of records (stream time - record timestamp).</td>
|
||||
<td>The max observed lateness in ms of records (stream time - record timestamp).</td>
|
||||
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>enforced-processing-rate</td>
|
||||
<td>The average number of enforced processings per second.</td>
|
||||
<td>The average number of enforced processings per sec.</td>
|
||||
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -2973,7 +2973,7 @@ active-process-ratio metrics which have a recording level of <code>info</code>:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>dropped-records-rate</td>
|
||||
<td>The average number of records dropped within this task.</td>
|
||||
<td>The average number of records dropped per sec within this task.</td>
|
||||
<td>kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3005,7 +3005,7 @@ active-process-ratio metrics which have a recording level of <code>info</code>:
|
|||
<code>emit-final-*</code> metrics are only available for windowed aggregations nodes, and the
|
||||
<code>record-e2e-latency-*</code> metrics are only available for source processor nodes and terminal nodes (nodes without successor
|
||||
nodes).
|
||||
All of the metrics have a recording level of <code>debug</code>, except for the <code>record-e2e-latency-*</code> metrics which have
|
||||
All the metrics have a recording level of <code>debug</code>, except for the <code>record-e2e-latency-*</code> metrics which have
|
||||
a recording level of <code>info</code>:
|
||||
<table class="data-table">
|
||||
<tbody>
|
||||
|
@ -3026,17 +3026,17 @@ active-process-ratio metrics which have a recording level of <code>info</code>:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>process-rate</td>
|
||||
<td>The average number of records processed by a source processor node per second.</td>
|
||||
<td>The average number of records processed by a source processor node per sec.</td>
|
||||
<td>kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>process-total</td>
|
||||
<td>The total number of records processed by a source processor node per second.</td>
|
||||
<td>The total number of records processed by a source processor node per sec.</td>
|
||||
<td>kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>suppression-emit-rate</td>
|
||||
<td>The rate at which records that have been emitted downstream from suppression operation nodes.</td>
|
||||
<td>The rate of records emitted per sec that have been emitted downstream from suppression operation nodes.</td>
|
||||
<td>kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3046,17 +3046,17 @@ active-process-ratio metrics which have a recording level of <code>info</code>:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>emit-final-latency-max</td>
|
||||
<td>The max latency to emit final records when a record could be emitted.</td>
|
||||
<td>The max latency in ms to emit final records when a record could be emitted.</td>
|
||||
<td>kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>emit-final-latency-avg</td>
|
||||
<td>The avg latency to emit final records when a record could be emitted.</td>
|
||||
<td>The avg latency in ms to emit final records when a record could be emitted.</td>
|
||||
<td>kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>emit-final-records-rate</td>
|
||||
<td>The rate of records emitted when records could be emitted.</td>
|
||||
<td>The rate of records emitted per sec when records could be emitted.</td>
|
||||
<td>kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3066,17 +3066,17 @@ active-process-ratio metrics which have a recording level of <code>info</code>:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>record-e2e-latency-avg</td>
|
||||
<td>The average end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
|
||||
<td>The average end-to-end latency in ms of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
|
||||
<td>kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>record-e2e-latency-max</td>
|
||||
<td>The maximum end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
|
||||
<td>The maximum end-to-end latency in ms of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
|
||||
<td>kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>record-e2e-latency-min</td>
|
||||
<td>The minimum end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
|
||||
<td>The minimum end-to-end latency in ms of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
|
||||
<td>kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3219,42 +3219,42 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>put-rate</td>
|
||||
<td>The average put rate for this store.</td>
|
||||
<td>The average put rate per sec for this store.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>put-if-absent-rate</td>
|
||||
<td>The average put-if-absent rate for this store.</td>
|
||||
<td>The average put-if-absent rate per sec for this store.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>get-rate</td>
|
||||
<td>The average get rate for this store.</td>
|
||||
<td>The average get rate per sec for this store.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>delete-rate</td>
|
||||
<td>The average delete rate for this store.</td>
|
||||
<td>The average delete rate per sec for this store.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>put-all-rate</td>
|
||||
<td>The average put-all rate for this store.</td>
|
||||
<td>The average put-all rate per sec for this store.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>all-rate</td>
|
||||
<td>The average all operation rate for this store.</td>
|
||||
<td>The average all operation rate per sec for this store.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>range-rate</td>
|
||||
<td>The average range rate for this store.</td>
|
||||
<td>The average range rate per sec for this store.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>prefix-scan-rate</td>
|
||||
<td>The average prefix-scan rate for this store.</td>
|
||||
<td>The average prefix-scan rate per sec for this store.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3269,7 +3269,7 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>suppression-buffer-size-avg</td>
|
||||
<td>The average total size, in bytes, of the buffered data over the sampling window.</td>
|
||||
<td>The average total size in bytes of the buffered data over the sampling window.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),in-memory-suppression-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3289,17 +3289,17 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>record-e2e-latency-avg</td>
|
||||
<td>The average end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
|
||||
<td>The average end-to-end latency in ms of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>record-e2e-latency-max</td>
|
||||
<td>The maximum end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
|
||||
<td>The maximum end-to-end latency in ms of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>record-e2e-latency-min</td>
|
||||
<td>The minimum end-to-end latency of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
|
||||
<td>The minimum end-to-end latency in ms of a record, measured by comparing the record timestamp with the system time when it has been fully processed by the node.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3309,17 +3309,17 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>iterator-duration-avg</td>
|
||||
<td>The average time spent between creating an iterator and closing it, in nanoseconds.</td>
|
||||
<td>The average time in ns spent between creating an iterator and closing it.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>iterator-duration-max</td>
|
||||
<td>The maximum time spent between creating an iterator and closing it, in nanoseconds.</td>
|
||||
<td>The maximum time in ns spent between creating an iterator and closing it.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>oldest-iterator-open-since-ms</td>
|
||||
<td>The UNIX timestamp the oldest still open iterator was created, in milliseconds.</td>
|
||||
<td>The system timestamp in ms the oldest still open iterator was created.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
</tbody>
|
||||
|
@ -3339,7 +3339,7 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</ul>
|
||||
|
||||
<strong>RocksDB Statistics-based Metrics:</strong>
|
||||
All of the following statistics-based metrics have a recording level of <code>debug</code> because collecting
|
||||
All the following statistics-based metrics have a recording level of <code>debug</code> because collecting
|
||||
statistics in <a href="https://github.com/facebook/rocksdb/wiki/Statistics#stats-level-and-performance-costs">RocksDB
|
||||
may have an impact on performance</a>.
|
||||
Statistics-based metrics are collected every minute from the RocksDB state stores.
|
||||
|
@ -3355,7 +3355,7 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>bytes-written-rate</td>
|
||||
<td>The average number of bytes written per second to the RocksDB state store.</td>
|
||||
<td>The average number of bytes written per sec to the RocksDB state store.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3375,7 +3375,7 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>memtable-bytes-flushed-rate</td>
|
||||
<td>The average number of bytes flushed per second from the memtable to disk.</td>
|
||||
<td>The average number of bytes flushed per sec from the memtable to disk.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3390,7 +3390,7 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>memtable-flush-time-avg</td>
|
||||
<td>The average duration of memtable flushes to disc in ms.</td>
|
||||
<td>The average duration in ms of memtable flushes to disc.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3400,7 +3400,7 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>memtable-flush-time-max</td>
|
||||
<td>The maximum duration of memtable flushes to disc in ms.</td>
|
||||
<td>The maximum duration in ms of memtable flushes to disc.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3420,27 +3420,27 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>write-stall-duration-avg</td>
|
||||
<td>The average duration of write stalls in ms.</td>
|
||||
<td>The average duration in ms of write stalls.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>write-stall-duration-total</td>
|
||||
<td>The total duration of write stalls in ms.</td>
|
||||
<td>The total duration in ms of write stalls.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>bytes-read-compaction-rate</td>
|
||||
<td>The average number of bytes read per second during compaction.</td>
|
||||
<td>The average number of bytes read per sec during compaction.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>bytes-written-compaction-rate</td>
|
||||
<td>The average number of bytes written per second during compaction.</td>
|
||||
<td>The average number of bytes written per sec during compaction.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>compaction-time-avg</td>
|
||||
<td>The average duration of disc compactions in ms.</td>
|
||||
<td>The average duration in ms of disc compactions.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3450,7 +3450,7 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>compaction-time-max</td>
|
||||
<td>The maximum duration of disc compactions in ms.</td>
|
||||
<td>The maximum duration in ms of disc compactions.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3467,7 +3467,7 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</table>
|
||||
|
||||
<strong>RocksDB Properties-based Metrics:</strong>
|
||||
All of the following properties-based metrics have a recording level of <code>info</code> and are recorded when the
|
||||
All the following properties-based metrics have a recording level of <code>info</code> and are recorded when the
|
||||
metrics are accessed.
|
||||
If a state store consists of multiple RocksDB instances, as is the case for WindowStores and SessionStores,
|
||||
each metric reports the sum over all the RocksDB instances of the state store, except for the block cache metrics
|
||||
|
@ -3489,17 +3489,17 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>cur-size-active-mem-table</td>
|
||||
<td>The approximate size of the active memtable in bytes.</td>
|
||||
<td>The approximate size in bytes of the active memtable.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>cur-size-all-mem-tables</td>
|
||||
<td>The approximate size of active and unflushed immutable memtables in bytes.</td>
|
||||
<td>The approximate size in bytes of active and unflushed immutable memtables.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>size-all-mem-tables</td>
|
||||
<td>The approximate size of active, unflushed immutable, and pinned immutable memtables in bytes.</td>
|
||||
<td>The approximate size in bytes of active, unflushed immutable, and pinned immutable memtables.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3565,17 +3565,17 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</tr>
|
||||
<tr>
|
||||
<td>block-cache-capacity</td>
|
||||
<td>The capacity of the block cache in bytes.</td>
|
||||
<td>The capacity in bytes of the block cache.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>block-cache-usage</td>
|
||||
<td>The memory size of the entries residing in block cache in bytes.</td>
|
||||
<td>The memory size in bytes of the entries residing in block cache.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>block-cache-pinned-usage</td>
|
||||
<td>The memory size for the entries being pinned in the block cache in bytes.</td>
|
||||
<td>The memory size in bytes for the entries being pinned in the block cache.</td>
|
||||
<td>kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
|
||||
</tr>
|
||||
<tr>
|
||||
|
@ -3597,7 +3597,7 @@ customized state stores; for built-in state stores, currently we have:
|
|||
</table>
|
||||
|
||||
<h5 class="anchor-heading"><a id="kafka_streams_cache_monitoring" class="anchor-link"></a><a href="#kafka_streams_cache_monitoring">Record Cache Metrics</a></h5>
|
||||
All of the following metrics have a recording level of <code>debug</code>:
|
||||
All the following metrics have a recording level of <code>debug</code>:
|
||||
|
||||
<table class="data-table">
|
||||
<tbody>
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.kafka.common.metrics.Sensor.RecordingLevel;
|
|||
import java.util.Map;
|
||||
|
||||
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.AVG_LATENCY_DESCRIPTION;
|
||||
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.LATENCY_DESCRIPTION_SUFFIX;
|
||||
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.LATENCY_SUFFIX;
|
||||
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.MAX_LATENCY_DESCRIPTION;
|
||||
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.PROCESSOR_NODE_LEVEL_GROUP;
|
||||
|
@ -68,8 +69,8 @@ public class ProcessorNodeMetrics {
|
|||
|
||||
private static final String EMIT_FINAL_LATENCY = EMITTED_RECORDS + LATENCY_SUFFIX;
|
||||
private static final String EMIT_FINAL_DESCRIPTION = "calls to emit final";
|
||||
private static final String EMIT_FINAL_AVG_LATENCY_DESCRIPTION = AVG_LATENCY_DESCRIPTION + EMIT_FINAL_DESCRIPTION;
|
||||
private static final String EMIT_FINAL_MAX_LATENCY_DESCRIPTION = MAX_LATENCY_DESCRIPTION + EMIT_FINAL_DESCRIPTION;
|
||||
private static final String EMIT_FINAL_AVG_LATENCY_DESCRIPTION = AVG_LATENCY_DESCRIPTION + EMIT_FINAL_DESCRIPTION + LATENCY_DESCRIPTION_SUFFIX;
|
||||
private static final String EMIT_FINAL_MAX_LATENCY_DESCRIPTION = MAX_LATENCY_DESCRIPTION + EMIT_FINAL_DESCRIPTION + LATENCY_DESCRIPTION_SUFFIX;
|
||||
|
||||
public static Sensor suppressionEmitSensor(final String threadId,
|
||||
final String taskId,
|
||||
|
|
|
@ -151,6 +151,7 @@ public class StreamsMetricsImpl implements StreamsMetrics {
|
|||
public static final String RATIO_DESCRIPTION = "The fraction of time the thread spent on ";
|
||||
public static final String AVG_LATENCY_DESCRIPTION = "The average latency of ";
|
||||
public static final String MAX_LATENCY_DESCRIPTION = "The maximum latency of ";
|
||||
public static final String LATENCY_DESCRIPTION_SUFFIX = " in milliseconds";
|
||||
public static final String RATE_DESCRIPTION_PREFIX = "The average number of ";
|
||||
public static final String RATE_DESCRIPTION_SUFFIX = " per second";
|
||||
|
||||
|
|
Loading…
Reference in New Issue