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

[#1658] fix(server): FileSegmentManagedBuffer#nioByteBuffer read result cacheable #1659

Merged
merged 1 commit into from
Jan 27, 2025

Conversation

xumanbu
Copy link
Contributor

@xumanbu xumanbu commented Apr 18, 2024

What changes were proposed in this pull request?

FileSegmentManagedBuffer#nioByteBuffer add a buffer to hold read file result to avoid multiple read file

Why are the changes needed?

Fix: #1658

Does this PR introduce any user-facing change?

No.

How was this patch tested?

existed UT

@xumanbu
Copy link
Contributor Author

xumanbu commented Apr 18, 2024

PTAL. @dingshun3016 @zuston @jerqi

Copy link

github-actions bot commented Apr 18, 2024

Test Results

 2 398 files  +14   2 398 suites  +14   4h 38m 33s ⏱️ + 6m 5s
   921 tests + 2     920 ✅ + 2   1 💤 ±0  0 ❌ ±0 
10 689 runs  +23  10 675 ✅ +23  14 💤 ±0  0 ❌ ±0 

Results for commit 8740feb. ± Comparison against base commit 5ab625b.

This pull request removes 1 and adds 3 tests. Note that renamed tests count towards both.
org.apache.uniffle.shuffle.manager.ShuffleManagerServerFactoryTest ‑ testShuffleManagerServerType
org.apache.uniffle.common.netty.buffer.FileSegmentManagedBufferTest ‑ testNioByteBuffer{File}
org.apache.uniffle.shuffle.manager.ShuffleManagerServerFactoryTest ‑ testShuffleManagerServerType{ServerType}[1]
org.apache.uniffle.shuffle.manager.ShuffleManagerServerFactoryTest ‑ testShuffleManagerServerType{ServerType}[2]

♻️ This comment has been updated with latest results.

@zuston zuston changed the title [#1658] fix(server) FileSegmentManagedBuffer#nioByteBuffer read result cacheable [#1658] fix(server): FileSegmentManagedBuffer#nioByteBuffer read result cacheable Apr 18, 2024
@zuston
Copy link
Member

zuston commented Apr 18, 2024

Could you help review this? @rickyma

Copy link
Contributor

@rickyma rickyma left a comment

Choose a reason for hiding this comment

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

Left some comments. I'm OK with this patch. But you need to test it more carefully. Because this is used in gRPC mode. It won't be invoked in Netty mode. So I don't know how it actually performs in a production environment. Because I currently use Netty in my environment.

@@ -39,6 +39,8 @@ public class FileSegmentManagedBuffer extends ManagedBuffer {
private final File file;
private final long offset;
private final int length;
private boolean isFilled;
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe we don't need isFilled at all. I think it can be replaced by readByteBuffer != null.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The readByteBuffer may be allocated but not filled with bytes, indicating a failure to read the file. Therefore, I believe adding another state for reliability is necessary.

} catch (IOException e) {
String errorMessage = "Error in reading " + this;
String errorMessage = "Error in reading file " + file + " " + this;
Copy link
Contributor

Choose a reason for hiding this comment

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

It's better if we use String.format.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok. fixed.

try {
if (channel != null) {
long size = channel.size();
errorMessage = "Error in reading " + this + " (actual file length " + size + ")";
errorMessage =
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed.

@zuston
Copy link
Member

zuston commented Apr 18, 2024

Left some comments. I'm OK with this patch. But you need to test it more carefully. Because this is used in gRPC mode. It won't be invoked in Netty mode. So I don't know how it actually performs in a production environment. Because I currently use Netty in my environment.

Oh I thought this is netty. I will take another look tomorrow.

@jerqi
Copy link
Contributor

jerqi commented Apr 19, 2024

Could you add a ut if this is a fix?

@xumanbu
Copy link
Contributor Author

xumanbu commented Apr 29, 2024

Could you add a ut if this is a fix?

ok. added.

Copy link
Member

@zuston zuston left a comment

Choose a reason for hiding this comment

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

Sorry for the late reply.

Nice improvement, this is useful for AQE skew tasks. Left minor comments. @xumanbu

return buf;
readByteBuffer.flip();
isFilled = true;
return readByteBuffer;
Copy link
Member

Choose a reason for hiding this comment

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

Can we make this as readOnlyBuffer? return buf.asReadOnlyBuffer();

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I have tested it, but it doesn't work. ReadOnlyBuffer will throw a ReadOnlyBufferException when calling buffer.array().

Copy link
Member

Choose a reason for hiding this comment

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

Pity. I want to ensure the readBuffer will not be changed after cached.

@jerqi
Copy link
Contributor

jerqi commented Jan 2, 2025

@xumanbu @zuston Will you continue this pull request?

@jerqi
Copy link
Contributor

jerqi commented Jan 20, 2025

@zuston @xumanbu Do we need to continue this pull request?

@xumanbu
Copy link
Contributor Author

xumanbu commented Jan 20, 2025

@zuston @xumanbu Do we need to continue this pull request?

I think this is necessary to continue, I'll pick it up.

@xumanbu xumanbu closed this Jan 23, 2025
@xumanbu xumanbu force-pushed the 1658_FileSegmentManagedBuffer branch from 8740feb to 08e7397 Compare January 23, 2025 08:22
@xumanbu
Copy link
Contributor Author

xumanbu commented Jan 23, 2025

@jerqi mistakenly discarded changes before committing. I've recommitted them. Could you help me reopen this PR?

@jerqi jerqi reopened this Jan 23, 2025
@xumanbu xumanbu requested a review from rickyma January 23, 2025 10:42
@xumanbu
Copy link
Contributor Author

xumanbu commented Jan 23, 2025

@zuston @rickyma Please help me see has any other issues

Copy link
Member

@zuston zuston left a comment

Choose a reason for hiding this comment

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

+1 for this. Thanks for your contribution. If no one reject this, I will merge this in the next 2 days.

@jerqi
Copy link
Contributor

jerqi commented Jan 26, 2025

@rickyma Do u have another suggestion?

@rickyma
Copy link
Contributor

rickyma commented Jan 26, 2025

@rickyma Do u have another suggestion?

Nope. LGTM. But I did not test this.

@zuston
Copy link
Member

zuston commented Jan 27, 2025

Ok. Let's merge this.

@zuston zuston merged commit 3590940 into apache:master Jan 27, 2025
44 checks passed
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.

[Bug] Optimized FileSegmentManagedBuffer.nioByteBuffer to avoid multiple read file
4 participants