Merge pull request #7 from normanmaurer/iouring_error

Correctly handle POLL*, handle errors, cleanup
This commit is contained in:
Josef Grieb 2020-08-30 14:59:03 +02:00 committed by GitHub
commit f77aa54f18
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 230 additions and 292 deletions

View File

@ -34,21 +34,13 @@ import io.netty.channel.RecvByteBufAllocator;
import io.netty.channel.socket.ChannelInputShutdownEvent; import io.netty.channel.socket.ChannelInputShutdownEvent;
import io.netty.channel.socket.ChannelInputShutdownReadComplete; import io.netty.channel.socket.ChannelInputShutdownReadComplete;
import io.netty.channel.socket.SocketChannelConfig; import io.netty.channel.socket.SocketChannelConfig;
import io.netty.channel.unix.Buffer; import io.netty.channel.unix.*;
import io.netty.channel.unix.FileDescriptor;
import io.netty.channel.unix.NativeInetAddress;
import io.netty.channel.unix.Socket;
import io.netty.channel.unix.UnixChannel;
import io.netty.channel.unix.UnixChannelUtil;
import io.netty.util.ReferenceCountUtil; import io.netty.util.ReferenceCountUtil;
import io.netty.util.internal.logging.InternalLogger; import io.netty.util.internal.logging.InternalLogger;
import io.netty.util.internal.logging.InternalLoggerFactory; import io.netty.util.internal.logging.InternalLoggerFactory;
import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.net.ConnectException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.net.NoRouteToHostException;
import java.net.SocketAddress; import java.net.SocketAddress;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.channels.AlreadyConnectedException; import java.nio.channels.AlreadyConnectedException;
@ -70,7 +62,6 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
protected volatile boolean active; protected volatile boolean active;
private boolean pollInScheduled = false; private boolean pollInScheduled = false;
//boolean uringInReadyPending;
boolean inputClosedSeenErrorOnRead; boolean inputClosedSeenErrorOnRead;
static final int SOCK_ADDR_LEN = 128; static final int SOCK_ADDR_LEN = 128;
@ -235,11 +226,7 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
connectPromise = null; connectPromise = null;
} }
ScheduledFuture<?> future = connectTimeoutFuture; cancelConnectTimeoutFuture();
if (future != null) {
future.cancel(false);
connectTimeoutFuture = null;
}
if (isRegistered()) { if (isRegistered()) {
// Need to check if we are on the EventLoop as doClose() may be triggered by the GlobalEventExecutor // Need to check if we are on the EventLoop as doClose() may be triggered by the GlobalEventExecutor
@ -271,7 +258,6 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
// Channel/ChannelHandlerContext.read() was called // Channel/ChannelHandlerContext.read() was called
@Override @Override
protected void doBeginRead() { protected void doBeginRead() {
System.out.println("Begin Read");
final AbstractUringUnsafe unsafe = (AbstractUringUnsafe) unsafe(); final AbstractUringUnsafe unsafe = (AbstractUringUnsafe) unsafe();
if (!pollInScheduled) { if (!pollInScheduled) {
unsafe.schedulePollIn(); unsafe.schedulePollIn();
@ -279,7 +265,7 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
} }
@Override @Override
protected void doWrite(ChannelOutboundBuffer in) throws Exception { protected void doWrite(ChannelOutboundBuffer in) {
logger.trace("IOUring doWrite message size: {}", in.size()); logger.trace("IOUring doWrite message size: {}", in.size());
if (writeScheduled) { if (writeScheduled) {
@ -290,22 +276,27 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
doWriteMultiple(in); doWriteMultiple(in);
//Object msg = in.current(); //Object msg = in.current();
//doWriteSingle((ByteBuf) msg); //doWriteSingle((ByteBuf) msg);
} else if(msgCount == 1) { } else if (msgCount == 1) {
Object msg = in.current(); Object msg = in.current();
doWriteSingle((ByteBuf) msg); doWriteSingle((ByteBuf) msg);
} }
} }
private void doWriteMultiple(ChannelOutboundBuffer in) throws Exception { private void doWriteMultiple(ChannelOutboundBuffer in) {
final IovecArrayPool iovecArray = ((IOUringEventLoop) eventLoop()).getIovecArrayPool(); final IovecArrayPool iovecArray = ((IOUringEventLoop) eventLoop()).getIovecArrayPool();
iovecMemoryAddress = iovecArray.createNewIovecMemoryAddress(); iovecMemoryAddress = iovecArray.createNewIovecMemoryAddress();
if (iovecMemoryAddress != -1) { if (iovecMemoryAddress != -1) {
in.forEachFlushedMessage(iovecArray); try {
in.forEachFlushedMessage(iovecArray);
} catch (Exception e) {
}
if (iovecArray.count() > 0) { if (iovecArray.count() > 0) {
submissionQueue().addWritev(socket.intValue(), iovecMemoryAddress, iovecArray.count()); submissionQueue().addWritev(socket.intValue(), iovecMemoryAddress, iovecArray.count());
submissionQueue().submit(); submissionQueue().submit();
writeScheduled = true;
} }
} }
//Todo error handling //Todo error handling
@ -313,24 +304,17 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
protected final void doWriteSingle(ByteBuf buf) { protected final void doWriteSingle(ByteBuf buf) {
if (buf.hasMemoryAddress()) { IOUringSubmissionQueue submissionQueue = submissionQueue();
//link poll<link>write operation submissionQueue.addWrite(socket.intValue(), buf.memoryAddress(), buf.readerIndex(),
addPollOut(); buf.writerIndex());
submissionQueue.submit();
IOUringEventLoop ioUringEventLoop = (IOUringEventLoop) eventLoop(); writeScheduled = true;
IOUringSubmissionQueue submissionQueue = ioUringEventLoop.getRingBuffer().getIoUringSubmissionQueue();
submissionQueue.addWrite(socket.intValue(), buf.memoryAddress(), buf.readerIndex(),
buf.writerIndex());
submissionQueue.submit();
writeScheduled = true;
}
} }
//POLLOUT //POLLOUT
private void addPollOut() { private void addPollOut() {
IOUringEventLoop ioUringEventLoop = (IOUringEventLoop) eventLoop(); IOUringSubmissionQueue submissionQueue = submissionQueue();
IOUringSubmissionQueue submissionQueue = ioUringEventLoop.getRingBuffer().getIoUringSubmissionQueue(); submissionQueue.addPollOut(socket.intValue());
submissionQueue.addPollOutLink(socket.intValue());
submissionQueue.submit(); submissionQueue.submit();
} }
@ -396,14 +380,43 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
return allocHandle; return allocHandle;
} }
void shutdownInput(boolean rdHup) {
logger.trace("shutdownInput Fd: {}", fd().intValue());
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(voidPromise());
}
} else if (!rdHup) {
inputClosedSeenErrorOnRead = true;
pipeline().fireUserEventTriggered(ChannelInputShutdownReadComplete.INSTANCE);
}
}
private void fireEventAndClose(Object evt) {
pipeline().fireUserEventTriggered(evt);
close(voidPromise());
}
void schedulePollIn() { void schedulePollIn() {
assert !pollInScheduled; assert !pollInScheduled;
if (!isActive() || shouldBreakIoUringInReady(config())) { if (!isActive() || shouldBreakIoUringInReady(config())) {
return; return;
} }
pollInScheduled = true; pollInScheduled = true;
IOUringEventLoop ioUringEventLoop = (IOUringEventLoop) eventLoop(); IOUringSubmissionQueue submissionQueue = submissionQueue();
IOUringSubmissionQueue submissionQueue = ioUringEventLoop.getRingBuffer().getIoUringSubmissionQueue();
submissionQueue.addPollIn(socket.intValue()); submissionQueue.addPollIn(socket.intValue());
submissionQueue.submit(); submissionQueue.submit();
} }
@ -413,11 +426,26 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
readComplete0(res); readComplete0(res);
} }
abstract void readComplete0(int res); protected abstract void readComplete0(int res);
/**
* Called once POLLRDHUP event is ready to be processed
*/
final void pollRdHup(int res) {
if (isActive()) {
// If it is still active, we need to call epollInReady as otherwise we may miss to
// read pending data from the underlying file descriptor.
// See https://github.com/netty/netty/issues/3709
pollIn(res);
} else {
// Just to be safe make sure the input marked as closed.
shutdownInput(true);
}
}
abstract void pollIn(int res); abstract void pollIn(int res);
void pollOut(int res) { final void pollOut(int res) {
// pending connect // pending connect
if (connectPromise != null) { if (connectPromise != null) {
// Note this method is invoked by the event loop only if the connection attempt was // Note this method is invoked by the event loop only if the connection attempt was
@ -440,50 +468,55 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
if (!connectStillInProgress) { if (!connectStillInProgress) {
// Check for null as the connectTimeoutFuture is only created if a connectTimeoutMillis > 0 is used // Check for null as the connectTimeoutFuture is only created if a connectTimeoutMillis > 0 is used
// See https://github.com/netty/netty/issues/1770 // See https://github.com/netty/netty/issues/1770
if (connectTimeoutFuture != null) { cancelConnectTimeoutFuture();
connectTimeoutFuture.cancel(false);
}
connectPromise = null; connectPromise = null;
} }
} }
} else if (!getSocket().isOutputShutdown()) {
doWrite(unsafe().outboundBuffer());
} }
} }
void writeComplete(int res) { final void writeComplete(int res) {
writeScheduled = false; writeScheduled = false;
ChannelOutboundBuffer channelOutboundBuffer = unsafe().outboundBuffer(); ChannelOutboundBuffer channelOutboundBuffer = unsafe().outboundBuffer();
if (iovecMemoryAddress != 0) { if (iovecMemoryAddress != -1) {
((IOUringEventLoop) eventLoop()).getIovecArrayPool().releaseIovec(iovecMemoryAddress); ((IOUringEventLoop) eventLoop()).getIovecArrayPool().releaseIovec(iovecMemoryAddress);
iovecMemoryAddress = -1;
} }
if (res > 0) { if (res >= 0) {
channelOutboundBuffer.removeBytes(res); channelOutboundBuffer.removeBytes(res);
doWrite(channelOutboundBuffer);
} else {
try { try {
doWrite(channelOutboundBuffer); if (ioResult("io_uring write", res) == 0) {
} catch (Exception e) { // We were not able to write everything, let's register for POLLOUT
e.printStackTrace(); addPollOut();
}
} catch (Throwable cause) {
handleWriteError(cause);
} }
} }
} }
final void connectComplete(int res) {
void connectComplete(int res) {
if (res == 0) { if (res == 0) {
fulfillConnectPromise(connectPromise, active); fulfillConnectPromise(connectPromise, active);
} else { } else {
if (res == ERRNO_EINPROGRESS_NEGATIVE) { if (res == ERRNO_EINPROGRESS_NEGATIVE) {
// connect not complete yet need to wait for poll_out event // connect not complete yet need to wait for poll_out event
IOUringSubmissionQueue submissionQueue = submissionQueue(); addPollOut();
submissionQueue.addPollOut(fd().intValue());
submissionQueue.submit();
} else { } else {
/* try {
if (res == -1 || res == -4) { Errors.throwConnectException("io_uring connect", res);
submissionQueue.addConnect(fd, channel.getRemoteAddressMemoryAddress(), } catch (Throwable cause) {
AbstractIOUringChannel.SOCK_ADDR_LEN); fulfillConnectPromise(connectPromise, cause);
submissionQueue.submit(); } finally {
break; // Check for null as the connectTimeoutFuture is only created if a connectTimeoutMillis > 0 is used
} // See https://github.com/netty/netty/issues/1770
*/ cancelConnectTimeoutFuture();
connectPromise = null;
}
} }
} }
} }
@ -504,9 +537,7 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
InetSocketAddress inetSocketAddress = (InetSocketAddress) remoteAddress; InetSocketAddress inetSocketAddress = (InetSocketAddress) remoteAddress;
NativeInetAddress address = NativeInetAddress.newInstance(inetSocketAddress.getAddress()); NativeInetAddress address = NativeInetAddress.newInstance(inetSocketAddress.getAddress());
socket.initAddress(address.address(), address.scopeId(), inetSocketAddress.getPort(),remoteAddressMemoryAddress); socket.initAddress(address.address(), address.scopeId(), inetSocketAddress.getPort(),remoteAddressMemoryAddress);
IOUringEventLoop ioUringEventLoop = (IOUringEventLoop) eventLoop(); final IOUringSubmissionQueue ioUringSubmissionQueue = submissionQueue();
final IOUringSubmissionQueue ioUringSubmissionQueue =
ioUringEventLoop.getRingBuffer().getIoUringSubmissionQueue();
ioUringSubmissionQueue.addConnect(socket.intValue(), remoteAddressMemoryAddress, SOCK_ADDR_LEN); ioUringSubmissionQueue.addConnect(socket.intValue(), remoteAddressMemoryAddress, SOCK_ADDR_LEN);
ioUringSubmissionQueue.submit(); ioUringSubmissionQueue.submit();
@ -537,9 +568,7 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
@Override @Override
public void operationComplete(ChannelFuture future) throws Exception { public void operationComplete(ChannelFuture future) throws Exception {
if (future.isCancelled()) { if (future.isCancelled()) {
if (connectTimeoutFuture != null) { cancelConnectTimeoutFuture();
connectTimeoutFuture.cancel(false);
}
connectPromise = null; connectPromise = null;
close(voidPromise()); close(voidPromise());
} }
@ -603,7 +632,7 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
/** /**
* Connect to the remote peer * Connect to the remote peer
*/ */
protected void doConnect(SocketAddress remoteAddress, SocketAddress localAddress) throws Exception { private void doConnect(SocketAddress remoteAddress, SocketAddress localAddress) throws Exception {
if (localAddress instanceof InetSocketAddress) { if (localAddress instanceof InetSocketAddress) {
checkResolvable((InetSocketAddress) localAddress); checkResolvable((InetSocketAddress) localAddress);
} }
@ -626,87 +655,16 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
} }
} }
// public void setRemote() {
// remote = remoteSocketAddr == null ?
// remoteAddress : computeRemoteAddr(remoteSocketAddr, socket.remoteAddress());
// }
private boolean doConnect0(SocketAddress remote) throws Exception {
boolean success = false;
try {
boolean connected = socket.connect(remote);
if (!connected) {
//setFlag(Native.EPOLLOUT);
}
success = true;
return connected;
} finally {
if (!success) {
doClose();
}
}
}
void shutdownInput(boolean rdHup) {
logger.trace("shutdownInput Fd: {}", this.socket.intValue());
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(voidPromise());
}
} else if (!rdHup) {
inputClosedSeenErrorOnRead = true;
pipeline().fireUserEventTriggered(ChannelInputShutdownReadComplete.INSTANCE);
}
}
//Todo we should move it to a error class
// copy unix Errors
static void throwConnectException(String method, int err)
throws IOException {
if (err == ERROR_EALREADY_NEGATIVE) {
throw new ConnectionPendingException();
}
if (err == ERROR_ENETUNREACH_NEGATIVE) {
throw new NoRouteToHostException();
}
if (err == ERROR_EISCONN_NEGATIVE) {
throw new AlreadyConnectedException();
}
if (err == ERRNO_ENOENT_NEGATIVE) {
throw new FileNotFoundException();
}
throw new ConnectException(method + "(..) failed: ");
}
private static boolean isAllowHalfClosure(ChannelConfig config) { private static boolean isAllowHalfClosure(ChannelConfig config) {
return config instanceof SocketChannelConfig && return config instanceof SocketChannelConfig &&
((SocketChannelConfig) config).isAllowHalfClosure(); ((SocketChannelConfig) config).isAllowHalfClosure();
} }
private void fireEventAndClose(Object evt) { private void cancelConnectTimeoutFuture() {
pipeline().fireUserEventTriggered(evt);
close(voidPromise());
}
void cancelTimeoutFuture() {
if (connectTimeoutFuture != null) { if (connectTimeoutFuture != null) {
connectTimeoutFuture.cancel(false); connectTimeoutFuture.cancel(false);
connectTimeoutFuture = null;
} }
connectPromise = null;
} }
private boolean doFinishConnect() throws Exception { private boolean doFinishConnect() throws Exception {
@ -718,35 +676,17 @@ abstract class AbstractIOUringChannel extends AbstractChannel implements UnixCha
return true; return true;
} }
IOUringSubmissionQueue submissionQueue = submissionQueue(); addPollOut();
submissionQueue.addPollOut(fd().intValue());
submissionQueue.submit();
return false; return false;
} }
void computeRemote() { private void computeRemote() {
if (requestedRemoteAddress instanceof InetSocketAddress) { if (requestedRemoteAddress instanceof InetSocketAddress) {
remote = computeRemoteAddr((InetSocketAddress) requestedRemoteAddress, socket.remoteAddress()); remote = computeRemoteAddr((InetSocketAddress) requestedRemoteAddress, socket.remoteAddress());
} }
} }
final boolean shouldBreakIoUringInReady(ChannelConfig config) { private boolean shouldBreakIoUringInReady(ChannelConfig config) {
return socket.isInputShutdown() && (inputClosedSeenErrorOnRead || !isAllowHalfClosure(config)); return socket.isInputShutdown() && (inputClosedSeenErrorOnRead || !isAllowHalfClosure(config));
} }
public long getRemoteAddressMemoryAddress() {
return remoteAddressMemoryAddress;
}
public ChannelPromise getConnectPromise() {
return connectPromise;
}
public ScheduledFuture<?> getConnectTimeoutFuture() {
return connectTimeoutFuture;
}
public SocketAddress getRequestedRemoteAddress() {
return requestedRemoteAddress;
}
} }

