Skip to content

Commit

Permalink
Refresh partition leader on FetchResponse REPLICA_NOT_AVAILABLE (#2955)
Browse files Browse the repository at this point in the history
And somewhat improved handling of partition errors on OffsetResponse.
  • Loading branch information
edenhill committed Jul 1, 2020
1 parent d54c645 commit 3bc3157
Show file tree
Hide file tree
Showing 4 changed files with 21 additions and 8 deletions.
7 changes: 5 additions & 2 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ librdkafka.
supported (by @sky92zwq).
* `./configure` arguments now take precedence over cached `configure` variables
from previous invocation.
* Fix theoeretical crash on coord request failure.
* Fix theoretical crash on coord request failure.


### Consumer fixes
Expand All @@ -74,7 +74,10 @@ librdkafka.
for partitions that were not being consumed (#2826).
* Initial consumer group joins should now be a couple of seconds quicker
thanks expedited query intervals (@benesch).
* Don't propagate temporary offset lookup errors to application
* Don't propagate temporary offset lookup errors to application.
* Immediately refresh topic metadata when partitions are reassigned to other
brokers, avoiding a fetch stall of up to `topic.metadata.refresh.interval.ms`. (#2955)


### Producer fixes

Expand Down
3 changes: 3 additions & 0 deletions src/rdkafka_broker.c
Original file line number Diff line number Diff line change
Expand Up @@ -4361,6 +4361,9 @@ rd_kafka_fetch_reply_handle (rd_kafka_broker_t *rkb,
case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE:
case RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION:
case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE:
case RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE:
case RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR:
case RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH:
/* Request metadata information update*/
rd_kafka_toppar_leader_unavailable(
rktp, "fetch", hdr.ErrorCode);
Expand Down
16 changes: 10 additions & 6 deletions src/rdkafka_partition.c
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ static void rd_kafka_toppar_lag_handle_Offset (rd_kafka_t *rk,
rktp->rktp_partition)))
err = RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION;

if (!err) {
if (!err && !rktpar->err) {
rd_kafka_toppar_lock(rktp);
rktp->rktp_lo_offset = rktpar->offset;
rd_kafka_toppar_unlock(rktp);
Expand Down Expand Up @@ -1333,11 +1333,15 @@ static void rd_kafka_toppar_handle_Offset (rd_kafka_t *rk,
rkbuf, request, offsets);
}

if (!err &&
(!(rktpar = rd_kafka_topic_partition_list_find(
offsets,
rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition))))
err = RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION;
if (!err) {
if (!(rktpar = rd_kafka_topic_partition_list_find(
offsets,
rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition)))
err = RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION;
else if (rktpar->err)
err = rktpar->err;
}

if (err) {
rd_rkb_dbg(rkb, TOPIC, "OFFSET",
Expand Down
3 changes: 3 additions & 0 deletions src/rdkafka_request.c
Original file line number Diff line number Diff line change
Expand Up @@ -451,6 +451,9 @@ rd_kafka_resp_err_t rd_kafka_handle_Offset (rd_kafka_t *rk,
RD_KAFKA_ERR_ACTION_REFRESH,
RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION,

RD_KAFKA_ERR_ACTION_REFRESH,
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE,

RD_KAFKA_ERR_ACTION_REFRESH|RD_KAFKA_ERR_ACTION_RETRY,
RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE,

Expand Down

0 comments on commit 3bc3157

Please sign in to comment.