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 2, 2020
1 parent 329c97f commit 6d13bb6
Show file tree
Hide file tree
Showing 7 changed files with 123 additions and 9 deletions.
6 changes: 4 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 @@ -76,9 +76,11 @@ 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
* Fix crash and/or inconsistent subscriptions when using multiple consumers
(in the same process) with wildcard topics on Windows.
* 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
7 changes: 6 additions & 1 deletion 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 Expand Up @@ -4452,7 +4455,9 @@ rd_kafka_fetch_reply_handle (rd_kafka_broker_t *rkb,
hdr.ErrorCode, tver->version,
NULL, rktp,
rktp->rktp_offsets.fetch_offset,
"Fetch failed: %s",
"Fetch from broker %"PRId32
" failed: %s",
rd_kafka_broker_id(rkb),
rd_kafka_err2str(hdr.ErrorCode));
break;
}
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
74 changes: 74 additions & 0 deletions tests/0104-fetch_from_follower_mock.c
Original file line number Diff line number Diff line change
Expand Up @@ -242,6 +242,78 @@ static void do_test_unknown_follower (void) {
}


/**
* @brief Issue #2955: Verify that fetch does not stall until next
* periodic metadata timeout when leader broker is no longer
* a replica.
*/
static void do_test_replica_not_available (void) {
const char *bootstraps;
rd_kafka_mock_cluster_t *mcluster;
rd_kafka_conf_t *conf;
rd_kafka_t *c;
const char *topic = "test";
const int msgcnt = 1000;

TEST_SAY(_C_MAG "[ Test REPLICA_NOT_AVAIALBLE ]\n");

mcluster = test_mock_cluster_new(3, &bootstraps);

/* Seed the topic with messages */
test_produce_msgs_easy_v(topic, 0, 0, 0, msgcnt, 1000,
"bootstrap.servers", bootstraps,
"batch.num.messages", "10",
NULL);

/* Set partition leader to broker 1. */
rd_kafka_mock_partition_set_leader(mcluster, topic, 0, 1);

test_conf_init(&conf, NULL, 0);
test_conf_set(conf, "bootstrap.servers", bootstraps);
test_conf_set(conf, "client.rack", "myrack");
test_conf_set(conf, "auto.offset.reset", "earliest");
test_conf_set(conf, "topic.metadata.refresh.interval.ms", "60000");
test_conf_set(conf, "fetch.error.backoff.ms", "1000");

c = test_create_consumer("mygroup", NULL, conf, NULL);

rd_kafka_mock_broker_push_request_errors(
mcluster,
1/*Broker 1*/,
1/*FetchRequest*/,
10,
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE,
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE,
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE,
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE,
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE,
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE,
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE,
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE,
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE,
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE);


test_consumer_assign_partition("REPLICA_NOT_AVAIALBLE", c, topic, 0,
RD_KAFKA_OFFSET_INVALID);

test_consumer_poll_no_msgs("Wait initial metadata", c, 0, 2000);

/* Switch leader to broker 2 so that metadata is updated,
* causing the consumer to start fetching from the new leader. */
rd_kafka_mock_partition_set_leader(mcluster, topic, 0, 2);

test_consumer_poll("Consume", c, 0, 1, 0, msgcnt, NULL);

test_consumer_close(c);

rd_kafka_destroy(c);

test_mock_cluster_destroy(mcluster);

TEST_SAY(_C_GRN "[ Test REPLICA_NOT_AVAIALBLE PASSED ]\n");
}


int main_0104_fetch_from_follower_mock (int argc, char **argv) {

Expand All @@ -257,5 +329,7 @@ int main_0104_fetch_from_follower_mock (int argc, char **argv) {

do_test_unknown_follower();

do_test_replica_not_available();

return 0;
}
23 changes: 23 additions & 0 deletions tests/test.c
Original file line number Diff line number Diff line change
Expand Up @@ -2613,6 +2613,29 @@ void test_consumer_assign_partition (const char *what, rd_kafka_t *rk,
}


void test_consumer_pause_resume_partition (rd_kafka_t *rk,
const char *topic, int32_t partition,
rd_bool_t pause) {
rd_kafka_topic_partition_list_t *part;
rd_kafka_resp_err_t err;

part = rd_kafka_topic_partition_list_new(1);
rd_kafka_topic_partition_list_add(part, topic, partition);

if (pause)
err = rd_kafka_pause_partitions(rk, part);
else
err = rd_kafka_resume_partitions(rk, part);

TEST_ASSERT(!err, "Failed to %s %s [%"PRId32"]: %s",
pause ? "pause":"resume",
topic, partition,
rd_kafka_err2str(err));

rd_kafka_topic_partition_list_destroy(part);
}


/**
* Message verification services
*
Expand Down
3 changes: 3 additions & 0 deletions tests/test.h
Original file line number Diff line number Diff line change
Expand Up @@ -520,6 +520,9 @@ void test_consumer_unassign (const char *what, rd_kafka_t *rk);
void test_consumer_assign_partition (const char *what, rd_kafka_t *rk,
const char *topic, int32_t partition,
int64_t offset);
void test_consumer_pause_resume_partition (rd_kafka_t *rk,
const char *topic, int32_t partition,
rd_bool_t pause);

void test_consumer_close (rd_kafka_t *rk);

Expand Down

0 comments on commit 6d13bb6

Please sign in to comment.