Move can_q_contain_fetched_msgs inside q_serve#4431
Move can_q_contain_fetched_msgs inside q_serve#4431Emanuele Sabellico (emasab) merged 6 commits intomasterfrom
Conversation
Milind L (milindl)
left a comment
There was a problem hiding this comment.
Implementation looks okay, left some comments for the test.
Please add changelog.md entry too
| do_test_rejoin_after_interval_expire(rd_true); | ||
| do_test_rejoin_after_interval_expire(rd_false, rd_false); | ||
| do_test_rejoin_after_interval_expire(rd_true, rd_false); | ||
| do_test_rejoin_after_interval_expire(rd_false, rd_true); |
There was a problem hiding this comment.
I think we should add a test for the specific case that brought this bug to light, too, which uses rd_kafka_conf_set_consume_cb to set a consume_cb, and the calls rd_kafka_poll.
There was a problem hiding this comment.
Added a new test, though test_wait_event is unable to capture RD_KAFKA_EVENT_REBALANCE not sure why, so haven't added any assert in the test. But added a poll, which should fail if the consumer has left the group.
77c65af to
69bdfa9
Compare
Milind L (milindl)
left a comment
There was a problem hiding this comment.
Approved with minor comments
| TEST_SAY("Consume callback\n"); | ||
| } | ||
|
|
||
| static void do_test_max_poll_reset_with_consumer_cb(void) { |
There was a problem hiding this comment.
Documentation comment above this test
We can also mention issue # for the original case
There was a problem hiding this comment.
Thanks, added.
69bdfa9 to
4796e42
Compare
4796e42 to
69e0942
Compare
6a2aea7 to
e69087d
Compare
* upstream/master: librdkafka v2.3.0 (confluentinc#4455) Fix for idempotent producer fatal errors, triggered after a possibly persisted message state (confluentinc#4438) Move can_q_contain_fetched_msgs inside q_serve (confluentinc#4431) [KIP-580] Exponential Backoff with Mock Broker Changes to Automate Testing. (confluentinc#4422) Update only the mklove version of OpenSSL to 3.0.11 (confluentinc#4454) Permanent errors during offset validation should be retried (confluentinc#4447) Increased flexver request size for Metadata request to include topic_id size (confluentinc#4453) Fix loop of OffsetForLeaderEpoch requests on quick leader changes (confluentinc#4433) Fix for stored offsets not being committed if they lacked the leader epoch (confluentinc#4442) Add leader epoch to control messages (confluentinc#4434) Refactored tmpabuf and fixed an insufficient buffer allocation (confluentinc#4449) Work around KIP-700 restrictions for DescribeCluster [KIP-430] [admin] KIP-430: Add authorized operations to describe API Fix segfault if assignor state is NULL, (confluentinc#4381)
Fixes #4421
Extended the existing test case to test if polling a queue forwarding to the consumer queue resets the max.poll.interval.ms. This test fails without the changes, but works with it.