Refactor ChunkedWriteHandler to remove synchronization which can have bad side effects like deadlocks. See #297 and #301

Conflicts:

	handler/src/main/java/io/netty/handler/stream/ChunkedWriteHandler.java
This commit is contained in:
norman 2012-05-04 10:31:06 +02:00 committed by Trustin Lee
parent f8f703e679
commit 280c65a28e

View File

@ -15,30 +15,23 @@
*/ */
package io.netty.handler.stream; package io.netty.handler.stream;
import static io.netty.channel.Channels.*;
import java.io.IOException;
import java.nio.channels.ClosedChannelException;
import java.util.Queue;
import io.netty.buffer.ChannelBuffers; import io.netty.buffer.ChannelBuffers;
import io.netty.channel.Channel; import io.netty.channel.Channel;
import io.netty.channel.ChannelDownstreamHandler;
import io.netty.channel.ChannelEvent;
import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelPipeline; import io.netty.channel.ChannelPipeline;
import io.netty.channel.ChannelStateEvent;
import io.netty.channel.ChannelUpstreamHandler;
import io.netty.channel.Channels;
import io.netty.channel.LifeCycleAwareChannelHandler;
import io.netty.channel.MessageEvent;
import io.netty.logging.InternalLogger; import io.netty.logging.InternalLogger;
import io.netty.logging.InternalLoggerFactory; import io.netty.logging.InternalLoggerFactory;
import io.netty.util.internal.QueueFactory; import io.netty.util.internal.QueueFactory;
import java.io.IOException;
import java.nio.channels.Channels;
import java.nio.channels.ClosedChannelException;
import java.util.Queue;
import java.util.concurrent.atomic.AtomicBoolean;
/** /**
* A {@link ChannelHandler} that adds support for writing a large data stream * A {@link ChannelHandler} that adds support for writing a large data stream
* asynchronously neither spending a lot of memory nor getting * asynchronously neither spending a lot of memory nor getting
@ -79,7 +72,8 @@ public class ChunkedWriteHandler implements ChannelUpstreamHandler, ChannelDowns
private final Queue<MessageEvent> queue = QueueFactory.createQueue(MessageEvent.class); private final Queue<MessageEvent> queue = QueueFactory.createQueue(MessageEvent.class);
private ChannelHandlerContext ctx; private volatile ChannelHandlerContext ctx;
private final AtomicBoolean flush = new AtomicBoolean(false);
private MessageEvent currentEvent; private MessageEvent currentEvent;
/** /**
@ -145,21 +139,21 @@ public class ChunkedWriteHandler implements ChannelUpstreamHandler, ChannelDowns
private void discard(ChannelHandlerContext ctx, boolean fireNow) { private void discard(ChannelHandlerContext ctx, boolean fireNow) {
ClosedChannelException cause = null; ClosedChannelException cause = null;
boolean fireExceptionCaught = false; boolean fireExceptionCaught = false;
for (;;) { for (;;) {
MessageEvent currentEvent = this.currentEvent; MessageEvent currentEvent = this.currentEvent;
if (this.currentEvent == null) { if (this.currentEvent == null) {
currentEvent = queue.poll(); currentEvent = queue.poll();
} else { } else {
this.currentEvent = null; this.currentEvent = null;
} }
if (currentEvent == null) { if (currentEvent == null) {
break; break;
} }
Object m = currentEvent.getMessage(); Object m = currentEvent.getMessage();
if (m instanceof ChunkedInput) { if (m instanceof ChunkedInput) {
closeInput((ChunkedInput) m); closeInput((ChunkedInput) m);
@ -172,7 +166,7 @@ public class ChunkedWriteHandler implements ChannelUpstreamHandler, ChannelDowns
currentEvent.getFuture().setFailure(cause); currentEvent.getFuture().setFailure(cause);
fireExceptionCaught = true; fireExceptionCaught = true;
} }
if (fireExceptionCaught) { if (fireExceptionCaught) {
if (fireNow) { if (fireNow) {
@ -183,96 +177,122 @@ public class ChunkedWriteHandler implements ChannelUpstreamHandler, ChannelDowns
} }
} }
private synchronized void flush(ChannelHandlerContext ctx, boolean fireNow) throws Exception { private void flush(ChannelHandlerContext ctx, boolean fireNow) throws Exception {
final Channel channel = ctx.channel(); boolean acquired = false;
if (!channel.isConnected()) { final Channel channel = ctx.getChannel();
discard(ctx, fireNow);
// use CAS to see if the have flush already running, if so we don't need to take futher actions
if (acquired = flush.compareAndSet(false, true)) {
try {
if (!channel.isConnected()) {
discard(ctx, fireNow);
}
while (channel.isWritable()) {
if (currentEvent == null) {
currentEvent = queue.poll();
}
if (currentEvent == null) {
break;
}
if (currentEvent.getFuture().isDone()) {
// Skip the current request because the previous partial write
// attempt for the current request has been failed.
currentEvent = null;
} else {
final MessageEvent currentEvent = this.currentEvent;
Object m = currentEvent.getMessage();
if (m instanceof ChunkedInput) {
final ChunkedInput chunks = (ChunkedInput) m;
Object chunk;
boolean endOfInput;
boolean suspend;
try {
chunk = chunks.nextChunk();
endOfInput = chunks.isEndOfInput();
if (chunk == null) {
chunk = ChannelBuffers.EMPTY_BUFFER;
// No need to suspend when reached at the end.
suspend = !endOfInput;
} else {
suspend = false;
}
} catch (Throwable t) {
this.currentEvent = null;
currentEvent.getFuture().setFailure(t);
if (fireNow) {
fireExceptionCaught(ctx, t);
} else {
fireExceptionCaughtLater(ctx, t);
}
closeInput(chunks);
break;
}
if (suspend) {
// ChunkedInput.nextChunk() returned null and it has
// not reached at the end of input. Let's wait until
// more chunks arrive. Nothing to write or notify.
break;
} else {
ChannelFuture writeFuture;
if (endOfInput) {
this.currentEvent = null;
writeFuture = currentEvent.getFuture();
// Register a listener which will close the input once the write is complete. This is needed because the Chunk may have
// some resource bound that can not be closed before its not written
//
// See https://github.com/netty/netty/issues/303
writeFuture.addListener(new ChannelFutureListener() {
@Override
public void operationComplete(ChannelFuture future) throws Exception {
closeInput(chunks);
}
});
} else {
writeFuture = future(channel);
writeFuture.addListener(new ChannelFutureListener() {
@Override
public void operationComplete(ChannelFuture future) throws Exception {
if (!future.isSuccess()) {
currentEvent.getFuture().setFailure(future.getCause());
closeInput((ChunkedInput) currentEvent.getMessage());
}
}
});
}
Channels.write(
ctx, writeFuture, chunk,
currentEvent.getRemoteAddress());
}
} else {
this.currentEvent = null;
ctx.sendDownstream(currentEvent);
}
}
if (!channel.isConnected()) {
discard(ctx, fireNow);
break;
}
}
} finally {
// mark the flush as done
flush.set(false);
}
} }
while (channel.isWritable()) { if (acquired && !channel.isConnected() || channel.isWritable() && !queue.isEmpty()) {
if (currentEvent == null) { flush(ctx, fireNow);
currentEvent = queue.poll();
}
if (currentEvent == null) {
break;
}
if (currentEvent.getFuture().isDone()) {
// Skip the current request because the previous partial write
// attempt for the current request has been failed.
currentEvent = null;
} else {
final MessageEvent currentEvent = this.currentEvent;
Object m = currentEvent.getMessage();
if (m instanceof ChunkedInput) {
ChunkedInput chunks = (ChunkedInput) m;
Object chunk;
boolean endOfInput;
boolean suspend;
try {
chunk = chunks.nextChunk();
endOfInput = chunks.isEndOfInput();
if (chunk == null) {
chunk = ChannelBuffers.EMPTY_BUFFER;
// No need to suspend when reached at the end.
suspend = !endOfInput;
} else {
suspend = false;
}
} catch (Throwable t) {
this.currentEvent = null;
currentEvent.getFuture().setFailure(t);
if (fireNow) {
fireExceptionCaught(ctx, t);
} else {
fireExceptionCaughtLater(ctx, t);
}
closeInput(chunks);
break;
}
if (suspend) {
// ChunkedInput.nextChunk() returned null and it has
// not reached at the end of input. Let's wait until
// more chunks arrive. Nothing to write or notify.
break;
} else {
ChannelFuture writeFuture;
if (endOfInput) {
this.currentEvent = null;
closeInput(chunks);
writeFuture = currentEvent.getFuture();
} else {
writeFuture = future(channel);
writeFuture.addListener(new ChannelFutureListener() {
@Override
public void operationComplete(ChannelFuture future)
throws Exception {
if (!future.isSuccess()) {
currentEvent.getFuture().setFailure(future.cause());
closeInput((ChunkedInput) currentEvent.getMessage());
}
}
});
}
Channels.write(
ctx, writeFuture, chunk,
currentEvent.getRemoteAddress());
}
} else {
this.currentEvent = null;
ctx.sendDownstream(currentEvent);
}
}
if (!channel.isConnected()) {
discard(ctx, fireNow);
break;
}
} }
} }
@ -285,18 +305,18 @@ public class ChunkedWriteHandler implements ChannelUpstreamHandler, ChannelDowns
} }
} }
} }
@Override @Override
public void beforeAdd(ChannelHandlerContext ctx) throws Exception { public void beforeAdd(ChannelHandlerContext ctx) throws Exception {
// nothing to do // nothing to do
} }
@Override @Override
public void afterAdd(ChannelHandlerContext ctx) throws Exception { public void afterAdd(ChannelHandlerContext ctx) throws Exception {
// nothing to do // nothing to do
} }
@Override @Override