librdkafka icon indicating copy to clipboard operation
librdkafka copied to clipboard

ASAN Stack buffer overflow on broker hostname

Open danog opened this issue 2 years ago • 3 comments

Description

Running php-rdkafka/librdkafka with ASAN enabled in production revealed a frequent stack buffer overflow in https://github.com/confluentinc/librdkafka/blob/v1.8.0/src/rdkafka_broker.c#L5908, while populating the nodename buffer with the metadata broker's host:port.
It seems really strange to me, given that 256 chars should be more than enough to store an ipv4:port couple, but ASAN's stack trace checks out:

=================================================================
==1190850==ERROR: AddressSanitizer: stack-buffer-overflow on address 0x7ff44b880ef0 at pc 0x7ff461958e02 bp 0x7ff44b880de0 sp 0x7ff44b880580
WRITE of size 17 at 0x7ff44b880ef0 thread T-1
    #0 0x7ff461958e01 in __interceptor_vsnprintf (/usr/lib/llvm-16/lib/clang/16/lib/linux/libclang_rt.asan-x86_64.so+0x6ee01) (BuildId: 8c0845aee3ffdc74a2f46d1eb4f0a349cdc69d67)
    #1 0x7ff46195a7a6 in __interceptor___snprintf_chk (/usr/lib/llvm-16/lib/clang/16/lib/linux/libclang_rt.asan-x86_64.so+0x707a6) (BuildId: 8c0845aee3ffdc74a2f46d1eb4f0a349cdc69d67)
    #2 0x7ff45883c213  (/lib/x86_64-linux-gnu/librdkafka.so.1+0x3e213) (BuildId: f9c5d378988521e9ec695c62aa965402ae47a7ee)
    #3 0x7ff458841613  (/lib/x86_64-linux-gnu/librdkafka.so.1+0x43613) (BuildId: f9c5d378988521e9ec695c62aa965402ae47a7ee)
    #4 0x7ff4588a59a0  (/lib/x86_64-linux-gnu/librdkafka.so.1+0xa79a0) (BuildId: f9c5d378988521e9ec695c62aa965402ae47a7ee)
    #5 0x7ff45886506a  (/lib/x86_64-linux-gnu/librdkafka.so.1+0x6706a) (BuildId: f9c5d378988521e9ec695c62aa965402ae47a7ee)
    #6 0x7ff45884d8e3  (/lib/x86_64-linux-gnu/librdkafka.so.1+0x4f8e3) (BuildId: f9c5d378988521e9ec695c62aa965402ae47a7ee)
    #7 0x7ff458859b59  (/lib/x86_64-linux-gnu/librdkafka.so.1+0x5bb59) (BuildId: f9c5d378988521e9ec695c62aa965402ae47a7ee)
    #8 0x7ff458859f79  (/lib/x86_64-linux-gnu/librdkafka.so.1+0x5bf79) (BuildId: f9c5d378988521e9ec695c62aa965402ae47a7ee)
    #9 0x7ff458819d67  (/lib/x86_64-linux-gnu/librdkafka.so.1+0x1bd67) (BuildId: f9c5d378988521e9ec695c62aa965402ae47a7ee)
    #10 0x7ff45e4bc934  (/lib/x86_64-linux-gnu/libc.so.6+0x94934) (BuildId: a43bfc8428df6623cd498c9c0caeb91aec9be4f9)
    #11 0x7ff45e54dbf3 in __clone (/lib/x86_64-linux-gnu/libc.so.6+0x125bf3) (BuildId: a43bfc8428df6623cd498c9c0caeb91aec9be4f9)

Address 0x7ff44b880ef0 is a wild pointer inside of access range of size 0x000000000011.
SUMMARY: AddressSanitizer: stack-buffer-overflow (/usr/lib/llvm-16/lib/clang/16/lib/linux/libclang_rt.asan-x86_64.so+0x6ee01) (BuildId: 8c0845aee3ffdc74a2f46d1eb4f0a349cdc69d67) in __interceptor_vsnprintf
Shadow bytes around the buggy address:
  0x7ff44b880c00: 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00
  0x7ff44b880c80: 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00
  0x7ff44b880d00: 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00
  0x7ff44b880d80: 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00
  0x7ff44b880e00: 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00
=>0x7ff44b880e80: f1 f1 f1 f1 00 00 f2 f2 00 00 00 00 f3 f3[f3]f3
  0x7ff44b880f00: 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00
  0x7ff44b880f80: 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00
  0x7ff44b881000: 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00
  0x7ff44b881080: 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00
  0x7ff44b881100: 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00
Shadow byte legend (one shadow byte represents 8 application bytes):
  Addressable:           00
  Partially addressable: 01 02 03 04 05 06 07
  Heap left redzone:       fa
  Freed heap region:       fd
  Stack left redzone:      f1
  Stack mid redzone:       f2
  Stack right redzone:     f3
  Stack after return:      f5
  Stack use after scope:   f8
  Global redzone:          f9
  Global init order:       f6
  Poisoned by user:        f7
  Container overflow:      fc
  Array cookie:            ac
  Intra object redzone:    bb
  ASan internal:           fe
  Left alloca redzone:     ca
  Right alloca redzone:    cb
==1190850==ABORTING

Resolving the addresses:

root@853250d0c853:~/librdkafka-1.8.0# cat a
0x3e213
0x43613
0xa79a0
0x6706a
0x4f8e3
0x5bb59
0x5bf79
0x1bd67
root@853250d0c853:~/librdkafka-1.8.0# for f in $( cat a); do addr2line addr -e /lib/x86_64-linux-gnu/librdkafka.so.1 $f 2>/dev/null | sed 's/.*src/src/g';done
??:0
/usr/include/x86_64-linux-gnu/bits/stdio2.h:71
src/rdkafka_broker.c:6141
src/rdkafka_metadata.c:470 (discriminator 2)
src/rdkafka_request.c:1880 (discriminator 2)
src/rdkafka_buf.c:495
src/rdkafka_op.c:821
src/rdkafka_queue.c:507
src/rdkafka.c:2058

Some stack frames are missing due to compiler optimizations, but it's very clear that the issue is triggered by the call to rd_kafka_broker_find in src/rdkafka_broker.c:6141 (rd_kafka_broker_update), which then calls rd_kafka_mk_nodename (and then rd_snprintf, __interceptor___snprintf_chk in the stack trace), passing a name and a port which together exceed the size of the stack-allocated nodename buffer (256 bytes).

Also curious is the fact that rd_kafka_mk_nodename is also invoked in rd_kafka_broker_update before calling rd_kafka_broker_find using the same values, with no buffer overflow occurring in this case; some memory corruption may be occurring between the two calls, possibly in the call to rd_kafka_broker_find_by_nodeid.

Checklist

Please provide the following information:

  • [x] librdkafka version (release number or git tag): v1.8.0 (from Ubuntu repos)
  • [x] Apache Kafka version: docker image confluentinc/cp-kafka:6.1.1
  • [x] librdkafka client configuration:
			'enable.auto.commit' => 'false',
			'enable.auto.offset.store' => 'true',
			'auto.commit.interval.ms' => '1000',
			'offset.store.method' => 'broker',
			'log_level' => getenv('RD_KAFKA_LOG_LEVEL') ?: '0',
			'group.id' => 'stuff',
			'metadata.broker.list' => '192.168.18.1:9091,192.168.18.1:9092,192.168.18.1:9093',
			'socket.timeout.ms' => (PHP_SAPI === 'cli' ? '2000' : '20'),
			'auto.offset.reset' => 'earliest',
			'queued.max.messages.kbytes' => '2097151',
			'topic.metadata.refresh.sparse' => 'true',
			'topic.metadata.refresh.interval.ms' => '1000',
			'max.poll.interval.ms' => '360000',
			'session.timeout.ms' => '60000',
			'heartbeat.interval.ms' => '1000',
			'socket.blocking.max.ms' => (PHP_SAPI === 'cli' ? '1000' : '1'),
			'fetch.wait.max.ms' => (PHP_SAPI === 'cli' ? '1000' : '1'),
			'message.max.bytes' => (string)(1024 * 1024 * 2),

			'queue.buffering.max.ms' => (PHP_SAPI === 'cli' ? '5000' : '100'),
			'linger.ms' => (PHP_SAPI === 'cli' ? '5000' : '100'), // alias for queue.buffering.max.ms
			'batch.size' => '65535',
			'compression.codec' => 'zstd',
  • [x] Operating system: Ubuntu 22.04 x86_64

danog avatar Nov 14 '23 13:11 danog

Thanks for the report @danog. Please also send your broker advertised listeners to check if they're longer than 256

emasab avatar Nov 24 '23 09:11 emasab

Hi @emasab, unfortunately we stopped using librdkafka due to the instabilities, will try to ask permission to run the code to get the listeners if I have some time, however looking at the code it's clear that there is some memory corruption happening, as the same listeners list is printed a few lines before the call that buffer overflows.

Incidentally we also tried using your kafka-rest API, and we're getting total hangs of the kafka-rest server with just this in the logs:

Dec  5 19:33:49 z52 docker/kafka-rest-52[1333]: [2023-12-05 16:33:49,922] ERROR Consumer read callback threw an unhandled exception id=io.confluent.kafkarest.v2.KafkaConsumerReadTask@c7b2a3e exception=null (io.confluent.kafkarest.v2.KafkaConsumerReadTask)

It's really unfortunate that all your solutions have shown critical issues in our high-load production deployment, we'll have to stick to kafka-pixy for now, even if it has issues with consumption with more than one pixy instance, however using a single kafka-pixy instance for everything works perfectly for months on end with no issues whatsoever.

danog avatar Dec 05 '23 17:12 danog

The call to

rd_kafka_mk_nodename(nodename, sizeof(nodename), mdb->host, mdb->port);

will eventually call snprintf(__VA_ARGS__), which will restrict the output size to 256. So I wonder why is this even possible of overflowing, when the length of mdb->host plus mdb->port is larger than 256?

davidhcefx avatar Mar 08 '24 05:03 davidhcefx