-
Notifications
You must be signed in to change notification settings - Fork 28.1k
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
[SPARK-24578][Core] Cap sub-region's size of returned nio buffer #21593
Conversation
Jenkins, ok to test |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
lgtm
I made a suggestion for another improvement we can do while we're at it, but its small and this is a really important fix, so its OK to leave it.
// SPARK-24578: cap the sub-region's size of returned nio buffer to improve the performance | ||
// for the case that the passed-in buffer has too many components. | ||
int length = Math.min(buf.readableBytes(), NIO_BUFFER_LIMIT); | ||
ByteBuffer buffer = buf.nioBuffer(buf.readerIndex(), length); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think you can go one step further here, and call buf.nioBuffers(int, int)
(plural)
https://github.com/netty/netty/blob/4.1/buffer/src/main/java/io/netty/buffer/ByteBuf.java#L2355
that will avoid the copying required to create the merged buffer (though its a bit complicated as you have to check for incomplete writes from any single target.write()
call).
Also OK to leave this for now as this is a pretty important fix.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, I will make a follow up PR to address this.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why not do this in this PR since this is a small change and we don't have a new release recently?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this pr is fixing a pretty serious issue, we know Wenbo is going to roll this out immediately, and I suspect even more users will. this fix is also "obviously correct" -- the followup here is not super complicated, but also will be more prone to bugs. So I'm inclined to just get this in.
anyway if @WenboZhao can do the other part today, then sure, but I think we should get this in quickly.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fair enough.
I just spent several minutes to write the following codes:
private int copyByteBuf(ByteBuf buf, WritableByteChannel target) throws IOException {
// SPARK-24578: cap the sub-region's size of returned nio buffer to improve the performance
// for the case that the passed-in buffer has too many components.
int length = Math.min(buf.readableBytes(), NIO_BUFFER_LIMIT);
ByteBuffer[] buffers = buf.nioBuffers(buf.readerIndex(), length);
int totalWritten = 0;
for (ByteBuffer buffer : buffers) {
int remaining = buffer.remaining();
int written = target.write(buffer);
totalWritten += written;
if (written < remaining) {
break;
}
}
buf.skipBytes(totalWritten);
return totalWritten;
}
Feel free to use them in your follow up PR.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@WenboZhao did you ever follow up on this, or at least file another jira for it? sorry if I missed it
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
for anybody watching this, eventually SPARK-25115 was opened (currently has a PR)
Test build #92114 has finished for PR 21593 at commit
|
retest this please |
Test build #92122 has finished for PR 21593 at commit
|
Thanks! Merging to master and 2.3. |
## What changes were proposed in this pull request? This PR tries to fix the performance regression introduced by SPARK-21517. In our production job, we performed many parallel computations, with high possibility, some task could be scheduled to a host-2 where it needs to read the cache block data from host-1. Often, this big transfer makes the cluster suffer time out issue (it will retry 3 times, each with 120s timeout, and then do recompute to put the cache block into the local MemoryStore). The root cause is that we don't do `consolidateIfNeeded` anymore as we are using ``` Unpooled.wrappedBuffer(chunks.length, getChunks(): _*) ``` in ChunkedByteBuffer. If we have many small chunks, it could cause the `buf.notBuffer(...)` have very bad performance in the case that we have to call `copyByteBuf(...)` many times. ## How was this patch tested? Existing unit tests and also test in production Author: Wenbo Zhao <wzhao@twosigma.com> Closes #21593 from WenboZhao/spark-24578. (cherry picked from commit 3f4bda7) Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
## What changes were proposed in this pull request? This PR tries to fix the performance regression introduced by SPARK-21517. In our production job, we performed many parallel computations, with high possibility, some task could be scheduled to a host-2 where it needs to read the cache block data from host-1. Often, this big transfer makes the cluster suffer time out issue (it will retry 3 times, each with 120s timeout, and then do recompute to put the cache block into the local MemoryStore). The root cause is that we don't do `consolidateIfNeeded` anymore as we are using ``` Unpooled.wrappedBuffer(chunks.length, getChunks(): _*) ``` in ChunkedByteBuffer. If we have many small chunks, it could cause the `buf.notBuffer(...)` have very bad performance in the case that we have to call `copyByteBuf(...)` many times. ## How was this patch tested? Existing unit tests and also test in production Author: Wenbo Zhao <wzhao@twosigma.com> Closes apache#21593 from WenboZhao/spark-24578. (cherry picked from commit 3f4bda7)
## What changes were proposed in this pull request? This PR tries to fix the performance regression introduced by SPARK-21517. In our production job, we performed many parallel computations, with high possibility, some task could be scheduled to a host-2 where it needs to read the cache block data from host-1. Often, this big transfer makes the cluster suffer time out issue (it will retry 3 times, each with 120s timeout, and then do recompute to put the cache block into the local MemoryStore). The root cause is that we don't do `consolidateIfNeeded` anymore as we are using ``` Unpooled.wrappedBuffer(chunks.length, getChunks(): _*) ``` in ChunkedByteBuffer. If we have many small chunks, it could cause the `buf.notBuffer(...)` have very bad performance in the case that we have to call `copyByteBuf(...)` many times. ## How was this patch tested? Existing unit tests and also test in production Author: Wenbo Zhao <wzhao@twosigma.com> Closes apache#21593 from WenboZhao/spark-24578. (cherry picked from commit 3f4bda7)
which release does have this commit? |
since 2.3.2 |
What changes were proposed in this pull request?
This PR tries to fix the performance regression introduced by SPARK-21517.
In our production job, we performed many parallel computations, with high possibility, some task could be scheduled to a host-2 where it needs to read the cache block data from host-1. Often, this big transfer makes the cluster suffer time out issue (it will retry 3 times, each with 120s timeout, and then do recompute to put the cache block into the local MemoryStore).
The root cause is that we don't do
consolidateIfNeeded
anymore as we are usingin ChunkedByteBuffer. If we have many small chunks, it could cause the
buf.notBuffer(...)
have very bad performance in the case that we have to callcopyByteBuf(...)
many times.How was this patch tested?
Existing unit tests and also test in production