Skip to content

librdkafka v2.6.1 Crash (Memory out-of-bounds read) during update metadata #5280

@Hayley-Q

Description

@Hayley-Q

Hi,

We're encountering a random core dump issue when using librdkafka v2.6.1 in our C++ program.

Stack

Program terminated with signal SIGSEGV, Segmentation fault.
#0  0x0000ffff86b5f408 in rd_kafka_metadata_update_op (rk=rk@entry=0xaaaafb2defd0, mdi=0xaaab0ea335f0) at rdkafka_metadata.c:2047
#1  0x0000ffff86ab5d00 in rd_kafka_poll_cb (rk=0xaaaafb2defd0, rkq=0xffff4c41b620, rko=0xaaab0ebcef20, cb_type=RD_KAFKA_Q_CB_CALLBACK, opaque=<optimized out>) at rdkafka.c:4149
#2  0x0000ffff86ae63fc in rd_kafka_q_serve (rkq=0xaaaafb2da760, timeout_ms=<optimized out>, max_cnt=max_cnt@entry=0, cb_type=cb_type@entry=RD_KAFKA_Q_CB_CALLBACK, callback=callback@entry=0x0, opaque=opaque@entry=0x0) at rdkafka_queue.c:578
#3  0x0000ffff86ab89f4 in rd_kafka_thread_main (arg=0xaaaafb2defd0) at rdkafka.c:2156
#4  0x0000ffff8e795018 in ?? () from /usr/lib64/libc.so.6
#5  0x0000ffff8e7fc81c in ?? () from /usr/lib64/libc.so.6

We've managed to get a symbolised stack trace from the core dump using a custom build of librdkafka v2.6.1 with debug symbols

Part of GDB analysis process

