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

KAFKA-12571: Eliminate LeaderEpochFileCache constructor dependency on logEndOffset #10426

Conversation

kowshik
Copy link
Contributor

@kowshik kowshik commented Mar 29, 2021

This PR is a precursor to the recovery logic refactor work (KAFKA-12553).

Problems:
For refactoring the recovery logic (KAFKA-12553), we would like to move the logic to initialize LeaderEpochFileCache out of the Log class and into a separate static function. In the future, once we successfully initialize LeaderEpochFileCache outside Log, we will be able pass it as a dependency into both the Log recovery module and Log class constructor. However, currently the LeaderEpochFileCache constructor takes a dependency on logEndOffset (via a callback), which poses the following problems:

  1. Blocks the instantiation of LeaderEpochFileCache outside Log class. Because, outside Log the logEndOffset is unavailable to be passed into LeaderEpochFileCache constructor. As a result, this situation blocks the recovery logic (KAFKA-12553) refactor work.
  2. It turns out the logEndOffset dependency is used only in 1 of the LeaderEpochFileCache methods: LeaderEpochFileCache.endOffsetFor, and just for 1 particular case. Therefore, it is overkill to pass it in the constructor as a dependency. Also a callback is generally not a neat way to access dependencies and it poses code readability problems too.

Solution:
This PR modifies the code such that we only pass the logEndOffset as a parameter into LeaderEpochFileCache.endOffsetFor whenever the method is called, thus eliminating the constructor dependency. This will also unblock the recovery logic refactor work (KAFKA-12553).

Tests:
I have modified the existing tests to suit the above refactor.

@kowshik
Copy link
Contributor Author

kowshik commented Mar 29, 2021

cc @junrao @dhruvilshah3 @satishd: this PR is ready for review.

@kowshik kowshik force-pushed the refactor_eliminate_LogEndOffset_dependency_on_LeaderEpochFileCache branch from 61ccacc to 4208aad Compare March 29, 2021 20:06
Copy link
Contributor

@dhruvilshah3 dhruvilshah3 left a comment

Choose a reason for hiding this comment

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

LGTM, thanks for the PR

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@kowshik : Thanks for the PR. LGTM. Waiting for the tests to pass.

@kowshik kowshik force-pushed the refactor_eliminate_LogEndOffset_dependency_on_LeaderEpochFileCache branch from 4208aad to c699987 Compare March 29, 2021 23:59
@kowshik
Copy link
Contributor Author

kowshik commented Mar 30, 2021

The rebase has started the Jenkins run now. We can wait for the tests to pass.

@kowshik kowshik force-pushed the refactor_eliminate_LogEndOffset_dependency_on_LeaderEpochFileCache branch from c699987 to b5d41d2 Compare March 30, 2021 06:52
Copy link
Member

@satishd satishd left a comment

Choose a reason for hiding this comment

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

thanks @kowshik for the PR, LGTM.

@junrao
Copy link
Contributor

junrao commented Mar 30, 2021

@kowshik : Do you know why the tests timed out? Thanks.

@kowshik kowshik force-pushed the refactor_eliminate_LogEndOffset_dependency_on_LeaderEpochFileCache branch 2 times, most recently from 4bd5bc1 to e01f613 Compare March 30, 2021 18:01
@kowshik kowshik force-pushed the refactor_eliminate_LogEndOffset_dependency_on_LeaderEpochFileCache branch from e01f613 to 53f8433 Compare March 30, 2021 18:12
@kowshik
Copy link
Contributor Author

kowshik commented Mar 30, 2021

@junrao Jenkins doesn't seem to be starting up tests after I rebased this morning. I'm not sure what's going on. I'll try again today.

@kowshik
Copy link
Contributor Author

kowshik commented Mar 31, 2021

@junrao The jenkins tests have finished. I checked the test failures, they seem to be unrelated to this PR:

JDK 11: org.apache.kafka.connect.integration.RebalanceSourceConnectorsIntegrationTest.testMultipleWorkersRejoining
JDK 11 & 15: org.apache.kafka.clients.admin.KafkaAdminClientTest.testClientSideTimeoutAfterFailureToReceiveResponse
JDK 15: org.apache.kafka.streams.integration.StreamsUpgradeTestIntegrationTest.classMethod

@junrao junrao merged commit b35704a into apache:trunk Mar 31, 2021
Terrdi pushed a commit to Terrdi/kafka that referenced this pull request Apr 1, 2021
… logEndOffset (apache#10426)

This PR is a precursor to the recovery logic refactor work (KAFKA-12553).

Problems:
For refactoring the recovery logic (KAFKA-12553), we would like to move the logic to initialize LeaderEpochFileCache out of the Log class and into a separate static function. In the future, once we successfully initialize LeaderEpochFileCache outside Log, we will be able pass it as a dependency into both the Log recovery module and Log class constructor. However, currently the LeaderEpochFileCache constructor takes a dependency on logEndOffset (via a callback), which poses the following problems:

Blocks the instantiation of LeaderEpochFileCache outside Log class. Because, outside Log the logEndOffset is unavailable to be passed into LeaderEpochFileCache constructor. As a result, this situation blocks the recovery logic (KAFKA-12553) refactor work.
It turns out the logEndOffset dependency is used only in 1 of the LeaderEpochFileCache methods: LeaderEpochFileCache.endOffsetFor, and just for 1 particular case. Therefore, it is overkill to pass it in the constructor as a dependency. Also a callback is generally not a neat way to access dependencies and it poses code readability problems too.

Solution:
This PR modifies the code such that we only pass the logEndOffset as a parameter into LeaderEpochFileCache.endOffsetFor whenever the method is called, thus eliminating the constructor dependency. This will also unblock the recovery logic refactor work (KAFKA-12553).

Tests:
I have modified the existing tests to suit the above refactor.

Reviewers: Dhruvil Shah <dhruvil@confluent.io>, Jun Rao <junrao@gmail.com>
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.

None yet

4 participants