Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[KIP-714] Avg and max produce latency plus more tests #4847

Merged
merged 1 commit into from
Oct 3, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@
librdkafka v2.6.0 is a feature release:

* [KIP-714] Complete consumer metrics support (#4808).
* [KIP-714] Produce latency average and maximum metrics support for parity with Java client (#4847).
* Fix for permanent fetch errors when using a newer Fetch RPC version with an older
inter broker protocol (#4806).

Expand Down
47 changes: 38 additions & 9 deletions src/rdkafka_broker.c
Original file line number Diff line number Diff line change
Expand Up @@ -1873,16 +1873,31 @@ static rd_kafka_buf_t *rd_kafka_waitresp_find(rd_kafka_broker_t *rkb,
rd_avg_add(&rkb->rkb_avg_rtt, rkbuf->rkbuf_ts_sent);
rd_avg_add(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_rtt,
rkbuf->rkbuf_ts_sent);
if (rkbuf->rkbuf_reqhdr.ApiKey == RD_KAFKAP_Fetch) {
rd_avg_add(&rkb->rkb_telemetry.rd_avg_current
.rkb_avg_fetch_latency,
rkbuf->rkbuf_ts_sent);
} else if (rkbuf->rkbuf_reqhdr.ApiKey ==
RD_KAFKAP_OffsetCommit) {
rd_avg_add(&rkb->rkb_rk->rk_telemetry.rd_avg_current
.rk_avg_commit_latency,
rkbuf->rkbuf_ts_sent);

switch (rkbuf->rkbuf_reqhdr.ApiKey) {
case RD_KAFKAP_Fetch:
if (rkb->rkb_rk->rk_type == RD_KAFKA_CONSUMER)
rd_avg_add(&rkb->rkb_telemetry.rd_avg_current
.rkb_avg_fetch_latency,
rkbuf->rkbuf_ts_sent);
break;
case RD_KAFKAP_OffsetCommit:
if (rkb->rkb_rk->rk_type == RD_KAFKA_CONSUMER)
rd_avg_add(
&rkb->rkb_rk->rk_telemetry.rd_avg_current
.rk_avg_commit_latency,
rkbuf->rkbuf_ts_sent);
break;
case RD_KAFKAP_Produce:
if (rkb->rkb_rk->rk_type == RD_KAFKA_PRODUCER)
rd_avg_add(&rkb->rkb_telemetry.rd_avg_current
.rkb_avg_produce_latency,
rkbuf->rkbuf_ts_sent);
break;
default:
break;
}

if (rkbuf->rkbuf_flags & RD_KAFKA_OP_F_BLOCKING &&
rd_atomic32_sub(&rkb->rkb_blocking_request_cnt, 1) == 1)
rd_kafka_brokers_broadcast_state_change(rkb->rkb_rk);
Expand Down Expand Up @@ -4818,11 +4833,17 @@ void rd_kafka_broker_destroy_final(rd_kafka_broker_t *rkb) {
&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_outbuf_latency);
rd_avg_destroy(
&rkb->rkb_telemetry.rd_avg_current.rkb_avg_outbuf_latency);

if (rkb->rkb_rk->rk_type == RD_KAFKA_CONSUMER) {
rd_avg_destroy(
&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_fetch_latency);
rd_avg_destroy(
&rkb->rkb_telemetry.rd_avg_current.rkb_avg_fetch_latency);
} else if (rkb->rkb_rk->rk_type == RD_KAFKA_PRODUCER) {
rd_avg_destroy(
&rkb->rkb_telemetry.rd_avg_current.rkb_avg_produce_latency);
rd_avg_destroy(&rkb->rkb_telemetry.rd_avg_rollover
.rkb_avg_produce_latency);
}


Expand Down Expand Up @@ -4937,6 +4958,7 @@ rd_kafka_broker_t *rd_kafka_broker_add(rd_kafka_t *rk,
rd_avg_init(&rkb->rkb_telemetry.rd_avg_current.rkb_avg_outbuf_latency,
RD_AVG_GAUGE, 0, 100 * 1000, 2,
rk->rk_conf.enable_metrics_push);

if (rk->rk_type == RD_KAFKA_CONSUMER) {
rd_avg_init(
&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_fetch_latency,
Expand All @@ -4946,6 +4968,13 @@ rd_kafka_broker_t *rd_kafka_broker_add(rd_kafka_t *rk,
&rkb->rkb_telemetry.rd_avg_current.rkb_avg_fetch_latency,
RD_AVG_GAUGE, 0, 500 * 1000, 2,
rk->rk_conf.enable_metrics_push);
} else if (rk->rk_type == RD_KAFKA_PRODUCER) {
rd_avg_init(
&rkb->rkb_telemetry.rd_avg_current.rkb_avg_produce_latency,
RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
rd_avg_init(
&rkb->rkb_telemetry.rd_avg_rollover.rkb_avg_produce_latency,
RD_AVG_GAUGE, 0, 500 * 1000, 2, rd_true);
}

rd_refcnt_init(&rkb->rkb_refcnt, 0);
Expand Down
17 changes: 11 additions & 6 deletions src/rdkafka_broker.h
Original file line number Diff line number Diff line change
Expand Up @@ -203,12 +203,14 @@ struct rd_kafka_broker_s { /* rd_kafka_broker_t */
rd_avg_t rkb_avg_rtt; /* Current RTT avg */
rd_avg_t rkb_avg_throttle; /* Current throttle avg */
rd_avg_t
rkb_avg_outbuf_latency; /**< Current latency
* between buf_enq0
* and writing to socket
*/
rd_avg_t rkb_avg_fetch_latency; /**< Current fetch
* latency avg */
rkb_avg_outbuf_latency; /**< Current latency
* between buf_enq0
* and writing to socket
*/
rd_avg_t rkb_avg_fetch_latency; /**< Current fetch
* latency avg */
rd_avg_t rkb_avg_produce_latency; /**< Current produce
* latency avg */
} rd_avg_current;

struct {
Expand All @@ -219,6 +221,9 @@ struct rd_kafka_broker_s { /* rd_kafka_broker_t */
* latency avg */
rd_avg_t rkb_avg_fetch_latency; /**< Rolled over fetch
* latency avg */
rd_avg_t
rkb_avg_produce_latency; /**< Rolled over produce
* latency avg */
} rd_avg_rollover;
} rkb_telemetry;

Expand Down
Loading