diff --git a/src/rdkafka_assignment.c b/src/rdkafka_assignment.c index 6d1f01913f..53abfcf4a5 100644 --- a/src/rdkafka_assignment.c +++ b/src/rdkafka_assignment.c @@ -315,21 +315,23 @@ static void rd_kafka_assignment_handle_OffsetFetch(rd_kafka_t *rk, rd_kafka_dbg( rk, CGRP, "OFFSET", "Offset fetch error for %d partition(s): %s", - offsets->cnt, rd_kafka_err2str(err)); + offsets ? offsets->cnt : 0, rd_kafka_err2str(err)); rd_kafka_consumer_err( rk->rk_consumer.q, rd_kafka_broker_id(rkb), err, 0, NULL, NULL, RD_KAFKA_OFFSET_INVALID, "Failed to fetch committed offsets for " "%d partition(s) in group \"%s\": %s", - offsets->cnt, rk->rk_group_id->str, + offsets ? offsets->cnt : 0, rk->rk_group_id->str, rd_kafka_err2str(err)); } } - /* Apply the fetched offsets to the assignment */ - rd_kafka_assignment_apply_offsets(rk, offsets, err); + if (offsets) { + /* Apply the fetched offsets to the assignment */ + rd_kafka_assignment_apply_offsets(rk, offsets, err); - rd_kafka_topic_partition_list_destroy(offsets); + rd_kafka_topic_partition_list_destroy(offsets); + } }