Eliminated unnecessary use of thread pool in the HTTP tunneling transport

This commit is contained in:
Trustin Lee 2009-06-30 10:30:48 +00:00
parent 012066cfa9
commit d35a8d29b1
5 changed files with 94 additions and 291 deletions

View File

@ -1,175 +0,0 @@
/*
* JBoss, Home of Professional Open Source
*
* Copyright 2008, Red Hat Middleware LLC, and individual contributors
* by the @author tags. See the COPYRIGHT.txt in the distribution for a
* full listing of individual contributors.
*
* This is free software; you can redistribute it and/or modify it
* under the terms of the GNU Lesser General Public License as
* published by the Free Software Foundation; either version 2.1 of
* the License, or (at your option) any later version.
*
* This software is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public
* License along with this software; if not, write to the Free
* Software Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA
* 02110-1301 USA, or see the FSF site: http://www.fsf.org.
*/
package org.jboss.netty.channel.socket.http;
import static org.jboss.netty.channel.Channels.*;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.channel.Channel;
import org.jboss.netty.channel.ChannelFuture;
/**
* @author The Netty Project (netty-dev@lists.jboss.org)
* @author Andy Taylor (andy.taylor@jboss.org)
* @author Trustin Lee (tlee@redhat.com)
* @version $Rev$, $Date$
*/
final class HttpTunnelWorker implements Runnable {
private final HttpTunnelingClientSocketChannel channel;
HttpTunnelWorker(HttpTunnelingClientSocketChannel channel) {
this.channel = channel;
}
public void run() {
channel.workerThread = Thread.currentThread();
while (channel.isOpen()) {
synchronized (channel.interestOpsLock) {
while (!channel.isReadable()) {
try {
// notify() is not called at all.
// close() and setInterestOps() calls Thread.interrupt()
channel.interestOpsLock.wait();
}
catch (InterruptedException e) {
if (!channel.isOpen()) {
break;
}
}
}
}
ChannelBuffer buf;
try {
buf = channel.receiveChunk();
}
catch (Throwable t) {
if (!channel.isOpen()) {
fireExceptionCaught(channel, t);
}
break;
}
if (buf != null) {
fireMessageReceived(channel, buf);
}
}
// Setting the workerThread to null will prevent any channel
// operations from interrupting this thread from now on.
channel.workerThread = null;
// Clean up.
close(channel, succeededFuture(channel));
}
static void write(
HttpTunnelingClientSocketChannel channel, ChannelFuture future,
Object message) {
try {
ChannelBuffer buf = (ChannelBuffer) message;
int writtenBytes = channel.sendChunk(buf);
future.setSuccess();
fireWriteComplete(channel, writtenBytes);
}
catch (Throwable t) {
future.setFailure(t);
fireExceptionCaught(channel, t);
}
}
static void setInterestOps(
HttpTunnelingClientSocketChannel channel, ChannelFuture future, int interestOps) {
// Override OP_WRITE flag - a user cannot change this flag.
interestOps &= ~Channel.OP_WRITE;
interestOps |= channel.getInterestOps() & Channel.OP_WRITE;
boolean changed = false;
try {
if (channel.getInterestOps() != interestOps) {
if ((interestOps & Channel.OP_READ) != 0) {
channel.setInterestOpsNow(Channel.OP_READ);
}
else {
channel.setInterestOpsNow(Channel.OP_NONE);
}
changed = true;
}
future.setSuccess();
if (changed) {
synchronized (channel.interestOpsLock) {
channel.setInterestOpsNow(interestOps);
// Notify the worker so it stops or continues reading.
Thread currentThread = Thread.currentThread();
Thread workerThread = channel.workerThread;
if (workerThread != null && currentThread != workerThread) {
workerThread.interrupt();
}
}
fireChannelInterestChanged(channel);
}
}
catch (Throwable t) {
future.setFailure(t);
fireExceptionCaught(channel, t);
}
}
static void close(HttpTunnelingClientSocketChannel channel, ChannelFuture future) {
boolean connected = channel.isConnected();
boolean bound = channel.isBound();
try {
channel.closeSocket();
if (channel.setClosed()) {
future.setSuccess();
if (connected) {
// Notify the worker so it stops reading.
Thread currentThread = Thread.currentThread();
Thread workerThread = channel.workerThread;
if (workerThread != null && currentThread != workerThread) {
workerThread.interrupt();
}
fireChannelDisconnected(channel);
}
if (bound) {
fireChannelUnbound(channel);
}
fireChannelClosed(channel);
} else {
future.setSuccess();
}
}
catch (Throwable t) {
future.setFailure(t);
fireExceptionCaught(channel, t);
}
}
}

