MINOR: Store metrics scope, total metrics (#5290)

1. Rename metrics scope of rocksDB window and session stores; also modify the store metrics accordingly with guidance on its correlations to metricsScope.

2. Add the missing total metrics for per-thread, per-task, per-node and per-store sensors.


Reviewers: John Roesler <john@confluent.io>, Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>
This commit is contained in:
Guozhang Wang 2018-06-28 14:32:02 -07:00 committed by GitHub
parent be0f10e190
commit 6bfaf4dc60
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 154 additions and 37 deletions

View File

@ -1321,22 +1321,42 @@ All the following metrics have a recording level of ``info``:
</tr> </tr>
<tr> <tr>
<td>commit-rate</td> <td>commit-rate</td>
<td>The average number of commits per second across all tasks.</td> <td>The average number of commits per second.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td> <td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr> </tr>
<tr>
<td>commit-total</td>
<td>The total number of commit calls across all tasks.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr>
<tr> <tr>
<td>poll-rate</td> <td>poll-rate</td>
<td>The average number of polls per second across all tasks.</td> <td>The average number of polls per second.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td> <td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr> </tr>
<tr>
<td>poll-total</td>
<td>The total number of poll calls across all tasks.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr>
<tr> <tr>
<td>process-rate</td> <td>process-rate</td>
<td>The average number of process calls per second across all tasks.</td> <td>The average number of process calls per second.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td> <td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr> </tr>
<tr>
<td>process-total</td>
<td>The total number of process calls across all tasks.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr>
<tr> <tr>
<td>punctuate-rate</td> <td>punctuate-rate</td>
<td>The average number of punctuates per second across all tasks.</td> <td>The average number of punctuates per second.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr>
<tr>
<td>punctuate-total</td>
<td>The total number of punctuate calls across all tasks.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td> <td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
@ -1344,11 +1364,21 @@ All the following metrics have a recording level of ``info``:
<td>The average number of newly created tasks per second.</td> <td>The average number of newly created tasks per second.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td> <td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr> </tr>
<tr>
<td>task-created-total</td>
<td>The total number of tasks created.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr>
<tr> <tr>
<td>task-closed-rate</td> <td>task-closed-rate</td>
<td>The average number of tasks closed per second.</td> <td>The average number of tasks closed per second.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td> <td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr> </tr>
<tr>
<td>task-closed-total</td>
<td>The total number of tasks closed.</td>
<td>kafka.streams:type=stream-metrics,client-id=([-.\w]+)</td>
</tr>
<tr> <tr>
<td>skipped-records-rate</td> <td>skipped-records-rate</td>
<td>The average number of skipped records per second.</td> <td>The average number of skipped records per second.</td>
@ -1386,11 +1416,16 @@ All the following metrics have a recording level of ``debug``:
<td>The average number of commit calls per second. </td> <td>The average number of commit calls per second. </td>
<td>kafka.streams:type=stream-task-metrics,client-id=([-.\w]+),task-id=([-.\w]+)</td> <td>kafka.streams:type=stream-task-metrics,client-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr> </tr>
<tr>
<td>commit-total</td>
<td>The total number of commit calls. </td>
<td>kafka.streams:type=stream-task-metrics,client-id=([-.\w]+),task-id=([-.\w]+)</td>
</tr>
</tbody> </tbody>
</table> </table>
<h5><a id="kafka_streams_node_monitoring" href="#kafka_streams_node_monitoring">Processor Node Metrics</a></h5> <h5><a id="kafka_streams_node_monitoring" href="#kafka_streams_node_monitoring">Processor Node Metrics</a></h5>
All the following metrics have a recording level of ``debug``: All the following metrics have a recording level of ``debug``:
<table class="data-table"> <table class="data-table">
<tbody> <tbody>
<tr> <tr>
@ -1443,33 +1478,60 @@ All the following metrics have a recording level of ``debug``:
<td>The average number of process operations per second. </td> <td>The average number of process operations per second. </td>
<td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td> <td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr> </tr>
<tr>
<td>process-total</td>
<td>The total number of process operations called. </td>
<td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr>
<tr> <tr>
<td>punctuate-rate</td> <td>punctuate-rate</td>
<td>The average number of punctuate operations per second. </td> <td>The average number of punctuate operations per second. </td>
<td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td> <td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr> </tr>
<tr>
<td>punctuate-total</td>
<td>The total number of punctuate operations called. </td>
<td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr>
<tr> <tr>
<td>create-rate</td> <td>create-rate</td>
<td>The average number of create operations per second. </td> <td>The average number of create operations per second. </td>
<td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td> <td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr> </tr>
<tr>
<td>create-total</td>
<td>The total number of create operations called. </td>
<td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr>
<tr> <tr>
<td>destroy-rate</td> <td>destroy-rate</td>
<td>The average number of destroy operations per second. </td> <td>The average number of destroy operations per second. </td>
<td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td> <td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr> </tr>
<tr>
<td>destroy-total</td>
<td>The total number of destroy operations called. </td>
<td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr>
<tr> <tr>
<td>forward-rate</td> <td>forward-rate</td>
<td>The average rate of records being forwarded downstream, from source nodes only, per second. </td> <td>The average rate of records being forwarded downstream, from source nodes only, per second. </td>
<td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td> <td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr> </tr>
<tr>
<td>forward-total</td>
<td>The total number of of records being forwarded downstream, from source nodes only. </td>
<td>kafka.streams:type=stream-processor-node-metrics,client-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)</td>
</tr>
</tbody> </tbody>
</table> </table>
<h5><a id="kafka_streams_store_monitoring" href="#kafka_streams_store_monitoring">State Store Metrics</a></h5> <h5><a id="kafka_streams_store_monitoring" href="#kafka_streams_store_monitoring">State Store Metrics</a></h5>
All the following metrics have a recording level of ``debug``: All the following metrics have a recording level of ``debug``. Note that the ``store-scope`` value is specified in <code>StoreSupplier#metricsScope()</code> for user's customized
state stores; for built-in state stores, currently we have <code>in-memory-state</code>, <code>in-memory-lru-state</code>, <code>rocksdb-state</code> (for RocksDB backed key-value store),
<code>rocksdb-window-state</code> (for RocksDB backed window store) and <code>rocksdb-session-state</code> (for RocksDB backed session store).
<table class="data-table"> <table class="data-table">
<tbody> <tbody>
<tr> <tr>
<th>Metric/Attribute name</th> <th>Metric/Attribute name</th>
@ -1479,143 +1541,188 @@ All the following metrics have a recording level of ``debug``:
<tr> <tr>
<td>put-latency-avg</td> <td>put-latency-avg</td>
<td>The average put execution time in ns. </td> <td>The average put execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>put-latency-max</td> <td>put-latency-max</td>
<td>The maximum put execution time in ns. </td> <td>The maximum put execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>put-if-absent-latency-avg</td> <td>put-if-absent-latency-avg</td>
<td>The average put-if-absent execution time in ns. </td> <td>The average put-if-absent execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>put-if-absent-latency-max</td> <td>put-if-absent-latency-max</td>
<td>The maximum put-if-absent execution time in ns. </td> <td>The maximum put-if-absent execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>get-latency-avg</td> <td>get-latency-avg</td>
<td>The average get execution time in ns. </td> <td>The average get execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>get-latency-max</td> <td>get-latency-max</td>
<td>The maximum get execution time in ns. </td> <td>The maximum get execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>delete-latency-avg</td> <td>delete-latency-avg</td>
<td>The average delete execution time in ns. </td> <td>The average delete execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>delete-latency-max</td> <td>delete-latency-max</td>
<td>The maximum delete execution time in ns. </td> <td>The maximum delete execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>put-all-latency-avg</td> <td>put-all-latency-avg</td>
<td>The average put-all execution time in ns. </td> <td>The average put-all execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>put-all-latency-max</td> <td>put-all-latency-max</td>
<td>The maximum put-all execution time in ns. </td> <td>The maximum put-all execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>all-latency-avg</td> <td>all-latency-avg</td>
<td>The average all operation execution time in ns. </td> <td>The average all operation execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>all-latency-max</td> <td>all-latency-max</td>
<td>The maximum all operation execution time in ns. </td> <td>The maximum all operation execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>range-latency-avg</td> <td>range-latency-avg</td>
<td>The average range execution time in ns. </td> <td>The average range execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>range-latency-max</td> <td>range-latency-max</td>
<td>The maximum range execution time in ns. </td> <td>The maximum range execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>flush-latency-avg</td> <td>flush-latency-avg</td>
<td>The average flush execution time in ns. </td> <td>The average flush execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>flush-latency-max</td> <td>flush-latency-max</td>
<td>The maximum flush execution time in ns. </td> <td>The maximum flush execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>restore-latency-avg</td> <td>restore-latency-avg</td>
<td>The average restore execution time in ns. </td> <td>The average restore execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>restore-latency-max</td> <td>restore-latency-max</td>
<td>The maximum restore execution time in ns. </td> <td>The maximum restore execution time in ns. </td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>put-rate</td> <td>put-rate</td>
<td>The average put rate for this store.</td> <td>The average put rate for this store.</td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>put-total</td>
<td>The total number of put calls for this store.</td>
<td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>put-if-absent-rate</td> <td>put-if-absent-rate</td>
<td>The average put-if-absent rate for this store.</td> <td>The average put-if-absent rate for this store.</td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>put-if-absent-total</td>
<td>The total number of put-if-absent calls for this store.</td>
<td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>get-rate</td> <td>get-rate</td>
<td>The average get rate for this store.</td> <td>The average get rate for this store.</td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>get-total</td>
<td>The total number of get calls for this store.</td>
<td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>delete-rate</td> <td>delete-rate</td>
<td>The average delete rate for this store.</td> <td>The average delete rate for this store.</td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>delete-total</td>
<td>The total number of delete calls for this store.</td>
<td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>put-all-rate</td> <td>put-all-rate</td>
<td>The average put-all rate for this store.</td> <td>The average put-all rate for this store.</td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>put-all-total</td>
<td>The total number of put-all calls for this store.</td>
<td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>all-rate</td> <td>all-rate</td>
<td>The average all operation rate for this store.</td> <td>The average all operation rate for this store.</td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>all-total</td>
<td>The total number of all operation calls for this store.</td>
<td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>range-rate</td> <td>range-rate</td>
<td>The average range rate for this store.</td> <td>The average range rate for this store.</td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>range-total</td>
<td>The total number of range calls for this store.</td>
<td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>flush-rate</td> <td>flush-rate</td>
<td>The average flush rate for this store.</td> <td>The average flush rate for this store.</td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>flush-total</td>
<td>The total number of flush calls for this store.</td>
<td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
<tr> <tr>
<td>restore-rate</td> <td>restore-rate</td>
<td>The average restore rate for this store.</td> <td>The average restore rate for this store.</td>
<td>kafka.streams:type=stream-[store-type]-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-type]-state-id=([-.\w]+)</td> <td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr>
<tr>
<td>restore-total</td>
<td>The total number of restore calls for this store.</td>
<td>kafka.streams:type=stream-[store-scope]-metrics,client-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)</td>
</tr> </tr>
</tbody> </tbody>
</table> </table>
<h5><a id="kafka_streams_cache_monitoring" href="#kafka_streams_cache_monitoring">Record Cache Metrics</a></h5> <h5><a id="kafka_streams_cache_monitoring" href="#kafka_streams_cache_monitoring">Record Cache Metrics</a></h5>
All the following metrics have a recording level of ``debug``: All the following metrics have a recording level of ``debug``:
<table class="data-table"> <table class="data-table">
<tbody> <tbody>