(gdb) bt
#0  0x0000ffff86b5f408 in rd_kafka_metadata_update_op (rk=rk@entry=0xaaaafb2defd0, mdi=0xaaab0ea335f0) at rdkafka_metadata.c:2047
#1  0x0000ffff86ab5d00 in rd_kafka_poll_cb (rk=0xaaaafb2defd0, rkq=0xffff4c41b620, rko=0xaaab0ebcef20, cb_type=RD_KAFKA_Q_CB_CALLBACK, opaque=<optimized out>) at rdkafka.c:4149
#2  0x0000ffff86ae63fc in rd_kafka_q_serve (rkq=0xaaaafb2da760, timeout_ms=<optimized out>, max_cnt=max_cnt@entry=0, cb_type=cb_type@entry=RD_KAFKA_Q_CB_CALLBACK, callback=callback@entry=0x0, opaque=opaque@entry=0x0) at rdkafka_queue.c:578
#3  0x0000ffff86ab89f4 in rd_kafka_thread_main (arg=0xaaaafb2defd0) at rdkafka.c:2156
#4  0x0000ffff8e795018 in ?? () from /usr/lib64/libc.so.6
#5  0x0000ffff8e7fc81c in ?? () from /usr/lib64/libc.so.6
(gdb) f 0
(gdb) p part
$39 = -1542205261
(gdb) p mdp->id
$40 = -1542205261
(gdb) p mdi->metadata->topics[i].partitions[j]->id
$41 = -1542205261
(gdb) p i
$42 = 0
(gdb) p j
$43 = 10
(gdb) p mdi->metadata->topics[i].partition_cnt
$44 = 16
(gdb) p mdi->metadata->topics[i].partitions[0]
$45 = {id = 15, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 0, replica_cnt = 0, replicas = 0x0, isr_cnt = 0, isrs = 0x0}
(gdb) p mdi->metadata->topics[i].partitions[1]
$46 = {id = 32, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = -1542201965, replica_cnt = 43681, replicas = 0x0, isr_cnt = 30, isrs = 0x41}
(gdb) p mdi->metadata->topics[i].partitions[2]
$47 = {id = 245578016, err = 43691, leader = 245577920, replica_cnt = 43691, replicas = 0xaaab0ea33750, isr_cnt = 33, isrs = 0xaaab0f7242d0}
(gdb) p mdi->metadata->topics[i].partitions[3]
$48 = {id = 251667072, err = 43691, leader = 448, replica_cnt = 0, replicas = 0x20, isr_cnt = -1542205325, isrs = 0xaaab0ea338d0}
(gdb) p mdi->metadata->topics[i].partitions[4]
$49 = {id = 0, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 161, replica_cnt = 0, replicas = 0xaaab0f2b9df0, isr_cnt = 259146960, isrs = 0xaaab0e9cadd0}
(gdb) p mdi->metadata->topics[i].partitions[5]
$50 = {id = 4, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 0, replica_cnt = 0, replicas = 0x0, isr_cnt = 0, isrs = 0x41}
(gdb) p mdi->metadata->topics[i].partitions[6]
$51 = {id = 245578112, err = 43691, leader = -1900987688, replica_cnt = 65535, replicas = 0xaaab0f0622c0, isr_cnt = 4, isrs = 0x0}
(gdb) p mdi->metadata->topics[i].partitions[7]
$52 = {id = 0, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 128, replica_cnt = 0, replicas = 0x20, isr_cnt = -1514850061, isrs = 0xaaab0f05d110}
(gdb) p mdi->metadata->topics[i].partitions[8]
$53 = {id = 640, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 32, replica_cnt = 0, replicas = 0xaaa1a413d293, isr_cnt = 0, isrs = 0xaaab0ea33a90}
(gdb) p mdi->metadata->topics[i].partitions[9]
$54 = {id = 33, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 253523040, replica_cnt = 43691, replicas = 0xaaab0f05cee0, isr_cnt = 704, isrs = 0x20}
(gdb) p mdi->metadata->topics[i].partitions[10]
$55 = {id = -1542205261, err = 43681, leader = 0, replica_cnt = 0, replicas = 0xaaab0ea33890, isr_cnt = 65, isrs = 0xaaab0ea33920}
(gdb) p mdi->metadata->topics[i].partitions[11]
$56 = {id = 245578304, err = 43691, leader = 245577648, replica_cnt = 43691, replicas = 0x21, isr_cnt = 252038880, isrs = 0xaaab0ea33920}
(gdb) p mdi->metadata->topics[i].partitions[12]
$57 = {id = 800, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 32, replica_cnt = 0, replicas = 0xaaa1a5c2a9e3, isr_cnt = 0, isrs = 0x4645674272676742}
(gdb) p mdi->metadata->topics[i].partitions[13]
$58 = {id = 65, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 252038880, replica_cnt = 43691, replicas = 0xaaab0ea33a40, isr_cnt = 245578000, isrs = 0x21}
(gdb) p mdi->metadata->topics[i].partitions[14]
$59 = {id = 254517104, err = 43691, leader = 256080032, replica_cnt = 43691, replicas = 0x380, isr_cnt = 32, isrs = 0xaaa1a5c2ac23}
(gdb) p mdi->metadata->topics[i].partitions[15]
$60 = {id = 0, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 3145828, replica_cnt = 0, replicas = 0xa1, isr_cnt = 259146960, isrs = 0xaaab0f1bdee0}
(gdb) p rkmce->rkmce_metadata_internal_topic.partitions
$61 = (rd_kafka_metadata_partition_internal_t *) 0xffff70ccccb8
(gdb) p rkmce->rkmce_metadata_internal_topic.partitions[0]
$62 = {id = 0, leader_epoch = 4937, racks = 0x0, racks_cnt = 0}
(gdb) p rkmce->rkmce_metadata_internal_topic.partitions[15]
$63 = {id = 15, leader_epoch = 4938, racks = 0x0, racks_cnt = 0}
(gdb) p rkmce->rkmce_metadata_internal_topic.partitions[part]
Cannot access memory at address 0xfff6d2a85d80
(gdb)

We checked the code of rd_kafka_metadata_update_op and find if the mdp->id is minus, then it can crash as out-of-bounds read when it runs to

Source code

current_leader_epoch =
                            rkmce->rkmce_metadata_internal_topic
                                .partitions[part]
                                .leader_epoch;

Part of GDB analysis process

(gdb) p part
$39 = -1542205261
(gdb) p mdi->metadata->topics[i].partition_cnt
$44 = 16
(gdb) p rkmce->rkmce_metadata_internal_topic.partitions
$61 = (rd_kafka_metadata_partition_internal_t *) 0xffff70ccccb8
(gdb) p rkmce->rkmce_metadata_internal_topic.partitions[0]
$62 = {id = 0, leader_epoch = 4937, racks = 0x0, racks_cnt = 0}
(gdb) p rkmce->rkmce_metadata_internal_topic.partitions[15]
$63 = {id = 15, leader_epoch = 4938, racks = 0x0, racks_cnt = 0}
(gdb) p rkmce->rkmce_metadata_internal_topic.partitions[part]
Cannot access memory at address 0xfff6d2a85d80

Also, for mdi->metadata->topics[i].partitions, it appears that not all partitions are in RD_KAFKA_RESP_ERR_NO_ERROR state.

