diff --git a/confluent_kafka/src/Producer.c b/confluent_kafka/src/Producer.c index bb5d24e4e..21fc720a8 100644 --- a/confluent_kafka/src/Producer.c +++ b/confluent_kafka/src/Producer.c @@ -259,7 +259,7 @@ Producer_producev (Handle *self, const void *value, size_t value_len, const void *key, size_t key_len, void *opaque, int64_t timestamp -#if RD_KAFKA_V_HEADERS +#ifdef RD_KAFKA_V_HEADERS ,rd_kafka_headers_t *headers #endif ) { @@ -272,7 +272,7 @@ Producer_producev (Handle *self, RD_KAFKA_V_VALUE((void *)value, (size_t)value_len), RD_KAFKA_V_TIMESTAMP(timestamp), -#if RD_KAFKA_V_HEADERS +#ifdef RD_KAFKA_V_HEADERS RD_KAFKA_V_HEADERS(headers), #endif RD_KAFKA_V_OPAQUE(opaque), @@ -386,7 +386,7 @@ static PyObject *Producer_produce (Handle *self, PyObject *args, value, value_len, key, key_len, msgstate, timestamp -#if RD_KAFKA_V_HEADERS +#ifdef RD_KAFKA_V_HEADERS ,rd_headers #endif );