View File

@ -123,6 +123,16 @@
<li>Fetched records having a negative timestamp.</li> <li>Fetched records having a negative timestamp.</li>
</ul> </ul>
<p>
We've also fixed the metrics name for time and session windowed store operations in 2.0. As a result, our current built-in stores
will have their store types in the metric names as <code>in-memory-state</code>, <code>in-memory-lru-state</code>,
<code>rocksdb-state</code>, <code>rocksdb-window-state</code>, and <code>rocksdb-session-state</code>. For example, a RocksDB time windowed store's
put operation metrics would now be
<code>kafka.streams:type=stream-rocksdb-window-state-metrics,client-id=([-.\w]+),task-id=([-.\w]+),rocksdb-window-state-id=([-.\w]+)</code>.
Users need to update their metrics collecting and reporting systems for their time and session windowed stores accordingly.
For more details, please read the <a href="/{{version}}/documentation/ops.html#kafka_streams_store_monitoring">State Store Metrics</a> section.
</p>
<p> <p>
We have added support for methods in <code>ReadOnlyWindowStore</code> which allows for querying a single window's key-value pair. We have added support for methods in <code>ReadOnlyWindowStore</code> which allows for querying a single window's key-value pair.
For users who have customized window store implementations on the above interface, they'd need to update their code to implement the newly added method as well. For users who have customized window store implementations on the above interface, they'd need to update their code to implement the newly added method as well.

View File

@ -48,7 +48,7 @@ public class RocksDbSessionBytesStoreSupplier implements SessionBytesStoreSuppli
@Override @Override
public String metricsScope() { public String metricsScope() {
return "rocksdb-session"; return "rocksdb-session-state";
} }
@Override @Override

View File

@ -63,7 +63,7 @@ public class RocksDbWindowBytesStoreSupplier implements WindowBytesStoreSupplier
@Override @Override
public String metricsScope() { public String metricsScope() {
return "rocksdb-window"; return "rocksdb-window-state";
} }
@Override @Override