View File

@ -21,9 +21,12 @@ import io.netty.channel.ChannelOutboundBuffer;
import io.netty.channel.ChannelPipeline; import io.netty.channel.ChannelPipeline;
import io.netty.channel.ChannelPromise; import io.netty.channel.ChannelPromise;
import io.netty.channel.ServerChannel; import io.netty.channel.ServerChannel;
import io.netty.channel.unix.Errors;
import java.net.SocketAddress; import java.net.SocketAddress;
import static io.netty.channel.unix.Errors.*;
abstract class AbstractIOUringServerChannel extends AbstractIOUringChannel implements ServerChannel { abstract class AbstractIOUringServerChannel extends AbstractIOUringChannel implements ServerChannel {
AbstractIOUringServerChannel(int fd) { AbstractIOUringServerChannel(int fd) {
@ -40,7 +43,7 @@ abstract class AbstractIOUringServerChannel extends AbstractIOUringChannel imple
} }
@Override @Override
protected void doWrite(ChannelOutboundBuffer in) throws Exception { protected void doWrite(ChannelOutboundBuffer in) {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@ -53,41 +56,43 @@ abstract class AbstractIOUringServerChannel extends AbstractIOUringChannel imple
final class UringServerChannelUnsafe extends AbstractIOUringChannel.AbstractUringUnsafe { final class UringServerChannelUnsafe extends AbstractIOUringChannel.AbstractUringUnsafe {
private final byte[] acceptedAddress = new byte[26]; private final byte[] acceptedAddress = new byte[26];
private void acceptSocket() {
@Override
void pollIn(int res) {
IOUringSubmissionQueue submissionQueue = submissionQueue(); IOUringSubmissionQueue submissionQueue = submissionQueue();
//Todo get network addresses //Todo get network addresses
submissionQueue.addAccept(fd().intValue()); submissionQueue.addAccept(fd().intValue());
submissionQueue.submit(); submissionQueue.submit();
} }
void readComplete0(int res) { @Override
if (res >= 0) { void pollIn(int res) {
final IOUringRecvByteAllocatorHandle allocHandle = acceptSocket();
(IOUringRecvByteAllocatorHandle) unsafe() }
.recvBufAllocHandle();
final ChannelPipeline pipeline = pipeline();
allocHandle.incMessagesRead(1); // TODO: Respect MAX_MESSAGES_READ
try { protected void readComplete0(int res) {
final Channel childChannel = newChildChannel(res); final IOUringRecvByteAllocatorHandle allocHandle =
(IOUringRecvByteAllocatorHandle) unsafe()
// all childChannels should poll POLLRDHUP .recvBufAllocHandle();
IOUringSubmissionQueue submissionQueue = submissionQueue(); final ChannelPipeline pipeline = pipeline();
submissionQueue.addPollRdHup(res); if (res >= 0) {
submissionQueue.submit(); allocHandle.incMessagesRead(1);
try {
pipeline.fireChannelRead(childChannel); pipeline.fireChannelRead(newChildChannel(res));
} catch (Exception e) { } catch (Throwable cause) {
e.printStackTrace(); allocHandle.readComplete();
pipeline.fireExceptionCaught(cause);
pipeline.fireChannelReadComplete();
}
acceptSocket();
} else {
allocHandle.readComplete();
// Check if we did fail because there was nothing to accept atm.
if (res != ERRNO_EAGAIN_NEGATIVE && res != ERRNO_EWOULDBLOCK_NEGATIVE) {
// Something bad happened. Convert to an exception.
pipeline.fireExceptionCaught(Errors.newIOException("io_uring accept", res));
}
pipeline.fireChannelReadComplete();
} }
allocHandle.readComplete();
pipeline.fireChannelReadComplete();
} else {
// TODO: Fix me
schedulePollIn();
}
} }
@Override @Override
@ -96,10 +101,5 @@ abstract class AbstractIOUringServerChannel extends AbstractIOUringChannel imple
promise.setFailure(new UnsupportedOperationException()); promise.setFailure(new UnsupportedOperationException());
} }
} }
@Override
protected void doClose() throws Exception {
super.doClose();
}
} }