View File

@ -27,9 +27,6 @@ import static org.jboss.netty.channel.Channels.*;
import java.net.InetSocketAddress;
import java.net.SocketAddress;
import java.net.URI;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import javax.net.ssl.SSLContext;
import javax.net.ssl.SSLEngine;
@ -43,6 +40,7 @@ import org.jboss.netty.channel.ChannelHandlerContext;
import org.jboss.netty.channel.ChannelPipeline;
import org.jboss.netty.channel.ChannelPipelineCoverage;
import org.jboss.netty.channel.ChannelSink;
import org.jboss.netty.channel.ChannelStateEvent;
import org.jboss.netty.channel.DefaultChannelPipeline;
import org.jboss.netty.channel.ExceptionEvent;
import org.jboss.netty.channel.MessageEvent;
@ -65,7 +63,6 @@ import org.jboss.netty.handler.codec.http.HttpVersion;
import org.jboss.netty.handler.ssl.SslHandler;
import org.jboss.netty.logging.InternalLogger;
import org.jboss.netty.logging.InternalLoggerFactory;
import org.jboss.netty.util.internal.LinkedTransferQueue;
/**
* @author The Netty Project (netty-dev@lists.jboss.org)
@ -82,7 +79,6 @@ class HttpTunnelingClientSocketChannel extends AbstractChannel
private static final String JSESSIONID = "JSESSIONID";
private final HttpTunnelingSocketChannelConfig config;
private final Lock reconnectLock = new ReentrantLock();
volatile boolean awaitingInitialResponse = true;
@ -95,8 +91,6 @@ class HttpTunnelingClientSocketChannel extends AbstractChannel
volatile boolean closed = false;
final BlockingQueue<ChannelBuffer> messages = new LinkedTransferQueue<ChannelBuffer>();
private final ClientSocketChannelFactory clientSocketChannelFactory;
volatile SocketChannel channel;
@ -138,17 +132,28 @@ class HttpTunnelingClientSocketChannel extends AbstractChannel
public boolean isConnected() {
return channel.isConnected();
}
@Override
public int getInterestOps() {
return channel.getInterestOps();
}
@Override
public boolean isWritable() {
return channel.isWritable();
}
@Override
public ChannelFuture setInterestOps(int interestOps) {
// TODO: Wrap the future.
return channel.setInterestOps(interestOps);
}
@Override
protected boolean setClosed() {
return super.setClosed();
}
@Override
protected void setInterestOpsNow(int interestOps) {
super.setInterestOpsNow(interestOps);
}
@Override
public ChannelFuture write(Object message, SocketAddress remoteAddress) {
if (remoteAddress == null || remoteAddress.equals(getRemoteAddress())) {
@ -231,38 +236,6 @@ class HttpTunnelingClientSocketChannel extends AbstractChannel
return size + hex.length() + HttpTunnelingClientSocketPipelineSink.LINE_TERMINATOR.length();
}
ChannelBuffer receiveChunk() {
ChannelBuffer buf = null;
try {
buf = messages.take();
}
catch (InterruptedException e) {
// Ignore
}
return buf;
}
void reconnect() throws Exception {
if (closed) {
throw new IllegalStateException("channel closed");
}
// XXX: What if a user writes something during the connection attempt?
if (reconnectLock.tryLock()) {
try {
awaitingInitialResponse = true;
connectAndSendHeaders(true, remoteAddress);
} finally {
reconnectLock.unlock();
}
} else {
try {
reconnectLock.lock();
} finally {
reconnectLock.unlock();
}
}
}
void closeSocket() {
if (setClosed()) {
// Send the end of chunk.
@ -295,10 +268,6 @@ class HttpTunnelingClientSocketChannel extends AbstractChannel
newSessionId = getSessionId(res, HttpHeaders.Names.SET_COOKIE2);
}
if (newSessionId == null) {
// XXX: Server does not support JSESSIONID?
}
// XXX: Utilize keep-alive if possible to reduce reconnection overhead.
// XXX: Consider non-200 status code.
// If the status code is not 200, no more reconnection attempt
@ -314,15 +283,13 @@ class HttpTunnelingClientSocketChannel extends AbstractChannel
} else {
ChannelBuffer content = res.getContent();
if (content.readable()) {
System.out.println("1: " + content.toString("ISO-8859-1"));
messages.offer(content);
fireMessageReceived(channel, content);
}
}
} else {
HttpChunk chunk = (HttpChunk) e.getMessage();
if (!chunk.isLast()) {
System.out.println("2: " + chunk.getContent());
messages.offer(chunk.getContent());
fireMessageReceived(channel, chunk.getContent());
} else {
readingChunks = false;
}
@ -330,6 +297,19 @@ class HttpTunnelingClientSocketChannel extends AbstractChannel
}
}
@Override
public void channelClosed(ChannelHandlerContext ctx, ChannelStateEvent e)
throws Exception {
if (sessionId != null) {
// TODO Reconnect.
} else {
// sessionId is null if:
// 1) A user closed the channel explicitly, or
// 2) The server does not support JSESSIONID.
channel.close();
}
}
@Override
public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) throws Exception {
fireExceptionCaught(

View File

@ -22,13 +22,10 @@
*/
package org.jboss.netty.channel.socket.http;
import java.util.concurrent.Executor;
import org.jboss.netty.channel.ChannelPipeline;
import org.jboss.netty.channel.ChannelSink;
import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
import org.jboss.netty.channel.socket.SocketChannel;
import org.jboss.netty.util.internal.ExecutorUtil;
/**
* Creates a client-side {@link SocketChannel} which connects to an
@ -42,39 +39,14 @@ import org.jboss.netty.util.internal.ExecutorUtil;
*/
public class HttpTunnelingClientSocketChannelFactory implements ClientSocketChannelFactory {
private final Executor workerExecutor;
private final ChannelSink sink;
ClientSocketChannelFactory clientSocketChannelFactory;
/**
*
* @param workerExecutor
*/
public HttpTunnelingClientSocketChannelFactory(ClientSocketChannelFactory clientSocketChannelFactory, Executor workerExecutor) {
this(workerExecutor, Runtime.getRuntime().availableProcessors());
this.clientSocketChannelFactory = clientSocketChannelFactory;
}
private final ChannelSink sink = new HttpTunnelingClientSocketPipelineSink();
private final ClientSocketChannelFactory clientSocketChannelFactory;
/**
* Creates a new instance.
*
* the {@link java.util.concurrent.Executor} which will execute the boss thread
* @param workerExecutor
* the {@link java.util.concurrent.Executor} which will execute the I/O worker threads
* @param workerCount
*/
public HttpTunnelingClientSocketChannelFactory(Executor workerExecutor, int workerCount) {
if (workerExecutor == null) {
throw new NullPointerException("workerExecutor");
}
if (workerCount <= 0) {
throw new IllegalArgumentException(
"workerCount (" + workerCount + ") " +
"must be a positive integer.");
}
this.workerExecutor = workerExecutor;
sink = new HttpTunnelingClientSocketPipelineSink(workerExecutor);
public HttpTunnelingClientSocketChannelFactory(ClientSocketChannelFactory clientSocketChannelFactory) {
this.clientSocketChannelFactory = clientSocketChannelFactory;
}
public SocketChannel newChannel(ChannelPipeline pipeline) {
@ -83,6 +55,5 @@ public class HttpTunnelingClientSocketChannelFactory implements ClientSocketChan
public void releaseExternalResources() {
clientSocketChannelFactory.releaseExternalResources();
ExecutorUtil.terminate(workerExecutor);
}
}

View File

@ -25,8 +25,8 @@ package org.jboss.netty.channel.socket.http;
import static org.jboss.netty.channel.Channels.*;
import java.net.SocketAddress;
import java.util.concurrent.Executor;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.channel.AbstractChannelSink;
import org.jboss.netty.channel.ChannelEvent;
import org.jboss.netty.channel.ChannelFuture;
@ -35,8 +35,6 @@ import org.jboss.netty.channel.ChannelPipeline;
import org.jboss.netty.channel.ChannelState;
import org.jboss.netty.channel.ChannelStateEvent;
import org.jboss.netty.channel.MessageEvent;
import org.jboss.netty.util.ThreadRenamingRunnable;
import org.jboss.netty.util.internal.IoWorkerRunnable;
/**
* @author The Netty Project (netty-dev@lists.jboss.org)
@ -47,10 +45,9 @@ import org.jboss.netty.util.internal.IoWorkerRunnable;
final class HttpTunnelingClientSocketPipelineSink extends AbstractChannelSink {
static final String LINE_TERMINATOR = "\r\n";
private final Executor workerExecutor;
HttpTunnelingClientSocketPipelineSink(Executor workerExecutor) {
this.workerExecutor = workerExecutor;
HttpTunnelingClientSocketPipelineSink() {
super();
}
public void eventSunk(
@ -64,31 +61,52 @@ final class HttpTunnelingClientSocketPipelineSink extends AbstractChannelSink {
switch (state) {
case OPEN:
if (Boolean.FALSE.equals(value)) {
HttpTunnelWorker.close(channel, future);
close(channel, future);
}
break;
case BOUND:
if (value != null) {
bind(channel, future, (SocketAddress) value);
} else {
HttpTunnelWorker.close(channel, future);
close(channel, future);
}
break;
case CONNECTED:
if (value != null) {
connect(channel, future, (HttpTunnelAddress) value);
} else {
HttpTunnelWorker.close(channel, future);
close(channel, future);
}
break;
case INTEREST_OPS:
HttpTunnelWorker.setInterestOps(channel, future, ((Integer) value).intValue());
final ChannelFuture actualFuture = future;
setInterestOps(channel.channel, ((Integer) value).intValue()).addListener(
new ChannelFutureListener() {
public void operationComplete(ChannelFuture future)
throws Exception {
if (future.isSuccess()) {
actualFuture.setSuccess();
} else {
actualFuture.setFailure(future.getCause());
}
}
});
break;
}
} else if (e instanceof MessageEvent) {
HttpTunnelWorker.write(
channel, future,
((MessageEvent) e).getMessage());
write(channel, (ChannelBuffer) ((MessageEvent) e).getMessage(), future);
}
}
private void write(HttpTunnelingClientSocketChannel channel, ChannelBuffer msg, ChannelFuture future) {
try {
int writtenBytes = channel.sendChunk(msg);
future.setSuccess();
fireWriteComplete(channel, writtenBytes);
}
catch (Throwable t) {
future.setFailure(t);
fireExceptionCaught(channel, t);
}
}
@ -110,8 +128,6 @@ final class HttpTunnelingClientSocketPipelineSink extends AbstractChannelSink {
HttpTunnelAddress remoteAddress) {
boolean bound = channel.isBound();
boolean connected = false;
boolean workerStarted = false;
future.addListener(ChannelFutureListener.CLOSE_ON_FAILURE);
@ -123,25 +139,36 @@ final class HttpTunnelingClientSocketPipelineSink extends AbstractChannelSink {
fireChannelBound(channel, channel.getLocalAddress());
}
fireChannelConnected(channel, channel.getRemoteAddress());
// Start the business.
workerExecutor.execute(
new IoWorkerRunnable(
new ThreadRenamingRunnable(
new HttpTunnelWorker(channel),
"HTTP Tunnel I/O client worker (channelId: " +
channel.getId() + ", " +
channel.getLocalAddress() + " => " +
channel.getRemoteAddress() + ')')));
workerStarted = true;
} catch (Throwable t) {
future.setFailure(t);
fireExceptionCaught(channel, t);
} finally {
if (connected && !workerStarted) {
HttpTunnelWorker.close(channel, future);
}
// FIXME: Rewrite exception handling.
}
}
private void close(
HttpTunnelingClientSocketChannel channel, ChannelFuture future) {
boolean connected = channel.isConnected();
boolean bound = channel.isBound();
try {
channel.closeSocket();
if (channel.setClosed()) {
future.setSuccess();
if (connected) {
fireChannelDisconnected(channel);
}
if (bound) {
fireChannelUnbound(channel);
}
fireChannelClosed(channel);
} else {
future.setSuccess();
}
}
catch (Throwable t) {
future.setFailure(t);
fireExceptionCaught(channel, t);
}
}
}

View File

@ -113,7 +113,7 @@ public class HttpTunnelingClientExample {
System.err.println("Only HTTP is supported.");
return;
}
HttpTunnelingClientSocketChannelFactory factory = new HttpTunnelingClientSocketChannelFactory(new OioClientSocketChannelFactory(Executors.newCachedThreadPool()), Executors.newCachedThreadPool());
HttpTunnelingClientSocketChannelFactory factory = new HttpTunnelingClientSocketChannelFactory(new OioClientSocketChannelFactory(Executors.newCachedThreadPool()));
ClientBootstrap bootstrap = new ClientBootstrap(factory);
bootstrap.getPipeline().addLast("decoder", new StringDecoder());
bootstrap.getPipeline().addLast("encoder", new StringEncoder());