netty5/transport-native-kqueue/src/main/java/io/netty/channel/kqueue/AbstractKQueueChannel.java

731 lines
29 KiB
Java
Raw Normal View History

/*
* Copyright 2016 The Netty Project
*
* The Netty Project licenses this file to you under the Apache License,
* version 2.0 (the "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at:
*
* https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations
* under the License.
*/
package io.netty.channel.kqueue;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import io.netty.buffer.ByteBufUtil;
import io.netty.buffer.Unpooled;
import io.netty.channel.AbstractChannel;
import io.netty.channel.Channel;
import io.netty.channel.ChannelConfig;
import io.netty.channel.ChannelException;
import io.netty.channel.ChannelMetadata;
2017-12-08 01:00:52 +01:00
import io.netty.channel.ChannelOutboundBuffer;
import io.netty.channel.ConnectTimeoutException;
import io.netty.channel.EventLoop;
import io.netty.channel.RecvByteBufAllocator;
import io.netty.channel.socket.ChannelInputShutdownEvent;
import io.netty.channel.socket.ChannelInputShutdownReadComplete;
import io.netty.channel.socket.SocketChannelConfig;
import io.netty.channel.unix.FileDescriptor;
import io.netty.channel.unix.UnixChannel;
import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.Future;
Clean up Future/Promises API (#11575) Motivation: The generics for the existing futures, promises, and listeners are too complicated. This complication comes from the existence of `ChannelPromise` and `ChannelFuture`, which forces listeners to care about the particular _type_ of future being listened on. Modification: * Add a `FutureContextListener` which can take a context object as an additional argument. This allows our listeners to have the channel piped through to them, so they don't need to rely on the `ChannelFuture.channel()` method. * Make the `FutureListener`, along with the `FutureContextListener` sibling, the default listener API, retiring the `GenericFutureListener` since we no longer need to abstract over the type of the future. * Change all uses of `ChannelPromise` to `Promise<Void>`. * Change all uses of `ChannelFuture` to `Future<Void>`. * Change all uses of `GenericFutureListener` to either `FutureListener` or `FutureContextListener` as needed. * Remove `ChannelFutureListener` and `GenericFutureListener`. * Introduce a `ChannelFutureListeners` enum to house the constants that previously lived in `ChannelFutureListener`. These constants now implement `FutureContextListener` and take the `Channel` as a context. * Remove `ChannelPromise` and `ChannelFuture` — all usages now rely on the plain `Future` and `Promise` APIs. * Add static factory methods to `DefaultPromise` that allow us to create promises that are initialised as successful or failed. * Remove `CompleteFuture`, `SucceededFuture`, `FailedFuture`, `CompleteChannelFuture`, `SucceededChannelFuture`, and `FailedChannelFuture`. * Remove `ChannelPromiseNotifier`. Result: Cleaner generics and more straight forward code.
2021-08-20 09:55:16 +02:00
import io.netty.util.concurrent.Promise;
import java.io.IOException;
KQueue write filter initial state (#7738) Motivation: KQueue implementations current have inconsistent behavior with Epoll implementations with respect to asynchronous sockets and connecting. In the Epoll transport we attempt to connect, if the connect call does not synchornously fail/succeed we set the EPOLLOUT which will be triggered by the kernel if the connection attempt succeeds or an error occurs. The connect API provides no way to asynchronously communicate an error so the Epoll implementation fires a EPOLLOUT event and puts the connect status in getsockopt(SO_ERROR). KQueue provides the same APIs but different behavior. If the EVFILT_WRITE is not enabled and the EVFILT_READ is enabled before connect is called, and there is an error the kernel may fire the EVFILT_READ filter and provide the Connection Refused error via read(). This is even true if we set the EVFILT_WRITE filter after calling connect because connect didn't synchornously complete. After the error has been delievered via read() a call to getsockopt(SO_ERROR) will return 0 indicating there is no error. This means we cannot rely upon the KQueue based kernel to deliver connection errors via the EVFILT_WRITE filter in the same way that the linux kernel does with the EPOLLOUT flag. ce241bd introduced a change which depends upon the behavior of the EVFILT_WRITE being set and may prematurely stop writing to the OS as a result, becaues we assume the OS will notify us when the socket is writable. However the current work around for the above described behavior is to initialize the EVFILT_WRITE to true for connection oriented protocols. This leads to prematurely exiting from the flush() which may lead to deadlock. Modifications: - KQueue should check when an error is obtained from read() if the connectPromise has not yet been completed, and if not complete it with a ConnectException Result: No more deadlock in KQueue due to asynchronous connect workaround.
2018-02-20 20:01:49 +01:00
import java.net.ConnectException;
import java.net.InetSocketAddress;
import java.net.SocketAddress;
import java.nio.ByteBuffer;
import java.nio.channels.AlreadyConnectedException;
import java.nio.channels.ConnectionPendingException;
import java.nio.channels.NotYetConnectedException;
import java.nio.channels.UnresolvedAddressException;
import java.util.concurrent.TimeUnit;
2017-12-08 01:00:52 +01:00
import static io.netty.channel.internal.ChannelUtils.WRITE_STATUS_SNDBUF_FULL;
import static io.netty.channel.unix.UnixChannelUtil.computeRemoteAddr;
import static java.util.Objects.requireNonNull;
abstract class AbstractKQueueChannel extends AbstractChannel implements UnixChannel {
private static final ChannelMetadata METADATA = new ChannelMetadata(false);
/**
* The future of the current connection attempt. If not null, subsequent
* connection attempts will fail.
*/
Clean up Future/Promises API (#11575) Motivation: The generics for the existing futures, promises, and listeners are too complicated. This complication comes from the existence of `ChannelPromise` and `ChannelFuture`, which forces listeners to care about the particular _type_ of future being listened on. Modification: * Add a `FutureContextListener` which can take a context object as an additional argument. This allows our listeners to have the channel piped through to them, so they don't need to rely on the `ChannelFuture.channel()` method. * Make the `FutureListener`, along with the `FutureContextListener` sibling, the default listener API, retiring the `GenericFutureListener` since we no longer need to abstract over the type of the future. * Change all uses of `ChannelPromise` to `Promise<Void>`. * Change all uses of `ChannelFuture` to `Future<Void>`. * Change all uses of `GenericFutureListener` to either `FutureListener` or `FutureContextListener` as needed. * Remove `ChannelFutureListener` and `GenericFutureListener`. * Introduce a `ChannelFutureListeners` enum to house the constants that previously lived in `ChannelFutureListener`. These constants now implement `FutureContextListener` and take the `Channel` as a context. * Remove `ChannelPromise` and `ChannelFuture` — all usages now rely on the plain `Future` and `Promise` APIs. * Add static factory methods to `DefaultPromise` that allow us to create promises that are initialised as successful or failed. * Remove `CompleteFuture`, `SucceededFuture`, `FailedFuture`, `CompleteChannelFuture`, `SucceededChannelFuture`, and `FailedChannelFuture`. * Remove `ChannelPromiseNotifier`. Result: Cleaner generics and more straight forward code.
2021-08-20 09:55:16 +02:00
private Promise<Void> connectPromise;
private Future<?> connectTimeoutFuture;
private SocketAddress requestedRemoteAddress;
private KQueueRegistration registration;
final BsdSocket socket;
private boolean readFilterEnabled;
private boolean writeFilterEnabled;
boolean readReadyRunnablePending;
boolean inputClosedSeenErrorOnRead;
protected volatile boolean active;
private volatile SocketAddress local;
private volatile SocketAddress remote;
Tighten contract between Channel and EventLoop by require the EventLoop on Channel construction. (#8587) Motivation: At the moment it’s possible to have a Channel in Netty that is not registered / assigned to an EventLoop until register(...) is called. This is suboptimal as if the Channel is not registered it is also not possible to do anything useful with a ChannelFuture that belongs to the Channel. We should think about if we should have the EventLoop as a constructor argument of a Channel and have the register / deregister method only have the effect of add a Channel to KQueue/Epoll/... It is also currently possible to deregister a Channel from one EventLoop and register it with another EventLoop. This operation defeats the threading model assumptions that are wide spread in Netty, and requires careful user level coordination to pull off without any concurrency issues. It is not a commonly used feature in practice, may be better handled by other means (e.g. client side load balancing), and therefore we propose removing this feature. Modifications: - Change all Channel implementations to require an EventLoop for construction ( + an EventLoopGroup for all ServerChannel implementations) - Remove all register(...) methods from EventLoopGroup - Add ChannelOutboundInvoker.register(...) which now basically means we want to register on the EventLoop for IO. - Change ChannelUnsafe.register(...) to not take an EventLoop as parameter (as the EventLoop is supplied on custruction). - Change ChannelFactory to take an EventLoop to create new Channels and introduce ServerChannelFactory which takes an EventLoop and one EventLoopGroup to create new ServerChannel instances. - Add ServerChannel.childEventLoopGroup() - Ensure all operations on the accepted Channel is done in the EventLoop of the Channel in ServerBootstrap - Change unit tests for new behaviour Result: A Channel always has an EventLoop assigned which will never change during its life-time. This ensures we are always be able to call any operation on the Channel once constructed (unit the EventLoop is shutdown). This also simplifies the logic in DefaultChannelPipeline a lot as we can always call handlerAdded / handlerRemoved directly without the need to wait for register() to happen. Also note that its still possible to deregister a Channel and register it again. It's just not possible anymore to move from one EventLoop to another (which was not really safe anyway). Fixes https://github.com/netty/netty/issues/8513.
2019-01-14 20:11:13 +01:00
AbstractKQueueChannel(Channel parent, EventLoop eventLoop, BsdSocket fd, boolean active) {
super(parent, eventLoop);
socket = requireNonNull(fd, "fd");
this.active = active;
if (active) {
// Directly cache the remote and local addresses
// See https://github.com/netty/netty/issues/2359
local = fd.localAddress();
remote = fd.remoteAddress();
}
}
Tighten contract between Channel and EventLoop by require the EventLoop on Channel construction. (#8587) Motivation: At the moment it’s possible to have a Channel in Netty that is not registered / assigned to an EventLoop until register(...) is called. This is suboptimal as if the Channel is not registered it is also not possible to do anything useful with a ChannelFuture that belongs to the Channel. We should think about if we should have the EventLoop as a constructor argument of a Channel and have the register / deregister method only have the effect of add a Channel to KQueue/Epoll/... It is also currently possible to deregister a Channel from one EventLoop and register it with another EventLoop. This operation defeats the threading model assumptions that are wide spread in Netty, and requires careful user level coordination to pull off without any concurrency issues. It is not a commonly used feature in practice, may be better handled by other means (e.g. client side load balancing), and therefore we propose removing this feature. Modifications: - Change all Channel implementations to require an EventLoop for construction ( + an EventLoopGroup for all ServerChannel implementations) - Remove all register(...) methods from EventLoopGroup - Add ChannelOutboundInvoker.register(...) which now basically means we want to register on the EventLoop for IO. - Change ChannelUnsafe.register(...) to not take an EventLoop as parameter (as the EventLoop is supplied on custruction). - Change ChannelFactory to take an EventLoop to create new Channels and introduce ServerChannelFactory which takes an EventLoop and one EventLoopGroup to create new ServerChannel instances. - Add ServerChannel.childEventLoopGroup() - Ensure all operations on the accepted Channel is done in the EventLoop of the Channel in ServerBootstrap - Change unit tests for new behaviour Result: A Channel always has an EventLoop assigned which will never change during its life-time. This ensures we are always be able to call any operation on the Channel once constructed (unit the EventLoop is shutdown). This also simplifies the logic in DefaultChannelPipeline a lot as we can always call handlerAdded / handlerRemoved directly without the need to wait for register() to happen. Also note that its still possible to deregister a Channel and register it again. It's just not possible anymore to move from one EventLoop to another (which was not really safe anyway). Fixes https://github.com/netty/netty/issues/8513.
2019-01-14 20:11:13 +01:00
AbstractKQueueChannel(Channel parent, EventLoop eventLoop, BsdSocket fd, SocketAddress remote) {
super(parent, eventLoop);
socket = requireNonNull(fd, "fd");
active = true;
// Directly cache the remote and local addresses
// See https://github.com/netty/netty/issues/2359
this.remote = remote;
local = fd.localAddress();
}
static boolean isSoErrorZero(BsdSocket fd) {
try {
return fd.getSoError() == 0;
} catch (IOException e) {
throw new ChannelException(e);
}
}
protected KQueueRegistration registration() {
assert registration != null;
return registration;
}
@Override
public final FileDescriptor fd() {
return socket;
}
@Override
public boolean isActive() {
return active;
}
@Override
public ChannelMetadata metadata() {
return METADATA;
}
@Override
protected void doClose() throws Exception {
active = false;
// Even if we allow half closed sockets we should give up on reading. Otherwise we may allow a read attempt on a
// socket which has not even been connected yet. This has been observed to block during unit tests.
inputClosedSeenErrorOnRead = true;
socket.close();
}
@Override
protected void doDisconnect() throws Exception {
doClose();
}
void resetCachedAddresses() {
local = socket.localAddress();
remote = socket.remoteAddress();
}
@Override
public boolean isOpen() {
return socket.isOpen();
}
@Override
protected final void doBeginRead() throws Exception {
// Channel.read() or ChannelHandlerContext.read() was called
final AbstractKQueueUnsafe unsafe = (AbstractKQueueUnsafe) unsafe();
unsafe.readPending = true;
// We must set the read flag here as it is possible the user didn't read in the last read loop, the
// executeReadReadyRunnable could read nothing, and if the user doesn't explicitly call read they will
// never get data after this.
readFilter(true);
// If auto read was toggled off on the last read loop then we may not be notified
// again if we didn't consume all the data. So we force a read operation here if there maybe more data.
if (unsafe.maybeMoreDataToRead) {
unsafe.executeReadReadyRunnable(config());
}
}
void register0(KQueueRegistration registration) {
this.registration = registration;
// Just in case the previous EventLoop was shutdown abruptly, or an event is still pending on the old EventLoop
// make sure the readReadyRunnablePending variable is reset so we will be able to execute the Runnable on the
// new EventLoop.
readReadyRunnablePending = false;
// Add the write event first so we get notified of connection refused on the client side!
if (writeFilterEnabled) {
evSet0(registration, Native.EVFILT_WRITE, Native.EV_ADD_CLEAR_ENABLE);
}
if (readFilterEnabled) {
evSet0(registration, Native.EVFILT_READ, Native.EV_ADD_CLEAR_ENABLE);
}
evSet0(registration, Native.EVFILT_SOCK, Native.EV_ADD, Native.NOTE_RDHUP);
}
KQueueEventLoop | EpollEventLoop may incorrectly update registration when FD is reused. Motivation: The current KQueueEventLoop implementation does not process concurrent domain socket channel registration/unregistration in the order they actual happen since unregistration are delated by an event loop task scheduling. When a domain socket is closed, it's file descriptor might be reused quickly and therefore trigger a new channel registration using the same descriptor. Consequently the KQueueEventLoop#add(AbstractKQueueChannel) method will overwrite the current inactive channels having the same descriptor and the delayed KQueueEventLoop#remove(AbstractKQueueChannel) will remove the active channel that replaced the inactive one. As active channels are registered, events for this file descriptor won't be processed anymore and the channels will never be closed. The same problem can also happen in EpollEventLoop. Beside this we also may never remove the AbstractEpollChannel from the internal map when it is unregistered which will prevent it from be GC'ed Modifications: - Change logic of native KQueue and Epoll implementations to ensure we correctly handle the case of FD reuse - Only try to update kevent / epoll if the Channel is still open (as otherwise it will be handled by kqueue / epoll itself) - Correctly remove AbstractEpollChannel from internal map in all cases - Make implementation of closeAll() consistent for Epoll and KQueueEventLoop Result: KQueue and Epoll native transports correctly handle FD reuse Co-authored-by: Norman Maurer <norman_maurer@apple.com>
2019-05-01 11:24:33 +02:00
void deregister0() {
// As unregisteredFilters() may have not been called because isOpen() returned false we just set both filters
// to false to ensure a consistent state in all cases.
readFilterEnabled = false;
writeFilterEnabled = false;
}
void unregisterFilters() throws Exception {
// Make sure we unregister our filters from kqueue!
readFilter(false);
writeFilter(false);
KQueueEventLoop | EpollEventLoop may incorrectly update registration when FD is reused. Motivation: The current KQueueEventLoop implementation does not process concurrent domain socket channel registration/unregistration in the order they actual happen since unregistration are delated by an event loop task scheduling. When a domain socket is closed, it's file descriptor might be reused quickly and therefore trigger a new channel registration using the same descriptor. Consequently the KQueueEventLoop#add(AbstractKQueueChannel) method will overwrite the current inactive channels having the same descriptor and the delayed KQueueEventLoop#remove(AbstractKQueueChannel) will remove the active channel that replaced the inactive one. As active channels are registered, events for this file descriptor won't be processed anymore and the channels will never be closed. The same problem can also happen in EpollEventLoop. Beside this we also may never remove the AbstractEpollChannel from the internal map when it is unregistered which will prevent it from be GC'ed Modifications: - Change logic of native KQueue and Epoll implementations to ensure we correctly handle the case of FD reuse - Only try to update kevent / epoll if the Channel is still open (as otherwise it will be handled by kqueue / epoll itself) - Correctly remove AbstractEpollChannel from internal map in all cases - Make implementation of closeAll() consistent for Epoll and KQueueEventLoop Result: KQueue and Epoll native transports correctly handle FD reuse Co-authored-by: Norman Maurer <norman_maurer@apple.com>
2019-05-01 11:24:33 +02:00
if (registration != null) {
evSet0(registration, Native.EVFILT_SOCK, Native.EV_DELETE, 0);
registration = null;
}
}
@Override
protected abstract AbstractKQueueUnsafe newUnsafe();
@Override
public abstract KQueueChannelConfig config();
/**
* Returns an off-heap copy of the specified {@link ByteBuf}, and releases the original one.
*/
protected final ByteBuf newDirectBuffer(ByteBuf buf) {
return newDirectBuffer(buf, buf);
}
/**
* Returns an off-heap copy of the specified {@link ByteBuf}, and releases the specified holder.
* The caller must ensure that the holder releases the original {@link ByteBuf} when the holder is released by
* this method.
*/
protected final ByteBuf newDirectBuffer(Object holder, ByteBuf buf) {
final int readableBytes = buf.readableBytes();
if (readableBytes == 0) {
ReferenceCountUtil.release(holder);
return Unpooled.EMPTY_BUFFER;
}
final ByteBufAllocator alloc = alloc();
if (alloc.isDirectBufferPooled()) {
return newDirectBuffer0(holder, buf, alloc, readableBytes);
}
final ByteBuf directBuf = ByteBufUtil.threadLocalDirectBuffer();
if (directBuf == null) {
return newDirectBuffer0(holder, buf, alloc, readableBytes);
}
directBuf.writeBytes(buf, buf.readerIndex(), readableBytes);
ReferenceCountUtil.safeRelease(holder);
return directBuf;
}
private static ByteBuf newDirectBuffer0(Object holder, ByteBuf buf, ByteBufAllocator alloc, int capacity) {
final ByteBuf directBuf = alloc.directBuffer(capacity);
directBuf.writeBytes(buf, buf.readerIndex(), capacity);
ReferenceCountUtil.safeRelease(holder);
return directBuf;
}
protected static void checkResolvable(InetSocketAddress addr) {
if (addr.isUnresolved()) {
throw new UnresolvedAddressException();
}
}
/**
* Read bytes into the given {@link ByteBuf} and return the amount.
*/
protected final int doReadBytes(ByteBuf byteBuf) throws Exception {
int writerIndex = byteBuf.writerIndex();
int localReadAmount;
unsafe().recvBufAllocHandle().attemptedBytesRead(byteBuf.writableBytes());
if (byteBuf.hasMemoryAddress()) {
localReadAmount = socket.readAddress(byteBuf.memoryAddress(), writerIndex, byteBuf.capacity());
} else {
ByteBuffer buf = byteBuf.internalNioBuffer(writerIndex, byteBuf.writableBytes());
localReadAmount = socket.read(buf, buf.position(), buf.limit());
}
if (localReadAmount > 0) {
byteBuf.writerIndex(writerIndex + localReadAmount);
}
return localReadAmount;
}
2017-12-08 01:00:52 +01:00
protected final int doWriteBytes(ChannelOutboundBuffer in, ByteBuf buf) throws Exception {
if (buf.hasMemoryAddress()) {
2017-12-08 01:00:52 +01:00
int localFlushedAmount = socket.writeAddress(buf.memoryAddress(), buf.readerIndex(), buf.writerIndex());
if (localFlushedAmount > 0) {
in.removeBytes(localFlushedAmount);
return 1;
}
} else {
final ByteBuffer nioBuf = buf.nioBufferCount() == 1?
2017-12-08 01:00:52 +01:00
buf.internalNioBuffer(buf.readerIndex(), buf.readableBytes()) : buf.nioBuffer();
int localFlushedAmount = socket.write(nioBuf, nioBuf.position(), nioBuf.limit());
if (localFlushedAmount > 0) {
nioBuf.position(nioBuf.position() + localFlushedAmount);
in.removeBytes(localFlushedAmount);
return 1;
}
}
2017-12-08 01:00:52 +01:00
return WRITE_STATUS_SNDBUF_FULL;
}
final boolean shouldBreakReadReady(ChannelConfig config) {
return socket.isInputShutdown() && (inputClosedSeenErrorOnRead || !isAllowHalfClosure(config));
}
private static boolean isAllowHalfClosure(ChannelConfig config) {
if (config instanceof KQueueDomainSocketChannelConfig) {
return ((KQueueDomainSocketChannelConfig) config).isAllowHalfClosure();
}
return config instanceof SocketChannelConfig &&
((SocketChannelConfig) config).isAllowHalfClosure();
}
final void clearReadFilter() {
// Only clear if registered with an EventLoop as otherwise
if (isRegistered()) {
final EventLoop loop = executor();
final AbstractKQueueUnsafe unsafe = (AbstractKQueueUnsafe) unsafe();
if (loop.inEventLoop()) {
unsafe.clearReadFilter0();
} else {
// schedule a task to clear the EPOLLIN as it is not safe to modify it directly
loop.execute(() -> {
if (!unsafe.readPending && !config().isAutoRead()) {
// Still no read triggered so clear it now
unsafe.clearReadFilter0();
}
});
}
} else {
// The EventLoop is not registered atm so just update the flags so the correct value
// will be used once the channel is registered
readFilterEnabled = false;
}
}
void readFilter(boolean readFilterEnabled) throws IOException {
if (this.readFilterEnabled != readFilterEnabled) {
this.readFilterEnabled = readFilterEnabled;
evSet(Native.EVFILT_READ, readFilterEnabled ? Native.EV_ADD_CLEAR_ENABLE : Native.EV_DELETE_DISABLE);
}
}
void writeFilter(boolean writeFilterEnabled) throws IOException {
if (this.writeFilterEnabled != writeFilterEnabled) {
this.writeFilterEnabled = writeFilterEnabled;
evSet(Native.EVFILT_WRITE, writeFilterEnabled ? Native.EV_ADD_CLEAR_ENABLE : Native.EV_DELETE_DISABLE);
}
}
private void evSet(short filter, short flags) {
KQueueEventLoop | EpollEventLoop may incorrectly update registration when FD is reused. Motivation: The current KQueueEventLoop implementation does not process concurrent domain socket channel registration/unregistration in the order they actual happen since unregistration are delated by an event loop task scheduling. When a domain socket is closed, it's file descriptor might be reused quickly and therefore trigger a new channel registration using the same descriptor. Consequently the KQueueEventLoop#add(AbstractKQueueChannel) method will overwrite the current inactive channels having the same descriptor and the delayed KQueueEventLoop#remove(AbstractKQueueChannel) will remove the active channel that replaced the inactive one. As active channels are registered, events for this file descriptor won't be processed anymore and the channels will never be closed. The same problem can also happen in EpollEventLoop. Beside this we also may never remove the AbstractEpollChannel from the internal map when it is unregistered which will prevent it from be GC'ed Modifications: - Change logic of native KQueue and Epoll implementations to ensure we correctly handle the case of FD reuse - Only try to update kevent / epoll if the Channel is still open (as otherwise it will be handled by kqueue / epoll itself) - Correctly remove AbstractEpollChannel from internal map in all cases - Make implementation of closeAll() consistent for Epoll and KQueueEventLoop Result: KQueue and Epoll native transports correctly handle FD reuse Co-authored-by: Norman Maurer <norman_maurer@apple.com>
2019-05-01 11:24:33 +02:00
if (isRegistered()) {
evSet0(registration, filter, flags);
}
}
private void evSet0(KQueueRegistration registration, short filter, short flags) {
evSet0(registration, filter, flags, 0);
}
private void evSet0(KQueueRegistration registration, short filter, short flags, int fflags) {
KQueueEventLoop | EpollEventLoop may incorrectly update registration when FD is reused. Motivation: The current KQueueEventLoop implementation does not process concurrent domain socket channel registration/unregistration in the order they actual happen since unregistration are delated by an event loop task scheduling. When a domain socket is closed, it's file descriptor might be reused quickly and therefore trigger a new channel registration using the same descriptor. Consequently the KQueueEventLoop#add(AbstractKQueueChannel) method will overwrite the current inactive channels having the same descriptor and the delayed KQueueEventLoop#remove(AbstractKQueueChannel) will remove the active channel that replaced the inactive one. As active channels are registered, events for this file descriptor won't be processed anymore and the channels will never be closed. The same problem can also happen in EpollEventLoop. Beside this we also may never remove the AbstractEpollChannel from the internal map when it is unregistered which will prevent it from be GC'ed Modifications: - Change logic of native KQueue and Epoll implementations to ensure we correctly handle the case of FD reuse - Only try to update kevent / epoll if the Channel is still open (as otherwise it will be handled by kqueue / epoll itself) - Correctly remove AbstractEpollChannel from internal map in all cases - Make implementation of closeAll() consistent for Epoll and KQueueEventLoop Result: KQueue and Epoll native transports correctly handle FD reuse Co-authored-by: Norman Maurer <norman_maurer@apple.com>
2019-05-01 11:24:33 +02:00
// Only try to add to changeList if the FD is still open, if not we already closed it in the meantime.
if (isOpen()) {
registration.evSet(filter, flags, fflags);
}
}
abstract class AbstractKQueueUnsafe extends AbstractUnsafe {
boolean readPending;
boolean maybeMoreDataToRead;
private KQueueRecvByteAllocatorHandle allocHandle;
private final Runnable readReadyRunnable = new Runnable() {
@Override
public void run() {
readReadyRunnablePending = false;
readReady(recvBufAllocHandle());
}
};
final void readReady(long numberBytesPending) {
KQueueRecvByteAllocatorHandle allocHandle = recvBufAllocHandle();
allocHandle.numberBytesPending(numberBytesPending);
readReady(allocHandle);
}
abstract void readReady(KQueueRecvByteAllocatorHandle allocHandle);
final void readReadyBefore() {
maybeMoreDataToRead = false;
}
final void readReadyFinally(ChannelConfig config) {
maybeMoreDataToRead = allocHandle.maybeMoreDataToRead();
if (allocHandle.isReadEOF() || readPending && maybeMoreDataToRead) {
// trigger a read again as there may be something left to read and because of ET we
// will not get notified again until we read everything from the socket
//
// It is possible the last fireChannelRead call could cause the user to call read() again, or if
// autoRead is true the call to channelReadComplete would also call read, but maybeMoreDataToRead is set
// to false before every read operation to prevent re-entry into readReady() we will not read from
// the underlying OS again unless the user happens to call read again.
executeReadReadyRunnable(config);
} else if (!readPending && !config.isAutoRead()) {
// Check if there is a readPending which was not processed yet.
// This could be for two reasons:
// * The user called Channel.read() or ChannelHandlerContext.read() in channelRead(...) method
// * The user called Channel.read() or ChannelHandlerContext.read() in channelReadComplete(...) method
//
// See https://github.com/netty/netty/issues/2254
clearReadFilter0();
}
}
KQueue write filter initial state (#7738) Motivation: KQueue implementations current have inconsistent behavior with Epoll implementations with respect to asynchronous sockets and connecting. In the Epoll transport we attempt to connect, if the connect call does not synchornously fail/succeed we set the EPOLLOUT which will be triggered by the kernel if the connection attempt succeeds or an error occurs. The connect API provides no way to asynchronously communicate an error so the Epoll implementation fires a EPOLLOUT event and puts the connect status in getsockopt(SO_ERROR). KQueue provides the same APIs but different behavior. If the EVFILT_WRITE is not enabled and the EVFILT_READ is enabled before connect is called, and there is an error the kernel may fire the EVFILT_READ filter and provide the Connection Refused error via read(). This is even true if we set the EVFILT_WRITE filter after calling connect because connect didn't synchornously complete. After the error has been delievered via read() a call to getsockopt(SO_ERROR) will return 0 indicating there is no error. This means we cannot rely upon the KQueue based kernel to deliver connection errors via the EVFILT_WRITE filter in the same way that the linux kernel does with the EPOLLOUT flag. ce241bd introduced a change which depends upon the behavior of the EVFILT_WRITE being set and may prematurely stop writing to the OS as a result, becaues we assume the OS will notify us when the socket is writable. However the current work around for the above described behavior is to initialize the EVFILT_WRITE to true for connection oriented protocols. This leads to prematurely exiting from the flush() which may lead to deadlock. Modifications: - KQueue should check when an error is obtained from read() if the connectPromise has not yet been completed, and if not complete it with a ConnectException Result: No more deadlock in KQueue due to asynchronous connect workaround.
2018-02-20 20:01:49 +01:00
final boolean failConnectPromise(Throwable cause) {
if (connectPromise != null) {
// SO_ERROR has been shown to return 0 on macOS if detect an error via read() and the write filter was
// not set before calling connect. This means finishConnect will not detect any error and would
// successfully complete the connectPromise and update the channel state to active (which is incorrect).
Clean up Future/Promises API (#11575) Motivation: The generics for the existing futures, promises, and listeners are too complicated. This complication comes from the existence of `ChannelPromise` and `ChannelFuture`, which forces listeners to care about the particular _type_ of future being listened on. Modification: * Add a `FutureContextListener` which can take a context object as an additional argument. This allows our listeners to have the channel piped through to them, so they don't need to rely on the `ChannelFuture.channel()` method. * Make the `FutureListener`, along with the `FutureContextListener` sibling, the default listener API, retiring the `GenericFutureListener` since we no longer need to abstract over the type of the future. * Change all uses of `ChannelPromise` to `Promise<Void>`. * Change all uses of `ChannelFuture` to `Future<Void>`. * Change all uses of `GenericFutureListener` to either `FutureListener` or `FutureContextListener` as needed. * Remove `ChannelFutureListener` and `GenericFutureListener`. * Introduce a `ChannelFutureListeners` enum to house the constants that previously lived in `ChannelFutureListener`. These constants now implement `FutureContextListener` and take the `Channel` as a context. * Remove `ChannelPromise` and `ChannelFuture` — all usages now rely on the plain `Future` and `Promise` APIs. * Add static factory methods to `DefaultPromise` that allow us to create promises that are initialised as successful or failed. * Remove `CompleteFuture`, `SucceededFuture`, `FailedFuture`, `CompleteChannelFuture`, `SucceededChannelFuture`, and `FailedChannelFuture`. * Remove `ChannelPromiseNotifier`. Result: Cleaner generics and more straight forward code.
2021-08-20 09:55:16 +02:00
Promise<Void> connectPromise = AbstractKQueueChannel.this.connectPromise;
KQueue write filter initial state (#7738) Motivation: KQueue implementations current have inconsistent behavior with Epoll implementations with respect to asynchronous sockets and connecting. In the Epoll transport we attempt to connect, if the connect call does not synchornously fail/succeed we set the EPOLLOUT which will be triggered by the kernel if the connection attempt succeeds or an error occurs. The connect API provides no way to asynchronously communicate an error so the Epoll implementation fires a EPOLLOUT event and puts the connect status in getsockopt(SO_ERROR). KQueue provides the same APIs but different behavior. If the EVFILT_WRITE is not enabled and the EVFILT_READ is enabled before connect is called, and there is an error the kernel may fire the EVFILT_READ filter and provide the Connection Refused error via read(). This is even true if we set the EVFILT_WRITE filter after calling connect because connect didn't synchornously complete. After the error has been delievered via read() a call to getsockopt(SO_ERROR) will return 0 indicating there is no error. This means we cannot rely upon the KQueue based kernel to deliver connection errors via the EVFILT_WRITE filter in the same way that the linux kernel does with the EPOLLOUT flag. ce241bd introduced a change which depends upon the behavior of the EVFILT_WRITE being set and may prematurely stop writing to the OS as a result, becaues we assume the OS will notify us when the socket is writable. However the current work around for the above described behavior is to initialize the EVFILT_WRITE to true for connection oriented protocols. This leads to prematurely exiting from the flush() which may lead to deadlock. Modifications: - KQueue should check when an error is obtained from read() if the connectPromise has not yet been completed, and if not complete it with a ConnectException Result: No more deadlock in KQueue due to asynchronous connect workaround.
2018-02-20 20:01:49 +01:00
AbstractKQueueChannel.this.connectPromise = null;
Clean up Future/Promises API (#11575) Motivation: The generics for the existing futures, promises, and listeners are too complicated. This complication comes from the existence of `ChannelPromise` and `ChannelFuture`, which forces listeners to care about the particular _type_ of future being listened on. Modification: * Add a `FutureContextListener` which can take a context object as an additional argument. This allows our listeners to have the channel piped through to them, so they don't need to rely on the `ChannelFuture.channel()` method. * Make the `FutureListener`, along with the `FutureContextListener` sibling, the default listener API, retiring the `GenericFutureListener` since we no longer need to abstract over the type of the future. * Change all uses of `ChannelPromise` to `Promise<Void>`. * Change all uses of `ChannelFuture` to `Future<Void>`. * Change all uses of `GenericFutureListener` to either `FutureListener` or `FutureContextListener` as needed. * Remove `ChannelFutureListener` and `GenericFutureListener`. * Introduce a `ChannelFutureListeners` enum to house the constants that previously lived in `ChannelFutureListener`. These constants now implement `FutureContextListener` and take the `Channel` as a context. * Remove `ChannelPromise` and `ChannelFuture` — all usages now rely on the plain `Future` and `Promise` APIs. * Add static factory methods to `DefaultPromise` that allow us to create promises that are initialised as successful or failed. * Remove `CompleteFuture`, `SucceededFuture`, `FailedFuture`, `CompleteChannelFuture`, `SucceededChannelFuture`, and `FailedChannelFuture`. * Remove `ChannelPromiseNotifier`. Result: Cleaner generics and more straight forward code.
2021-08-20 09:55:16 +02:00
if (connectPromise.tryFailure(cause instanceof ConnectException? cause
KQueue write filter initial state (#7738) Motivation: KQueue implementations current have inconsistent behavior with Epoll implementations with respect to asynchronous sockets and connecting. In the Epoll transport we attempt to connect, if the connect call does not synchornously fail/succeed we set the EPOLLOUT which will be triggered by the kernel if the connection attempt succeeds or an error occurs. The connect API provides no way to asynchronously communicate an error so the Epoll implementation fires a EPOLLOUT event and puts the connect status in getsockopt(SO_ERROR). KQueue provides the same APIs but different behavior. If the EVFILT_WRITE is not enabled and the EVFILT_READ is enabled before connect is called, and there is an error the kernel may fire the EVFILT_READ filter and provide the Connection Refused error via read(). This is even true if we set the EVFILT_WRITE filter after calling connect because connect didn't synchornously complete. After the error has been delievered via read() a call to getsockopt(SO_ERROR) will return 0 indicating there is no error. This means we cannot rely upon the KQueue based kernel to deliver connection errors via the EVFILT_WRITE filter in the same way that the linux kernel does with the EPOLLOUT flag. ce241bd introduced a change which depends upon the behavior of the EVFILT_WRITE being set and may prematurely stop writing to the OS as a result, becaues we assume the OS will notify us when the socket is writable. However the current work around for the above described behavior is to initialize the EVFILT_WRITE to true for connection oriented protocols. This leads to prematurely exiting from the flush() which may lead to deadlock. Modifications: - KQueue should check when an error is obtained from read() if the connectPromise has not yet been completed, and if not complete it with a ConnectException Result: No more deadlock in KQueue due to asynchronous connect workaround.
2018-02-20 20:01:49 +01:00
: new ConnectException("failed to connect").initCause(cause))) {
closeIfClosed();
return true;
}
}
return false;
}
final void writeReady() {
if (connectPromise != null) {
// pending connect which is now complete so handle it.
finishConnect();
} else if (!socket.isOutputShutdown()) {
// directly call super.flush0() to force a flush now
super.flush0();
}
}
/**
* Shutdown the input side of the channel.
*/
void shutdownInput(boolean readEOF) {
// We need to take special care of calling finishConnect() if readEOF is true and we not
// fullfilled the connectPromise yet. If we fail to do so the connectPromise will be failed
// with a ClosedChannelException as a close() will happen and so the FD is closed before we
// have a chance to call finishConnect() later on. Calling finishConnect() here will ensure
// we observe the correct exception in case of a connect failure.
if (readEOF && connectPromise != null) {
finishConnect();
}
if (!socket.isInputShutdown()) {
if (isAllowHalfClosure(config())) {
try {
socket.shutdown(true, false);
} catch (IOException ignored) {
// We attempted to shutdown and failed, which means the input has already effectively been
// shutdown.
fireEventAndClose(ChannelInputShutdownEvent.INSTANCE);
return;
} catch (NotYetConnectedException ignore) {
// We attempted to shutdown and failed, which means the input has already effectively been
// shutdown.
}
pipeline().fireUserEventTriggered(ChannelInputShutdownEvent.INSTANCE);
} else {
close(newPromise());
}
} else if (!readEOF) {
inputClosedSeenErrorOnRead = true;
pipeline().fireUserEventTriggered(ChannelInputShutdownReadComplete.INSTANCE);
}
}
final void readEOF() {
// This must happen before we attempt to read. This will ensure reading continues until an error occurs.
final KQueueRecvByteAllocatorHandle allocHandle = recvBufAllocHandle();
allocHandle.readEOF();
if (isActive()) {
// If it is still active, we need to call readReady as otherwise we may miss to
// read pending data from the underlying file descriptor.
// See https://github.com/netty/netty/issues/3709
readReady(allocHandle);
} else {
// Just to be safe make sure the input marked as closed.
shutdownInput(true);
}
}
@Override
public KQueueRecvByteAllocatorHandle recvBufAllocHandle() {
if (allocHandle == null) {
allocHandle = new KQueueRecvByteAllocatorHandle(
(RecvByteBufAllocator.ExtendedHandle) super.recvBufAllocHandle());
}
return allocHandle;
}
@Override
protected final void flush0() {
// Flush immediately only when there's no pending flush.
// If there's a pending flush operation, event loop will call forceFlush() later,
// and thus there's no need to call it now.
if (!writeFilterEnabled) {
super.flush0();
}
}
final void executeReadReadyRunnable(ChannelConfig config) {
if (readReadyRunnablePending || !isActive() || shouldBreakReadReady(config)) {
return;
}
readReadyRunnablePending = true;
executor().execute(readReadyRunnable);
}
protected final void clearReadFilter0() {
assert executor().inEventLoop();
try {
readPending = false;
readFilter(false);
} catch (IOException e) {
// When this happens there is something completely wrong with either the filedescriptor or epoll,
// so fire the exception through the pipeline and close the Channel.
pipeline().fireExceptionCaught(e);
unsafe().close(newPromise());
}
}
private void fireEventAndClose(Object evt) {
pipeline().fireUserEventTriggered(evt);
close(newPromise());
}
@Override
public void connect(
Clean up Future/Promises API (#11575) Motivation: The generics for the existing futures, promises, and listeners are too complicated. This complication comes from the existence of `ChannelPromise` and `ChannelFuture`, which forces listeners to care about the particular _type_ of future being listened on. Modification: * Add a `FutureContextListener` which can take a context object as an additional argument. This allows our listeners to have the channel piped through to them, so they don't need to rely on the `ChannelFuture.channel()` method. * Make the `FutureListener`, along with the `FutureContextListener` sibling, the default listener API, retiring the `GenericFutureListener` since we no longer need to abstract over the type of the future. * Change all uses of `ChannelPromise` to `Promise<Void>`. * Change all uses of `ChannelFuture` to `Future<Void>`. * Change all uses of `GenericFutureListener` to either `FutureListener` or `FutureContextListener` as needed. * Remove `ChannelFutureListener` and `GenericFutureListener`. * Introduce a `ChannelFutureListeners` enum to house the constants that previously lived in `ChannelFutureListener`. These constants now implement `FutureContextListener` and take the `Channel` as a context. * Remove `ChannelPromise` and `ChannelFuture` — all usages now rely on the plain `Future` and `Promise` APIs. * Add static factory methods to `DefaultPromise` that allow us to create promises that are initialised as successful or failed. * Remove `CompleteFuture`, `SucceededFuture`, `FailedFuture`, `CompleteChannelFuture`, `SucceededChannelFuture`, and `FailedChannelFuture`. * Remove `ChannelPromiseNotifier`. Result: Cleaner generics and more straight forward code.
2021-08-20 09:55:16 +02:00
final SocketAddress remoteAddress, final SocketAddress localAddress, Promise<Void> promise) {
if (!promise.setUncancellable() || !ensureOpen(promise)) {
return;
}
try {
if (connectPromise != null) {
throw new ConnectionPendingException();
}
boolean wasActive = isActive();
if (doConnect(remoteAddress, localAddress)) {
fulfillConnectPromise(promise, wasActive);
} else {
connectPromise = promise;
requestedRemoteAddress = remoteAddress;
// Schedule connect timeout.
int connectTimeoutMillis = config().getConnectTimeoutMillis();
if (connectTimeoutMillis > 0) {
connectTimeoutFuture = executor().schedule(() -> {
Clean up Future/Promises API (#11575) Motivation: The generics for the existing futures, promises, and listeners are too complicated. This complication comes from the existence of `ChannelPromise` and `ChannelFuture`, which forces listeners to care about the particular _type_ of future being listened on. Modification: * Add a `FutureContextListener` which can take a context object as an additional argument. This allows our listeners to have the channel piped through to them, so they don't need to rely on the `ChannelFuture.channel()` method. * Make the `FutureListener`, along with the `FutureContextListener` sibling, the default listener API, retiring the `GenericFutureListener` since we no longer need to abstract over the type of the future. * Change all uses of `ChannelPromise` to `Promise<Void>`. * Change all uses of `ChannelFuture` to `Future<Void>`. * Change all uses of `GenericFutureListener` to either `FutureListener` or `FutureContextListener` as needed. * Remove `ChannelFutureListener` and `GenericFutureListener`. * Introduce a `ChannelFutureListeners` enum to house the constants that previously lived in `ChannelFutureListener`. These constants now implement `FutureContextListener` and take the `Channel` as a context. * Remove `ChannelPromise` and `ChannelFuture` — all usages now rely on the plain `Future` and `Promise` APIs. * Add static factory methods to `DefaultPromise` that allow us to create promises that are initialised as successful or failed. * Remove `CompleteFuture`, `SucceededFuture`, `FailedFuture`, `CompleteChannelFuture`, `SucceededChannelFuture`, and `FailedChannelFuture`. * Remove `ChannelPromiseNotifier`. Result: Cleaner generics and more straight forward code.
2021-08-20 09:55:16 +02:00
Promise<Void> connectPromise = AbstractKQueueChannel.this.connectPromise;
if (connectPromise != null && !connectPromise.isDone()
&& connectPromise.tryFailure(new ConnectTimeoutException(
"connection timed out: " + remoteAddress))) {
close(newPromise());
}
}, connectTimeoutMillis, TimeUnit.MILLISECONDS);
}
Clean up Future/Promises API (#11575) Motivation: The generics for the existing futures, promises, and listeners are too complicated. This complication comes from the existence of `ChannelPromise` and `ChannelFuture`, which forces listeners to care about the particular _type_ of future being listened on. Modification: * Add a `FutureContextListener` which can take a context object as an additional argument. This allows our listeners to have the channel piped through to them, so they don't need to rely on the `ChannelFuture.channel()` method. * Make the `FutureListener`, along with the `FutureContextListener` sibling, the default listener API, retiring the `GenericFutureListener` since we no longer need to abstract over the type of the future. * Change all uses of `ChannelPromise` to `Promise<Void>`. * Change all uses of `ChannelFuture` to `Future<Void>`. * Change all uses of `GenericFutureListener` to either `FutureListener` or `FutureContextListener` as needed. * Remove `ChannelFutureListener` and `GenericFutureListener`. * Introduce a `ChannelFutureListeners` enum to house the constants that previously lived in `ChannelFutureListener`. These constants now implement `FutureContextListener` and take the `Channel` as a context. * Remove `ChannelPromise` and `ChannelFuture` — all usages now rely on the plain `Future` and `Promise` APIs. * Add static factory methods to `DefaultPromise` that allow us to create promises that are initialised as successful or failed. * Remove `CompleteFuture`, `SucceededFuture`, `FailedFuture`, `CompleteChannelFuture`, `SucceededChannelFuture`, and `FailedChannelFuture`. * Remove `ChannelPromiseNotifier`. Result: Cleaner generics and more straight forward code.
2021-08-20 09:55:16 +02:00
promise.addListener(future -> {
if (future.isCancelled()) {
if (connectTimeoutFuture != null) {
connectTimeoutFuture.cancel(false);
}
connectPromise = null;
close(newPromise());
}
});
}
} catch (Throwable t) {
closeIfClosed();
promise.tryFailure(annotateConnectException(t, remoteAddress));
}
}
Clean up Future/Promises API (#11575) Motivation: The generics for the existing futures, promises, and listeners are too complicated. This complication comes from the existence of `ChannelPromise` and `ChannelFuture`, which forces listeners to care about the particular _type_ of future being listened on. Modification: * Add a `FutureContextListener` which can take a context object as an additional argument. This allows our listeners to have the channel piped through to them, so they don't need to rely on the `ChannelFuture.channel()` method. * Make the `FutureListener`, along with the `FutureContextListener` sibling, the default listener API, retiring the `GenericFutureListener` since we no longer need to abstract over the type of the future. * Change all uses of `ChannelPromise` to `Promise<Void>`. * Change all uses of `ChannelFuture` to `Future<Void>`. * Change all uses of `GenericFutureListener` to either `FutureListener` or `FutureContextListener` as needed. * Remove `ChannelFutureListener` and `GenericFutureListener`. * Introduce a `ChannelFutureListeners` enum to house the constants that previously lived in `ChannelFutureListener`. These constants now implement `FutureContextListener` and take the `Channel` as a context. * Remove `ChannelPromise` and `ChannelFuture` — all usages now rely on the plain `Future` and `Promise` APIs. * Add static factory methods to `DefaultPromise` that allow us to create promises that are initialised as successful or failed. * Remove `CompleteFuture`, `SucceededFuture`, `FailedFuture`, `CompleteChannelFuture`, `SucceededChannelFuture`, and `FailedChannelFuture`. * Remove `ChannelPromiseNotifier`. Result: Cleaner generics and more straight forward code.
2021-08-20 09:55:16 +02:00
private void fulfillConnectPromise(Promise<Void> promise, boolean wasActive) {
if (promise == null) {
// Closed via cancellation and the promise has been notified already.
return;
}
active = true;
Clean up Future/Promises API (#11575) Motivation: The generics for the existing futures, promises, and listeners are too complicated. This complication comes from the existence of `ChannelPromise` and `ChannelFuture`, which forces listeners to care about the particular _type_ of future being listened on. Modification: * Add a `FutureContextListener` which can take a context object as an additional argument. This allows our listeners to have the channel piped through to them, so they don't need to rely on the `ChannelFuture.channel()` method. * Make the `FutureListener`, along with the `FutureContextListener` sibling, the default listener API, retiring the `GenericFutureListener` since we no longer need to abstract over the type of the future. * Change all uses of `ChannelPromise` to `Promise<Void>`. * Change all uses of `ChannelFuture` to `Future<Void>`. * Change all uses of `GenericFutureListener` to either `FutureListener` or `FutureContextListener` as needed. * Remove `ChannelFutureListener` and `GenericFutureListener`. * Introduce a `ChannelFutureListeners` enum to house the constants that previously lived in `ChannelFutureListener`. These constants now implement `FutureContextListener` and take the `Channel` as a context. * Remove `ChannelPromise` and `ChannelFuture` — all usages now rely on the plain `Future` and `Promise` APIs. * Add static factory methods to `DefaultPromise` that allow us to create promises that are initialised as successful or failed. * Remove `CompleteFuture`, `SucceededFuture`, `FailedFuture`, `CompleteChannelFuture`, `SucceededChannelFuture`, and `FailedChannelFuture`. * Remove `ChannelPromiseNotifier`. Result: Cleaner generics and more straight forward code.
2021-08-20 09:55:16 +02:00
// Get the state as trySuccess() may trigger an ChannelFutureListeners that will close the Channel.
// We still need to ensure we call fireChannelActive() in this case.
boolean active = isActive();
// trySuccess() will return false if a user cancelled the connection attempt.
Clean up Future/Promises API (#11575) Motivation: The generics for the existing futures, promises, and listeners are too complicated. This complication comes from the existence of `ChannelPromise` and `ChannelFuture`, which forces listeners to care about the particular _type_ of future being listened on. Modification: * Add a `FutureContextListener` which can take a context object as an additional argument. This allows our listeners to have the channel piped through to them, so they don't need to rely on the `ChannelFuture.channel()` method. * Make the `FutureListener`, along with the `FutureContextListener` sibling, the default listener API, retiring the `GenericFutureListener` since we no longer need to abstract over the type of the future. * Change all uses of `ChannelPromise` to `Promise<Void>`. * Change all uses of `ChannelFuture` to `Future<Void>`. * Change all uses of `GenericFutureListener` to either `FutureListener` or `FutureContextListener` as needed. * Remove `ChannelFutureListener` and `GenericFutureListener`. * Introduce a `ChannelFutureListeners` enum to house the constants that previously lived in `ChannelFutureListener`. These constants now implement `FutureContextListener` and take the `Channel` as a context. * Remove `ChannelPromise` and `ChannelFuture` — all usages now rely on the plain `Future` and `Promise` APIs. * Add static factory methods to `DefaultPromise` that allow us to create promises that are initialised as successful or failed. * Remove `CompleteFuture`, `SucceededFuture`, `FailedFuture`, `CompleteChannelFuture`, `SucceededChannelFuture`, and `FailedChannelFuture`. * Remove `ChannelPromiseNotifier`. Result: Cleaner generics and more straight forward code.
2021-08-20 09:55:16 +02:00
boolean promiseSet = promise.trySuccess(null);
// Regardless if the connection attempt was cancelled, channelActive() event should be triggered,
// because what happened is what happened.
if (!wasActive && active) {
pipeline().fireChannelActive();
readIfIsAutoRead();
}
// If a user cancelled the connection attempt, close the channel, which is followed by channelInactive().
if (!promiseSet) {
close(newPromise());
}
}
Clean up Future/Promises API (#11575) Motivation: The generics for the existing futures, promises, and listeners are too complicated. This complication comes from the existence of `ChannelPromise` and `ChannelFuture`, which forces listeners to care about the particular _type_ of future being listened on. Modification: * Add a `FutureContextListener` which can take a context object as an additional argument. This allows our listeners to have the channel piped through to them, so they don't need to rely on the `ChannelFuture.channel()` method. * Make the `FutureListener`, along with the `FutureContextListener` sibling, the default listener API, retiring the `GenericFutureListener` since we no longer need to abstract over the type of the future. * Change all uses of `ChannelPromise` to `Promise<Void>`. * Change all uses of `ChannelFuture` to `Future<Void>`. * Change all uses of `GenericFutureListener` to either `FutureListener` or `FutureContextListener` as needed. * Remove `ChannelFutureListener` and `GenericFutureListener`. * Introduce a `ChannelFutureListeners` enum to house the constants that previously lived in `ChannelFutureListener`. These constants now implement `FutureContextListener` and take the `Channel` as a context. * Remove `ChannelPromise` and `ChannelFuture` — all usages now rely on the plain `Future` and `Promise` APIs. * Add static factory methods to `DefaultPromise` that allow us to create promises that are initialised as successful or failed. * Remove `CompleteFuture`, `SucceededFuture`, `FailedFuture`, `CompleteChannelFuture`, `SucceededChannelFuture`, and `FailedChannelFuture`. * Remove `ChannelPromiseNotifier`. Result: Cleaner generics and more straight forward code.
2021-08-20 09:55:16 +02:00
private void fulfillConnectPromise(Promise<Void> promise, Throwable cause) {
if (promise == null) {
// Closed via cancellation and the promise has been notified already.
return;
}
// Use tryFailure() instead of setFailure() to avoid the race against cancel().
promise.tryFailure(cause);
closeIfClosed();
}
private void finishConnect() {
// Note this method is invoked by the event loop only if the connection attempt was
// neither cancelled nor timed out.
assert executor().inEventLoop();
boolean connectStillInProgress = false;
try {
boolean wasActive = isActive();
if (!doFinishConnect()) {
connectStillInProgress = true;
return;
}
fulfillConnectPromise(connectPromise, wasActive);
} catch (Throwable t) {
fulfillConnectPromise(connectPromise, annotateConnectException(t, requestedRemoteAddress));
} finally {
if (!connectStillInProgress) {
// Check for null as the connectTimeoutFuture is only created if a connectTimeoutMillis > 0 is used
// See https://github.com/netty/netty/issues/1770
if (connectTimeoutFuture != null) {
connectTimeoutFuture.cancel(false);
}
connectPromise = null;
}
}
}
private boolean doFinishConnect() throws Exception {
if (socket.finishConnect()) {
writeFilter(false);
if (requestedRemoteAddress instanceof InetSocketAddress) {
remote = computeRemoteAddr((InetSocketAddress) requestedRemoteAddress, socket.remoteAddress());
}
requestedRemoteAddress = null;
return true;
}
writeFilter(true);
return false;
}
}
@Override
protected void doBind(SocketAddress local) throws Exception {
if (local instanceof InetSocketAddress) {
checkResolvable((InetSocketAddress) local);
}
socket.bind(local);
this.local = socket.localAddress();
}
/**
* Connect to the remote peer
*/
protected boolean doConnect(SocketAddress remoteAddress, SocketAddress localAddress) throws Exception {
if (localAddress instanceof InetSocketAddress) {
checkResolvable((InetSocketAddress) localAddress);
}
InetSocketAddress remoteSocketAddr = remoteAddress instanceof InetSocketAddress
? (InetSocketAddress) remoteAddress : null;
if (remoteSocketAddr != null) {
checkResolvable(remoteSocketAddr);
}
if (remote != null) {
// Check if already connected before trying to connect. This is needed as connect(...) will not return -1
// and set errno to EISCONN if a previous connect(...) attempt was setting errno to EINPROGRESS and finished
// later.
throw new AlreadyConnectedException();
}
if (localAddress != null) {
socket.bind(localAddress);
}
boolean connected = doConnect0(remoteAddress, localAddress);
if (connected) {
remote = remoteSocketAddr == null?
remoteAddress : computeRemoteAddr(remoteSocketAddr, socket.remoteAddress());
}
// We always need to set the localAddress even if not connected yet as the bind already took place.
//
// See https://github.com/netty/netty/issues/3463
local = socket.localAddress();
return connected;
}
protected boolean doConnect0(SocketAddress remoteAddress, SocketAddress localAddress) throws Exception {
boolean success = false;
try {
boolean connected = socket.connect(remoteAddress);
if (!connected) {
writeFilter(true);
}
success = true;
return connected;
} finally {
if (!success) {
doClose();
}
}
}
@Override
protected SocketAddress localAddress0() {
return local;
}
@Override
protected SocketAddress remoteAddress0() {
return remote;
}
}