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-14522 Rewrite/Move of RemoteIndexCache to storage module. #13275

Merged
merged 10 commits into from Jul 11, 2023

Conversation

satishd
Copy link
Member

@satishd satishd commented Feb 18, 2023

KAFKA-14522 Rewrite/Move of RemoteIndexCache to the storage module.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

Copy link
Contributor

@showuon showuon left a comment

Choose a reason for hiding this comment

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

Had an early review, left some comments. Thanks.

@satishd satishd force-pushed the KAFKA-14522 branch 2 times, most recently from da17d44 to 90b914c Compare February 19, 2023 15:19
@satishd satishd marked this pull request as ready for review February 25, 2023 12:14
@satishd satishd force-pushed the KAFKA-14522 branch 2 times, most recently from 81ceeee to e49a357 Compare March 1, 2023 06:57
@satishd
Copy link
Member Author

satishd commented Mar 1, 2023

Thanks @showuon for your review. Addressed the comments with the latest commit.

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.

@satishd : Thanks for the PR. A few comments below.

try {
entry.markForCleanup();
} catch (IOException e) {
throw new KafkaException(e);
Copy link
Contributor

Choose a reason for hiding this comment

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

The constructor declares throwing IOException. Why do we need to convert IOException to KafkaException? Ditto in other methods like init()?

Copy link
Member Author

Choose a reason for hiding this comment

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

This exception is not directly thrown to the constructor. It is inside an anonymous inner class of LinkedHashMap#removeEldestEntry. This method is not declared with IOException.

@satishd
Copy link
Member Author

satishd commented Mar 2, 2023

Thanks @junrao for your review. Addressed your comments inline and/or updated with the latest commit.

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.

@satishd : Thanks for the updated PR. A few more comments.

@divijvaidya divijvaidya added the tiered-storage Pull requests associated with KIP-405 (Tiered Storage) label Jun 16, 2023
@satishd satishd force-pushed the KAFKA-14522 branch 2 times, most recently from b290f00 to 22a9c72 Compare July 4, 2023 06:14
@satishd
Copy link
Member Author

satishd commented Jul 4, 2023

@junrao @showuon Addressed the review comments with the latest commits. It also includes the recent changes on RemoteIndexCache in trunk.

@satishd satishd requested review from junrao and showuon July 4, 2023 06:39
@showuon
Copy link
Contributor

showuon commented Jul 5, 2023

Will take a look this week or next week. Thanks.

Copy link
Contributor

@showuon showuon left a comment

Choose a reason for hiding this comment

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

@satishd , thanks for the update. Left some comments.

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.

@satishd : Thanks for the updated PR. Just a few minor comments.

cleanerThread.start();
}

public LinkedBlockingQueue<Entry> expiredIndexes() {
Copy link
Contributor

Choose a reason for hiding this comment

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

This and the next two methods are only used for tests. Should we expose them at the package level?

Copy link
Member Author

@satishd satishd Jul 10, 2023

Choose a reason for hiding this comment

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

Tests are written at different package level in core module. We can revisit this once the tests are also moved to java and the respective module/package.

log.error("Cleaner thread received interruption but remote index cache is not closed", ex);
throw new KafkaException(ex);
} else {
log.debug("Cleaner thread was interrupted on cache shutdown");
Copy link
Contributor

Choose a reason for hiding this comment

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

ShutdownableThread already logs the shutdown.

Copy link
Member Author

Choose a reason for hiding this comment

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

This comment is very specific to the interrupted scenario.

// Delete any .deleted files remained from the earlier run of the broker.
try (Stream<Path> paths = Files.list(cacheDir.toPath())) {
paths.forEach(path -> {
if (path.endsWith(LogFileUtils.DELETED_FILE_SUFFIX)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Should we clean up the tmp file too?

Cleanedup index file suffix usages and other minor cleaups
@satishd
Copy link
Member Author

satishd commented Jul 10, 2023

Thanks @junrao for the review comment. Addressed them with the latest commits and/or inline.

@satishd
Copy link
Member Author

satishd commented Jul 10, 2023

Thanks @divijvaidya for the review. Addressed them with inline and/or with the latest commits.

Copy link
Contributor

@showuon showuon left a comment

Choose a reason for hiding this comment

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

LGTM!

Copy link
Contributor

@divijvaidya divijvaidya left a comment

Choose a reason for hiding this comment

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

Left a last round of minor comments. I am good to ship this PR once they are resolved.

@satishd
Copy link
Member Author

satishd commented Jul 10, 2023

Thanks @divijvaidya for the latest review. Addressed them with the latest commit.

Copy link
Contributor

@jeqo jeqo left a comment

Choose a reason for hiding this comment

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

Just a few comments related to TxnIndex potentially being optional.

@satishd
Copy link
Member Author

satishd commented Jul 11, 2023

Just a few comments related to TxnIndex potentially being optional.

@jeqo There is already KAFKA-14993 to address that. @kamalcph was working on that. I did not want to add those changes to this PR as it is tracked separately.

Copy link
Contributor

@jeqo jeqo left a comment

Choose a reason for hiding this comment

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

Sure, thanks for the reminder. LGTM, thanks @satishd !

Copy link
Collaborator

@kamalcph kamalcph left a comment

Choose a reason for hiding this comment

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

LGTM

@satishd satishd requested a review from junrao July 11, 2023 08:55
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.

@satishd : Thanks for the updated PR. LGTM

@satishd
Copy link
Member Author

satishd commented Jul 11, 2023

A few tests are failed but those are not related to the changes in the PR.

@satishd satishd merged commit 7e2f878 into apache:trunk Jul 11, 2023
1 check failed
Cerchie pushed a commit to Cerchie/kafka that referenced this pull request Jul 25, 2023
…he#13275)

KAFKA-14522 Rewrite and Move of RemoteIndexCache to storage module.
Cleanedup index file suffix usages and other minor cleanups

Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
tiered-storage Pull requests associated with KIP-405 (Tiered Storage)
Projects
None yet
7 participants