Skip to content

Conversation

@normanmaurer
Copy link
Contributor

@normanmaurer normanmaurer commented Aug 14, 2018

…d by > 1 ByteBuffer.

What changes were proposed in this pull request?

Check how many ByteBuffer are used and depending on it do either call nioBuffer(...) or nioBuffers(...) to eliminate extra memory copies.

This is related to netty/netty#8176.

How was this patch tested?

Unit tests added.

@dbtsai
Copy link
Member

dbtsai commented Aug 14, 2018

add to whitelist

@normanmaurer normanmaurer force-pushed the composite_byte_buf_mem_copy branch from 9cf1180 to 2ccca98 Compare August 14, 2018 17:18
@normanmaurer normanmaurer changed the title Eliminate extra memory copy done when a ByteBuf is used that is backe… [SPARK-25115] [Core] Eliminate extra memory copy done when a ByteBuf is used that is backed by > 1 ByteBuffer. Aug 14, 2018
…is used that is backed by > 1 ByteBuffer.

Check how many ByteBuffer are used and depending on it do either call nioBuffer(...) or nioBuffers(...) to eliminate extra memory copies.

This is related to netty/netty#8176.

Unit tests added.
@normanmaurer normanmaurer force-pushed the composite_byte_buf_mem_copy branch from 2ccca98 to beb1454 Compare August 14, 2018 17:34
for (ByteBuffer buffer: buffers) {
int remaining = buffer.remaining();
int w = target.write(buffer);
written += w;
Copy link
Member

@kiszk kiszk Aug 14, 2018

Choose a reason for hiding this comment

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

Can we guarantee written does not cause overflow while we accumulate int values?

Copy link
Member

Choose a reason for hiding this comment

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

We are using Int in the current codebase, so this should not be a concern. Also, ByteBuf is using Int to index the byte; as a result, it's impossible to overflow written given they're from the same ByteBuf.

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);
Copy link
Contributor

Choose a reason for hiding this comment

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

Is this limit needed now at all? I think your change was suggested as a future enhancement when reviewing SPARK-24578 but I never noticed a PR to implement it until yours...

@attilapiros

Copy link
Member

Choose a reason for hiding this comment

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

@vanzin This is to avoid memory copy when writing a large ByteBuffer. You merged this actually: #12083

Copy link
Contributor

Choose a reason for hiding this comment

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

My understanding is that the code here is avoiding the copy by using nioBuffers() already, so that limit shouldn't be needed, right? Or maybe I'm missing something.

Copy link
Member

Choose a reason for hiding this comment

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

@vanzin The ByteBuffer here may be just a large ByteBuffer. See my comment here: #12083 (comment)

Copy link
Contributor

Choose a reason for hiding this comment

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

I re-read that discussion and you're right. I know this has been checked in, but the comment is now stale; the limit is there because of the behavior of the JRE code, not because of the composite buffer. It would be good to update it.

Copy link
Member

@dbtsai dbtsai Aug 15, 2018

Choose a reason for hiding this comment

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

Out of my curiosity, how did we come out with number of NIO_BUFFER_LIMIT 256KB?

In Hadoop, they are using 8KB

For most OSes, in the write(ByteBuffer[]) API in sun.nio.ch.IOUtil, it goes one buffer at a time, and gets a temporary direct buffer from the BufferCache, up to a limit of IOUtil#IOV_MAX which is 1KB.

Copy link
Contributor

Choose a reason for hiding this comment

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

That was only briefly discussed in the PR that Ryan linked above... the original code actually used 512k.

I think Hadoop's limit is a little low, but maybe 256k is a bit high. IIRC socket buffers are 32k by default on Linux, so it seems unlikely you'd be able to write 256k in one call (ignoring what IOUtil does internally). But maybe in practice it works ok.

If anyone has the time to test this out that would be great.

Copy link
Member

Choose a reason for hiding this comment

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

Maybe @normanmaurer has some insight?

Copy link
Member

Choose a reason for hiding this comment

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

IIRC socket buffers are 32k by default on Linux, so it seems unlikely you'd be able to write 256k in one call (ignoring what IOUtil does internally). But maybe in practice it works ok.

After reading the context in #12083 and this discussion, I want to provide a possibility about 256k in one call can work in practice. As in our scenario, user will change /proc/sys/net/core/wmem_default based on their online behavior, generally we'll set this value larger than wmem_default.
image
So maybe 256k of NIO_BUFFER_LIMIT is ok here? We just need add more annotation to remind what params related with this value.

@dbtsai
Copy link
Member

dbtsai commented Aug 14, 2018

Add @squito who reviewed this code before.

@SparkQA
Copy link

SparkQA commented Aug 14, 2018

Test build #94756 has finished for PR 22105 at commit beb1454.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Aug 14, 2018

Test build #94755 has finished for PR 22105 at commit 2ccca98.

  • This patch fails from timeout after a configured wait of `340m`.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zsxwing
Copy link
Member

zsxwing commented Aug 14, 2018

@normanmaurer LGTM. Thanks for the fix. I totally forgot this issue.

@dbtsai
Copy link
Member

dbtsai commented Aug 15, 2018

LGTM. Merged into master. Thanks.

@asfgit asfgit closed this in 92fd7f3 Aug 15, 2018
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.

7 participants