Part of GDB analysis process

(gdb) p i
$42 = 0
(gdb) p j
$43 = 10
(gdb) p mdi->metadata->topics[i].partition_cnt
$44 = 16
(gdb) p mdi->metadata->topics[i].partitions[0]
$45 = {id = 15, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 0, replica_cnt = 0, replicas = 0x0, isr_cnt = 0, isrs = 0x0}
(gdb) p mdi->metadata->topics[i].partitions[1]
$46 = {id = 32, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = -1542201965, replica_cnt = 43681, replicas = 0x0, isr_cnt = 30, isrs = 0x41}
(gdb) p mdi->metadata->topics[i].partitions[2]
$47 = {id = 245578016, err = 43691, leader = 245577920, replica_cnt = 43691, replicas = 0xaaab0ea33750, isr_cnt = 33, isrs = 0xaaab0f7242d0}
(gdb) p mdi->metadata->topics[i].partitions[3]
$48 = {id = 251667072, err = 43691, leader = 448, replica_cnt = 0, replicas = 0x20, isr_cnt = -1542205325, isrs = 0xaaab0ea338d0}
(gdb) p mdi->metadata->topics[i].partitions[4]
$49 = {id = 0, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 161, replica_cnt = 0, replicas = 0xaaab0f2b9df0, isr_cnt = 259146960, isrs = 0xaaab0e9cadd0}
(gdb) p mdi->metadata->topics[i].partitions[5]
$50 = {id = 4, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 0, replica_cnt = 0, replicas = 0x0, isr_cnt = 0, isrs = 0x41}
(gdb) p mdi->metadata->topics[i].partitions[6]
$51 = {id = 245578112, err = 43691, leader = -1900987688, replica_cnt = 65535, replicas = 0xaaab0f0622c0, isr_cnt = 4, isrs = 0x0}
(gdb) p mdi->metadata->topics[i].partitions[7]
$52 = {id = 0, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 128, replica_cnt = 0, replicas = 0x20, isr_cnt = -1514850061, isrs = 0xaaab0f05d110}
(gdb) p mdi->metadata->topics[i].partitions[8]
$53 = {id = 640, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 32, replica_cnt = 0, replicas = 0xaaa1a413d293, isr_cnt = 0, isrs = 0xaaab0ea33a90}
(gdb) p mdi->metadata->topics[i].partitions[9]
$54 = {id = 33, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 253523040, replica_cnt = 43691, replicas = 0xaaab0f05cee0, isr_cnt = 704, isrs = 0x20}
(gdb) p mdi->metadata->topics[i].partitions[10]
$55 = {id = -1542205261, err = 43681, leader = 0, replica_cnt = 0, replicas = 0xaaab0ea33890, isr_cnt = 65, isrs = 0xaaab0ea33920}
(gdb) p mdi->metadata->topics[i].partitions[11]
$56 = {id = 245578304, err = 43691, leader = 245577648, replica_cnt = 43691, replicas = 0x21, isr_cnt = 252038880, isrs = 0xaaab0ea33920}
(gdb) p mdi->metadata->topics[i].partitions[12]
$57 = {id = 800, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 32, replica_cnt = 0, replicas = 0xaaa1a5c2a9e3, isr_cnt = 0, isrs = 0x4645674272676742}
(gdb) p mdi->metadata->topics[i].partitions[13]
$58 = {id = 65, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 252038880, replica_cnt = 43691, replicas = 0xaaab0ea33a40, isr_cnt = 245578000, isrs = 0x21}
(gdb) p mdi->metadata->topics[i].partitions[14]
$59 = {id = 254517104, err = 43691, leader = 256080032, replica_cnt = 43691, replicas = 0x380, isr_cnt = 32, isrs = 0xaaa1a5c2ac23}
(gdb) p mdi->metadata->topics[i].partitions[15]
$60 = {id = 0, err = RD_KAFKA_RESP_ERR_NO_ERROR, leader = 3145828, replica_cnt = 0, replicas = 0xa1, isr_cnt = 259146960, isrs = 0xaaab0f1bdee0}
Cannot access memory at address 0xfff6d2a85d80

It seems that when the value of err is not RD_KAFKA_RESP_ERR_NO_ERROR, the value of id is not reliable.

Can the negative value check of part (that is, mdp->id) be added to rd_kafka_metadata_update_op ?
or the err validity check of elements in mdi->metadata->topics[i].partitions?

Metadata

Metadata

Assignees

No one assigned

    Labels

    No labels
    No labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions