Skip to content

Conversation

@original-brownbear
Copy link
Contributor

@original-brownbear original-brownbear commented Feb 22, 2019

Fixes issues with allocating huge amounts of direct memory in Netty under heavy write load.
The initial issue reported here was that writing a large message to multiple nodes was incurring a large amount of direct memory allocations when writing a large cluster state to multiple nodes, but there were also reports of ES generally consuming a lot of direct memory in heavy bulk request processing in the past sporadically.

The general reason for these problems lies in the fact that we were invoking Netty's write method faster than the network/other side could read and with large individual messages.

  • Large individual messages are a problem, as they force Netty to copy the whole message to direct memory before writing if the message is backed by a byte[] initially. So situations like writing 100MB to 50 connections simultaneously could result in allocating up to 5G if writes and/or GCing of buffers are slow.
  • Writing faster than the network can transport the data is a problem for the same reason as large writes are a problem: Netty will copy every message received by its write method to direct memory and queue it to be written without ever blocking the caller to write, which can lead to runaway allocations if writes are initiated faster than they can physically be written long term.

The fix here addresses both issues:

  • Only write if the connection signals that it is writable (defined as being below a certain amount of queues writes and the underlying channel being writable in the OP_WRITE sense).
  • Manually copy to small direct byte buffers right before invoking write to avoid big allocations and only allocate buffers for what can be immediately written.
    • Also, small buffers pool much more efficiently than larger ones which are often not pooled at all.

We could potentially do even better here memory wise (if necessary) by pooling the direct ByteBuf used for writing ourselves instead of using the Netty allocator, but we'd need data from benchmarking to support and measure the impact of that kind of effort.

@elasticmachine
Copy link
Collaborator

Pinging @elastic/es-distributed

@original-brownbear
Copy link
Contributor Author

Jenkins run elasticsearch-ci/2
Jenkins run elasticsearch-ci/packaging-sample

@original-brownbear
Copy link
Contributor Author

Jenkins run elasticsearch-ci/2

Copy link
Contributor

@henningandersen henningandersen left a comment

Choose a reason for hiding this comment

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

Thanks for your work on this @original-brownbear . I have left comments - please bear in mind that I may be wrong on all of them since this is new territory for me.

}

@Override
public void channelInactive(ChannelHandlerContext ctx) throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

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

I find this a bit suspicious since channelInactive is from ChannelInboundHandler?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ChannelInboundHandler has state callbacks for the channel in general, it's not specific to reading if that's what makes you suspicious? :)

return;
}
while (channel.isWritable()) {
if (currentWrite == null) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Are we sure that there is only one thread in here at a time? If not, we need to guard the reads/writes to currentWrite.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We shouldn't be calling flush from outside the event loop ever an all other invocations of this method do happen in event loop callbacks so we should be good here. If you look at other Netty handler implementations (like compression implementations or so) you'll see that they make the same assumption about flush.

if (channel.isActive() == false) {
if (currentWrite != null) {
currentWrite.promise.tryFailure(new ClosedChannelException());
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe set currentWrite to null to only trigger once?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That's ok currentWrite.promise.tryFailure is idempotent and I think it's easier to let Netty guard us here than do it ourselves and maybe make a mistake :)

ctx.flush();
if (channel.isActive() == false) {
WriteOperation queuedWrite;
while ((queuedWrite = queuedWrites.poll()) != null) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I think we should also notify currentWrite if it is not null and thus this is the same as is done in the beginning of this method (extract to method?).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

currentWrite will have its callback fail on the event loop because we just enqueued a flush for it. If we touch it here as well it should be a noop.

@original-brownbear original-brownbear changed the title [WIP] Chunked Netty Writes Chunked Netty Writes Feb 28, 2019
continue;
}
final int bufferSize = Math.min(write.buf.readableBytes(), 1 << 18);
writeBuffer = ctx.alloc().directBuffer(bufferSize, bufferSize);
Copy link
Contributor

Choose a reason for hiding this comment

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

Could we skip the copy if the write is smaller than 1<<18?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Maybe, it's probably a noop since Netty will itself do the same copying internally if we don't do it here. I mainly did it here to make profiling a little easier (only one source of allocations for writes).

Also, I think we could do even better here performance wise if we want/need to by using a different allocator that just caches a single direct ByteBuf, so I figured this way of extracting the alloc to a single place makes it a little easier to experiment with that kind of thing. I'll add some more comments on that to the PR description in a bit.

@original-brownbear original-brownbear changed the title Chunked Netty Writes Chunk + Throttle Netty Writes Feb 28, 2019
@original-brownbear original-brownbear marked this pull request as ready for review February 28, 2019 14:55
Copy link
Contributor

@Tim-Brooks Tim-Brooks left a comment

Choose a reason for hiding this comment

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

Some comments.


private final Netty4Transport transport;

private final ConcurrentLinkedQueue<WriteOperation> queuedWrites = new ConcurrentLinkedQueue<>();
Copy link
Contributor

Choose a reason for hiding this comment

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

As you mention in another comment, Netty channel handlers are only called from the event loop thread, so I do not see the need for a concurrent implementation here.


private WriteOperation currentWrite;

Netty4MessageChannelHandler(Netty4Transport transport) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm not sure that this should be in this (Netty4MessageChannelHandler). You are copying heap bytes -> direct bytes. That works fine for plaintext. But this handler comes before the SslHandler. Which means in the security use case, this is heap bytes -> direct bytes -> heap bytes (temporary byte array in SSLEngine) encrypted-> heap buffer -> direct bytes (in filterOutboundMessages). So this will introduce more copies.

I think that either:

  1. This should be a outbound handler that comes after the SslHandler
  2. This should not copy the bytes to a direct buffer and instead just limit the heap bytes that are passed to the next handler until the writability callback is invoked.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Right :) I went with option 2. now. That also makes the whole issue of potentially trying to write released bytes a lot safer/cleaner. Once/if the buffer is below our write chunk size we can simply pass it down outright and use retained slices in the writes preceding that situation.
(+ we save some objects for writes that are smaller than the buffer size :))

