Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[KIP-580] Exponential Backoff with Mock Broker Changes to Automate Testing. #4422

Merged
merged 15 commits into from
Sep 29, 2023

Conversation

mahajanadhitya
Copy link
Contributor

No description provided.

@cla-assistant
Copy link

cla-assistant bot commented Sep 1, 2023

CLA assistant check
All committers have signed the CLA.

Copy link
Contributor

@milindl milindl left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

General comments:

  1. Change INTRODUCTION.md to update list of supported KIPs.
  2. Change INTRODUCTION.md section "Producer message delivery failure" and mention retry.backoff.max.ms as well.
  3. Update CHANGELOG.md accordingly.
  4. There are a whole lot of memory leaks in the tests, fix them. At the very least, clean up the resources you're creating (the request lists, partition lists, duplicated configs etc) if you can't run valgrind. Any subsequent memory leaks I can run valgrind on the test to find out.
  5. Run make style-fix on the code. If you have difficulty doing that with macOS, I can do that once you've made all the other changes.
  6. Fix the warnings. Lots of warnings are there while compiling.

src/rdinterval.h Outdated Show resolved Hide resolved
src/rdinterval.h Outdated Show resolved Hide resolved
src/rdkafka_broker.c Outdated Show resolved Hide resolved
src/rdkafka_broker.c Outdated Show resolved Hide resolved
src/rdkafka_broker.c Outdated Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
src/rdkafka_conf.h Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
Copy link
Contributor

@milindl milindl left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Mostly minor comments, but two major comments regarding calculation errors in unit conversion

src/rdkafka_broker.c Outdated Show resolved Hide resolved
src/rdkafka_int.h Show resolved Hide resolved
src/rdkafka_mock_handlers.c Outdated Show resolved Hide resolved
src/rdkafka_mock_int.h Show resolved Hide resolved
src/rdkafka_mock_int.h Outdated Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
src/rdkafka_partition.c Outdated Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
@milindl
Copy link
Contributor

milindl commented Sep 27, 2023

Some notes for test failures:

0090: the backoff for each message in a batch needs to be the same, currently it's different due to jitter. In rd_kafka_retry_msgq calculate jitter just once and use it for each message.
0075 and 0076: expects linear backoff, add retry.backoff.max.ms

@milindl
Copy link
Contributor

milindl commented Sep 27, 2023

And besides this, please increase the timeout for 0143.

@emasab emasab changed the title Exponential Backoff with Mock Broker Changes to Automate Testing. [KIP-580] Exponential Backoff with Mock Broker Changes to Automate Testing. Sep 27, 2023
Copy link
Contributor

@emasab emasab left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Partial comments

src/rdkafka_partition.c Outdated Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
tests/test.c Outdated Show resolved Hide resolved
tests/0143-exponential_backoff.c Outdated Show resolved Hide resolved
tests/0075-retry.c Show resolved Hide resolved
INTRODUCTION.md Outdated Show resolved Hide resolved
src/rdkafka_partition.c Outdated Show resolved Hide resolved
CHANGELOG.md Outdated Show resolved Hide resolved
src/rdkafka_mock_int.h Show resolved Hide resolved
src/rdinterval.h Outdated Show resolved Hide resolved
CHANGELOG.md Outdated Show resolved Hide resolved
CONFIGURATION.md Outdated Show resolved Hide resolved
src/rdkafka_mock_int.h Show resolved Hide resolved
src/rdkafka_mock.c Show resolved Hide resolved
src/rdkafka_conf.c Show resolved Hide resolved
src/rdkafka_metadata.c Show resolved Hide resolved
src/rdkafka_topic.c Outdated Show resolved Hide resolved
src/rdkafka_topic.c Outdated Show resolved Hide resolved
src/rdkafka_topic.c Outdated Show resolved Hide resolved
src/rdinterval.h Show resolved Hide resolved
Copy link
Contributor

@emasab emasab left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Awesome PR @mahajanadhitya @milindl ! This is a big improvement.

@milindl milindl merged commit 6dc7c71 into master Sep 29, 2023
@milindl milindl deleted the feature/KIP-580 branch September 29, 2023 14:23
axelandersson added a commit to axelandersson/librdkafka that referenced this pull request Oct 5, 2023
* 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)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants