1# Statistics
2
3librdkafka may be configured to emit internal metrics at a fixed interval
4by setting the `statistics.interval.ms` configuration property to a value > 0
5and registering a `stats_cb` (or similar, depending on language).
6
7The stats are provided as a JSON object string.
8
9**Note**: The metrics returned may not be completely consistent between
10          brokers, toppars and totals, due to the internal asynchronous
11          nature of librdkafka.
12          E.g., the top level `tx` total may be less than the sum of
13          the broker `tx` values which it represents.
14
15
16## General structure
17
18All fields that contain sizes are are in bytes unless otherwise noted.
19
20```
21{
22 <Top-level fields>
23 "brokers": {
24    <brokers fields>,
25    "toppars": { <toppars fields> }
26 },
27 "topics": {
28   <topic fields>,
29   "partitions": {
30     <partitions fields>
31   }
32 }
33[, "cgrp": { <cgrp fields> } ]
34[, "eos": { <eos fields> } ]
35}
36```
37
38## Field type
39
40Fields are represented as follows:
41 * string - UTF8 string.
42 * int - Integer counter (64 bits wide). Ever increasing.
43 * int gauge - Integer gauge (64 bits wide). Will be reset to 0 on each stats emit.
44 * object - Nested JSON object.
45 * bool - `true` or `false`.
46
47
48## Top-level
49
50Field | Type | Example | Description
51----- | ---- | ------- | -----------
52name | string | `"rdkafka#producer-1"` | Handle instance name
53client_id | string | `"rdkafka"` | The configured (or default) `client.id`
54type | string | `"producer"` | Instance type (producer or consumer)
55ts | int | 12345678912345 | librdkafka's internal monotonic clock (micro seconds)
56time | int | | Wall clock time in seconds since the epoch
57replyq | int gauge | | Number of ops (callbacks, events, etc) waiting in queue for application to serve with rd_kafka_poll()
58msg_cnt | int gauge | | Current number of messages in producer queues
59msg_size | int gauge | | Current total size of messages in producer queues
60msg_max | int | | Threshold: maximum number of messages allowed allowed on the producer queues
61msg_size_max | int | | Threshold: maximum total size of messages allowed on the producer queues
62tx | int | | Total number of requests sent to Kafka brokers
63tx_bytes | int | | Total number of bytes transmitted to Kafka brokers
64rx | int | | Total number of responses received from Kafka brokers
65rx_bytes | int | | Total number of bytes received from Kafka brokers
66txmsgs | int | | Total number of messages transmitted (produced) to Kafka brokers
67txmsg_bytes | int | | Total number of message bytes (including framing, such as per-Message framing and MessageSet/batch framing) transmitted to Kafka brokers
68rxmsgs | int | | Total number of messages consumed, not including ignored messages (due to offset, etc), from Kafka brokers.
69rxmsg_bytes | int | | Total number of message bytes (including framing) received from Kafka brokers
70simple_cnt | int gauge | | Internal tracking of legacy vs new consumer API state
71metadata_cache_cnt | int gauge | | Number of topics in the metadata cache.
72brokers | object | | Dict of brokers, key is broker name, value is object. See **brokers** below
73topics | object | | Dict of topics, key is topic name, value is object. See **topics** below
74cgrp | object | | Consumer group metrics. See **cgrp** below
75eos | object | | EOS / Idempotent producer state and metrics. See **eos** below
76
77## brokers
78
79Per broker statistics.
80
81Field | Type | Example | Description
82----- | ---- | ------- | -----------
83name | string | `"example.com:9092/13"` | Broker hostname, port and broker id
84nodeid | int | 13 | Broker id (-1 for bootstraps)
85nodename | string | `"example.com:9092"` | Broker hostname
86source | string | `"configured"` | Broker source (learned, configured, internal, logical)
87state | string | `"UP"` | Broker state (INIT, DOWN, CONNECT, AUTH, APIVERSION_QUERY, AUTH_HANDSHAKE, UP, UPDATE)
88stateage | int gauge | | Time since last broker state change (microseconds)
89outbuf_cnt | int gauge | | Number of requests awaiting transmission to broker
90outbuf_msg_cnt | int gauge | | Number of messages awaiting transmission to broker
91waitresp_cnt | int gauge | | Number of requests in-flight to broker awaiting response
92waitresp_msg_cnt | int gauge | | Number of messages in-flight to broker awaitign response
93tx | int | | Total number of requests sent
94txbytes | int | | Total number of bytes sent
95txerrs | int | | Total number of transmission errors
96txretries | int | | Total number of request retries
97req_timeouts | int | | Total number of requests timed out
98rx | int | | Total number of responses received
99rxbytes | int | | Total number of bytes received
100rxerrs | int | | Total number of receive errors
101rxcorriderrs | int | | Total number of unmatched correlation ids in response (typically for timed out requests)
102rxpartial | int | | Total number of partial MessageSets received. The broker may return partial responses if the full MessageSet could not fit in remaining Fetch response size.
103req | object | | Request type counters. Object key is the request name, value is the number of requests sent.
104zbuf_grow | int | | Total number of decompression buffer size increases
105buf_grow | int | | Total number of buffer size increases (deprecated, unused)
106wakeups | int | | Broker thread poll wakeups
107connects | int | | Number of connection attempts, including successful and failed, and name resolution failures.
108disconnects | int | | Number of disconnects (triggered by broker, network, load-balancer, etc.).
109int_latency | object | | Internal producer queue latency in microseconds. See *Window stats* below
110outbuf_latency | object | | Internal request queue latency in microseconds. This is the time between a request is enqueued on the transmit (outbuf) queue and the time the request is written to the TCP socket. Additional buffering and latency may be incurred by the TCP stack and network. See *Window stats* below
111rtt | object | | Broker latency / round-trip time in microseconds. See *Window stats* below
112throttle | object | | Broker throttling time in milliseconds. See *Window stats* below
113toppars | object | | Partitions handled by this broker handle. Key is "topic-partition". See *brokers.toppars* below
114
115
116## Window stats
117
118Rolling window statistics. The values are in microseconds unless otherwise stated.
119
120Field | Type | Example | Description
121----- | ---- | ------- | -----------
122min | int gauge | | Smallest value
123max | int gauge | | Largest value
124avg | int gauge | | Average value
125sum | int gauge | | Sum of values
126cnt | int gauge | | Number of values sampled
127stddev | int gauge | | Standard deviation (based on histogram)
128hdrsize | int gauge | | Memory size of Hdr Histogram
129p50 | int gauge | | 50th percentile
130p75 | int gauge | | 75th percentile
131p90 | int gauge | | 90th percentile
132p95 | int gauge | | 95th percentile
133p99 | int gauge | | 99th percentile
134p99_99 | int gauge | | 99.99th percentile
135outofrange | int gauge | | Values skipped due to out of histogram range
136
137
138## brokers.toppars
139
140Topic partition assigned to broker.
141
142Field | Type | Example | Description
143----- | ---- | ------- | -----------
144topic | string | `"mytopic"` | Topic name
145partition | int | 3 | Partition id
146
147## topics
148
149Field | Type | Example | Description
150----- | ---- | ------- | -----------
151topic | string | `"myatopic"` | Topic name
152metadata_age | int gauge | | Age of metadata from broker for this topic (milliseconds)
153batchsize | object | | Batch sizes in bytes. See *Window stats*·
154batchcnt | object | | Batch message counts. See *Window stats*·
155partitions | object | | Partitions dict, key is partition id. See **partitions** below.
156
157
158## partitions
159
160Field | Type | Example | Description
161----- | ---- | ------- | -----------
162partition | int | 3 | Partition Id (-1 for internal UA/UnAssigned partition)
163broker | int | | The id of the broker that messages are currently being fetched from
164leader | int | | Current leader broker id
165desired | bool | | Partition is explicitly desired by application
166unknown | bool | | Partition not seen in topic metadata from broker
167msgq_cnt | int gauge | | Number of messages waiting to be produced in first-level queue
168msgq_bytes | int gauge | | Number of bytes in msgq_cnt
169xmit_msgq_cnt | int gauge | | Number of messages ready to be produced in transmit queue
170xmit_msgq_bytes | int gauge | | Number of bytes in xmit_msgq
171fetchq_cnt | int gauge | | Number of pre-fetched messages in fetch queue
172fetchq_size | int gauge | | Bytes in fetchq
173fetch_state | string | `"active"` | Consumer fetch state for this partition (none, stopping, stopped, offset-query, offset-wait, active).
174query_offset | int gauge | | Current/Last logical offset query
175next_offset | int gauge | | Next offset to fetch
176app_offset | int gauge | | Offset of last message passed to application + 1
177stored_offset | int gauge | | Offset to be committed
178committed_offset | int gauge | | Last committed offset
179eof_offset | int gauge | | Last PARTITION_EOF signaled offset
180lo_offset | int gauge | | Partition's low watermark offset on broker
181hi_offset | int gauge | | Partition's high watermark offset on broker
182ls_offset | int gauge | | Partition's last stable offset on broker, or same as hi_offset is broker version is less than 0.11.0.0.
183consumer_lag | int gauge | | Difference between (hi_offset or ls_offset) - max(app_offset, committed_offset). hi_offset is used when isolation.level=read_uncommitted, otherwise ls_offset.
184txmsgs | int | | Total number of messages transmitted (produced)
185txbytes | int | | Total number of bytes transmitted for txmsgs
186rxmsgs | int | | Total number of messages consumed, not including ignored messages (due to offset, etc).
187rxbytes | int | | Total number of bytes received for rxmsgs
188msgs | int | | Total number of messages received (consumer, same as rxmsgs), or total number of messages produced (possibly not yet transmitted) (producer).
189rx_ver_drops | int | | Dropped outdated messages
190msgs_inflight | int gauge | | Current number of messages in-flight to/from broker
191next_ack_seq | int gauge | | Next expected acked sequence (idempotent producer)
192next_err_seq | int gauge | | Next expected errored sequence (idempotent producer)
193acked_msgid | int | | Last acked internal message id (idempotent producer)
194
195## cgrp
196
197Field | Type | Example | Description
198----- | ---- | ------- | -----------
199state | string | "up"    | Local consumer group handler's state.
200stateage | int gauge | | Time elapsed since last state change (milliseconds).
201joinstate | string | "assigned" | Local consumer group handler's join state.
202rebalance_age | int gauge | | Time elapsed since last rebalance (assign or revoke) (milliseconds).
203rebalance_cnt | int | | Total number of rebalances (assign or revoke).
204rebalance_reason | string | | Last rebalance reason, or empty string.
205assignment_size | int gauge | | Current assignment's partition count.
206
207
208## eos
209
210Field | Type | Example | Description
211----- | ---- | ------- | -----------
212idemp_state | string | "Assigned" | Current idempotent producer id state.
213idemp_stateage | int gauge | | Time elapsed since last idemp_state change (milliseconds).
214txn_state | string | "InTransaction" | Current transactional producer state.
215txn_stateage | int gauge | | Time elapsed since last txn_state change (milliseconds).
216txn_may_enq | bool | | Transactional state allows enqueuing (producing) new messages.
217producer_id | int gauge | | The currently assigned Producer ID (or -1).
218producer_epoch | int gauge | | The current epoch (or -1).
219epoch_cnt | int | | The number of Producer ID assignments since start.
220
221
222# Example output
223
224This (prettified) example output is from a short-lived producer using the following command:
225`rdkafka_performance -b localhost -P -t test -T 1000 -Y 'cat >> stats.json'`.
226
227Note: this output is prettified using `jq .`, the JSON object emitted by librdkafka does not contain line breaks.
228
229```json
230{
231  "name": "rdkafka#producer-1",
232  "client_id": "rdkafka",
233  "type": "producer",
234  "ts": 5016483227792,
235  "time": 1527060869,
236  "replyq": 0,
237  "msg_cnt": 22710,
238  "msg_size": 704010,
239  "msg_max": 500000,
240  "msg_size_max": 1073741824,
241  "simple_cnt": 0,
242  "metadata_cache_cnt": 1,
243  "brokers": {
244    "localhost:9092/2": {
245      "name": "localhost:9092/2",
246      "nodeid": 2,
247      "nodename": "localhost:9092",
248      "source": "learned",
249      "state": "UP",
250      "stateage": 9057234,
251      "outbuf_cnt": 0,
252      "outbuf_msg_cnt": 0,
253      "waitresp_cnt": 0,
254      "waitresp_msg_cnt": 0,
255      "tx": 320,
256      "txbytes": 84283332,
257      "txerrs": 0,
258      "txretries": 0,
259      "req_timeouts": 0,
260      "rx": 320,
261      "rxbytes": 15708,
262      "rxerrs": 0,
263      "rxcorriderrs": 0,
264      "rxpartial": 0,
265      "zbuf_grow": 0,
266      "buf_grow": 0,
267      "wakeups": 591067,
268      "int_latency": {
269        "min": 86,
270        "max": 59375,
271        "avg": 23726,
272        "sum": 5694616664,
273        "stddev": 13982,
274        "p50": 28031,
275        "p75": 36095,
276        "p90": 39679,
277        "p95": 43263,
278        "p99": 48639,
279        "p99_99": 59391,
280        "outofrange": 0,
281        "hdrsize": 11376,
282        "cnt": 240012
283      },
284      "rtt": {
285        "min": 1580,
286        "max": 3389,
287        "avg": 2349,
288        "sum": 79868,
289        "stddev": 474,
290        "p50": 2319,
291        "p75": 2543,
292        "p90": 3183,
293        "p95": 3199,
294        "p99": 3391,
295        "p99_99": 3391,
296        "outofrange": 0,
297        "hdrsize": 13424,
298        "cnt": 34
299      },
300      "throttle": {
301        "min": 0,
302        "max": 0,
303        "avg": 0,
304        "sum": 0,
305        "stddev": 0,
306        "p50": 0,
307        "p75": 0,
308        "p90": 0,
309        "p95": 0,
310        "p99": 0,
311        "p99_99": 0,
312        "outofrange": 0,
313        "hdrsize": 17520,
314        "cnt": 34
315      },
316      "toppars": {
317        "test-1": {
318          "topic": "test",
319          "partition": 1
320        }
321      }
322    },
323    "localhost:9093/3": {
324      "name": "localhost:9093/3",
325      "nodeid": 3,
326      "nodename": "localhost:9093",
327      "source": "learned",
328      "state": "UP",
329      "stateage": 9057209,
330      "outbuf_cnt": 0,
331      "outbuf_msg_cnt": 0,
332      "waitresp_cnt": 0,
333      "waitresp_msg_cnt": 0,
334      "tx": 310,
335      "txbytes": 84301122,
336      "txerrs": 0,
337      "txretries": 0,
338      "req_timeouts": 0,
339      "rx": 310,
340      "rxbytes": 15104,
341      "rxerrs": 0,
342      "rxcorriderrs": 0,
343      "rxpartial": 0,
344      "zbuf_grow": 0,
345      "buf_grow": 0,
346      "wakeups": 607956,
347      "int_latency": {
348        "min": 82,
349        "max": 58069,
350        "avg": 23404,
351        "sum": 5617432101,
352        "stddev": 14021,
353        "p50": 27391,
354        "p75": 35839,
355        "p90": 39679,
356        "p95": 42751,
357        "p99": 48639,
358        "p99_99": 58111,
359        "outofrange": 0,
360        "hdrsize": 11376,
361        "cnt": 240016
362      },
363      "rtt": {
364        "min": 1704,
365        "max": 3572,
366        "avg": 2493,
367        "sum": 87289,
368        "stddev": 559,
369        "p50": 2447,
370        "p75": 2895,
371        "p90": 3375,
372        "p95": 3407,
373        "p99": 3583,
374        "p99_99": 3583,
375        "outofrange": 0,
376        "hdrsize": 13424,
377        "cnt": 35
378      },
379      "throttle": {
380        "min": 0,
381        "max": 0,
382        "avg": 0,
383        "sum": 0,
384        "stddev": 0,
385        "p50": 0,
386        "p75": 0,
387        "p90": 0,
388        "p95": 0,
389        "p99": 0,
390        "p99_99": 0,
391        "outofrange": 0,
392        "hdrsize": 17520,
393        "cnt": 35
394      },
395      "toppars": {
396        "test-0": {
397          "topic": "test",
398          "partition": 0
399        }
400      }
401    },
402    "localhost:9094/4": {
403      "name": "localhost:9094/4",
404      "nodeid": 4,
405      "nodename": "localhost:9094",
406      "source": "learned",
407      "state": "UP",
408      "stateage": 9057207,
409      "outbuf_cnt": 0,
410      "outbuf_msg_cnt": 0,
411      "waitresp_cnt": 0,
412      "waitresp_msg_cnt": 0,
413      "tx": 1,
414      "txbytes": 25,
415      "txerrs": 0,
416      "txretries": 0,
417      "req_timeouts": 0,
418      "rx": 1,
419      "rxbytes": 272,
420      "rxerrs": 0,
421      "rxcorriderrs": 0,
422      "rxpartial": 0,
423      "zbuf_grow": 0,
424      "buf_grow": 0,
425      "wakeups": 4,
426      "int_latency": {
427        "min": 0,
428        "max": 0,
429        "avg": 0,
430        "sum": 0,
431        "stddev": 0,
432        "p50": 0,
433        "p75": 0,
434        "p90": 0,
435        "p95": 0,
436        "p99": 0,
437        "p99_99": 0,
438        "outofrange": 0,
439        "hdrsize": 11376,
440        "cnt": 0
441      },
442      "rtt": {
443        "min": 0,
444        "max": 0,
445        "avg": 0,
446        "sum": 0,
447        "stddev": 0,
448        "p50": 0,
449        "p75": 0,
450        "p90": 0,
451        "p95": 0,
452        "p99": 0,
453        "p99_99": 0,
454        "outofrange": 0,
455        "hdrsize": 13424,
456        "cnt": 0
457      },
458      "throttle": {
459        "min": 0,
460        "max": 0,
461        "avg": 0,
462        "sum": 0,
463        "stddev": 0,
464        "p50": 0,
465        "p75": 0,
466        "p90": 0,
467        "p95": 0,
468        "p99": 0,
469        "p99_99": 0,
470        "outofrange": 0,
471        "hdrsize": 17520,
472        "cnt": 0
473      },
474      "toppars": {}
475    }
476  },
477  "topics": {
478    "test": {
479      "topic": "test",
480      "metadata_age": 9060,
481      "batchsize": {
482        "min": 99,
483        "max": 391805,
484        "avg": 272593,
485        "sum": 18808985,
486        "stddev": 180408,
487        "p50": 393215,
488        "p75": 393215,
489        "p90": 393215,
490        "p95": 393215,
491        "p99": 393215,
492        "p99_99": 393215,
493        "outofrange": 0,
494        "hdrsize": 14448,
495        "cnt": 69
496      },
497      "batchcnt": {
498        "min": 1,
499        "max": 10000,
500        "avg": 6956,
501        "sum": 480028,
502        "stddev": 4608,
503        "p50": 10047,
504        "p75": 10047,
505        "p90": 10047,
506        "p95": 10047,
507        "p99": 10047,
508        "p99_99": 10047,
509        "outofrange": 0,
510        "hdrsize": 8304,
511        "cnt": 69
512      },
513      "partitions": {
514        "0": {
515          "partition": 0,
516          "broker": 3,
517          "leader": 3,
518          "desired": false,
519          "unknown": false,
520          "msgq_cnt": 1,
521          "msgq_bytes": 31,
522          "xmit_msgq_cnt": 0,
523          "xmit_msgq_bytes": 0,
524          "fetchq_cnt": 0,
525          "fetchq_size": 0,
526          "fetch_state": "none",
527          "query_offset": 0,
528          "next_offset": 0,
529          "app_offset": -1001,
530          "stored_offset": -1001,
531          "commited_offset": -1001,
532          "committed_offset": -1001,
533          "eof_offset": -1001,
534          "lo_offset": -1001,
535          "hi_offset": -1001,
536          "consumer_lag": -1,
537          "txmsgs": 2150617,
538          "txbytes": 66669127,
539          "rxmsgs": 0,
540          "rxbytes": 0,
541          "msgs": 2160510,
542          "rx_ver_drops": 0
543        },
544        "1": {
545          "partition": 1,
546          "broker": 2,
547          "leader": 2,
548          "desired": false,
549          "unknown": false,
550          "msgq_cnt": 0,
551          "msgq_bytes": 0,
552          "xmit_msgq_cnt": 0,
553          "xmit_msgq_bytes": 0,
554          "fetchq_cnt": 0,
555          "fetchq_size": 0,
556          "fetch_state": "none",
557          "query_offset": 0,
558          "next_offset": 0,
559          "app_offset": -1001,
560          "stored_offset": -1001,
561          "commited_offset": -1001,
562          "committed_offset": -1001,
563          "eof_offset": -1001,
564          "lo_offset": -1001,
565          "hi_offset": -1001,
566          "consumer_lag": -1,
567          "txmsgs": 2150136,
568          "txbytes": 66654216,
569          "rxmsgs": 0,
570          "rxbytes": 0,
571          "msgs": 2159735,
572          "rx_ver_drops": 0
573        },
574        "-1": {
575          "partition": -1,
576          "broker": -1,
577          "leader": -1,
578          "desired": false,
579          "unknown": false,
580          "msgq_cnt": 0,
581          "msgq_bytes": 0,
582          "xmit_msgq_cnt": 0,
583          "xmit_msgq_bytes": 0,
584          "fetchq_cnt": 0,
585          "fetchq_size": 0,
586          "fetch_state": "none",
587          "query_offset": 0,
588          "next_offset": 0,
589          "app_offset": -1001,
590          "stored_offset": -1001,
591          "commited_offset": -1001,
592          "committed_offset": -1001,
593          "eof_offset": -1001,
594          "lo_offset": -1001,
595          "hi_offset": -1001,
596          "consumer_lag": -1,
597          "txmsgs": 0,
598          "txbytes": 0,
599          "rxmsgs": 0,
600          "rxbytes": 0,
601          "msgs": 1177,
602          "rx_ver_drops": 0
603        }
604      }
605    }
606  },
607  "tx": 631,
608  "tx_bytes": 168584479,
609  "rx": 631,
610  "rx_bytes": 31084,
611  "txmsgs": 4300753,
612  "txmsg_bytes": 133323343,
613  "rxmsgs": 0,
614  "rxmsg_bytes": 0
615}
616```
617