if (channel.isActive() == false) {
WriteOperation queuedWrite;
while ((queuedWrite = queuedWrites.poll()) != null) {
queuedWrite.promise.tryFailure(new ClosedChannelException());
Copy link
Contributor

Choose a reason for hiding this comment

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

It is confusing that these are cleared in the doFlush call. Can we not just clear and release everything in the channelInactive callback? Or the close callback if you change this to an outbound handler and put it after the ssl handler?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This solves the problem of potentially trying to flush released bytes because we can't just rely on the channel inactive hook as far as I can see because it triggers in a later iteration of the event-loop. From io.netty.channel.AbstractChannel.AbstractUnsafe#close:

            // Fail all the queued messages
            try {
                outboundBuffer.failFlushed(CLOSED_CHANNEL_EXCEPTION);
                outboundBuffer.close(CLOSED_CHANNEL_EXCEPTION);
            } finally {

                if (wasActive && !isActive()) {
                    invokeLater(new OneTimeTask() {
                        @Override
                        public void run() {
                            pipeline.fireChannelInactive();
                        }
                    });
                }

if (currentWrite == null) {
break;
}
final WriteOperation write = currentWrite;
Copy link
Contributor

Choose a reason for hiding this comment

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

I think you will need to guard against the promise from already failing. The prior channel write could have failed and failed the the promise associated with the write operation. I imagine that this will kill the channel and cause everything to shutdown. But it still seems safest to prevent potentially flushing bytes that have already been released.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That's why I put the cleanup loop that you mentioned above. This way we def. fail all the writes and drain the queue on failures before going for another write in the loop. And since the only way, the callbacks/futures that would release the bytes is the ctx.flush we do at the end of the loop body I think we're safe here?

@original-brownbear
Copy link
Contributor Author

@tbrooks8 thanks for pointing out the threading situation :) (my bad for getting confused by the SslHandler here :(), I simplified the logic accordingly.

@travishaagen
Copy link

@original-brownbear , instead of creating new instances of ClosedChannelException,
I suggest you consider instantiating a static final instance that can be
reused. I was first introduced to the idea here:

http://normanmaurer.me/blog/2013/11/09/The-hidden-performance-costs-of-instantiating-Throwables/

You can see from the search results below that the Netty project often uses this technique,
but I did see that they sometimes do not:

https://github.com/netty/netty/search?q=new+ClosedChannelException%28%29&unscoped_q=new+ClosedChannelException%28%29

In a nutshell, this will reduce object creation and eliminate the performance cost of
instantiating a Throwable, where the cost is from the stacktrace generation. As a
rule of thumb, whenever a Throwable is used for flow-control it may be a good idea.
If you care about the stacktrace for logging, then you would not want to do this.

@original-brownbear
Copy link
Contributor Author

original-brownbear commented Mar 2, 2019

@travishaagen since the connections here are intended to be long-lived and the messages generally have a number of other objects attached for their callbacks I don't think we have to worry about the cost of setting up these ClosedChannelExceptions here because they will be rare and there are other overheads as well.

@original-brownbear
Copy link
Contributor Author

@tbrooks8 ping :) can you take another look, please? Thanks!

@Tim-Brooks
Copy link
Contributor

Yes I will take a look tomorrow.

Copy link
Contributor

@Tim-Brooks Tim-Brooks left a comment

Choose a reason for hiding this comment

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

LGTM

@original-brownbear
Copy link
Contributor Author

@tbrooks8 thanks!

@original-brownbear original-brownbear merged commit 0cdadf5 into elastic:master Mar 7, 2019
@original-brownbear original-brownbear deleted the chunked-netty branch March 7, 2019 04:17
original-brownbear added a commit to original-brownbear/elasticsearch that referenced this pull request Mar 7, 2019
* Chunk large writes and throttle on a non-writable channel to reduce direct memory usage by Netty
original-brownbear added a commit that referenced this pull request Mar 7, 2019
* Chunk large writes and throttle on a non-writable channel to reduce direct memory usage by Netty
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants