Fix stack-use-after-scope in KafkaProducer::produce()

ASan report:

    ==1==ERROR: AddressSanitizer: stack-use-after-scope on address 0x7f7540186b20 at pc 0x55656ef458ad bp 0x7f754114acf0 sp 0x7f754114a4b8
    READ of size 2 at 0x7f7540186b20 thread T2 (TCPHandler)
        0 0x55656ef458ac in __asan_memcpy (/usr/bin/clickhouse+0x11b028ac) (BuildId: 6ba63057df561f6dc16fb8fd7623fd858bf327d1)
        1 0x55659f62daf8 in rd_kafka_msg_new00 build_docker/./contrib/librdkafka/src/rdkafka_msg.c:192:3
        2 0x55659f62daf8 in rd_kafka_msg_new0 build_docker/./contrib/librdkafka/src/rdkafka_msg.c:257:8
        3 0x55659f630635 in rd_kafka_producev build_docker/./contrib/librdkafka/src/rdkafka_msg.c:642:23
        4 0x55659f4a865c in cppkafka::Producer::do_produce() build_docker/./contrib/cppkafka/src/producer.cpp:128:19
        5 0x55659f4a8306 in cppkafka::Producer::produce() build_docker/./contrib/cppkafka/src/producer.cpp:69:5
        6 0x5565961e748c in DB::KafkaProducer::produce() build_docker/./src/Storages/Kafka/KafkaProducer.cpp:70:23
        7 0x5565962155b2 in DB::MessageQueueSink::consume(DB::Chunk) build_docker/./src/Storages/MessageQueueSink.cpp:64:23
        8 0x5565980b9e8e in DB::SinkToStorage::onConsume(DB::Chunk) build_docker/./src/Processors/Sinks/SinkToStorage.cpp:18:5

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
This commit is contained in:
Azat Khuzhin 2023-05-10 12:15:17 +02:00
parent a15d088fd2
commit 2f9a7cba8a

View File

@ -52,8 +52,8 @@ void KafkaProducer::produce(const String & message, size_t rows_in_message, cons
if (key_column_index)
{
const auto & key_column = assert_cast<const ColumnString &>(*columns[key_column_index.value()]);
const auto key_data = key_column.getDataAt(last_row).toString();
builder.key(cppkafka::Buffer(key_data.data(), key_data.size()));
const auto key_data = key_column.getDataAt(last_row);
builder.key(cppkafka::Buffer(key_data.data, key_data.size));
}
if (timestamp_column_index)