Motivation:
If SO_LINGER is used shutdownOutput() and close() syscalls will block until either all data was send or until the timeout exceed. This is a problem when we try to execute them on the EventLoop as this means the EventLoop may be blocked and so can not process any other I/O.
Modifications:
- Add AbstractUnsafe.closeExecutor() which returns null by default and use this Executor for close if not null.
- Override the closeExecutor() in NioSocketChannel and EpollSocketChannel and return GlobalEventExecutor.INSTANCE if getSoLinger() > 0
- use closeExecutor() in shutdownInput(...) in NioSocketChannel and EpollSocketChannel
Result:
No more blocking of the EventLoop if SO_LINGER is used and shutdownOutput() or close() is called.
Motivation:
isRoot() is an expensive operation. We should avoid calling it if
possible.
Modifications:
Move the isRoot() checks to the end of the 'if' block, so that isRoot()
is evaluated only when really necessary.
Result:
isRoot() is evaluated only when SO_BROADCAST is set and the bind address
is anylocal address.
Related:
- 8b2fb2b985
Motivation:
The commit mentioned above introduced a regression where
channelReadComplete() event is swallowed by a handler which was added
dynamically.
Modifications:
Do not suppress channelReadComplete() if the current handler's
channelRead() method was not invoked at all, so that a just-added
handler does not suppress channelReadComplete().
Result:
Regression is gone, and channelReadComplete() is invoked when necessary.
Related:
- 375b9e1307
Motivation:
Even if a handler called ctx.fireChannelReadComplete(), the next handler
should not get its channelReadComplete() invoked if fireChannelRead()
was not invoked before.
Modifications:
- Ensure channelReadComplete() is invoked only when the handler of the
current context actually produced a message, because otherwise there's
no point of triggering channelReadComplete().
i.e. channelReadComplete() must follow channelRead().
- Fix a bug where ctx.read() was not called if the handler of the
current context did not produce any message, making the connection
stall. Read the new comment for more information.
Result:
- channelReadComplete() is invoked only when it makes sense.
- No stale connection
Motivation:
Because of a re-entrance bug in PendingWriteQueue it was possible to get the queue corrupted and also trigger an IllegalStateException caused by multiple recycling of the internal PendingWrite objects.
Modifications:
- Correctly guard against re-entrance
Result:
No more IllegalStateException possible
Motiviation:
When using domain sockets on linux it is supported to recv and send file descriptors. This can be used to pass around for example sockets.
Modifications:
- Add support for recv and send file descriptors when using EpollDomainSocketChannel.
- Allow to obtain the file descriptor for an Epoll*Channel so it can be send via domain sockets.
Result:
recv and send of file descriptors is supported now.
Motivation:
As the ByteBuf is not set to null after release it we may try to release it again in handleReadException()
Modifications:
- set ByteBuf to null to avoid another byteBuf.release() to be called in handleReadException()
Result:
No IllegalReferenceCountException anymore
Motivation:
ctx.fireChannelReadComplete() should only be called if something is produced during a channelRead(...) operation. Also we must ensure that it will be called
if channelRead(...) produced something at some point as channelRead(...) maybe called multiple times by the transport before channelReadComplete(...) is called.
Modifications:
- Ensure channelReadComplete(...) only triggers ctx.fireChannelReadComplete() when a previous channelRead(...) call produced a message
- Ensure read() is called of more data is needed
Result:
Correct semantic with channelReadComplete(...) events and also ensure no stales
Motivation:
Fix a minor documentation bug in
ChannelHandlerContext#fireChannelReadComplete.
Modifications:
ChannelHandlerContext#fireChannelReadComplete no longer references an
incorrect method in its javadoc.
Results:
Documentation is correct.
Motivation:
We only provided a constructor in DefaultFileRegion that takes a FileChannel which means the File itself needs to get opened on construction. This has the problem that if you want to write a lot of Files very fast you may end up with may open FD's even if they are not needed yet. This can lead to hit the open FD limit of the OS.
Modifications:
Add a new constructor to DefaultFileRegion which allows to construct it from a File. The FileChannel will only be obtained when transferTo(...) is called or the DefaultFileRegion is explicit open'ed via open() (this is needed for the native epoll transport)
Result:
Less resource usage when writing a lot of DefaultFileRegion.
Related: #3212
Motivation:
When SslHandler and ChunkedWriteHandler exists in a pipeline together,
it is possible that ChunkedWriteHandler.channelWritabilityChanged()
invokes SslHandler.flush() and vice versa. Because they can feed each
other (i.e. ChunkedWriteHandler.channelWritabilityChanged() ->
SslHandler.flush() -> ChunkedWriteHandler.channelWritabilityChanged() ->
..), they can fall into an inconsistent state due to reentrance (e.g.
bad MAC record at the remote peer due to incorrect ordering.)
Modifications:
- Trigger channelWritabilityChanged() using EventLoop.execute() when
there's a chance where channelWritabilityChanged() can cause a
reentrance issue
- Fix test failures caused by the modification
Result:
Fix the handler reentrance issues related with a
channelWritabilityChanged() event
Related: #3212
Motivation:
PendingWriteQueue.recycle() updates its data structure after triggering
a channelWritabilityChanged() event. It causes a rare corruption such as
double free when channelWritabilityChanged() method accesses the
PendingWriteQueue.
Modifications:
Update the state of PendingWriteQueue before triggering an event.
Result:
Fix a rare double-free problem
Related: #3190
Motivation:
When an outbound handler method raises an exception, its promise is
marked as failed. If the promise is done already, the exception is
logged.
When the promise is void, exceptionCaught() must be triggered to notify
a user. However, AbstractChannelHandlerContext simply swallows it.
Modifications:
Do not swallow an exception when the promise is void.
Result:
A user who uses a void promise for an outbound operation will be
notified on failure.
Related: #3189
Motivation:
OIO transport implementations block for at most 1 second to wait for
additional messages (or accepted connections).
However, because AbstractOioMessageChannel defers the channelRead()
events for the messages read so far until there's nothing to read up to
maxMessagesPerRead, any read operation will be followed by a 1-second
delay.
Modifications:
Fire channelRead() events as soon as doRead() returns so that there is
no 1 second delay between the actual read and the channelRead() event.
Result:
No more weird 1-second delay
Related: #3156
Motivation:
Let's say we have a channel with the following pipeline configuration:
HEAD --> [E1] H1 --> [E2] H2 --> TAIL
when the channel is deregistered, the channelUnregistered() methods of
H1 and H2 will be invoked from the executor thread of E1 and E2
respectively. To ensure that the channelUnregistered() methods are
invoked from the correct thread, new one-time tasks will be created
accordingly and be scheduled via Executor.execute(Runnable).
As soon as the one-time tasks are scheduled,
DefaultChannelPipeline.fireChannelUnregistered() will start to remove
all handlers from the pipeline via teardownAll(). This process is
performed in reversed order of event propagation. i.e. H2 is removed
first, and then H1 is removed.
If the channelUnregistered() event has been passed to H2 before H2 is
removed, a user does not see any problem.
If H2 has been removed before channelUnregistered() event is passed to
H2, a user will often see the following confusing warning message:
An exceptionCaught() event was fired, and it reached at the tail of
the pipeline. It usually means the last handler in the pipeline did
not handle the exception.
Modifications:
To ensure that the handlers are removed *after* all events are
propagated, traverse the pipeline in ascending order before performing
the actual removal.
Result:
A user does not get the confusing warning message anymore.
Motivation:
AbstractUnsafe considers two possibilities during channel registration. First,
the channel may be an outgoing connection, in which case it will be registered
before becoming active. Second, the channel may be an incoming connection in,
which case the channel will already be active when it is registered. To handle
the second case, AbstractUnsafe checks if the channel is active after
registration and calls ChannelPipeline.fireChannelActive() if so. However, if
an active channel is deregistered and then re-registered this logic causes a
second fireChannelActive() to be invoked. This is unexpected; it is reasonable
for handlers to assume that this method will only be invoked once per channel.
Modifications:
This change introduces a flag into AbstractUnsafe to recognize if this is the
first or a subsequent registration. ChannelPipeline.fireChannelActive() is only
possible for the first registration.
Result:
ChannelPipeline.fireChannelActive() is only called once.
Motivation:
Found performance issues via FindBugs and PMD.
Modifications:
- Removed unnecessary boxing/unboxing operations in DefaultTextHeaders.convertToInt(CharSequence) and DefaultTextHeaders.convertToLong(CharSequence). A boxed primitive is created from a string, just to extract the unboxed primitive value.
- Added a static modifier for DefaultHttp2Connection.ParentChangedEvent class. This class is an inner class, but does not use its embedded reference to the object which created it. This reference makes the instances of the class larger, and may keep the reference to the creator object alive longer than necessary.
- Added a static compiled Pattern to avoid compile it each time it is used when we need to replace some part of authority.
- Improved using of StringBuilders.
Result:
Performance improvements.
Motivation:
ChannelPromiseAggregator and ChannelPromiseNotifiers only allow
consumers to work with Channels as the result type. Generic versions
of these classes allow consumers to aggregate or broadcast the results
of an asynchronous execution with other result types.
Modifications:
Add PromiseAggregator and PromiseNotifier. Add unit tests for both.
Remove code in ChannelPromiseAggregator and ChannelPromiseNotifier and
modify them to extend the new base classes.
Result:
Consumers can now aggregate or broadcast the results of an asynchronous
execution with results types other than Channel.
Related: #2945
Motivation:
Some special handlers such as TrafficShapingHandler need to override the
writability of a Channel to throttle the outbound traffic.
Modifications:
Add a new indexed property called 'user-defined writability flag' to
ChannelOutboundBuffer so that a handler can override the writability of
a Channel easily.
Result:
A handler can override the writability of a Channel using an unsafe API.
For example:
Channel ch = ...;
ch.unsafe().outboundBuffer().setUserDefinedWritability(1, false);
Related: #2034
Motivation:
Some users want to mock Bootstrap (or ServerBootstrap), and thus they
should not be final but be fully overridable and extensible.
Modifications:
Remove finals wherever possible
Result:
@daschl is happy.
Related: #2964
Motivation:
Writing a zero-length FileRegion to an NIO channel will lead to an
infinite loop.
Modification:
- Do not write a zero-length FileRegion by protecting with proper 'if'.
- Update the testsuite
Result:
Another bug fixed
Motivation:
When a datagram packet is sent to a destination where nobody actually listens to,
the server O/S will respond with an ICMP Port Unreachable packet.
The ICMP Port Unreachable packet is translated into PortUnreachableException by JDK.
PortUnreachableException is not a harmful exception that prevents a user from sending a datagram.
Therefore, we should not close a datagram channel when PortUnreachableException is caught.
Modifications:
- Do not close a channel when the caught exception is PortUnreachableException.
Result:
A datagram channel is not closed unexpectedly anymore.
Motivation:
JDK's exception messages triggered by a connection attempt failure do
not contain the related remote address in its message. We currently
append the remote address to ConnectException's message, but I found
that we need to cover more exception types such as SocketException.
Modifications:
- Add AbstractUnsafe.annotateConnectException() to de-duplicate the
code that appends the remote address
Result:
- Less duplication
- A transport implementor can annotate connection attempt failure
message more easily
Motiviation:
Before this change, autoRead was a volatile boolean accessed directly. Any thread that invoked the DefaultChannelConfig#setAutoRead(boolean) method would read the current value of autoRead, and then set a new value. If the old value did not match the new value, some action would be immediately taken as part of the same method call.
As volatile only provides happens-before consistency, there was no guarantee that the calling thread was actually the thread mutating the state of the autoRead variable (such that it should be the one to invoke the follow-up actions). For example, with 3 threads:
* Thread 1: get = false
* Thread 1: set = true
* Thread 1: invokes read()
* Thread 2: get = true
* Thread 3: get = true
* Thread 2: set = false
* Thread 2: invokes autoReadCleared()
* Event Loop receives notification from the Selector that data is available, but as autoRead has been cleared, cancels the operation and removes read interest
* Thread 3: set = true
This results in a livelock - autoRead is set true, but no reads will happen even if data is available (as readyOps). The only way around this livelock currently is to set autoRead to false, and then back to true.
Modifications:
Write access to the autoRead variable is now made using the getAndSet() method of an AtomicIntegerFieldUpdater, AUTOREAD_UPDATER. This also changed the type of the underlying autoRead variable to be an integer, as no AtomicBooleanFieldUpdater class exists. Boolean logic is retained by assuming that 1 is true and 0 is false.
Result:
There is no longer a race condition between retrieving the old value of the autoRead variable and setting a new value.
Motivation:
We used the wrong EventExecutor to notify for bind failures if a late registration was done.
Modifications:
Use the correct EventExecutor to notify and only use the GlobelEventExecutor if the registration fails itself.
Result:
The correct Thread will do the notification.
When a ChannelOutboundBuffer contains ByteBufs followed by a FileRegion,
removeBytes() will fail with a ClassCastException. It should break the
loop instead.
f31c630c8c was causing
SocketGatheringWriteTest to fail because it does not take the case where
an empty buffer exists in a gathering write.
When there is an empty buffer in a gathering write, the number of
buffers returned by ChannelOutboundBuffer.nioBuffer() and the actual
number of write attemps can differ.
To remove the write requests correctly, a byte transport must use
ChannelOutboundBuffer.removeBytes()
Motivation:
Because of an incorrect logic in teh EmbeddedChannel constructor it is not possible to use EmbeddedChannel with a ChannelInitializer as constructor argument. This is because it adds the internal LastInboundHandler to its ChannelPipeline before it register itself to the EventLoop.
Modifications:
First register self to EventLoop before add LastInboundHandler to the ChannelPipeline.
Result:
It's now possible to use EmbeddedChannel with ChannelInitializer.
Motivation:
Due a regression NioSocketChannel.doWrite(...) will throw a ClassCastException if you do something like:
channel.write(bytebuf);
channel.write(fileregion);
channel.flush();
Modifications:
Correctly handle writing of different message types by using the correct message count while loop over them.
Result:
No more ClassCastException
Motivation:
The previous fix did disable the caching of ByteBuffers completely which can cause performance regressions. This fix makes sure we use nioBuffers() for all writes in NioSocketChannel and so prevent data-corruptions. This is still kind of a workaround which will be replaced by a more fundamental fix later.
Modifications:
- Revert 4059c9f354
- Use nioBuffers() for all writes to prevent data-corruption
Result:
No more data-corruption but still retain the original speed.
Motivation:
At the moment we expand the ByteBuffer[] when we have more then 1024 ByteBuffer to write and replace the stored instance in its FastThreadLocal. This is not needed and may even harm performance on linux as IOV_MAX is 1024 and so this may cause the JVM to do an array copy.
Modifications:
Just exit the nioBuffers() method if we can not fit more ByteBuffer in the array. This way we will pick them up on the next call.
Result:
Remove uncessary array copy and simplify the code.
Motivation:
We cache the ByteBuffers in ChannelOutboundBuffer.nioBuffers() for the Entries in the ChannelOutboundBuffer to reduce some overhead. The problem is this can lead to data-corruption if an incomplete write happens and next time we try to do a non-gathering write.
To fix this we should remove the caching which does not help a lot anyway and just make the code buggy.
Modifications:
Remove the caching of ByteBuffers.
Result:
No more data-corruption.
Motivation:
Sometimes ChannelHandler need to queue writes to some point and then process these. We currently have no datastructure for this so the user will use an Queue or something like this. The problem is with this Channel.isWritable() will not work as expected and so the user risk to write to fast. That's exactly what happened in our SslHandler. For this purpose we need to add a special datastructure which will also take care of update the Channel and so be sure that Channel.isWritable() works as expected.
Modifications:
- Add PendingWriteQueue which can be used for this purpose
- Make use of PendingWriteQueue in SslHandler
Result:
It is now possible to queue writes in a ChannelHandler and still have Channel.isWritable() working as expected. This also fixes#2752.
Motivation:
While trying to merge our ChannelOutboundBuffer changes we've made last
week, I realized that we have quite a bit of conflicting changes at 4.1
and master. It was primarily because we added
ChannelOutboundBuffer.beforeAdd() and moved some logic there, such as
direct buffer conversion.
However, this is not possible with the changes we've made for 4.0. We
made ChannelOutboundBuffer final for example.
Maintaining multiple branch is already getting painful and having
different core will make it even worse, so I think we should keep the
differences between 4.0 and other branches minimal.
Modifications:
- Move ChannelOutboundBuffer.safeRelease() to ReferenceCountUtil
- Add ByteBufUtil.threadLocalBuffer()
- Backported from ThreadLocalPooledDirectByteBuf
- Make most methods in AbstractUnsafe final
- Add AbstractChannel.filterOutboundMessage() so that a transport can
convert a message to another (e.g. heap -> off-heap), and also
reject unsupported messages
- Move all direct buffer conversions to filterOutboundMessage()
- Move all type checks to filterOutboundMessage()
- Move AbstractChannel.checkEOF() to OioByteStreamChannel, because it's
the only place it is used at all
- Remove ChannelOutboundBuffer.current(Object), because it's not used
anymore
- Add protected direct buffer conversion methods to AbstractNioChannel
and AbstractEpollChannel so that they can be used by their subtypes
- Update all transport implementations according to the changes above
Result:
- The missing extension point in 4.0 has been added.
- AbstractChannel.filterOutboundMessage()
- Thanks to the new extension point, we moved all transport-specific
logic from ChannelOutboundBuffer to each transport implementation
- We can copy most of the transport implementations in 4.0 to 4.1 and
master now, so that we have much less merge conflict when we modify
the core.
Motivation:
We expose ChannelOutboundBuffer in Channel.Unsafe but it is not possible
to create a new ChannelOutboundBuffer without an AbstractChannel. This
makes it impossible to write a Channel implementation that does not
extend AbstractChannel.
Modifications:
- Change ChannelOutboundBuffer to take a Channel as constructor argument.
- Add javadocs
Result:
ChannelOutboundBuffer can be used with a Channel implemention that does
not extend AbstractChannel.
Motivation:
Our ChannelOutboundBuffer implementation is not based on ArrayDeque anymore so we can remove the license notice for it.
Modifications:
Remove license of deque and entry in NOTICE.
Result:
Cleaned up licenses
Motivation:
When a ChannelOutboundBuffer contains a series of entries whose messages
are all empty buffers, EpollSocketChannel sometimes fails to remove
them. As a result, the result of the write(EmptyByteBuf) is never
notified, making the user application hang.
Modifications:
- Add ChannelOutboundBuffer.removeBytes(long) method that updates the
progress of the entries and removes them as much as the specified
number of written bytes. It also updates the reader index of
partially flushed buffer.
- Make both NioSocketChannel and EpollSocketChannel use it to reduce
code duplication
- Replace EpollSocketChannel.updateOutboundBuffer()
- Refactor EpollSocketChannel.doWrite() for simplicity
- Split doWrite() into doWriteSingle() and doWriteMultiple()
- Do not add a zero-length buffer to IovArray
- Do not perform any real I/O when the size of IovArray is 0
Result:
Another regression is gone.
Related issue: #2717, #2710, #2704, #2693
Motivation:
When ChannelOutboundBuffer.nioBuffers() iterates over the linked list of
entries, it is not supposed to visit unflushed entries, but it does.
Modifications:
- Make sure ChannelOutboundBuffer.nioBuffers() stops the iteration before
it visits an unflushed entry
- Add isFlushedEntry() to reduce the chance of the similar mistakes
Result:
Another regression is gone.
Motivation:
ChannelOutboundBuffer.forEachFlushedMessage() visits even an unflushed
messages.
Modifications:
Stop the loop if the currently visiting entry is unflushedEntry.
Result:
forEachFlushedMessage() behaves correctly.
- ChannelOutboundBuffer.Entry.buffers -> bufs for consistency
- Make Native.IOV_MAX final because it's a constant
- Naming changes
- FlushedMessageProcessor -> MessageProcessor just in case we can
reuse it for unflushed messages in the future
- Add ChannelOutboundBuffer.Entry.recycle() that does not return the
next entry, and use it wherever possible
- Javadoc clean-up
Motivation:
While benchmarking the native transport, I noticed that gathering write
is not as fast as expected. It was due to the fact that we have to do a
lot of array copies to put the buffer addresses into the iovec struct
array.
Modifications:
Introduce a new class called IovArray, which allows to fill buffers
directly into an off-heap array of iovec structs, so that it can be
passed over to JNI without any extra array copies.
Result:
Big performance improvement when doing gathering writes:
Before:
[nmaurer@xxx]~% wrk/wrk -H 'Host: localhost' -H 'Accept: text/html,application/xhtml+xml,application/xml;q=0.9,*/*;q=0.8' -H 'Connection: keep-alive' -d 120 -c 256 -t 16 --pipeline 256 http://xxx:8080/plaintext
Running 2m test @ http://xxx:8080/plaintext
16 threads and 256 connections
Thread Stats Avg Stdev Max +/- Stdev
Latency 23.44ms 16.37ms 259.57ms 91.77%
Req/Sec 181.99k 31.69k 304.60k 78.12%
346544071 requests in 2.00m, 46.48GB read
Requests/sec: 2887885.09
Transfer/sec: 396.59MB
After:
[nmaurer@xxx]~% wrk/wrk -H 'Host: localhost' -H 'Accept: text/html,application/xhtml+xml,application/xml;q=0.9,*/*;q=0.8' -H 'Connection: keep-alive' -d 120 -c 256 -t 16 --pipeline 256 http://xxx:8080/plaintext
Running 2m test @ http://xxx:8080/plaintext
16 threads and 256 connections
Thread Stats Avg Stdev Max +/- Stdev
Latency 21.93ms 16.33ms 305.73ms 92.34%
Req/Sec 194.56k 33.75k 309.33k 77.04%
369617503 requests in 2.00m, 49.57GB read
Requests/sec: 3080169.65
Transfer/sec: 423.00MB
Motivation:
73dfd7c01b introduced various test
failures because:
- EpollSocketChannel.doWrite() raised a NullPointerException when
notifying the write progress.
- ChannelOutboundBuffer.nioBuffers() did not expand the internal array
when the pending entries contained more than 1024 buffers, dropping
the remainder.
Modifications:
- Fix the NPE in EpollSocketChannel by removing an unnecessary progress
update
- Expand the thread-local buffer array if there is not enough room,
which was the original behavior dropped by the offending commit
Result:
Regression is gone.
Motiviation:
ChannelOuboundBuffer uses often too much memory. This is especially a problem if you want to serve a lot of connections. This is due the fact that it uses 2 arrays internally. One if used as a circular buffer and store the Entries that are never released (ChannelOutboundBuffer is pooled) and one is used to hold the ByteBuffers that are used for gathering writes.
Modifications:
Rewrite ChannelOutboundBuffer to remove these two arrays by:
- Make Entry recyclable and use it as linked Node
- Remove the circular buffer which was used for the Entries as we use a Linked-List like structure now
- Remove the array that did hold the ByteBuffers and replace it by an ByteBuffer array that is hold by a FastThreadLocal. We use a fixed capacity of 1024 here which is fine as we share these anyway.
- ChannelOuboundBuffer is not recyclable anymore as it is now a "light-weight" object. We recycle the internally used Entries instead.
Result:
Less memory footprint and resource usage. Performance seems to be a bit better but most likely as we not need to expand any arrays anymore.
Benchmark before change:
[nmaurer@xxx]~% wrk/wrk -H 'Host: localhost' -H 'Accept: text/html,application/xhtml+xml,application/xml;q=0.9,*/*;q=0.8' -H 'Connection: keep-alive' -d 120 -c 256 -t 16 --pipeline 256 http://xxx:8080/plaintext
Running 2m test @ http://xxx:8080/plaintext
16 threads and 256 connections
Thread Stats Avg Stdev Max +/- Stdev
Latency 26.88ms 67.47ms 1.26s 97.97%
Req/Sec 191.81k 28.22k 255.63k 83.86%
364806639 requests in 2.00m, 48.92GB read
Requests/sec: 3040101.23
Transfer/sec: 417.49MB
Benchmark after change:
[nmaurer@xxx]~% wrk/wrk -H 'Host: localhost' -H 'Accept: text/html,application/xhtml+xml,application/xml;q=0.9,*/*;q=0.8' -H 'Connection: keep-alive' -d 120 -c 256 -t 16 --pipeline 256 http://xxx:8080/plaintext
Running 2m test @ http://xxx:8080/plaintext
16 threads and 256 connections
Thread Stats Avg Stdev Max +/- Stdev
Latency 22.22ms 17.22ms 301.77ms 90.13%
Req/Sec 194.98k 41.98k 328.38k 70.50%
371816023 requests in 2.00m, 49.86GB read
Requests/sec: 3098461.44
Transfer/sec: 425.51MB
Motivation:
We have some inconsistency when handling writes. Sometimes we call ChannelOutboundBuffer.progress(...) also for complete writes and sometimes not. We should call it always.
Modifications:
Correctly call ChannelOuboundBuffer.progress(...) for complete and incomplete writes.
Result:
Consistent behavior
Motivation:
ChannelOutboundBuffer is basically a circular array queue of its entry
objects. Once an entry is created in the array, it is never nulled out
to reduce the allocation cost.
However, because it is a circular queue, the array almost always ends up
with as many entry instances as the size of the array, regardless of the
number of pending writes.
At worst case, a channel might have only 1 pending writes at maximum
while creating 32 entry objects, where 32 is the initial capacity of the
array.
Modifications:
- Reduce the initial capacity of the circular array queue to 4.
- Make the initial capacity of the circular array queue configurable
Result:
We spend 4 times less memory for entry objects under certain
circumstances.
Motivation:
At the moment ChannelOutboundBuffer.nioBuffers() returns null if something is contained in the ChannelOutboundBuffer which is not a ByteBuf. This is a problem for two reasons:
1 - In the javadocs we state that it will never return null
2 - We may do a not optimal write as there may be things that could be written via gathering writes
Modifications:
Change ChannelOutboundBuffer.nioBuffers() to never return null but have it contain all ByteBuffer that were found before the non ByteBuf. This way we can do a gathering write and also conform to the javadocs.
Result:
Better speed and also correct implementation in terms of the api.
Motivation:
Now Netty has a few problems with null values.
Modifications:
- Check File in DiskFileUpload.toString().
If File is null we will get NPE when calling toString() method.
- Check Result<String> in MqttDecoder.decodeConnectionPayload(...).
- Check Unsafe before calling unsafe.getClass() in PlatformDependent0 static block.
- Removed unnecessary null check in WebSocket08FrameEncoder.encode(...).
Because msg.content() can not return null.
- Removed unnecessary null checks in ConcurrentHashMapV8.removeTreeNode(TreeNode<K,V>).
- Removed unnecessary null check in OioDatagramChannel.doReadMessages(List<Object>).
Because tmpPacket.getSocketAddress() always returns new SocketAddress instance.
- Removed unnecessary null check in OioServerSocketChannel.doReadMessages(List<Object>).
Because socket.accept() always returns new Socket instance.
- Pass Unpooled.buffer(0) instead of null inside CloseWebSocketFrame(boolean, int) constructor.
If we will pass null we will get NPE in super class constructor.
- Added throw new IllegalStateException in GlobalEventExecutor.awaitInactivity(long, TimeUnit) if it will be called before GlobalEventExecutor.execute(Runnable).
Because now we will get NPE. IllegalStateException will be better in this case.
- Fixed null check in OpenSslServerContext.setTicketKeys(byte[]).
Now we throw new NPE if byte[] is not null.
Result:
Added new null checks when it is necessary, removed unnecessary null checks and fixed some NPE problems.
Motivation:
Fix some typos in Netty.
Modifications:
- Fix potentially dangerous use of non-short-circuit logic in Recycler.transfer(Stack<?>).
- Removed double 'the the' in javadoc of EmbeddedChannel.
- Write to log an exception message if we can not get SOMAXCONN in the NetUtil's static block.
Motivation:
As a DatagramChannel supports to write to multiple remote peers we must not close the Channel once a IOException accours as this error may be only valid for one remote peer.
Modification:
Continue writing on IOException.
Result:
DatagramChannel can be used even after an IOException accours during writing.
Motivation:
Because of a missing return statement we may produce a NPE when try to fullfill the connect ChannelPromise when it was fullfilled before.
Modification:
Add missing return statement.
Result:
No more NPE.
Motivation:
When an exception is thrown during try to send DatagramPacket or SctpMessage a buffer may leak.
Modification:
Correctly handle allocated buffers in case of exception
Result:
No more leaks
Motivation:
When a bind fails AbstractBootstrap will use the GlobalEventExecutor to notify the ChannelPromise. We should use the EventLoop of the Channel if possible.
Modification:
Use EventLoop of the Channel if possible to use the correct Thread to notify and so guaranteer the right order of events.
Result:
Use the correct EventLoop for notification
Motivation:
We use the nanoTime of the scheduledTasks to calculate the milli-seconds to wait for a select operation to select something. Once these elapsed we check if there was something selected or some task is ready for processing. Unfortunally we not take into account scheduled tasks here so the selection loop will continue if only scheduled tasks are ready for processing. This will delay the execution of these tasks.
Modification:
- Check if a scheduled task is ready after selecting
- also make a tiny change in NioEventLoop to not trigger a rebuild if nothing was selected because the timeout was reached a few times in a row.
Result:
Execute scheduled tasks on time.
Motivation:
When a select rebuild was triggered the reference to the SelectionKey is not updated in AbstractNioChannel. This will cause a CancelledKeyException later.
Modification:
Correctly update SelectionKey reference after rebuild
Result:
Fix exception
Motivation:
Recycler is used in many places to reduce GC-pressure but is still not as fast as possible because of the internal datastructures used.
Modification:
- Rewrite Recycler to use a WeakOrderQueue which makes minimal guaranteer about order and visibility for max performance.
- Recycling of the same object multiple times without acquire it will fail.
- Introduce a RecyclableMpscLinkedQueueNode which can be used for MpscLinkedQueueNodes that use Recycler
These changes are based on @belliottsmith 's work that was part of #2504.
Result:
Huge increase in performance.
4.0 branch without this commit:
Benchmark (size) Mode Samples Score Score error Units
i.n.m.i.RecyclableArrayListBenchmark.recycleSameThread 00000 thrpt 20 116026994.130 2763381.305 ops/s
i.n.m.i.RecyclableArrayListBenchmark.recycleSameThread 00256 thrpt 20 110823170.627 3007221.464 ops/s
i.n.m.i.RecyclableArrayListBenchmark.recycleSameThread 01024 thrpt 20 118290272.413 7143962.304 ops/s
i.n.m.i.RecyclableArrayListBenchmark.recycleSameThread 04096 thrpt 20 120560396.523 6483323.228 ops/s
i.n.m.i.RecyclableArrayListBenchmark.recycleSameThread 16384 thrpt 20 114726607.428 2960013.108 ops/s
i.n.m.i.RecyclableArrayListBenchmark.recycleSameThread 65536 thrpt 20 119385917.899 3172913.684 ops/s
Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 297.617 sec - in io.netty.microbench.internal.RecyclableArrayListBenchmark
4.0 branch with this commit:
Benchmark (size) Mode Samples Score Score error Units
i.n.m.i.RecyclableArrayListBenchmark.recycleSameThread 00000 thrpt 20 204158855.315 5031432.145 ops/s
i.n.m.i.RecyclableArrayListBenchmark.recycleSameThread 00256 thrpt 20 205179685.861 1934137.841 ops/s
i.n.m.i.RecyclableArrayListBenchmark.recycleSameThread 01024 thrpt 20 209906801.437 8007811.254 ops/s
i.n.m.i.RecyclableArrayListBenchmark.recycleSameThread 04096 thrpt 20 214288320.053 6413126.689 ops/s
i.n.m.i.RecyclableArrayListBenchmark.recycleSameThread 16384 thrpt 20 215940902.649 7837706.133 ops/s
i.n.m.i.RecyclableArrayListBenchmark.recycleSameThread 65536 thrpt 20 211141994.206 5017868.542 ops/s
Tests run: 1, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 297.648 sec - in io.netty.microbench.internal.RecyclableArrayListBenchmark
Motivation:
LocalServerChannel.doClose() calls LocalChannelRegistry.unregister(localAddress); without check if localAddress is null and so produce a NPE when pass null the used ConcurrentHashMapV8
Modification:
Check for localAddress != null before try to remove it from Map. Also added a unit test which showed the stacktrace of the error.
Result:
No more NPE during doClose().
Motivation:
At the moment AbstractBoostrap.bind(...) will always use the GlobalEventExecutor to notify the returned ChannelFuture if the registration is not done yet. This should only be done if the registration fails later. If it completes successful we should just notify with the EventLoop of the Channel.
Modification:
Use EventLoop of the Channel if possible to use the correct Thread to notify and so guaranteer the right order of events.
Result:
Use the correct EventLoop for notification
Motivation:
When Netty runs in a managed environment such as web application server,
Netty needs to provide an explicit way to remove the thread-local
variables it created to prevent class loader leaks.
FastThreadLocal uses different execution paths for storing a
thread-local variable depending on the type of the current thread.
It increases the complexity of thread-local removal.
Modifications:
- Moved FastThreadLocal and FastThreadLocalThread out of the internal
package so that a user can use it.
- FastThreadLocal now keeps track of all thread local variables it has
initialized, and calling FastThreadLocal.removeAll() will remove all
thread-local variables of the caller thread.
- Added FastThreadLocal.size() for diagnostics and tests
- Introduce InternalThreadLocalMap which is a mixture of hard-wired
thread local variable fields and extensible indexed variables
- FastThreadLocal now uses InternalThreadLocalMap to implement a
thread-local variable.
- Added ThreadDeathWatcher.unwatch() so that PooledByteBufAllocator
tells it to stop watching when its thread-local cache has been freed
by FastThreadLocal.removeAll().
- Added FastThreadLocalTest to ensure that removeAll() works
- Added microbenchmark for FastThreadLocal and JDK ThreadLocal
- Upgraded to JMH 0.9
Result:
- A user can remove all thread-local variables Netty created, as long as
he or she did not exit from the current thread. (Note that there's no
way to remove a thread-local variable from outside of the thread.)
- FastThreadLocal exposes more useful operations such as isSet() because
we always implement a thread local variable via InternalThreadLocalMap
instead of falling back to JDK ThreadLocal.
- FastThreadLocalBenchmark shows that this change improves the
performance of FastThreadLocal even more.
Motivation:
The code in ChannelOutboundBuffer can be simplified by using AtomicLongFieldUpdater.addAndGet(...)
Modification:
Replace our manual looping with AtomicLongFieldUpdater.addAndGet(...)
Result:
Cleaner code
Motivation:
If ChannelOutboundBuffer.addFlush() is called multiple times and flushed != unflushed it will still loop through all entries that are not flushed yet even if it is not needed anymore as these were marked uncancellable before.
Modifications:
Check if new messages were added since addFlush() was called and only if this was the case loop through all entries and try to mark the uncancellable.
Result:
Less overhead when ChannelOuboundBuffer.addFlush() is called multiple times without new messages been added.
Motivation:
Allow to make use of our new FastThreadLocal whereever possible
Modification:
Make use of an array to store FastThreadLocals and so allow to also use it in PooledByteBufAllocator that is instanced by users.
The maximal size of the array is configurable per system property to allow to tune it if needed. As default we use 64 entries which should be good enough.
Result:
More flexible usage of FastThreadLocal
Motivation:
Provide a faster ThreadLocal implementation
Modification:
Add a "FastThreadLocal" which uses an EnumMap and a predefined fixed set of possible thread locals (all of the static instances created by netty) that is around 10-20% faster than standard ThreadLocal in my benchmarks (and can be seen having an effect in the direct PooledByteBufAllocator benchmark that uses the DEFAULT ByteBufAllocator which uses this FastThreadLocal, as opposed to normal instantiations that do not, and in the new RecyclableArrayList benchmark);
Result:
Improved performance
Motivation:
Each of DefaultChannelPipeline instance creates an head and tail that wraps a handler. These are used to chain together other DefaultChannelHandlerContext that are created once a new ChannelHandler is added. There are a few things here that can be improved in terms of memory usage and initialization time.
Modification:
- Only generate the name for the tail and head one time as it will never change anyway
- Rename DefaultChannelHandlerContext to AbstractChannelHandlerContext and make it abstract
- Create a new DefaultChannelHandlerContext that is used when a ChannelHandler is added to the DefaultChannelPipeline
- Rename TailHandler to TailContext and HeadHandler to HeadContext and let them extend AbstractChannelHandlerContext. This way we can save 2 object creations per DefaultChannelPipeline
Result:
- Less memory usage because we have 2 less objects per DefaultChannelPipeline
- Faster creation of DefaultChannelPipeline as we not need to generate the name for the head and tail
Motivation:
At the moment ChannelFlushPromiseNotifier.add(....) takes an int value for pendingDataSize, which may be too small as a user may need to use a long. This can for example be useful when a user writes a FileRegion etc. Beside this the notify* method names are kind of missleading as these should not contain *Future* because it is about ChannelPromises.
Modification:
Add a new add(...) method that takes a long for pendingDataSize and @deprecated the old method. Beside this also @deprecated all *Future* methods and add methods that have *Promise* in the method name to better reflect usage.
Result:
ChannelFlushPromiseNotifier can be used with bigger data.
Motivation:
On some ill-configured systems, InetAddress.getLocalHost() fails. NioSocketChannelTest calls java.net.Socket.connect() and it internally invoked InetAddress.getLocalHost(), which causes the test failures in NioSocketChannelTes on such an ill-configured system.
Modifications:
Use NetUtil.LOCALHOST explicitly.
Result:
NioSocketChannelTest should not fail anymore.
Motivation:
At the moment we sometimes use only RecvByteBufAllocator.guess() to guess the next size and the use the ByteBufAllocator.* directly to allocate the buffer. We should always use RecvByteBufAllocator.allocate(...) all the time as this makes the behavior easier to adjust.
Modifications:
Change the read() implementations to make use of RecvByteBufAllocator.
Result:
Behavior is more consistent.
Motivation:
DefaultChannelPipeline.firstContext() should return null when the ipeline is empty. This is not the case atm.
Modification:
Fix incorrect check in DefaultChannelPipeline.firstContext() and add unit tests.
Result:
Correctly return null when DefaultChannelPipeline.firstContext() is called on empty pipeline.
Motivation:
Because Thread.currentThread().interrupt() will unblock Selector.select() we need to take special care when check if we need to rebuild the Selector. If the unblock was caused by the interrupt() we will clear it and move on as this is most likely a bug in a custom ChannelHandler or a library the user makes use of.
Modification:
Clear the interrupt state of the Thread if the Selector was unblock because of an interrupt and the returned keys was 0.
Result:
No more busy loop caused by Thread.currentThread().interrupt()
Motivation:
As discussed in #2250, it will become much less complicated to implement
deregistration and reregistration of a channel once #2250 is resolved.
Therefore, there's no need to deprecate deregister() and
channelUnregistered().
Modification:
- Undeprecate deregister() and channelUnregistered()
- Remove SuppressWarnings annotations where applicable
Result:
We (including @jakobbuchgraber) are now ready to play with #2250 at
master
Motivation:
I had the NioSocketChannelTest.testFlushCloseReentrance() fail sometimes on one of my linux installation. This change let it pass all the time.
Modification:
Set the SO_SNDBUF to a small value to force split writes
Result:
Test is passing all the time where it was sometimes fail before.