Prefer MessageBuf over Queue where possible

- Also replaced thread safe queues with non-thread-safe ones where
  possible
- Unpooled.wrappedBuffer(Queue<T>) does not wrap MessageBuf anymore
This commit is contained in:
Trustin Lee 2012-06-12 17:02:00 +09:00
parent e1faea035e
commit ecd0ae5406
25 changed files with 117 additions and 92 deletions

View File

@ -113,6 +113,9 @@ public final class Unpooled {
}
public static <T> MessageBuf<T> wrappedBuffer(Queue<T> queue) {
if (queue instanceof MessageBuf) {
return (MessageBuf<T>) queue;
}
return new QueueBackedMessageBuf<T>(queue);
}

View File

@ -20,8 +20,8 @@ import io.netty.channel.Channel;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.CombinedChannelHandler;
import io.netty.handler.codec.PrematureChannelClosureException;
import io.netty.util.internal.QueueFactory;
import java.util.ArrayDeque;
import java.util.Queue;
import java.util.concurrent.atomic.AtomicLong;
@ -45,7 +45,7 @@ import java.util.concurrent.atomic.AtomicLong;
public class HttpClientCodec extends CombinedChannelHandler {
/** A queue that is used for correlating a request and a response. */
final Queue<HttpMethod> queue = QueueFactory.createQueue();
final Queue<HttpMethod> queue = new ArrayDeque<HttpMethod>();
/** If true, decoding stops (i.e. pass-through) */
volatile boolean done;

View File

@ -20,8 +20,8 @@ import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.embedded.EmbeddedByteChannel;
import io.netty.handler.codec.MessageToMessageCodec;
import io.netty.util.internal.QueueFactory;
import java.util.ArrayDeque;
import java.util.Queue;
/**
@ -48,7 +48,7 @@ import java.util.Queue;
*/
public abstract class HttpContentEncoder extends MessageToMessageCodec<HttpMessage, HttpMessage, Object, Object> {
private final Queue<String> acceptEncodingQueue = QueueFactory.createQueue();
private final Queue<String> acceptEncodingQueue = new ArrayDeque<String>();
private EmbeddedByteChannel encoder;
/**

View File

@ -15,12 +15,14 @@
*/
package io.netty.handler.codec.spdy;
import io.netty.util.internal.QueueFactory;
import java.util.Comparator;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.atomic.AtomicInteger;
final class SpdySession {
@ -176,8 +178,7 @@ final class SpdySession {
private final AtomicInteger sendWindowSize;
private final AtomicInteger receiveWindowSize;
private volatile int receiveWindowSizeLowerBound;
private final ConcurrentLinkedQueue<Object> pendingWriteQueue =
new ConcurrentLinkedQueue<Object>();
private final BlockingQueue<Object> pendingWriteQueue = QueueFactory.createQueue();
public StreamState(
byte priority, boolean remoteSideClosed, boolean localSideClosed,

View File

@ -25,7 +25,6 @@ import io.netty.channel.ChannelInboundMessageHandler;
import io.netty.channel.ChannelOutboundMessageHandler;
import java.nio.channels.ClosedChannelException;
import java.util.Queue;
import java.util.concurrent.atomic.AtomicInteger;
/**
@ -97,7 +96,7 @@ public class SpdySessionHandler
@Override
public void inboundBufferUpdated(ChannelHandlerContext ctx) throws Exception {
Queue<Object> in = ctx.inboundMessageBuffer();
MessageBuf<Object> in = ctx.inboundMessageBuffer();
for (;;) {
Object msg = in.poll();
if (msg == null) {
@ -443,7 +442,7 @@ public class SpdySessionHandler
@Override
public void flush(ChannelHandlerContext ctx, ChannelFuture future) throws Exception {
Queue<Object> in = ctx.outboundMessageBuffer();
MessageBuf<Object> in = ctx.outboundMessageBuffer();
for (;;) {
Object msg = in.poll();
if (msg == null) {

View File

@ -16,17 +16,16 @@
package io.netty.handler.codec;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.MessageBuf;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelOutboundMessageHandlerAdapter;
import java.util.Queue;
public abstract class MessageToByteEncoder<I> extends ChannelOutboundMessageHandlerAdapter<I> {
@Override
public void flush(ChannelHandlerContext ctx, ChannelFuture future) throws Exception {
Queue<I> in = ctx.outboundMessageBuffer();
MessageBuf<I> in = ctx.outboundMessageBuffer();
ByteBuf out = ctx.nextOutboundByteBuffer();
for (;;) {

View File

@ -21,8 +21,6 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.ChannelInboundMessageHandler;
import java.util.Queue;
public abstract class MessageToMessageDecoder<I, O>
extends ChannelInboundHandlerAdapter implements ChannelInboundMessageHandler<I> {
@ -34,7 +32,7 @@ public abstract class MessageToMessageDecoder<I, O>
@Override
public void inboundBufferUpdated(ChannelHandlerContext ctx)
throws Exception {
Queue<I> in = ctx.inboundMessageBuffer();
MessageBuf<I> in = ctx.inboundMessageBuffer();
boolean notify = false;
for (;;) {
try {

View File

@ -15,17 +15,16 @@
*/
package io.netty.handler.codec;
import io.netty.buffer.MessageBuf;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelOutboundMessageHandlerAdapter;
import java.util.Queue;
public abstract class MessageToMessageEncoder<I, O> extends ChannelOutboundMessageHandlerAdapter<I> {
@Override
public void flush(ChannelHandlerContext ctx, ChannelFuture future) throws Exception {
Queue<I> in = ctx.outboundMessageBuffer();
MessageBuf<I> in = ctx.outboundMessageBuffer();
for (;;) {
try {
Object msg = in.poll();

View File

@ -15,13 +15,13 @@
*/
package io.netty.example.factorial;
import io.netty.buffer.MessageBuf;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundMessageHandlerAdapter;
import java.math.BigInteger;
import java.util.Queue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.logging.Level;
@ -99,7 +99,7 @@ public class FactorialClientHandler extends ChannelInboundMessageHandlerAdapter<
private void sendNumbers() {
// Do not send more than 4096 numbers.
boolean finished = false;
Queue<Object> out = ctx.nextOutboundMessageBuffer();
MessageBuf<Object> out = ctx.nextOutboundMessageBuffer();
while (out.size() < 4096) {
if (i <= count) {
out.add(Integer.valueOf(i));

View File

@ -22,8 +22,6 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundMessageHandler;
import io.netty.channel.ChannelOutboundMessageHandler;
import java.util.Queue;
public class MessageLoggingHandler
extends LoggingHandler
implements ChannelInboundMessageHandler<Object>, ChannelOutboundMessageHandler<Object> {
@ -62,12 +60,12 @@ public class MessageLoggingHandler
@Override
public void inboundBufferUpdated(ChannelHandlerContext ctx)
throws Exception {
Queue<Object> buf = ctx.inboundMessageBuffer();
MessageBuf<Object> buf = ctx.inboundMessageBuffer();
if (logger.isEnabled(internalLevel)) {
logger.log(internalLevel, format(ctx, formatBuffer("RECEIVED", buf)));
}
Queue<Object> out = ctx.nextInboundMessageBuffer();
MessageBuf<Object> out = ctx.nextInboundMessageBuffer();
for (;;) {
Object o = buf.poll();
if (o == null) {
@ -81,12 +79,12 @@ public class MessageLoggingHandler
@Override
public void flush(ChannelHandlerContext ctx, ChannelFuture future)
throws Exception {
Queue<Object> buf = ctx.outboundMessageBuffer();
MessageBuf<Object> buf = ctx.outboundMessageBuffer();
if (logger.isEnabled(internalLevel)) {
logger.log(internalLevel, format(ctx, formatBuffer("WRITE", buf)));
}
Queue<Object> out = ctx.nextOutboundMessageBuffer();
MessageBuf<Object> out = ctx.nextOutboundMessageBuffer();
for (;;) {
Object o = buf.poll();
if (o == null) {
@ -97,7 +95,7 @@ public class MessageLoggingHandler
ctx.flush(future);
}
protected String formatBuffer(String message, Queue<Object> buf) {
protected String formatBuffer(String message, MessageBuf<Object> buf) {
return message + '(' + buf.size() + "): " + buf;
}
}

View File

@ -248,7 +248,7 @@ public class ServerBootstrap {
@Override
public void inboundBufferUpdated(ChannelHandlerContext ctx) {
Queue<Channel> in = ctx.inboundMessageBuffer();
MessageBuf<Channel> in = ctx.inboundMessageBuffer();
for (;;) {
Channel child = in.poll();
if (child == null) {

View File

@ -26,7 +26,6 @@ import java.net.SocketAddress;
import java.nio.channels.ClosedChannelException;
import java.util.ArrayDeque;
import java.util.Deque;
import java.util.Queue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@ -648,7 +647,7 @@ public abstract class AbstractChannel extends DefaultAttributeMap implements Cha
}
}
} else {
Queue<Object> out = ctx.outboundMessageBuffer();
MessageBuf<Object> out = ctx.outboundMessageBuffer();
int oldSize = out.size();
try {
doFlushMessageBuffer(out);
@ -717,7 +716,7 @@ public abstract class AbstractChannel extends DefaultAttributeMap implements Cha
protected void doFlushByteBuffer(ByteBuf buf) throws Exception {
throw new UnsupportedOperationException();
}
protected void doFlushMessageBuffer(Queue<Object> buf) throws Exception {
protected void doFlushMessageBuffer(MessageBuf<Object> buf) throws Exception {
throw new UnsupportedOperationException();
}

View File

@ -81,7 +81,7 @@ public abstract class AbstractServerChannel extends AbstractChannel implements S
}
@Override
protected void doFlushMessageBuffer(Queue<Object> buf) throws Exception {
protected void doFlushMessageBuffer(MessageBuf<Object> buf) throws Exception {
throw new UnsupportedOperationException();
}

View File

@ -30,7 +30,7 @@ public abstract class ChannelInboundMessageHandlerAdapter<I>
@Override
public final void inboundBufferUpdated(ChannelHandlerContext ctx) throws Exception {
Queue<I> in = ctx.inboundMessageBuffer();
MessageBuf<I> in = ctx.inboundMessageBuffer();
for (;;) {
I msg = in.poll();
if (msg == null) {

View File

@ -0,0 +1,42 @@
/*
* Copyright 2012 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:
*
* http://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;
public class IllegalBufferAccessException extends ChannelPipelineException {
private static final long serialVersionUID = -1313647533364810258L;
private static final String DEFAULT_MESSAGE =
"The buffers that belong to a " + ChannelHandlerContext.class.getSimpleName() +
" must be accessed from the thread of the associated " +
EventExecutor.class.getSimpleName() + ".";
public IllegalBufferAccessException() {
this(DEFAULT_MESSAGE);
}
public IllegalBufferAccessException(String message, Throwable cause) {
super(message, cause);
}
public IllegalBufferAccessException(String message) {
super(message);
}
public IllegalBufferAccessException(Throwable cause) {
super(cause);
}
}

View File

@ -100,7 +100,7 @@ public abstract class AbstractEmbeddedChannel extends AbstractChannel {
return state == 1;
}
public Queue<Object> lastInboundMessageBuffer() {
public MessageBuf<Object> lastInboundMessageBuffer() {
return lastInboundMessageBuffer;
}

View File

@ -15,11 +15,11 @@
*/
package io.netty.channel.embedded;
import io.netty.buffer.MessageBuf;
import io.netty.channel.ChannelBufferType;
import io.netty.channel.ChannelHandler;
import java.util.ArrayDeque;
import java.util.Queue;
public class EmbeddedMessageChannel extends AbstractEmbeddedChannel {
@ -32,13 +32,13 @@ public class EmbeddedMessageChannel extends AbstractEmbeddedChannel {
return ChannelBufferType.MESSAGE;
}
public Queue<Object> inboundBuffer() {
public MessageBuf<Object> inboundBuffer() {
return pipeline().inboundMessageBuffer();
}
@SuppressWarnings("unchecked")
public Queue<Object> lastOutboundBuffer() {
return (Queue<Object>) lastOutboundBuffer;
public MessageBuf<Object> lastOutboundBuffer() {
return (MessageBuf<Object>) lastOutboundBuffer;
}
public Object readOutbound() {
@ -66,13 +66,7 @@ public class EmbeddedMessageChannel extends AbstractEmbeddedChannel {
}
@Override
protected void doFlushMessageBuffer(Queue<Object> buf) throws Exception {
for (;;) {
Object o = buf.poll();
if (o == null) {
break;
}
lastOutboundBuffer().add(o);
}
protected void doFlushMessageBuffer(MessageBuf<Object> buf) throws Exception {
buf.drainTo(lastOutboundBuffer());
}
}

View File

@ -15,6 +15,7 @@
*/
package io.netty.channel.local;
import io.netty.buffer.MessageBuf;
import io.netty.channel.AbstractChannel;
import io.netty.channel.Channel;
import io.netty.channel.ChannelBufferType;
@ -31,7 +32,6 @@ import java.nio.channels.AlreadyConnectedException;
import java.nio.channels.ClosedChannelException;
import java.nio.channels.ConnectionPendingException;
import java.nio.channels.NotYetConnectedException;
import java.util.Queue;
/**
* A {@link Channel} for the local transport.
@ -203,7 +203,7 @@ public class LocalChannel extends AbstractChannel {
}
@Override
protected void doFlushMessageBuffer(Queue<Object> buf) throws Exception {
protected void doFlushMessageBuffer(MessageBuf<Object> buf) throws Exception {
if (state < 2) {
throw new NotYetConnectedException();
}
@ -214,14 +214,7 @@ public class LocalChannel extends AbstractChannel {
final LocalChannel peer = this.peer;
assert peer != null;
Queue<Object> out = peer.pipeline().inboundMessageBuffer();
for (;;) {
Object msg = buf.poll();
if (msg == null) {
break;
}
out.add(msg);
}
buf.drainTo(peer.pipeline().inboundMessageBuffer());
peer.eventLoop().execute(new Runnable() {
@Override

View File

@ -15,13 +15,13 @@
*/
package io.netty.channel.socket.nio;
import io.netty.buffer.MessageBuf;
import io.netty.channel.Channel;
import io.netty.channel.ChannelBufferType;
import io.netty.channel.ChannelPipeline;
import java.io.IOException;
import java.nio.channels.SelectableChannel;
import java.util.Queue;
abstract class AbstractNioMessageChannel extends AbstractNioChannel {
@ -46,7 +46,7 @@ abstract class AbstractNioMessageChannel extends AbstractNioChannel {
assert eventLoop().inEventLoop();
final ChannelPipeline pipeline = pipeline();
final Queue<Object> msgBuf = pipeline.inboundMessageBuffer();
final MessageBuf<Object> msgBuf = pipeline.inboundMessageBuffer();
boolean closed = false;
boolean read = false;
try {
@ -82,7 +82,7 @@ abstract class AbstractNioMessageChannel extends AbstractNioChannel {
}
@Override
protected void doFlushMessageBuffer(Queue<Object> buf) throws Exception {
protected void doFlushMessageBuffer(MessageBuf<Object> buf) throws Exception {
final int writeSpinCount = config().getWriteSpinCount() - 1;
while (!buf.isEmpty()) {
boolean wrote = false;
@ -100,6 +100,6 @@ abstract class AbstractNioMessageChannel extends AbstractNioChannel {
}
}
protected abstract int doReadMessages(Queue<Object> buf) throws Exception;
protected abstract int doWriteMessages(Queue<Object> buf, boolean lastSpin) throws Exception;
protected abstract int doReadMessages(MessageBuf<Object> buf) throws Exception;
protected abstract int doWriteMessages(MessageBuf<Object> buf, boolean lastSpin) throws Exception;
}

View File

@ -16,6 +16,7 @@
package io.netty.channel.socket.nio;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.MessageBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelException;
import io.netty.channel.ChannelFuture;
@ -39,7 +40,6 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Queue;
/**
* Provides an NIO based {@link io.netty.channel.socket.DatagramChannel}.
@ -160,7 +160,7 @@ public final class NioDatagramChannel
}
@Override
protected int doReadMessages(Queue<Object> buf) throws Exception {
protected int doReadMessages(MessageBuf<Object> buf) throws Exception {
DatagramChannel ch = javaChannel();
ByteBuffer data = ByteBuffer.allocate(config().getReceivePacketSize());
InetSocketAddress remoteAddress = (InetSocketAddress) ch.receive(data);
@ -174,7 +174,7 @@ public final class NioDatagramChannel
}
@Override
protected int doWriteMessages(Queue<Object> buf, boolean lastSpin) throws Exception {
protected int doWriteMessages(MessageBuf<Object> buf, boolean lastSpin) throws Exception {
DatagramPacket packet = (DatagramPacket) buf.peek();
ByteBuf data = packet.data();
ByteBuffer nioData;

View File

@ -15,6 +15,7 @@
*/
package io.netty.channel.socket.nio;
import io.netty.buffer.MessageBuf;
import io.netty.channel.ChannelException;
import io.netty.channel.socket.DefaultServerSocketChannelConfig;
import io.netty.channel.socket.ServerSocketChannelConfig;
@ -25,7 +26,6 @@ import java.net.SocketAddress;
import java.nio.channels.SelectionKey;
import java.nio.channels.ServerSocketChannel;
import java.nio.channels.SocketChannel;
import java.util.Queue;
public class NioServerSocketChannel extends AbstractNioMessageChannel
implements io.netty.channel.socket.ServerSocketChannel {
@ -84,7 +84,7 @@ public class NioServerSocketChannel extends AbstractNioMessageChannel
}
@Override
protected int doReadMessages(Queue<Object> buf) throws Exception {
protected int doReadMessages(MessageBuf<Object> buf) throws Exception {
SocketChannel ch = javaChannel().accept();
if (ch == null) {
return 0;
@ -116,7 +116,7 @@ public class NioServerSocketChannel extends AbstractNioMessageChannel
}
@Override
protected int doWriteMessages(Queue<Object> buf, boolean lastSpin) throws Exception {
protected int doWriteMessages(MessageBuf<Object> buf, boolean lastSpin) throws Exception {
throw new UnsupportedOperationException();
}
}

View File

@ -15,12 +15,12 @@
*/
package io.netty.channel.socket.oio;
import io.netty.buffer.MessageBuf;
import io.netty.channel.Channel;
import io.netty.channel.ChannelBufferType;
import io.netty.channel.ChannelPipeline;
import java.io.IOException;
import java.util.Queue;
abstract class AbstractOioMessageChannel extends AbstractOioChannel {
@ -44,7 +44,7 @@ abstract class AbstractOioMessageChannel extends AbstractOioChannel {
assert eventLoop().inEventLoop();
final ChannelPipeline pipeline = pipeline();
final Queue<Object> msgBuf = pipeline.inboundMessageBuffer();
final MessageBuf<Object> msgBuf = pipeline.inboundMessageBuffer();
boolean closed = false;
boolean read = false;
try {
@ -75,12 +75,12 @@ abstract class AbstractOioMessageChannel extends AbstractOioChannel {
}
@Override
protected void doFlushMessageBuffer(Queue<Object> buf) throws Exception {
protected void doFlushMessageBuffer(MessageBuf<Object> buf) throws Exception {
while (!buf.isEmpty()) {
doWriteMessages(buf);
}
}
protected abstract int doReadMessages(Queue<Object> buf) throws Exception;
protected abstract void doWriteMessages(Queue<Object> buf) throws Exception;
protected abstract int doReadMessages(MessageBuf<Object> buf) throws Exception;
protected abstract void doWriteMessages(MessageBuf<Object> buf) throws Exception;
}

View File

@ -16,6 +16,7 @@
package io.netty.channel.socket.oio;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.MessageBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelException;
import io.netty.channel.ChannelFuture;
@ -35,7 +36,6 @@ import java.net.SocketAddress;
import java.net.SocketException;
import java.net.SocketTimeoutException;
import java.util.Locale;
import java.util.Queue;
public class OioDatagramChannel extends AbstractOioMessageChannel
implements DatagramChannel {
@ -148,7 +148,7 @@ public class OioDatagramChannel extends AbstractOioMessageChannel
}
@Override
protected int doReadMessages(Queue<Object> buf) throws Exception {
protected int doReadMessages(MessageBuf<Object> buf) throws Exception {
int packetSize = config().getReceivePacketSize();
byte[] data = new byte[packetSize];
tmpPacket.setData(data);
@ -173,7 +173,7 @@ public class OioDatagramChannel extends AbstractOioMessageChannel
}
@Override
protected void doWriteMessages(Queue<Object> buf) throws Exception {
protected void doWriteMessages(MessageBuf<Object> buf) throws Exception {
DatagramPacket p = (DatagramPacket) buf.poll();
ByteBuf data = p.data();
int length = data.readableBytes();

View File

@ -15,6 +15,7 @@
*/
package io.netty.channel.socket.oio;
import io.netty.buffer.MessageBuf;
import io.netty.channel.ChannelException;
import io.netty.channel.socket.DefaultServerSocketChannelConfig;
import io.netty.channel.socket.ServerSocketChannel;
@ -28,7 +29,6 @@ import java.net.ServerSocket;
import java.net.Socket;
import java.net.SocketAddress;
import java.net.SocketTimeoutException;
import java.util.Queue;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
@ -124,7 +124,7 @@ public class OioServerSocketChannel extends AbstractOioMessageChannel
}
@Override
protected int doReadMessages(Queue<Object> buf) throws Exception {
protected int doReadMessages(MessageBuf<Object> buf) throws Exception {
if (socket.isClosed()) {
return -1;
}
@ -169,7 +169,7 @@ public class OioServerSocketChannel extends AbstractOioMessageChannel
}
@Override
protected void doWriteMessages(Queue<Object> buf) throws Exception {
protected void doWriteMessages(MessageBuf<Object> buf) throws Exception {
throw new UnsupportedOperationException();
}
}

View File

@ -245,7 +245,7 @@ public class LocalTransportThreadModelTest {
ch.eventLoop().execute(new Runnable() {
@Override
public void run() {
Queue<Object> buf = ch.pipeline().inboundMessageBuffer();
MessageBuf<Object> buf = ch.pipeline().inboundMessageBuffer();
for (int j = start; j < end; j ++) {
buf.add(Integer.valueOf(j));
}
@ -285,7 +285,7 @@ public class LocalTransportThreadModelTest {
ch.pipeline().context(h6).executor().execute(new Runnable() {
@Override
public void run() {
Queue<Object> buf = ch.pipeline().outboundMessageBuffer();
MessageBuf<Object> buf = ch.pipeline().outboundMessageBuffer();
for (int j = start; j < end; j ++) {
buf.add(Integer.valueOf(j));
}
@ -407,7 +407,7 @@ public class LocalTransportThreadModelTest {
Assert.assertSame(t, Thread.currentThread());
}
Queue<Integer> in = ctx.inboundMessageBuffer();
MessageBuf<Integer> in = ctx.inboundMessageBuffer();
ByteBuf out = ctx.nextInboundByteBuffer();
for (;;) {
@ -432,7 +432,7 @@ public class LocalTransportThreadModelTest {
boolean swallow = this == ctx.pipeline().first();
ByteBuf in = ctx.outboundByteBuffer();
Queue<Object> out = ctx.nextOutboundMessageBuffer();
MessageBuf<Object> out = ctx.nextOutboundMessageBuffer();
while (in.readableBytes() >= 4) {
int msg = in.readInt();
int expected = outCnt ++;
@ -493,7 +493,7 @@ public class LocalTransportThreadModelTest {
}
ByteBuf in = ctx.inboundByteBuffer();
Queue<Object> out = ctx.nextInboundMessageBuffer();
MessageBuf<Object> out = ctx.nextInboundMessageBuffer();
while (in.readableBytes() >= 4) {
int msg = in.readInt();
@ -510,7 +510,7 @@ public class LocalTransportThreadModelTest {
ChannelFuture future) throws Exception {
Assert.assertSame(t, Thread.currentThread());
Queue<Integer> in = ctx.outboundMessageBuffer();
MessageBuf<Integer> in = ctx.outboundMessageBuffer();
ByteBuf out = ctx.nextOutboundByteBuffer();
for (;;) {
@ -566,8 +566,8 @@ public class LocalTransportThreadModelTest {
Assert.assertSame(t, Thread.currentThread());
}
Queue<Object> in = ctx.inboundMessageBuffer();
Queue<Object> out = ctx.nextInboundMessageBuffer();
MessageBuf<Object> in = ctx.inboundMessageBuffer();
MessageBuf<Object> out = ctx.nextInboundMessageBuffer();
for (;;) {
Object msg = in.poll();
if (msg == null) {
@ -586,8 +586,8 @@ public class LocalTransportThreadModelTest {
ChannelFuture future) throws Exception {
Assert.assertSame(t, Thread.currentThread());
Queue<Object> in = ctx.outboundMessageBuffer();
Queue<Object> out = ctx.nextOutboundMessageBuffer();
MessageBuf<Object> in = ctx.outboundMessageBuffer();
MessageBuf<Object> out = ctx.nextOutboundMessageBuffer();
for (;;) {
Object msg = in.poll();
if (msg == null) {
@ -642,7 +642,7 @@ public class LocalTransportThreadModelTest {
Assert.assertSame(t, Thread.currentThread());
}
Queue<Object> in = ctx.inboundMessageBuffer();
MessageBuf<Object> in = ctx.inboundMessageBuffer();
for (;;) {
Object msg = in.poll();
if (msg == null) {
@ -658,8 +658,8 @@ public class LocalTransportThreadModelTest {
ChannelFuture future) throws Exception {
Assert.assertSame(t, Thread.currentThread());
Queue<Object> in = ctx.outboundMessageBuffer();
Queue<Object> out = ctx.nextOutboundMessageBuffer();
MessageBuf<Object> in = ctx.outboundMessageBuffer();
MessageBuf<Object> out = ctx.nextOutboundMessageBuffer();
for (;;) {
Object msg = in.poll();
if (msg == null) {