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

Fluctuations and resets in committed offsets due to temporal network outage #4427

Closed
7 tasks done
krasin-ga opened this issue Sep 10, 2023 · 1 comment · Fixed by #4447
Closed
7 tasks done

Fluctuations and resets in committed offsets due to temporal network outage #4427

krasin-ga opened this issue Sep 10, 2023 · 1 comment · Fixed by #4447

Comments

@krasin-ga
Copy link

krasin-ga commented Sep 10, 2023

Description

We were testing a network outage scenario where one of three data centers became unavailable and noticed strange fluctuations and resets in committed offsets after the data center went back online. I've observed some anomalies in the logs that might be related to it.

Temporary errors in host resolution that result in the resetting of offsets

[thrd:main]: test-topic [2]: offset reset (at offset INVALID (leader epoch 6059), broker 1014) to offset BEGINNING (leader epoch -1): Unable to validate offset and epoch: Local: Host resolution failure: Local: Partition log truncation detected

[thrd:main]: test-topic [19]: offset reset (at offset BEGINNING (leader epoch -1), broker 1014) to offset BEGINNING (leader epoch -1): failed to query logical offset: Local: Host resolution failure

Suspicious updates of committed offsets

2023-09-08T00:13:46.246Z // Started at correct offset

Consumer in the group "testbench-1000": "[thrd:main]: Partition test-topic [0] start fetching at offset 57842809 (leader epoch 7007)" Code: "FETCH"; SysLevel: Debug;

2023-09-08T00:13:50.051Z // Race condition? Committed offset and leader epoch for partition 0 is from partition 7 (see log below)

Consumer in the group "testbench-1000": "[thrd:main]: Topic test-topic [0]: stored offset INVALID (leader epoch -1), committed offset 55171745 (leader epoch 6476): not including in commit" Code: "OFFSET"; SysLevel: Debug;

2023-09-08T00:13:50.058Z

Consumer in the group "testbench-1000": "[thrd:main]: Topic test-topic [7]: stored offset 55181424 (leader epoch 6476), committed offset 55171745 (leader epoch 6476): setting stored offset 55181424 (leader epoch 6476) for commit" Code: "OFFSET"; SysLevel: Debug;

2023-09-08T00:13:55.056Z // Back to normal

Consumer in the group "testbench-1000": "[thrd:main]: Topic test-topic [0]: stored offset 57842809 (leader epoch 7007), committed offset 57842809 (leader epoch 7007): not including in commit" Code: "OFFSET";"

Here is the graph displaying the committed offsets by partitions for that consumer group:
offsets

Please note that on our test bench the probability of encountering a race condition increases, because the Kubernetes pods running the consumer are constantly being throttled.

Checklist

Please provide the following information:

  • librdkafka version (release number or git tag): v2.2.0
  • Apache Kafka version: v2.7.2
  • librdkafka client configuration:
auto.offset.reset: earliest
  • Operating system: Debian 11.7
  • Provide logs (with debug=.. as necessary) from librdkafka
  • Provide broker log excerpts: N/A
  • Critical issue
@emasab
Copy link
Contributor

emasab commented Sep 19, 2023

The offset reset was triggered by RD_KAFKA_RESP_ERR__RESOLVE that corresponds to a host resolution failure during an offset validation.

                        /* Permanent error */
                        rd_kafka_offset_reset(
                            rktp, rd_kafka_broker_id(rkb),
                            RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID,
                                               rktp->rktp_leader_epoch),
                            RD_KAFKA_RESP_ERR__LOG_TRUNCATION,
                            "Unable to validate offset and epoch: %s",
                            rd_kafka_err2str(err));

We need to remove this reset and retry even in case of permanent error here, as in Java

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging a pull request may close this issue.

2 participants