Skip to content

Commit

Permalink
Fixes #4249
Browse files Browse the repository at this point in the history
  • Loading branch information
emasab committed Apr 18, 2023
1 parent d1d62b6 commit fe111b0
Show file tree
Hide file tree
Showing 2 changed files with 37 additions and 0 deletions.
22 changes: 22 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
@@ -1,3 +1,20 @@
# librdkafka v2.1.1

librdkafka v2.1.1 is a bugfix release:

* Avoid a duplicate message when a fetch response is received
in the middle of an offset validation request (#todo).

## Fixes

### Consumer fixes

* A duplicate message can be emitted when a fetch response is received
in the middle of an offset validation request. Solved by discarding
the fetched message if the state is not `ACTIVE`.



# librdkafka v2.1.0

librdkafka v2.1.0 is a feature release:
Expand Down Expand Up @@ -64,6 +81,11 @@ librdkafka v2.1.0 is a feature release:
any of the **seek**, **pause**, **resume** or **rebalancing** operation, `on_consume`
interceptors might be called incorrectly (maybe multiple times) for not consumed messages.

### Consume API

* A duplicate message can be emitted when a fetch response is received
in the middle of an offset validation request.



# librdkafka v2.0.2
Expand Down
15 changes: 15 additions & 0 deletions src/rdkafka_fetcher.c
Original file line number Diff line number Diff line change
Expand Up @@ -508,6 +508,21 @@ rd_kafka_fetch_reply_handle_partition(rd_kafka_broker_t *rkb,
return RD_KAFKA_RESP_ERR_NO_ERROR;
}

/* Make sure toppar is in ACTIVE state. */
if (unlikely(rktp->rktp_fetch_state != RD_KAFKA_TOPPAR_FETCH_ACTIVE)) {
rd_kafka_toppar_unlock(rktp);
rd_rkb_dbg(rkb, MSG, "FETCH",
"%.*s [%" PRId32
"]: partition not in state ACTIVE: "
"discarding fetch response",
RD_KAFKAP_STR_PR(topic), hdr.Partition);
rd_kafka_toppar_destroy(rktp); /* from get */
rd_kafka_buf_skip(rkbuf, hdr.MessageSetSize);
if (aborted_txns)
rd_kafka_aborted_txns_destroy(aborted_txns);
return RD_KAFKA_RESP_ERR_NO_ERROR;
}

fetch_version = rktp->rktp_fetch_version;
rd_kafka_toppar_unlock(rktp);

Expand Down

0 comments on commit fe111b0

Please sign in to comment.