View File

@ -34,6 +34,8 @@ import java.net.SocketAddress;
import java.io.IOException; import java.io.IOException;
import java.util.concurrent.Executor; import java.util.concurrent.Executor;
import static io.netty.channel.unix.Errors.ioResult;
abstract class AbstractIOUringStreamChannel extends AbstractIOUringChannel implements DuplexChannel { abstract class AbstractIOUringStreamChannel extends AbstractIOUringChannel implements DuplexChannel {
private static final InternalLogger logger = InternalLoggerFactory.getInstance(AbstractIOUringStreamChannel.class); private static final InternalLogger logger = InternalLoggerFactory.getInstance(AbstractIOUringStreamChannel.class);
@ -47,18 +49,15 @@ abstract class AbstractIOUringStreamChannel extends AbstractIOUringChannel imple
AbstractIOUringStreamChannel(Channel parent, LinuxSocket fd, SocketAddress remote) { AbstractIOUringStreamChannel(Channel parent, LinuxSocket fd, SocketAddress remote) {
super(parent, fd, remote); super(parent, fd, remote);
// Add EPOLLRDHUP so we are notified once the remote peer close the connection.
} }
@Override @Override
public ChannelFuture shutdown() { public ChannelFuture shutdown() {
System.out.println("AbstractStreamChannel shutdown");
return shutdown(newPromise()); return shutdown(newPromise());
} }
@Override @Override
public ChannelFuture shutdown(final ChannelPromise promise) { public ChannelFuture shutdown(final ChannelPromise promise) {
ChannelFuture shutdownOutputFuture = shutdownOutput(); ChannelFuture shutdownOutputFuture = shutdownOutput();
if (shutdownOutputFuture.isDone()) { if (shutdownOutputFuture.isDone()) {
shutdownOutputDone(shutdownOutputFuture, promise); shutdownOutputDone(shutdownOutputFuture, promise);
@ -173,7 +172,6 @@ abstract class AbstractIOUringStreamChannel extends AbstractIOUringChannel imple
private static void shutdownDone(ChannelFuture shutdownOutputFuture, private static void shutdownDone(ChannelFuture shutdownOutputFuture,
ChannelFuture shutdownInputFuture, ChannelFuture shutdownInputFuture,
ChannelPromise promise) { ChannelPromise promise) {
System.out.println("AbstractStreamChannel ShutdownDone");
Throwable shutdownOutputCause = shutdownOutputFuture.cause(); Throwable shutdownOutputCause = shutdownOutputFuture.cause();
Throwable shutdownInputCause = shutdownInputFuture.cause(); Throwable shutdownInputCause = shutdownInputFuture.cause();
if (shutdownOutputCause != null) { if (shutdownOutputCause != null) {
@ -189,6 +187,15 @@ abstract class AbstractIOUringStreamChannel extends AbstractIOUringChannel imple
} }
} }
@Override
protected void doRegister() throws Exception {
super.doRegister();
// all non-server channels should poll POLLRDHUP
IOUringSubmissionQueue submissionQueue = submissionQueue();
submissionQueue.addPollRdHup(fd().intValue());
submissionQueue.submit();
}
class IOUringStreamUnsafe extends AbstractUringUnsafe { class IOUringStreamUnsafe extends AbstractUringUnsafe {
// Overridden here just to be able to access this method from AbstractEpollStreamChannel // Overridden here just to be able to access this method from AbstractEpollStreamChannel
@ -197,8 +204,14 @@ abstract class AbstractIOUringStreamChannel extends AbstractIOUringChannel imple
return super.prepareToClose(); return super.prepareToClose();
} }
private ByteBuf readBuffer;
@Override @Override
void pollIn(int res) { void pollIn(int res) {
readFromSocket();
}
private void readFromSocket() {
final ChannelConfig config = config(); final ChannelConfig config = config();
final ByteBufAllocator allocator = config.getAllocator(); final ByteBufAllocator allocator = config.getAllocator();
@ -206,44 +219,42 @@ abstract class AbstractIOUringStreamChannel extends AbstractIOUringChannel imple
allocHandle.reset(config); allocHandle.reset(config);
ByteBuf byteBuf = allocHandle.allocate(allocator); ByteBuf byteBuf = allocHandle.allocate(allocator);
doReadBytes(byteBuf); IOUringSubmissionQueue submissionQueue = submissionQueue();
}
private ByteBuf readBuffer;
public void doReadBytes(ByteBuf byteBuf) {
assert readBuffer == null;
IOUringEventLoop ioUringEventLoop = (IOUringEventLoop) eventLoop();
IOUringSubmissionQueue submissionQueue = ioUringEventLoop.getRingBuffer().getIoUringSubmissionQueue();
unsafe().recvBufAllocHandle().attemptedBytesRead(byteBuf.writableBytes()); unsafe().recvBufAllocHandle().attemptedBytesRead(byteBuf.writableBytes());
if (byteBuf.hasMemoryAddress()) { assert readBuffer == null;
readBuffer = byteBuf; readBuffer = byteBuf;
submissionQueue.addRead(socket.intValue(), byteBuf.memoryAddress(), submissionQueue.addRead(socket.intValue(), byteBuf.memoryAddress(),
byteBuf.writerIndex(), byteBuf.capacity()); byteBuf.writerIndex(), byteBuf.capacity());
submissionQueue.submit(); submissionQueue.submit();
}
} }
void readComplete0(int localReadAmount) { // TODO: Respect MAX_MESSAGE_READ.
protected void readComplete0(int res) {
boolean close = false; boolean close = false;
ByteBuf byteBuf = null;
final IOUringRecvByteAllocatorHandle allocHandle = final IOUringRecvByteAllocatorHandle allocHandle =
(IOUringRecvByteAllocatorHandle) unsafe() (IOUringRecvByteAllocatorHandle) unsafe()
.recvBufAllocHandle(); .recvBufAllocHandle();
final ChannelPipeline pipeline = pipeline(); final ChannelPipeline pipeline = pipeline();
ByteBuf byteBuf = this.readBuffer;
this.readBuffer = null;
assert byteBuf != null;
boolean writable = true;
try { try {
logger.trace("EventLoop Read Res: {}", localReadAmount); if (res < 0) {
logger.trace("EventLoop Fd: {}", fd().intValue()); // If res is negative we should pass it to ioResult(...) which will either throw
byteBuf = this.readBuffer; // or convert it to 0 if we could not read because the socket was not readable.
this.readBuffer = null; allocHandle.lastBytesRead(ioResult("io_uring read", res));
} else if (res > 0) {
if (localReadAmount > 0) { byteBuf.writerIndex(byteBuf.writerIndex() + res);
byteBuf.writerIndex(byteBuf.writerIndex() + localReadAmount); allocHandle.lastBytesRead(res);
} else {
// EOF which we signal with -1.
allocHandle.lastBytesRead(-1);
} }
allocHandle.lastBytesRead(localReadAmount);
if (allocHandle.lastBytesRead() <= 0) { if (allocHandle.lastBytesRead() <= 0) {
// nothing was read, release the buffer. // nothing was read, release the buffer.
byteBuf.release(); byteBuf.release();
@ -259,13 +270,21 @@ abstract class AbstractIOUringStreamChannel extends AbstractIOUringChannel imple
} }
allocHandle.incMessagesRead(1); allocHandle.incMessagesRead(1);
writable = byteBuf.isWritable();
pipeline.fireChannelRead(byteBuf); pipeline.fireChannelRead(byteBuf);
byteBuf = null; byteBuf = null;
allocHandle.readComplete();
pipeline.fireChannelReadComplete();
} catch (Throwable t) { } catch (Throwable t) {
handleReadException(pipeline, byteBuf, t, close, allocHandle); handleReadException(pipeline, byteBuf, t, close, allocHandle);
} }
if (!close) {
if (!writable) {
// Let's schedule another read.
readFromSocket();
} else {
// We did not fill the whole ByteBuf so we should break the "read loop" and try again later.
pipeline.fireChannelReadComplete();
}
}
} }
private void handleReadException(ChannelPipeline pipeline, ByteBuf byteBuf, private void handleReadException(ChannelPipeline pipeline, ByteBuf byteBuf,

View File

@ -15,7 +15,6 @@
*/ */
package io.netty.channel.uring; package io.netty.channel.uring;
import io.netty.channel.unix.FileDescriptor;
import io.netty.util.internal.SystemPropertyUtil; import io.netty.util.internal.SystemPropertyUtil;
final class IOUring { final class IOUring {
@ -31,9 +30,8 @@ final class IOUring {
static final int OP_CONNECT = 16; static final int OP_CONNECT = 16;
static final int POLLMASK_IN = 1; static final int POLLMASK_IN = 1;
static final int POLLMASK_OUT_LINK = 4;
static final int POLLMASK_RDHUP = 8192;
static final int POLLMASK_OUT = 4; static final int POLLMASK_OUT = 4;
static final int POLLMASK_RDHUP = 8192;
static { static {
Throwable cause = null; Throwable cause = null;

View File

@ -55,7 +55,7 @@ final class IOUringCompletionQueue {
this.ringFd = ringFd; this.ringFd = ringFd;
} }
public int process(IOUringCompletionQueueCallback callback) throws Exception { public int process(IOUringCompletionQueueCallback callback) {
int i = 0; int i = 0;
for (;;) { for (;;) {
long head = toUnsignedLong(PlatformDependent.getIntVolatile(kHeadAddress)); long head = toUnsignedLong(PlatformDependent.getIntVolatile(kHeadAddress));

View File

@ -153,11 +153,7 @@ final class IOUringEventLoop extends SingleThreadEventLoop implements
} }
} }
try { completionQueue.process(this);
completionQueue.process(this);
} catch (Exception e) {
//Todo handle exception
}
if (hasTasks()) { if (hasTasks()) {
runAllTasks(); runAllTasks();
@ -201,15 +197,7 @@ final class IOUringEventLoop extends SingleThreadEventLoop implements
if (writeChannel == null) { if (writeChannel == null) {
break; break;
} }
//localFlushAmount -> res ((AbstractIOUringChannel.AbstractUringUnsafe) writeChannel.unsafe()).writeComplete(res);
logger.trace("EventLoop Write Res: {}", res);
logger.trace("EventLoop Fd: {}", fd);
if (res == SOCKET_ERROR_EPIPE) {
writeChannel.shutdownInput(false);
} else {
((AbstractIOUringChannel.AbstractUringUnsafe) writeChannel.unsafe()).writeComplete(res);
}
break; break;
case IOUring.IO_TIMEOUT: case IOUring.IO_TIMEOUT:
if (res == ETIME) { if (res == ETIME) {
@ -225,14 +213,7 @@ final class IOUringEventLoop extends SingleThreadEventLoop implements
} }
if (eventfd.intValue() == fd) { if (eventfd.intValue() == fd) {
pendingWakeup = false; pendingWakeup = false;
// We need to consume the data as otherwise we would see another event handleEventFd(submissionQueue);
// in the completionQueue without
// an extra eventfd_write(....)
Native.eventFdRead(eventfd.intValue());
submissionQueue.addPollIn(eventfd.intValue());
// Submit so its picked up
submissionQueue.submit();
} else { } else {
AbstractIOUringChannel channel = channels.get(fd); AbstractIOUringChannel channel = channels.get(fd);
if (channel == null) { if (channel == null) {
@ -246,9 +227,7 @@ final class IOUringEventLoop extends SingleThreadEventLoop implements
((AbstractIOUringChannel.AbstractUringUnsafe) channel.unsafe()).pollOut(res); ((AbstractIOUringChannel.AbstractUringUnsafe) channel.unsafe()).pollOut(res);
break; break;
case IOUring.POLLMASK_RDHUP: case IOUring.POLLMASK_RDHUP:
if (!channel.isActive()) { ((AbstractIOUringChannel.AbstractUringUnsafe) channel.unsafe()).pollRdHup(res);
channel.shutdownInput(true);
}
break; break;
default: default:
break; break;
@ -266,18 +245,27 @@ final class IOUringEventLoop extends SingleThreadEventLoop implements
case IOUring.OP_CONNECT: case IOUring.OP_CONNECT:
AbstractIOUringChannel channel = channels.get(fd); AbstractIOUringChannel channel = channels.get(fd);
System.out.println("Connect res: " + res);
if (channel != null) { if (channel != null) {
((AbstractIOUringChannel.AbstractUringUnsafe) channel.unsafe()).connectComplete(res); ((AbstractIOUringChannel.AbstractUringUnsafe) channel.unsafe()).connectComplete(res);
} }
break; break;
default: default:
break; break;
} }
return true; return true;
} }
private void handleEventFd(IOUringSubmissionQueue submissionQueue) {
// We need to consume the data as otherwise we would see another event
// in the completionQueue without
// an extra eventfd_write(....)
Native.eventFdRead(eventfd.intValue());
submissionQueue.addPollIn(eventfd.intValue());
// Submit so its picked up
submissionQueue.submit();
}
@Override @Override
protected void cleanup() { protected void cleanup() {
try { try {

View File

@ -126,12 +126,7 @@ final class IOUringSubmissionQueue {
long uData = convertToUserData(op, fd, pollMask); long uData = convertToUserData(op, fd, pollMask);
PlatformDependent.putLong(sqe + SQE_USER_DATA_FIELD, uData); PlatformDependent.putLong(sqe + SQE_USER_DATA_FIELD, uData);
//poll<link>read or accept operation PlatformDependent.putByte(sqe + SQE_FLAGS_FIELD, (byte) 0);
if (op == 6 && (pollMask == IOUring.POLLMASK_OUT_LINK)) {
PlatformDependent.putByte(sqe + SQE_FLAGS_FIELD, (byte) IOSQE_IO_LINK);
} else {
PlatformDependent.putByte(sqe + SQE_FLAGS_FIELD, (byte) 0);
}
//c union set Rw-Flags or accept_flags //c union set Rw-Flags or accept_flags
if (op != IOUring.OP_ACCEPT) { if (op != IOUring.OP_ACCEPT) {
@ -152,8 +147,6 @@ final class IOUringSubmissionQueue {
PlatformDependent.putInt(sqe + SQE_RW_FLAGS_FIELD, pollMask); PlatformDependent.putInt(sqe + SQE_RW_FLAGS_FIELD, pollMask);
} }
logger.trace("UserDataField: {}", PlatformDependent.getLong(sqe + SQE_USER_DATA_FIELD)); logger.trace("UserDataField: {}", PlatformDependent.getLong(sqe + SQE_USER_DATA_FIELD));
logger.trace("BufferAddress: {}", PlatformDependent.getLong(sqe + SQE_ADDRESS_FIELD)); logger.trace("BufferAddress: {}", PlatformDependent.getLong(sqe + SQE_ADDRESS_FIELD));
logger.trace("Length: {}", PlatformDependent.getInt(sqe + SQE_LEN_FIELD)); logger.trace("Length: {}", PlatformDependent.getInt(sqe + SQE_LEN_FIELD));
@ -174,10 +167,6 @@ final class IOUringSubmissionQueue {
return addPoll(fd, IOUring.POLLMASK_IN); return addPoll(fd, IOUring.POLLMASK_IN);
} }
public boolean addPollOutLink(int fd) {
return addPoll(fd, IOUring.POLLMASK_OUT_LINK);
}
public boolean addPollRdHup(int fd) { public boolean addPollRdHup(int fd) {
return addPoll(fd, IOUring.POLLMASK_RDHUP); return addPoll(fd, IOUring.POLLMASK_RDHUP);
} }

View File

@ -54,7 +54,7 @@ final class IovecArrayPool implements MessageProcessor {
count = 0; count = 0;
if (remainingIovec.empty()) { if (remainingIovec.empty()) {
//Todo allocate new Memory // Todo allocate new Memory
return -1; return -1;
} }
long index = remainingIovec.pop(); long index = remainingIovec.pop();
@ -135,7 +135,7 @@ final class IovecArrayPool implements MessageProcessor {
} }
@Override @Override
public boolean processMessage(Object msg) throws Exception { public boolean processMessage(Object msg) {
if (msg instanceof ByteBuf) { if (msg instanceof ByteBuf) {
ByteBuf buffer = (ByteBuf) msg; ByteBuf buffer = (ByteBuf) msg;
return add(buffer, buffer.readerIndex(), buffer.readableBytes()); return add(buffer, buffer.readerIndex(), buffer.readableBytes());

View File

@ -107,7 +107,7 @@ public final class Errors {
} }
} }
static void throwConnectException(String method, int err) public static void throwConnectException(String method, int err)
throws IOException { throws IOException {
if (err == ERROR_EALREADY_NEGATIVE) { if (err == ERROR_EALREADY_NEGATIVE) {
throw new ConnectionPendingException(); throw new ConnectionPendingException();

View File

@ -929,30 +929,34 @@ public abstract class AbstractChannel extends DefaultAttributeMap implements Cha
try { try {
doWrite(outboundBuffer); doWrite(outboundBuffer);
} catch (Throwable t) { } catch (Throwable t) {
if (t instanceof IOException && config().isAutoClose()) { handleWriteError(t);
/**
* Just call {@link #close(ChannelPromise, Throwable, boolean)} here which will take care of
* failing all flushed messages and also ensure the actual close of the underlying transport
* will happen before the promises are notified.
*
* This is needed as otherwise {@link #isActive()} , {@link #isOpen()} and {@link #isWritable()}
* may still return {@code true} even if the channel should be closed as result of the exception.
*/
initialCloseCause = t;
close(voidPromise(), t, newClosedChannelException(t), false);
} else {
try {
shutdownOutput(voidPromise(), t);
} catch (Throwable t2) {
initialCloseCause = t;
close(voidPromise(), t2, newClosedChannelException(t), false);
}
}
} finally { } finally {
inFlush0 = false; inFlush0 = false;
} }
} }
protected final void handleWriteError(Throwable t) {
if (t instanceof IOException && config().isAutoClose()) {
/**
* Just call {@link #close(ChannelPromise, Throwable, boolean)} here which will take care of
* failing all flushed messages and also ensure the actual close of the underlying transport
* will happen before the promises are notified.
*
* This is needed as otherwise {@link #isActive()} , {@link #isOpen()} and {@link #isWritable()}
* may still return {@code true} even if the channel should be closed as result of the exception.
*/
initialCloseCause = t;
close(voidPromise(), t, newClosedChannelException(t), false);
} else {
try {
shutdownOutput(voidPromise(), t);
} catch (Throwable t2) {
initialCloseCause = t;
close(voidPromise(), t2, newClosedChannelException(t), false);
}
}
}
private ClosedChannelException newClosedChannelException(Throwable cause) { private ClosedChannelException newClosedChannelException(Throwable cause) {
ClosedChannelException exception = new ClosedChannelException(); ClosedChannelException exception = new ClosedChannelException();
if (cause != null) { if (cause != null) {