Decouple EventLoop details from the IO handling for each transport to… (#8680)

* Decouble EventLoop details from the IO handling for each transport to allow easy re-use of code and customization

Motiviation:

As today extending EventLoop implementations to add custom logic / metrics / instrumentations is only possible in a very limited way if at all. This is due the fact that most implementations are final or even package-private. That said even if these would be public there are the ability to do something useful with these is very limited as the IO processing and task processing are very tightly coupled. All of the mentioned things are a big pain point in netty 4.x and need improvement.

Modifications:

This changeset decoubled the IO processing logic from the task processing logic for the main transport (NIO, Epoll, KQueue) by introducing the concept of an IoHandler. The IoHandler itself is responsible to wait for IO readiness and process these IO events. The execution of the IoHandler itself is done by the SingleThreadEventLoop as part of its EventLoop processing. This allows to use the same EventLoopGroup (MultiThreadEventLoupGroup) for all the mentioned transports by just specify a different IoHandlerFactory during construction.

Beside this core API change this changeset also allows to easily extend SingleThreadEventExecutor / SingleThreadEventLoop to add custom logic to it which then can be reused by all the transports. The ideas are very similar to what is provided by ScheduledThreadPoolExecutor (that is part of the JDK). This allows for example things like:

  * Adding instrumentation / metrics:
    * how many Channels are registered on an SingleThreadEventLoop
    * how many Channels were handled during the IO processing in an EventLoop run
    * how many task were handled during the last EventLoop / EventExecutor run
    * how many outstanding tasks we have
    ...
    ...
  * Implementing custom strategies for choosing the next EventExecutor / EventLoop to use based on these metrics.
  * Use different Promise / Future / ScheduledFuture implementations
  * decorate Runnable / Callables when submitted to the EventExecutor / EventLoop

As a lot of functionalities are folded into the MultiThreadEventLoopGroup and SingleThreadEventLoopGroup this changeset also removes:

  * AbstractEventLoop
  * AbstractEventLoopGroup
  * EventExecutorChooser
  * EventExecutorChooserFactory
  * DefaultEventLoopGroup
  * DefaultEventExecutor
  * DefaultEventExecutorGroup

Result:

Fixes https://github.com/netty/netty/issues/8514 .
This commit is contained in:
Norman Maurer 2019-01-23 08:32:05 +01:00 committed by GitHub
parent 5cfb107822
commit 3d6e6136a9
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
172 changed files with 3133 additions and 2137 deletions

View File

@ -25,9 +25,10 @@ import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.embedded.EmbeddedChannel;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
@ -124,7 +125,7 @@ public class HttpClientCodecTest {
final CountDownLatch serverChannelLatch = new CountDownLatch(1);
final CountDownLatch responseReceivedLatch = new CountDownLatch(1);
try {
sb.group(new NioEventLoopGroup(2));
sb.group(new MultithreadEventLoopGroup(2, NioHandler.newFactory()));
sb.channel(NioServerSocketChannel.class);
sb.childHandler(new ChannelInitializer<Channel>() {
@Override
@ -168,7 +169,7 @@ public class HttpClientCodecTest {
}
});
cb.group(new NioEventLoopGroup(1));
cb.group(new MultithreadEventLoopGroup(1, NioHandler.newFactory()));
cb.channel(NioSocketChannel.class);
cb.option(ChannelOption.ALLOW_HALF_CLOSURE, true);
cb.handler(new ChannelInitializer<Channel>() {

View File

@ -25,7 +25,8 @@ import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.ChannelPromise;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.http.HttpHeaderNames;
@ -298,7 +299,8 @@ public class DataCompressionHttp2Test {
any(ByteBuf.class), anyInt(), anyBoolean());
final CountDownLatch serverChannelLatch = new CountDownLatch(1);
sb.group(new NioEventLoopGroup(), new NioEventLoopGroup());
sb.group(new MultithreadEventLoopGroup(NioHandler.newFactory()),
new MultithreadEventLoopGroup(NioHandler.newFactory()));
sb.channel(NioServerSocketChannel.class);
sb.childHandler(new ChannelInitializer<Channel>() {
@Override
@ -322,7 +324,7 @@ public class DataCompressionHttp2Test {
}
});
cb.group(new NioEventLoopGroup());
cb.group(new MultithreadEventLoopGroup(NioHandler.newFactory()));
cb.channel(NioSocketChannel.class);
cb.handler(new ChannelInitializer<Channel>() {
@Override

View File

@ -17,7 +17,9 @@ package io.netty.handler.codec.http2;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.local.LocalHandler;
import io.netty.handler.codec.http2.Http2Connection.Endpoint;
import io.netty.handler.codec.http2.Http2Stream.State;
import io.netty.util.concurrent.Future;
@ -61,7 +63,7 @@ public class DefaultHttp2ConnectionTest {
private DefaultHttp2Connection server;
private DefaultHttp2Connection client;
private static LocalEventLoopGroup group;
private static EventLoopGroup group;
@Mock
private Http2Connection.Listener clientListener;
@ -71,7 +73,7 @@ public class DefaultHttp2ConnectionTest {
@BeforeClass
public static void beforeClass() {
group = new LocalEventLoopGroup(2);
group = new MultithreadEventLoopGroup(2, LocalHandler.newFactory());
}
@AfterClass

View File

@ -30,9 +30,10 @@ import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOutboundHandlerAdapter;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.ChannelPromise;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.local.LocalServerChannel;
import io.netty.handler.codec.http2.Http2TestUtil.FrameCountDown;
import io.netty.handler.codec.http2.Http2TestUtil.Http2Runnable;
@ -1191,7 +1192,7 @@ public class Http2ConnectionRoundtripTest {
final AtomicReference<Http2ConnectionHandler> serverHandlerRef = new AtomicReference<>();
final CountDownLatch serverInitLatch = new CountDownLatch(1);
sb.group(new LocalEventLoopGroup());
sb.group(new MultithreadEventLoopGroup(LocalHandler.newFactory()));
sb.channel(LocalServerChannel.class);
sb.childHandler(new ChannelInitializer<Channel>() {
@Override
@ -1211,7 +1212,7 @@ public class Http2ConnectionRoundtripTest {
}
});
cb.group(new LocalEventLoopGroup());
cb.group(new MultithreadEventLoopGroup(LocalHandler.newFactory()));
cb.channel(LocalChannel.class);
cb.handler(new ChannelInitializer<Channel>() {
@Override

View File

@ -28,9 +28,10 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.local.LocalServerChannel;
import org.junit.After;
import org.junit.AfterClass;
@ -61,7 +62,7 @@ public class Http2MultiplexCodecBuilderTest {
@BeforeClass
public static void init() {
group = new LocalEventLoopGroup(1);
group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
}
@Before

View File

@ -25,9 +25,10 @@ import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.ChannelPromise;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.local.LocalServerChannel;
import io.netty.handler.codec.http.DefaultFullHttpRequest;
import io.netty.handler.codec.http.DefaultHttpContent;
@ -508,7 +509,7 @@ public class HttpToHttp2ConnectionHandlerTest {
sb = new ServerBootstrap();
cb = new Bootstrap();
sb.group(new LocalEventLoopGroup());
sb.group(new MultithreadEventLoopGroup(LocalHandler.newFactory()));
sb.channel(LocalServerChannel.class);
sb.childHandler(new ChannelInitializer<Channel>() {
@Override
@ -525,7 +526,7 @@ public class HttpToHttp2ConnectionHandlerTest {
}
});
cb.group(new LocalEventLoopGroup());
cb.group(new MultithreadEventLoopGroup(LocalHandler.newFactory()));
cb.channel(LocalChannel.class);
cb.handler(new ChannelInitializer<Channel>() {
@Override

View File

@ -26,10 +26,11 @@ import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.ChannelPromise;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.local.LocalServerChannel;
import io.netty.handler.codec.http.DefaultFullHttpRequest;
import io.netty.handler.codec.http.DefaultFullHttpResponse;
@ -667,7 +668,7 @@ public class InboundHttp2ToHttpAdapterTest {
sb = new ServerBootstrap();
cb = new Bootstrap();
sb.group(new LocalEventLoopGroup());
sb.group(new MultithreadEventLoopGroup(LocalHandler.newFactory()));
sb.channel(LocalServerChannel.class);
sb.childHandler(new ChannelInitializer<Channel>() {
@Override
@ -695,7 +696,7 @@ public class InboundHttp2ToHttpAdapterTest {
}
});
cb.group(new LocalEventLoopGroup());
cb.group(new MultithreadEventLoopGroup(LocalHandler.newFactory()));
cb.channel(LocalChannel.class);
cb.handler(new ChannelInitializer<Channel>() {
@Override

View File

@ -27,8 +27,9 @@ import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.embedded.EmbeddedChannel;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.EncoderException;
@ -240,7 +241,7 @@ public class Lz4FrameEncoderTest extends AbstractEncoderTest {
@Test(timeout = 3000)
public void writingAfterClosedChannelDoesNotNPE() throws InterruptedException {
EventLoopGroup group = new NioEventLoopGroup(2);
EventLoopGroup group = new MultithreadEventLoopGroup(2, NioHandler.newFactory());
Channel serverChannel = null;
Channel clientChannel = null;
final CountDownLatch latch = new CountDownLatch(1);

View File

@ -24,7 +24,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.concurrent.AbstractExecutorService;
import java.util.concurrent.Callable;
import java.util.concurrent.RunnableFuture;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
/**
@ -36,39 +36,25 @@ public abstract class AbstractEventExecutor extends AbstractExecutorService impl
static final long DEFAULT_SHUTDOWN_QUIET_PERIOD = 2;
static final long DEFAULT_SHUTDOWN_TIMEOUT = 15;
private final EventExecutorGroup parent;
private final Collection<EventExecutor> selfCollection = Collections.<EventExecutor>singleton(this);
protected AbstractEventExecutor() {
this(null);
}
protected AbstractEventExecutor(EventExecutorGroup parent) {
this.parent = parent;
}
@Override
public EventExecutorGroup parent() {
return parent;
}
@Override
public EventExecutor next() {
return this;
}
@Override
public boolean inEventLoop() {
public final boolean inEventLoop() {
return inEventLoop(Thread.currentThread());
}
@Override
public Iterator<EventExecutor> iterator() {
public final Iterator<EventExecutor> iterator() {
return selfCollection.iterator();
}
@Override
public Future<?> shutdownGracefully() {
public final Future<?> shutdownGracefully() {
return shutdownGracefully(DEFAULT_SHUTDOWN_QUIET_PERIOD, DEFAULT_SHUTDOWN_TIMEOUT, TimeUnit.SECONDS);
}
@ -110,59 +96,59 @@ public abstract class AbstractEventExecutor extends AbstractExecutorService impl
}
@Override
public Future<?> submit(Runnable task) {
public final Future<?> submit(Runnable task) {
return (Future<?>) super.submit(task);
}
@Override
public <T> Future<T> submit(Runnable task, T result) {
public final <T> Future<T> submit(Runnable task, T result) {
return (Future<T>) super.submit(task, result);
}
@Override
public <T> Future<T> submit(Callable<T> task) {
public final <T> Future<T> submit(Callable<T> task) {
return (Future<T>) super.submit(task);
}
@Override
protected final <T> RunnableFuture<T> newTaskFor(Runnable runnable, T value) {
return new PromiseTask<>(this, runnable, value);
protected <T> RunnableFuture<T> newTaskFor(Runnable runnable, T value) {
return newRunnableFuture(this.newPromise(), runnable, value);
}
@Override
protected final <T> RunnableFuture<T> newTaskFor(Callable<T> callable) {
return new PromiseTask<>(this, callable);
}
@Override
public ScheduledFuture<?> schedule(Runnable command, long delay,
TimeUnit unit) {
throw new UnsupportedOperationException();
}
@Override
public <V> ScheduledFuture<V> schedule(Callable<V> callable, long delay, TimeUnit unit) {
throw new UnsupportedOperationException();
}
@Override
public ScheduledFuture<?> scheduleAtFixedRate(Runnable command, long initialDelay, long period, TimeUnit unit) {
throw new UnsupportedOperationException();
}
@Override
public ScheduledFuture<?> scheduleWithFixedDelay(Runnable command, long initialDelay, long delay, TimeUnit unit) {
throw new UnsupportedOperationException();
protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable) {
return newRunnableFuture(this.newPromise(), callable);
}
/**
* Try to execute the given {@link Runnable} and just log if it throws a {@link Throwable}.
*/
protected static void safeExecute(Runnable task) {
static void safeExecute(Runnable task) {
try {
task.run();
} catch (Throwable t) {
logger.warn("A task raised an exception. Task: {}", task, t);
}
}
/**
* Returns a new {@link RunnableFuture} build on top of the given {@link Promise} and {@link Callable}.
*
* This can be used if you want to override {@link #newTaskFor(Callable)} and return a different
* {@link RunnableFuture}.
*/
private static <V> RunnableFuture<V> newRunnableFuture(Promise<V> promise, Callable<V> task) {
return new RunnableFutureAdapter<>(promise, task);
}
/**
* Returns a new {@link RunnableFuture} build on top of the given {@link Promise} and {@link Runnable} and
* {@code value}.
*
* This can be used if you want to override {@link #newTaskFor(Runnable, V)} and return a different
* {@link RunnableFuture}.
*/
private static <V> RunnableFuture<V> newRunnableFuture(Promise<V> promise, Runnable task, V value) {
return new RunnableFutureAdapter<>(promise, Executors.callable(task, value));
}
}

View File

@ -18,40 +18,54 @@ package io.netty.util.concurrent;
import io.netty.util.internal.DefaultPriorityQueue;
import io.netty.util.internal.ObjectUtil;
import io.netty.util.internal.PriorityQueue;
import io.netty.util.internal.PriorityQueueNode;
import java.util.Comparator;
import java.util.Queue;
import java.util.concurrent.Callable;
import java.util.concurrent.Delayed;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
/**
* Abstract base class for {@link EventExecutor}s that want to support scheduling.
*/
public abstract class AbstractScheduledEventExecutor extends AbstractEventExecutor {
static final long START_TIME = System.nanoTime();
private static final Comparator<ScheduledFutureTask<?>> SCHEDULED_FUTURE_TASK_COMPARATOR =
new Comparator<ScheduledFutureTask<?>>() {
private static final Comparator<RunnableScheduledFutureNode<?>> SCHEDULED_FUTURE_TASK_COMPARATOR =
new Comparator<RunnableScheduledFutureNode<?>>() {
@Override
public int compare(ScheduledFutureTask<?> o1, ScheduledFutureTask<?> o2) {
public int compare(RunnableScheduledFutureNode<?> o1, RunnableScheduledFutureNode<?> o2) {
return o1.compareTo(o2);
}
};
PriorityQueue<ScheduledFutureTask<?>> scheduledTaskQueue;
private PriorityQueue<RunnableScheduledFutureNode<?>> scheduledTaskQueue;
protected AbstractScheduledEventExecutor() {
}
protected AbstractScheduledEventExecutor(EventExecutorGroup parent) {
super(parent);
/**
* The time elapsed since initialization of this class in nanoseconds. This may return a negative number just like
* {@link System#nanoTime()}.
*/
public static long nanoTime() {
return System.nanoTime() - START_TIME;
}
protected static long nanoTime() {
return ScheduledFutureTask.nanoTime();
/**
* The deadline (in nanoseconds) for a given delay (in nanoseconds).
*/
static long deadlineNanos(long delay) {
long deadlineNanos = nanoTime() + delay;
// Guard against overflow
return deadlineNanos < 0 ? Long.MAX_VALUE : deadlineNanos;
}
PriorityQueue<ScheduledFutureTask<?>> scheduledTaskQueue() {
PriorityQueue<RunnableScheduledFutureNode<?>> scheduledTaskQueue() {
if (scheduledTaskQueue == null) {
scheduledTaskQueue = new DefaultPriorityQueue<>(
SCHEDULED_FUTURE_TASK_COMPARATOR,
@ -61,7 +75,7 @@ public abstract class AbstractScheduledEventExecutor extends AbstractEventExecut
return scheduledTaskQueue;
}
private static boolean isNullOrEmpty(Queue<ScheduledFutureTask<?>> queue) {
private static boolean isNullOrEmpty(Queue<RunnableScheduledFutureNode<?>> queue) {
return queue == null || queue.isEmpty();
}
@ -70,18 +84,18 @@ public abstract class AbstractScheduledEventExecutor extends AbstractEventExecut
*
* This method MUST be called only when {@link #inEventLoop()} is {@code true}.
*/
protected void cancelScheduledTasks() {
protected final void cancelScheduledTasks() {
assert inEventLoop();
PriorityQueue<ScheduledFutureTask<?>> scheduledTaskQueue = this.scheduledTaskQueue;
PriorityQueue<RunnableScheduledFutureNode<?>> scheduledTaskQueue = this.scheduledTaskQueue;
if (isNullOrEmpty(scheduledTaskQueue)) {
return;
}
final ScheduledFutureTask<?>[] scheduledTasks =
scheduledTaskQueue.toArray(new ScheduledFutureTask<?>[0]);
final RunnableScheduledFutureNode<?>[] scheduledTasks =
scheduledTaskQueue.toArray(new RunnableScheduledFutureNode<?>[0]);
for (ScheduledFutureTask<?> task: scheduledTasks) {
task.cancelWithoutRemove(false);
for (RunnableScheduledFutureNode<?> task: scheduledTasks) {
task.cancel(false);
}
scheduledTaskQueue.clearIgnoringIndexes();
@ -90,19 +104,21 @@ public abstract class AbstractScheduledEventExecutor extends AbstractEventExecut
/**
* @see #pollScheduledTask(long)
*/
protected final Runnable pollScheduledTask() {
protected final RunnableScheduledFuture<?> pollScheduledTask() {
return pollScheduledTask(nanoTime());
}
/**
* Return the {@link Runnable} which is ready to be executed with the given {@code nanoTime}.
* You should use {@link #nanoTime()} to retrieve the correct {@code nanoTime}.
*
* This method MUST be called only when {@link #inEventLoop()} is {@code true}.
*/
protected final Runnable pollScheduledTask(long nanoTime) {
protected final RunnableScheduledFuture<?> pollScheduledTask(long nanoTime) {
assert inEventLoop();
Queue<ScheduledFutureTask<?>> scheduledTaskQueue = this.scheduledTaskQueue;
ScheduledFutureTask<?> scheduledTask = scheduledTaskQueue == null ? null : scheduledTaskQueue.peek();
Queue<RunnableScheduledFutureNode<?>> scheduledTaskQueue = this.scheduledTaskQueue;
RunnableScheduledFutureNode<?> scheduledTask = scheduledTaskQueue == null ? null : scheduledTaskQueue.peek();
if (scheduledTask == null) {
return null;
}
@ -116,30 +132,39 @@ public abstract class AbstractScheduledEventExecutor extends AbstractEventExecut
/**
* Return the nanoseconds when the next scheduled task is ready to be run or {@code -1} if no task is scheduled.
*
* This method MUST be called only when {@link #inEventLoop()} is {@code true}.
*/
protected final long nextScheduledTaskNano() {
Queue<ScheduledFutureTask<?>> scheduledTaskQueue = this.scheduledTaskQueue;
ScheduledFutureTask<?> scheduledTask = scheduledTaskQueue == null ? null : scheduledTaskQueue.peek();
Queue<RunnableScheduledFutureNode<?>> scheduledTaskQueue = this.scheduledTaskQueue;
RunnableScheduledFutureNode<?> scheduledTask = scheduledTaskQueue == null ? null : scheduledTaskQueue.peek();
if (scheduledTask == null) {
return -1;
}
return Math.max(0, scheduledTask.deadlineNanos() - nanoTime());
}
final ScheduledFutureTask<?> peekScheduledTask() {
Queue<ScheduledFutureTask<?>> scheduledTaskQueue = this.scheduledTaskQueue;
final RunnableScheduledFuture<?> peekScheduledTask() {
Queue<RunnableScheduledFutureNode<?>> scheduledTaskQueue = this.scheduledTaskQueue;
if (scheduledTaskQueue == null) {
return null;
}
return scheduledTaskQueue.peek();
RunnableScheduledFutureNode<?> node = scheduledTaskQueue.peek();
if (node == null) {
return null;
}
return node;
}
/**
* Returns {@code true} if a scheduled task is ready for processing.
*
* This method MUST be called only when {@link #inEventLoop()} is {@code true}.
*/
protected final boolean hasScheduledTasks() {
Queue<ScheduledFutureTask<?>> scheduledTaskQueue = this.scheduledTaskQueue;
ScheduledFutureTask<?> scheduledTask = scheduledTaskQueue == null ? null : scheduledTaskQueue.peek();
assert inEventLoop();
Queue<RunnableScheduledFutureNode<?>> scheduledTaskQueue = this.scheduledTaskQueue;
RunnableScheduledFutureNode<?> scheduledTask = scheduledTaskQueue == null ? null : scheduledTaskQueue.peek();
return scheduledTask != null && scheduledTask.deadlineNanos() <= nanoTime();
}
@ -150,10 +175,9 @@ public abstract class AbstractScheduledEventExecutor extends AbstractEventExecut
if (delay < 0) {
delay = 0;
}
validateScheduled0(delay, unit);
return schedule(new ScheduledFutureTask<Void>(
this, command, null, ScheduledFutureTask.deadlineNanos(unit.toNanos(delay))));
RunnableScheduledFuture<?> task = newScheduledTaskFor(Executors.callable(command),
deadlineNanos(unit.toNanos(delay)), 0);
return schedule(task);
}
@Override
@ -163,10 +187,8 @@ public abstract class AbstractScheduledEventExecutor extends AbstractEventExecut
if (delay < 0) {
delay = 0;
}
validateScheduled0(delay, unit);
return schedule(new ScheduledFutureTask<>(
this, callable, ScheduledFutureTask.deadlineNanos(unit.toNanos(delay))));
RunnableScheduledFuture<V> task = newScheduledTaskFor(callable, deadlineNanos(unit.toNanos(delay)), 0);
return schedule(task);
}
@Override
@ -181,12 +203,10 @@ public abstract class AbstractScheduledEventExecutor extends AbstractEventExecut
throw new IllegalArgumentException(
String.format("period: %d (expected: > 0)", period));
}
validateScheduled0(initialDelay, unit);
validateScheduled0(period, unit);
return schedule(new ScheduledFutureTask<>(
this, Executors.<Void>callable(command, null),
ScheduledFutureTask.deadlineNanos(unit.toNanos(initialDelay)), unit.toNanos(period)));
RunnableScheduledFuture<?> task = newScheduledTaskFor(Executors.<Void>callable(command, null),
deadlineNanos(unit.toNanos(initialDelay)), unit.toNanos(period));
return schedule(task);
}
@Override
@ -202,45 +222,39 @@ public abstract class AbstractScheduledEventExecutor extends AbstractEventExecut
String.format("delay: %d (expected: > 0)", delay));
}
validateScheduled0(initialDelay, unit);
validateScheduled0(delay, unit);
return schedule(new ScheduledFutureTask<>(
this, Executors.<Void>callable(command, null),
ScheduledFutureTask.deadlineNanos(unit.toNanos(initialDelay)), -unit.toNanos(delay)));
}
@SuppressWarnings("deprecation")
private void validateScheduled0(long amount, TimeUnit unit) {
validateScheduled(amount, unit);
RunnableScheduledFuture<?> task = newScheduledTaskFor(Executors.<Void>callable(command, null),
deadlineNanos(unit.toNanos(initialDelay)), -unit.toNanos(delay));
return schedule(task);
}
/**
* Sub-classes may override this to restrict the maximal amount of time someone can use to schedule a task.
*
* @deprecated will be removed in the future.
* Add the {@link RunnableScheduledFuture} for execution.
*/
@Deprecated
protected void validateScheduled(long amount, TimeUnit unit) {
// NOOP
}
<V> ScheduledFuture<V> schedule(final ScheduledFutureTask<V> task) {
protected final <V> ScheduledFuture<V> schedule(final RunnableScheduledFuture<V> task) {
if (inEventLoop()) {
scheduledTaskQueue().add(task);
add0(task);
} else {
execute(new Runnable() {
@Override
public void run() {
scheduledTaskQueue().add(task);
add0(task);
}
});
}
return task;
}
final void removeScheduled(final ScheduledFutureTask<?> task) {
private <V> void add0(RunnableScheduledFuture<V> task) {
final RunnableScheduledFutureNode node;
if (task instanceof RunnableScheduledFutureNode) {
node = (RunnableScheduledFutureNode) task;
} else {
node = new DefaultRunnableScheduledFutureNode<V>(task);
}
scheduledTaskQueue().add(node);
}
final void removeScheduled(final RunnableScheduledFutureNode<?> task) {
if (inEventLoop()) {
scheduledTaskQueue().removeTyped(task);
} else {
@ -252,4 +266,197 @@ public abstract class AbstractScheduledEventExecutor extends AbstractEventExecut
});
}
}
/**
* Returns a new {@link RunnableFuture} build on top of the given {@link Promise} and {@link Callable}.
*
* This can be used if you want to override {@link #newTaskFor(Callable)} and return a different
* {@link RunnableFuture}.
*/
protected static <V> RunnableScheduledFuture<V> newRunnableScheduledFuture(
AbstractScheduledEventExecutor executor, Promise<V> promise, Callable<V> task,
long deadlineNanos, long periodNanos) {
return new RunnableScheduledFutureAdapter<V>(executor, promise, task, deadlineNanos, periodNanos);
}
/**
* Returns a {@code RunnableScheduledFuture} for the given values.
*/
protected <V> RunnableScheduledFuture<V> newScheduledTaskFor(
Callable<V> callable, long deadlineNanos, long period) {
return newRunnableScheduledFuture(this, this.newPromise(), callable, deadlineNanos, period);
}
interface RunnableScheduledFutureNode<V> extends PriorityQueueNode, RunnableScheduledFuture<V> { }
private static final class DefaultRunnableScheduledFutureNode<V> implements RunnableScheduledFutureNode<V> {
private final RunnableScheduledFuture<V> future;
private int queueIndex = INDEX_NOT_IN_QUEUE;
DefaultRunnableScheduledFutureNode(RunnableScheduledFuture<V> future) {
this.future = future;
}
@Override
public long deadlineNanos() {
return future.deadlineNanos();
}
@Override
public long delayNanos() {
return future.delayNanos();
}
@Override
public long delayNanos(long currentTimeNanos) {
return future.delayNanos(currentTimeNanos);
}
@Override
public RunnableScheduledFuture<V> addListener(
GenericFutureListener<? extends Future<? super V>> listener) {
future.addListener(listener);
return this;
}
@Override
public RunnableScheduledFuture<V> addListeners(
GenericFutureListener<? extends Future<? super V>>... listeners) {
future.addListeners(listeners);
return this;
}
@Override
public RunnableScheduledFuture<V> removeListener(
GenericFutureListener<? extends Future<? super V>> listener) {
future.removeListener(listener);
return this;
}
@Override
public RunnableScheduledFuture<V> removeListeners(
GenericFutureListener<? extends Future<? super V>>... listeners) {
future.removeListeners(listeners);
return this;
}
@Override
public boolean isPeriodic() {
return future.isPeriodic();
}
@Override
public int priorityQueueIndex(DefaultPriorityQueue<?> queue) {
return queueIndex;
}
@Override
public void priorityQueueIndex(DefaultPriorityQueue<?> queue, int i) {
queueIndex = i;
}
@Override
public void run() {
future.run();
}
@Override
public boolean cancel(boolean mayInterruptIfRunning) {
return future.cancel(mayInterruptIfRunning);
}
@Override
public boolean isCancelled() {
return future.isCancelled();
}
@Override
public boolean isDone() {
return future.isDone();
}
@Override
public V get() throws InterruptedException, ExecutionException {
return future.get();
}
@Override
public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
return future.get(timeout, unit);
}
@Override
public long getDelay(TimeUnit unit) {
return future.getDelay(unit);
}
@Override
public int compareTo(Delayed o) {
return future.compareTo(o);
}
@Override
public RunnableFuture<V> sync() throws InterruptedException {
future.sync();
return this;
}
@Override
public RunnableFuture<V> syncUninterruptibly() {
future.syncUninterruptibly();
return this;
}
@Override
public RunnableFuture<V> await() throws InterruptedException {
future.await();
return this;
}
@Override
public RunnableFuture<V> awaitUninterruptibly() {
future.awaitUninterruptibly();
return this;
}
@Override
public boolean isSuccess() {
return future.isSuccess();
}
@Override
public boolean isCancellable() {
return future.isCancellable();
}
@Override
public Throwable cause() {
return future.cause();
}
@Override
public boolean await(long timeout, TimeUnit unit) throws InterruptedException {
return future.await(timeout, unit);
}
@Override
public boolean await(long timeoutMillis) throws InterruptedException {
return future.await(timeoutMillis);
}
@Override
public boolean awaitUninterruptibly(long timeout, TimeUnit unit) {
return future.awaitUninterruptibly(timeout, unit);
}
@Override
public boolean awaitUninterruptibly(long timeoutMillis) {
return future.awaitUninterruptibly(timeoutMillis);
}
@Override
public V getNow() {
return future.getNow();
}
}
}

View File

@ -1,75 +0,0 @@
/*
* 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.util.concurrent;
import java.util.concurrent.Executor;
import java.util.concurrent.ThreadFactory;
/**
* Default {@link SingleThreadEventExecutor} implementation which just execute all submitted task in a
* serial fashion.
*/
public final class DefaultEventExecutor extends SingleThreadEventExecutor {
public DefaultEventExecutor() {
this((EventExecutorGroup) null);
}
public DefaultEventExecutor(ThreadFactory threadFactory) {
this(null, threadFactory);
}
public DefaultEventExecutor(Executor executor) {
this(null, executor);
}
public DefaultEventExecutor(EventExecutorGroup parent) {
this(parent, new DefaultThreadFactory(DefaultEventExecutor.class));
}
public DefaultEventExecutor(EventExecutorGroup parent, ThreadFactory threadFactory) {
super(parent, threadFactory, true);
}
public DefaultEventExecutor(EventExecutorGroup parent, Executor executor) {
super(parent, executor, true);
}
public DefaultEventExecutor(EventExecutorGroup parent, ThreadFactory threadFactory, int maxPendingTasks,
RejectedExecutionHandler rejectedExecutionHandler) {
super(parent, threadFactory, true, maxPendingTasks, rejectedExecutionHandler);
}
public DefaultEventExecutor(EventExecutorGroup parent, Executor executor, int maxPendingTasks,
RejectedExecutionHandler rejectedExecutionHandler) {
super(parent, executor, true, maxPendingTasks, rejectedExecutionHandler);
}
@Override
protected void run() {
for (;;) {
Runnable task = takeTask();
if (task != null) {
task.run();
updateLastExecutionTime();
}
if (confirmShutdown()) {
break;
}
}
}
}

View File

@ -1,73 +0,0 @@
/*
* 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:
*
* 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.util.concurrent;
import io.netty.util.internal.UnstableApi;
import java.util.concurrent.atomic.AtomicInteger;
/**
* Default implementation which uses simple round-robin to choose next {@link EventExecutor}.
*/
@UnstableApi
public final class DefaultEventExecutorChooserFactory implements EventExecutorChooserFactory {
public static final DefaultEventExecutorChooserFactory INSTANCE = new DefaultEventExecutorChooserFactory();
private DefaultEventExecutorChooserFactory() { }
@SuppressWarnings("unchecked")
@Override
public EventExecutorChooser newChooser(EventExecutor[] executors) {
if (isPowerOfTwo(executors.length)) {
return new PowerOfTwoEventExecutorChooser(executors);
} else {
return new GenericEventExecutorChooser(executors);
}
}
private static boolean isPowerOfTwo(int val) {
return (val & -val) == val;
}
private static final class PowerOfTwoEventExecutorChooser implements EventExecutorChooser {
private final AtomicInteger idx = new AtomicInteger();
private final EventExecutor[] executors;
PowerOfTwoEventExecutorChooser(EventExecutor[] executors) {
this.executors = executors;
}
@Override
public EventExecutor next() {
return executors[idx.getAndIncrement() & executors.length - 1];
}
}
private static final class GenericEventExecutorChooser implements EventExecutorChooser {
private final AtomicInteger idx = new AtomicInteger();
private final EventExecutor[] executors;
GenericEventExecutorChooser(EventExecutor[] executors) {
this.executors = executors;
}
@Override
public EventExecutor next() {
return executors[Math.abs(idx.getAndIncrement() % executors.length)];
}
}
}

View File

@ -15,12 +15,11 @@
*/
package io.netty.util.concurrent;
import java.util.concurrent.Executor;
import java.util.concurrent.ThreadFactory;
/**
* Default implementation of {@link MultithreadEventExecutorGroup} which will use {@link DefaultEventExecutor} instances
* to handle the tasks.
* Default implementation of {@link MultithreadEventExecutorGroup} which will use {@link SingleThreadEventExecutor}
* instances to handle the tasks.
*/
public class DefaultEventExecutorGroup extends MultithreadEventExecutorGroup {
/**
@ -53,9 +52,4 @@ public class DefaultEventExecutorGroup extends MultithreadEventExecutorGroup {
RejectedExecutionHandler rejectedHandler) {
super(nThreads, threadFactory, maxPendingTasks, rejectedHandler);
}
@Override
protected EventExecutor newChild(Executor executor, Object... args) throws Exception {
return new DefaultEventExecutor(this, executor, (Integer) args[0], (RejectedExecutionHandler) args[1]);
}
}

View File

@ -30,11 +30,6 @@ public interface EventExecutor extends EventExecutorGroup {
@Override
EventExecutor next();
/**
* Return the {@link EventExecutorGroup} which is the parent of this {@link EventExecutor},
*/
EventExecutorGroup parent();
/**
* Calls {@link #inEventLoop(Thread)} with {@link Thread#currentThread()} as argument
*/

View File

@ -1,42 +0,0 @@
/*
* 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:
*
* 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.util.concurrent;
import io.netty.util.internal.UnstableApi;
/**
* Factory that creates new {@link EventExecutorChooser}s.
*/
@UnstableApi
public interface EventExecutorChooserFactory {
/**
* Returns a new {@link EventExecutorChooser}.
*/
EventExecutorChooser newChooser(EventExecutor[] executors);
/**
* Chooses the next {@link EventExecutor} to use.
*/
@UnstableApi
interface EventExecutorChooser {
/**
* Returns the new {@link EventExecutor} to use.
*/
EventExecutor next();
}
}

View File

@ -40,16 +40,23 @@ public final class GlobalEventExecutor extends AbstractScheduledEventExecutor {
private static final long SCHEDULE_QUIET_PERIOD_INTERVAL = TimeUnit.SECONDS.toNanos(1);
public static final GlobalEventExecutor INSTANCE = new GlobalEventExecutor();
private static final RunnableScheduledFutureAdapter<Void> QUIET_PERIOD_TASK;
public static final GlobalEventExecutor INSTANCE;
final BlockingQueue<Runnable> taskQueue = new LinkedBlockingQueue<>();
final ScheduledFutureTask<Void> quietPeriodTask = new ScheduledFutureTask<>(
this, Executors.<Void>callable(new Runnable() {
@Override
public void run() {
// NOOP
}
}, null), ScheduledFutureTask.deadlineNanos(SCHEDULE_QUIET_PERIOD_INTERVAL), -SCHEDULE_QUIET_PERIOD_INTERVAL);
static {
INSTANCE = new GlobalEventExecutor();
QUIET_PERIOD_TASK = new RunnableScheduledFutureAdapter<>(
INSTANCE, INSTANCE.newPromise(), Executors.callable(new Runnable() {
@Override
public void run() {
// NOOP
}
}, null), deadlineNanos(SCHEDULE_QUIET_PERIOD_INTERVAL), -SCHEDULE_QUIET_PERIOD_INTERVAL);
INSTANCE.scheduledTaskQueue().add(QUIET_PERIOD_TASK);
}
private final BlockingQueue<Runnable> taskQueue = new LinkedBlockingQueue<Runnable>();
// because the GlobalEventExecutor is a singleton, tasks submitted to it can come from arbitrary threads and this
// can trigger the creation of a thread from arbitrary thread groups; for this reason, the thread factory must not
@ -64,7 +71,6 @@ public final class GlobalEventExecutor extends AbstractScheduledEventExecutor {
private final Future<?> terminationFuture = new FailedFuture<>(this, new UnsupportedOperationException());
private GlobalEventExecutor() {
scheduledTaskQueue().add(quietPeriodTask);
}
/**
@ -72,10 +78,10 @@ public final class GlobalEventExecutor extends AbstractScheduledEventExecutor {
*
* @return {@code null} if the executor thread has been interrupted or waken up.
*/
Runnable takeTask() {
private Runnable takeTask() {
BlockingQueue<Runnable> taskQueue = this.taskQueue;
for (;;) {
ScheduledFutureTask<?> scheduledTask = peekScheduledTask();
RunnableScheduledFuture<?> scheduledTask = peekScheduledTask();
if (scheduledTask == null) {
Runnable task = null;
try {
@ -250,14 +256,14 @@ public final class GlobalEventExecutor extends AbstractScheduledEventExecutor {
logger.warn("Unexpected exception from the global event executor: ", t);
}
if (task != quietPeriodTask) {
if (task != QUIET_PERIOD_TASK) {
continue;
}
}
Queue<ScheduledFutureTask<?>> scheduledTaskQueue = GlobalEventExecutor.this.scheduledTaskQueue;
Queue<RunnableScheduledFutureNode<?>> scheduledTaskQueue = scheduledTaskQueue();
// Terminate if there is no task in the queue (except the noop task).
if (taskQueue.isEmpty() && (scheduledTaskQueue == null || scheduledTaskQueue.size() == 1)) {
if (taskQueue.isEmpty() && scheduledTaskQueue.size() <= 1) {
// Mark the current thread as stopped.
// The following CAS must always success and must be uncontended,
// because only one thread should be running at the same time.
@ -265,7 +271,7 @@ public final class GlobalEventExecutor extends AbstractScheduledEventExecutor {
assert stopped;
// Check if there are pending entries added by execute() or schedule*() while we do CAS above.
if (taskQueue.isEmpty() && (scheduledTaskQueue == null || scheduledTaskQueue.size() == 1)) {
if (taskQueue.isEmpty() && scheduledTaskQueue.size() <= 1) {
// A) No new task was added and thus there's nothing to handle
// -> safe to terminate because there's nothing left to do
// B) A new thread started and handled all the new tasks.

View File

@ -20,6 +20,7 @@ import io.netty.util.internal.logging.InternalLoggerFactory;
import java.util.ArrayDeque;
import java.util.Queue;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
/**
@ -56,11 +57,6 @@ public final class ImmediateEventExecutor extends AbstractEventExecutor {
private ImmediateEventExecutor() { }
@Override
public boolean inEventLoop() {
return true;
}
@Override
public boolean inEventLoop(Thread thread) {
return true;
@ -138,6 +134,27 @@ public final class ImmediateEventExecutor extends AbstractEventExecutor {
return new ImmediateProgressivePromise<>(this);
}
@Override
public ScheduledFuture<?> schedule(Runnable command, long delay,
TimeUnit unit) {
throw new UnsupportedOperationException();
}
@Override
public <V> ScheduledFuture<V> schedule(Callable<V> callable, long delay, TimeUnit unit) {
throw new UnsupportedOperationException();
}
@Override
public ScheduledFuture<?> scheduleAtFixedRate(Runnable command, long initialDelay, long period, TimeUnit unit) {
throw new UnsupportedOperationException();
}
@Override
public ScheduledFuture<?> scheduleWithFixedDelay(Runnable command, long initialDelay, long delay, TimeUnit unit) {
throw new UnsupportedOperationException();
}
static class ImmediatePromise<V> extends DefaultPromise<V> {
ImmediatePromise(EventExecutor executor) {
super(executor);

View File

@ -15,36 +15,62 @@
*/
package io.netty.util.concurrent;
import io.netty.util.internal.EmptyArrays;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.Set;
import java.util.List;
import java.util.concurrent.Executor;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
/**
* Abstract base class for {@link EventExecutorGroup} implementations that handles their tasks with multiple threads at
* {@link EventExecutorGroup} implementation that handles their tasks with multiple threads at
* the same time.
*/
public abstract class MultithreadEventExecutorGroup extends AbstractEventExecutorGroup {
public class MultithreadEventExecutorGroup extends AbstractEventExecutorGroup {
private final EventExecutor[] children;
private final Set<EventExecutor> readonlyChildren;
private final List<EventExecutor> readonlyChildren;
private final AtomicInteger terminatedChildren = new AtomicInteger();
private final Promise<?> terminationFuture = new DefaultPromise(GlobalEventExecutor.INSTANCE);
private final EventExecutorChooserFactory.EventExecutorChooser chooser;
private final boolean powerOfTwo;
/**
* Create a new instance.
*
* @param nThreads the number of threads that will be used by this instance.
* @param threadFactory the ThreadFactory to use, or {@code null} if the default should be used.
* @param args arguments which will passed to each {@link #newChild(Executor, Object...)} call
*/
protected MultithreadEventExecutorGroup(int nThreads, ThreadFactory threadFactory, Object... args) {
this(nThreads, threadFactory == null ? null : new ThreadPerTaskExecutor(threadFactory), args);
public MultithreadEventExecutorGroup(int nThreads, ThreadFactory threadFactory) {
this(nThreads, threadFactory, SingleThreadEventExecutor.DEFAULT_MAX_PENDING_EXECUTOR_TASKS,
RejectedExecutionHandlers.reject());
}
/**
* Create a new instance.
*
* @param nThreads the number of threads that will be used by this instance.
* @param executor the {@link Executor} to use, or {@code null} if the default should be used.
*/
public MultithreadEventExecutorGroup(int nThreads, Executor executor) {
this(nThreads, executor, SingleThreadEventExecutor.DEFAULT_MAX_PENDING_EXECUTOR_TASKS,
RejectedExecutionHandlers.reject());
}
/**
* Create a new instance.
*
* @param nThreads the number of threads that will be used by this instance.
* @param threadFactory the {@link ThreadFactory} to use, or {@code null} if the default should be used.
* @param maxPendingTasks the maximum number of pending tasks before new tasks will be rejected.
* @param rejectedHandler the {@link RejectedExecutionHandler} to use.
*/
public MultithreadEventExecutorGroup(int nThreads, ThreadFactory threadFactory,
int maxPendingTasks, RejectedExecutionHandler rejectedHandler) {
this(nThreads, threadFactory, maxPendingTasks, rejectedHandler, EmptyArrays.EMPTY_OBJECTS);
}
/**
@ -52,10 +78,28 @@ public abstract class MultithreadEventExecutorGroup extends AbstractEventExecuto
*
* @param nThreads the number of threads that will be used by this instance.
* @param executor the Executor to use, or {@code null} if the default should be used.
* @param args arguments which will passed to each {@link #newChild(Executor, Object...)} call
* @param maxPendingTasks the maximum number of pending tasks before new tasks will be rejected.
* @param rejectedHandler the {@link RejectedExecutionHandler} to use.
*/
protected MultithreadEventExecutorGroup(int nThreads, Executor executor, Object... args) {
this(nThreads, executor, DefaultEventExecutorChooserFactory.INSTANCE, args);
public MultithreadEventExecutorGroup(int nThreads, Executor executor,
int maxPendingTasks, RejectedExecutionHandler rejectedHandler) {
this(nThreads, executor, maxPendingTasks, rejectedHandler, EmptyArrays.EMPTY_OBJECTS);
}
/**
* Create a new instance.
*
* @param nThreads the number of threads that will be used by this instance.
* @param threadFactory the ThreadFactory to use, or {@code null} if the default should be used.
* @param maxPendingTasks the maximum number of pending tasks before new tasks will be rejected.
* @param rejectedHandler the {@link RejectedExecutionHandler} to use.
* @param args arguments which will passed to each {@link #newChild(Executor, int,
* RejectedExecutionHandler, Object...)} call
*/
protected MultithreadEventExecutorGroup(int nThreads, ThreadFactory threadFactory, int maxPendingTasks,
RejectedExecutionHandler rejectedHandler, Object... args) {
this(nThreads, threadFactory == null ? null : new ThreadPerTaskExecutor(threadFactory),
maxPendingTasks, rejectedHandler, args);
}
/**
@ -63,29 +107,31 @@ public abstract class MultithreadEventExecutorGroup extends AbstractEventExecuto
*
* @param nThreads the number of threads that will be used by this instance.
* @param executor the Executor to use, or {@code null} if the default should be used.
* @param chooserFactory the {@link EventExecutorChooserFactory} to use.
* @param args arguments which will passed to each {@link #newChild(Executor, Object...)} call
* @param maxPendingTasks the maximum number of pending tasks before new tasks will be rejected.
* @param rejectedHandler the {@link RejectedExecutionHandler} to use.
* @param args arguments which will passed to each {@link #newChild(Executor, int,
* RejectedExecutionHandler, Object...)} call
*/
protected MultithreadEventExecutorGroup(int nThreads, Executor executor,
EventExecutorChooserFactory chooserFactory, Object... args) {
protected MultithreadEventExecutorGroup(int nThreads, Executor executor, int maxPendingTasks,
RejectedExecutionHandler rejectedHandler, Object... args) {
if (nThreads <= 0) {
throw new IllegalArgumentException(String.format("nThreads: %d (expected: > 0)", nThreads));
}
if (executor == null) {
executor = new ThreadPerTaskExecutor(newDefaultThreadFactory());
executor = new ThreadPerTaskExecutor(new DefaultThreadFactory(getClass()));
}
children = new EventExecutor[nThreads];
powerOfTwo = isPowerOfTwo(children.length);
for (int i = 0; i < nThreads; i ++) {
boolean success = false;
try {
children[i] = newChild(executor, args);
children[i] = newChild(executor, maxPendingTasks, rejectedHandler, args);
success = true;
} catch (Exception e) {
// TODO: Think about if this is a good exception type
throw new IllegalStateException("failed to create a child event loop", e);
throw new IllegalStateException("failed to create a child event executor", e);
} finally {
if (!success) {
for (int j = 0; j < i; j ++) {
@ -108,8 +154,6 @@ public abstract class MultithreadEventExecutorGroup extends AbstractEventExecuto
}
}
chooser = chooserFactory.newChooser(children);
final FutureListener<Object> terminationListener = new FutureListener<Object>() {
@Override
public void operationComplete(Future<Object> future) throws Exception {
@ -122,24 +166,37 @@ public abstract class MultithreadEventExecutorGroup extends AbstractEventExecuto
for (EventExecutor e: children) {
e.terminationFuture().addListener(terminationListener);
}
Set<EventExecutor> childrenSet = new LinkedHashSet<>(children.length);
Collections.addAll(childrenSet, children);
readonlyChildren = Collections.unmodifiableSet(childrenSet);
readonlyChildren = Collections.unmodifiableList(Arrays.asList(children));
}
protected ThreadFactory newDefaultThreadFactory() {
return new DefaultThreadFactory(getClass());
private final AtomicInteger idx = new AtomicInteger();
/**
* The {@link EventExecutor}s that are used by this {@link MultithreadEventExecutorGroup}.
*/
protected final List<EventExecutor> executors() {
return readonlyChildren;
}
/**
* Returns the next {@link EventExecutor} to use. The default implementation will use round-robin, but you may
* override this to change the selection algorithm.
*/
@Override
public EventExecutor next() {
return chooser.next();
if (powerOfTwo) {
return children[idx.getAndIncrement() & children.length - 1];
}
return children[Math.abs(idx.getAndIncrement() % children.length)];
}
private static boolean isPowerOfTwo(int val) {
return (val & -val) == val;
}
@Override
public Iterator<EventExecutor> iterator() {
return readonlyChildren.iterator();
return executors().iterator();
}
/**
@ -147,18 +204,25 @@ public abstract class MultithreadEventExecutorGroup extends AbstractEventExecuto
* 1:1 to the threads it use.
*/
public final int executorCount() {
return children.length;
return executors().size();
}
/**
* Create a new EventExecutor which will later then accessible via the {@link #next()} method. This method will be
* Create a new EventExecutor which will later then accessible via the {@link #next()} method. This method will be
* called for each thread that will serve this {@link MultithreadEventExecutorGroup}.
*
* As this method is called from within the constructor you can only use the parameters passed into the method when
* overriding this method.
*/
protected abstract EventExecutor newChild(Executor executor, Object... args) throws Exception;
protected EventExecutor newChild(Executor executor, int maxPendingTasks,
RejectedExecutionHandler rejectedExecutionHandler,
Object... args) {
assert args.length == 0;
return new SingleThreadEventExecutor(executor, maxPendingTasks, rejectedExecutionHandler);
}
@Override
public Future<?> shutdownGracefully(long quietPeriod, long timeout, TimeUnit unit) {
public final Future<?> shutdownGracefully(long quietPeriod, long timeout, TimeUnit unit) {
for (EventExecutor l: children) {
l.shutdownGracefully(quietPeriod, timeout, unit);
}
@ -166,20 +230,20 @@ public abstract class MultithreadEventExecutorGroup extends AbstractEventExecuto
}
@Override
public Future<?> terminationFuture() {
public final Future<?> terminationFuture() {
return terminationFuture;
}
@Override
@Deprecated
public void shutdown() {
public final void shutdown() {
for (EventExecutor l: children) {
l.shutdown();
}
}
@Override
public boolean isShuttingDown() {
public final boolean isShuttingDown() {
for (EventExecutor l: children) {
if (!l.isShuttingDown()) {
return false;
@ -189,7 +253,7 @@ public abstract class MultithreadEventExecutorGroup extends AbstractEventExecuto
}
@Override
public boolean isShutdown() {
public final boolean isShutdown() {
for (EventExecutor l: children) {
if (!l.isShutdown()) {
return false;
@ -199,7 +263,7 @@ public abstract class MultithreadEventExecutorGroup extends AbstractEventExecuto
}
@Override
public boolean isTerminated() {
public final boolean isTerminated() {
for (EventExecutor l: children) {
if (!l.isTerminated()) {
return false;
@ -209,7 +273,7 @@ public abstract class MultithreadEventExecutorGroup extends AbstractEventExecuto
}
@Override
public boolean awaitTermination(long timeout, TimeUnit unit)
public final boolean awaitTermination(long timeout, TimeUnit unit)
throws InterruptedException {
long deadline = System.nanoTime() + unit.toNanos(timeout);
loop: for (EventExecutor l: children) {

View File

@ -224,7 +224,6 @@ public final class NonStickyEventExecutorGroup implements EventExecutorGroup {
private final int maxTaskExecutePerRun;
NonStickyOrderedEventExecutor(EventExecutor executor, int maxTaskExecutePerRun) {
super(executor);
this.executor = executor;
this.maxTaskExecutePerRun = maxTaskExecutePerRun;
}
@ -287,11 +286,6 @@ public final class NonStickyEventExecutorGroup implements EventExecutorGroup {
return false;
}
@Override
public boolean inEventLoop() {
return false;
}
@Override
public boolean isShuttingDown() {
return executor.isShutdown();
@ -344,5 +338,28 @@ public final class NonStickyEventExecutorGroup implements EventExecutorGroup {
}
}
}
@Override
public ScheduledFuture<?> schedule(Runnable command, long delay,
TimeUnit unit) {
throw new UnsupportedOperationException();
}
@Override
public <V> ScheduledFuture<V> schedule(Callable<V> callable, long delay, TimeUnit unit) {
throw new UnsupportedOperationException();
}
@Override
public ScheduledFuture<?> scheduleAtFixedRate(
Runnable command, long initialDelay, long period, TimeUnit unit) {
throw new UnsupportedOperationException();
}
@Override
public ScheduledFuture<?> scheduleWithFixedDelay(
Runnable command, long initialDelay, long delay, TimeUnit unit) {
throw new UnsupportedOperationException();
}
}
}

View File

@ -125,6 +125,26 @@ class PromiseTask<V> extends DefaultPromise<V> implements RunnableFuture<V> {
return super.setUncancellable();
}
@Override
public Promise<V> addListener(GenericFutureListener<? extends Future<? super V>> listener) {
return super.addListener(listener);
}
@Override
public Promise<V> addListeners(GenericFutureListener<? extends Future<? super V>>... listeners) {
return super.addListeners(listeners);
}
@Override
public Promise<V> removeListener(GenericFutureListener<? extends Future<? super V>> listener) {
return super.removeListener(listener);
}
@Override
public Promise<V> removeListeners(GenericFutureListener<? extends Future<? super V>>... listeners) {
return super.removeListeners(listeners);
}
@Override
protected StringBuilder toStringBuilder() {
StringBuilder buf = super.toStringBuilder();

View File

@ -0,0 +1,47 @@
/*
* Copyright 2018 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.util.concurrent;
/**
* A combination of {@link java.util.concurrent.RunnableFuture} and {@link Future}.
*/
@SuppressWarnings("ClassNameSameAsAncestorName")
public interface RunnableFuture<V> extends java.util.concurrent.RunnableFuture<V>, Future<V> {
@Override
RunnableFuture<V> addListener(GenericFutureListener<? extends Future<? super V>> listener);
@Override
RunnableFuture<V> addListeners(GenericFutureListener<? extends Future<? super V>>... listeners);
@Override
RunnableFuture<V> removeListener(GenericFutureListener<? extends Future<? super V>> listener);
@Override
RunnableFuture<V> removeListeners(GenericFutureListener<? extends Future<? super V>>... listeners);
@Override
RunnableFuture<V> sync() throws InterruptedException;
@Override
RunnableFuture<V> syncUninterruptibly();
@Override
RunnableFuture<V> await() throws InterruptedException;
@Override
RunnableFuture<V> awaitUninterruptibly();
}

View File

@ -0,0 +1,192 @@
/*
* Copyright 2018 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.util.concurrent;
import io.netty.util.internal.ObjectUtil;
import io.netty.util.internal.StringUtil;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
final class RunnableFutureAdapter<V> implements RunnableFuture<V> {
private final Promise<V> promise;
private final Callable<V> task;
RunnableFutureAdapter(Promise<V> promise, Callable<V> task) {
this.promise = ObjectUtil.checkNotNull(promise, "promise");
this.task = ObjectUtil.checkNotNull(task, "task");
}
@Override
public boolean isSuccess() {
return promise.isSuccess();
}
@Override
public boolean isCancellable() {
return promise.isCancellable();
}
@Override
public Throwable cause() {
return promise.cause();
}
@Override
public RunnableFuture<V> addListener(GenericFutureListener<? extends Future<? super V>> listener) {
promise.addListener(listener);
return this;
}
@Override
public RunnableFuture<V> addListeners(GenericFutureListener<? extends Future<? super V>>... listeners) {
promise.addListeners(listeners);
return this;
}
@Override
public RunnableFuture<V> removeListener(GenericFutureListener<? extends Future<? super V>> listener) {
promise.removeListener(listener);
return this;
}
@Override
public RunnableFuture<V> removeListeners(GenericFutureListener<? extends Future<? super V>>... listeners) {
promise.removeListeners(listeners);
return this;
}
@Override
public RunnableFuture<V> sync() throws InterruptedException {
promise.sync();
return this;
}
@Override
public RunnableFuture<V> syncUninterruptibly() {
promise.syncUninterruptibly();
return this;
}
@Override
public RunnableFuture<V> await() throws InterruptedException {
promise.await();
return this;
}
@Override
public RunnableFuture<V> awaitUninterruptibly() {
promise.awaitUninterruptibly();
return this;
}
@Override
public boolean await(long timeout, TimeUnit unit) throws InterruptedException {
return promise.await(timeout, unit);
}
@Override
public boolean await(long timeoutMillis) throws InterruptedException {
return promise.await(timeoutMillis);
}
@Override
public boolean awaitUninterruptibly(long timeout, TimeUnit unit) {
return promise.awaitUninterruptibly(timeout, unit);
}
@Override
public boolean awaitUninterruptibly(long timeoutMillis) {
return promise.awaitUninterruptibly(timeoutMillis);
}
@Override
public V getNow() {
return promise.getNow();
}
@Override
public void run() {
try {
if (promise.setUncancellable()) {
V result = task.call();
promise.setSuccess(result);
}
} catch (Throwable e) {
promise.setFailure(e);
}
}
@Override
public boolean cancel(boolean mayInterruptIfRunning) {
return promise.cancel(mayInterruptIfRunning);
}
@Override
public boolean isCancelled() {
return promise.isCancelled();
}
@Override
public boolean isDone() {
return promise.isDone();
}
@Override
public V get() throws InterruptedException, ExecutionException {
return promise.get();
}
@Override
public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
return promise.get(timeout, unit);
}
@Override
public String toString() {
StringBuilder buf = new StringBuilder(64)
.append(StringUtil.simpleClassName(this))
.append('@')
.append(Integer.toHexString(hashCode()));
if (!isDone()) {
buf.append("(incomplete)");
} else {
Throwable cause = cause();
if (cause != null) {
buf.append("(failure: ")
.append(cause)
.append(')');
} else {
Object result = getNow();
if (result == null) {
buf.append("(success)");
} else {
buf.append("(success: ")
.append(result)
.append(')');
}
}
}
return buf.append(" task: ")
.append(task)
.append(')').toString();
}
}

View File

@ -0,0 +1,54 @@
/*
* Copyright 2018 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.util.concurrent;
/**
* A combination of {@link java.util.concurrent.RunnableScheduledFuture}, {@link RunnableFuture} and
* {@link ScheduledFuture}.
*/
@SuppressWarnings("ClassNameSameAsAncestorName")
public interface RunnableScheduledFuture<V> extends
java.util.concurrent.RunnableScheduledFuture<V>, RunnableFuture<V>, ScheduledFuture<V> {
/**
* Returns the deadline in nanos when the {@link #run()} method should be called again.
*/
long deadlineNanos();
/**
* Returns the delay in nanos when the {@link #run()} method should be called again.
*/
long delayNanos();
/**
* Returns the delay in nanos (taking the given {@code currentTimeNanos} into account) when the
* {@link #run()} method should be called again.
*/
long delayNanos(long currentTimeNanos);
@Override
RunnableScheduledFuture<V> addListener(GenericFutureListener<? extends Future<? super V>> listener);
@Override
RunnableScheduledFuture<V> addListeners(GenericFutureListener<? extends Future<? super V>>... listeners);
@Override
RunnableScheduledFuture<V> removeListener(GenericFutureListener<? extends Future<? super V>> listener);
@Override
RunnableScheduledFuture<V> removeListeners(GenericFutureListener<? extends Future<? super V>>... listeners);
}

View File

@ -0,0 +1,299 @@
/*
* Copyright 2013 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.util.concurrent;
import io.netty.util.internal.DefaultPriorityQueue;
import io.netty.util.internal.ObjectUtil;
import io.netty.util.internal.StringUtil;
import java.util.concurrent.Callable;
import java.util.concurrent.Delayed;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicLong;
@SuppressWarnings("ComparableImplementedButEqualsNotOverridden")
final class RunnableScheduledFutureAdapter<V> implements RunnableScheduledFuture<V>,
AbstractScheduledEventExecutor.RunnableScheduledFutureNode<V> {
private static final AtomicLong NEXT_TASK_ID = new AtomicLong();
private final long id = NEXT_TASK_ID.getAndIncrement();
private long deadlineNanos;
/* 0 - no repeat, >0 - repeat at fixed rate, <0 - repeat with fixed delay */
private final long periodNanos;
private int queueIndex = INDEX_NOT_IN_QUEUE;
private final AbstractScheduledEventExecutor executor;
private final Promise<V> promise;
private final Callable<V> callable;
RunnableScheduledFutureAdapter(AbstractScheduledEventExecutor executor, Promise<V> promise, Callable<V> callable,
long deadlineNanos, long periodNanos) {
this.executor = ObjectUtil.checkNotNull(executor, "executor");
this.promise = ObjectUtil.checkNotNull(promise, "promise");
this.callable = ObjectUtil.checkNotNull(callable, "callable");
this.deadlineNanos = deadlineNanos;
this.periodNanos = periodNanos;
}
@Override
public long deadlineNanos() {
return deadlineNanos;
}
@Override
public long delayNanos() {
return Math.max(0, deadlineNanos() - AbstractScheduledEventExecutor.nanoTime());
}
@Override
public long delayNanos(long currentTimeNanos) {
return Math.max(0, deadlineNanos() - (currentTimeNanos - AbstractScheduledEventExecutor.START_TIME));
}
@Override
public long getDelay(TimeUnit unit) {
return unit.convert(delayNanos(), TimeUnit.NANOSECONDS);
}
@Override
public int compareTo(Delayed o) {
if (this == o) {
return 0;
}
RunnableScheduledFutureAdapter<?> that = (RunnableScheduledFutureAdapter<?>) o;
long d = deadlineNanos() - that.deadlineNanos();
if (d < 0) {
return -1;
} else if (d > 0) {
return 1;
} else if (id < that.id) {
return -1;
} else if (id == that.id) {
throw new Error();
} else {
return 1;
}
}
@Override
public void run() {
try {
if (!isPeriodic()) {
if (promise.setUncancellable()) {
V result = callable.call();
promise.setSuccess(result);
}
} else {
// check if is done as it may was cancelled
if (!isCancelled()) {
callable.call();
if (!executor.isShutdown()) {
long p = periodNanos;
if (p > 0) {
deadlineNanos += p;
} else {
deadlineNanos = AbstractScheduledEventExecutor.nanoTime() - p;
}
if (!isCancelled()) {
executor.schedule(this);
}
}
}
}
} catch (Throwable cause) {
promise.setFailure(cause);
}
}
/**
* {@inheritDoc}
*
* @param mayInterruptIfRunning this value has no effect in this implementation.
*/
@Override
public boolean cancel(boolean mayInterruptIfRunning) {
boolean canceled = promise.cancel(mayInterruptIfRunning);
if (canceled) {
executor.removeScheduled(this);
}
return canceled;
}
@Override
public boolean isSuccess() {
return promise.isSuccess();
}
@Override
public boolean isCancellable() {
return promise.isCancellable();
}
@Override
public Throwable cause() {
return promise.cause();
}
@Override
public RunnableScheduledFuture<V> addListener(GenericFutureListener<? extends Future<? super V>> listener) {
promise.addListener(listener);
return this;
}
@Override
public RunnableScheduledFuture<V> addListeners(GenericFutureListener<? extends Future<? super V>>... listeners) {
promise.addListeners(listeners);
return this;
}
@Override
public RunnableScheduledFuture<V> removeListener(GenericFutureListener<? extends Future<? super V>> listener) {
promise.removeListener(listener);
return this;
}
@Override
public RunnableScheduledFuture<V> removeListeners(GenericFutureListener<? extends Future<? super V>>... listeners) {
promise.removeListeners(listeners);
return this;
}
@Override
public RunnableScheduledFuture<V> sync() throws InterruptedException {
promise.sync();
return this;
}
@Override
public RunnableScheduledFuture<V> syncUninterruptibly() {
promise.syncUninterruptibly();
return this;
}
@Override
public RunnableScheduledFuture<V> await() throws InterruptedException {
promise.await();
return this;
}
@Override
public RunnableScheduledFuture<V> awaitUninterruptibly() {
promise.awaitUninterruptibly();
return this;
}
@Override
public boolean await(long timeout, TimeUnit unit) throws InterruptedException {
return promise.await(timeout, unit);
}
@Override
public boolean await(long timeoutMillis) throws InterruptedException {
return promise.await(timeoutMillis);
}
@Override
public boolean awaitUninterruptibly(long timeout, TimeUnit unit) {
return promise.awaitUninterruptibly(timeout, unit);
}
@Override
public boolean awaitUninterruptibly(long timeoutMillis) {
return promise.awaitUninterruptibly(timeoutMillis);
}
@Override
public V getNow() {
return promise.getNow();
}
@Override
public boolean isPeriodic() {
return periodNanos != 0;
}
@Override
public boolean isCancelled() {
return promise.isCancelled();
}
@Override
public boolean isDone() {
return promise.isDone();
}
@Override
public V get() throws InterruptedException, ExecutionException {
return promise.get();
}
@Override
public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
return promise.get(timeout, unit);
}
@Override
public int priorityQueueIndex(DefaultPriorityQueue<?> queue) {
return queueIndex;
}
@Override
public void priorityQueueIndex(DefaultPriorityQueue<?> queue, int i) {
queueIndex = i;
}
@Override
public String toString() {
StringBuilder buf = new StringBuilder(64)
.append(StringUtil.simpleClassName(this))
.append('@')
.append(Integer.toHexString(hashCode()));
if (!isDone()) {
buf.append("(incomplete)");
} else {
Throwable cause = cause();
if (cause != null) {
buf.append("(failure: ")
.append(cause)
.append(')');
} else {
Object result = getNow();
if (result == null) {
buf.append("(success)");
} else {
buf.append("(success: ")
.append(result)
.append(')');
}
}
}
return buf.append(" task: ")
.append(callable)
.append(", id: ")
.append(id)
.append(", deadline: ")
.append(deadlineNanos)
.append(", period: ")
.append(periodNanos)
.append(')').toString();
}
}

View File

@ -1,197 +0,0 @@
/*
* Copyright 2013 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.util.concurrent;
import io.netty.util.internal.DefaultPriorityQueue;
import io.netty.util.internal.PriorityQueueNode;
import java.util.Queue;
import java.util.concurrent.Callable;
import java.util.concurrent.Delayed;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
@SuppressWarnings("ComparableImplementedButEqualsNotOverridden")
final class ScheduledFutureTask<V> extends PromiseTask<V> implements ScheduledFuture<V>, PriorityQueueNode {
private static final AtomicLong nextTaskId = new AtomicLong();
private static final long START_TIME = System.nanoTime();
static long nanoTime() {
return System.nanoTime() - START_TIME;
}
static long deadlineNanos(long delay) {
long deadlineNanos = nanoTime() + delay;
// Guard against overflow
return deadlineNanos < 0 ? Long.MAX_VALUE : deadlineNanos;
}
private final long id = nextTaskId.getAndIncrement();
private long deadlineNanos;
/* 0 - no repeat, >0 - repeat at fixed rate, <0 - repeat with fixed delay */
private final long periodNanos;
private int queueIndex = INDEX_NOT_IN_QUEUE;
ScheduledFutureTask(
AbstractScheduledEventExecutor executor,
Runnable runnable, V result, long nanoTime) {
this(executor, toCallable(runnable, result), nanoTime);
}
ScheduledFutureTask(
AbstractScheduledEventExecutor executor,
Callable<V> callable, long nanoTime, long period) {
super(executor, callable);
if (period == 0) {
throw new IllegalArgumentException("period: 0 (expected: != 0)");
}
deadlineNanos = nanoTime;
periodNanos = period;
}
ScheduledFutureTask(
AbstractScheduledEventExecutor executor,
Callable<V> callable, long nanoTime) {
super(executor, callable);
deadlineNanos = nanoTime;
periodNanos = 0;
}
@Override
protected EventExecutor executor() {
return super.executor();
}
public long deadlineNanos() {
return deadlineNanos;
}
public long delayNanos() {
return Math.max(0, deadlineNanos() - nanoTime());
}
public long delayNanos(long currentTimeNanos) {
return Math.max(0, deadlineNanos() - (currentTimeNanos - START_TIME));
}
@Override
public long getDelay(TimeUnit unit) {
return unit.convert(delayNanos(), TimeUnit.NANOSECONDS);
}
@Override
public int compareTo(Delayed o) {
if (this == o) {
return 0;
}
ScheduledFutureTask<?> that = (ScheduledFutureTask<?>) o;
long d = deadlineNanos() - that.deadlineNanos();
if (d < 0) {
return -1;
} else if (d > 0) {
return 1;
} else if (id < that.id) {
return -1;
} else if (id == that.id) {
throw new Error();
} else {
return 1;
}
}
@Override
public void run() {
assert executor().inEventLoop();
try {
if (periodNanos == 0) {
if (setUncancellableInternal()) {
V result = task.call();
setSuccessInternal(result);
}
} else {
// check if is done as it may was cancelled
if (!isCancelled()) {
task.call();
if (!executor().isShutdown()) {
long p = periodNanos;
if (p > 0) {
deadlineNanos += p;
} else {
deadlineNanos = nanoTime() - p;
}
if (!isCancelled()) {
// scheduledTaskQueue can never be null as we lazy init it before submit the task!
Queue<ScheduledFutureTask<?>> scheduledTaskQueue =
((AbstractScheduledEventExecutor) executor()).scheduledTaskQueue;
assert scheduledTaskQueue != null;
scheduledTaskQueue.add(this);
}
}
}
}
} catch (Throwable cause) {
setFailureInternal(cause);
}
}
/**
* {@inheritDoc}
*
* @param mayInterruptIfRunning this value has no effect in this implementation.
*/
@Override
public boolean cancel(boolean mayInterruptIfRunning) {
boolean canceled = super.cancel(mayInterruptIfRunning);
if (canceled) {
((AbstractScheduledEventExecutor) executor()).removeScheduled(this);
}
return canceled;
}
boolean cancelWithoutRemove(boolean mayInterruptIfRunning) {
return super.cancel(mayInterruptIfRunning);
}
@Override
protected StringBuilder toStringBuilder() {
StringBuilder buf = super.toStringBuilder();
buf.setCharAt(buf.length() - 1, ',');
return buf.append(" id: ")
.append(id)
.append(", deadline: ")
.append(deadlineNanos)
.append(", period: ")
.append(periodNanos)
.append(')');
}
@Override
public int priorityQueueIndex(DefaultPriorityQueue<?> queue) {
return queueIndex;
}
@Override
public void priorityQueueIndex(DefaultPriorityQueue<?> queue, int i) {
queueIndex = i;
}
}

View File

@ -18,7 +18,6 @@ package io.netty.util.concurrent;
import io.netty.util.internal.ObjectUtil;
import io.netty.util.internal.PlatformDependent;
import io.netty.util.internal.SystemPropertyUtil;
import io.netty.util.internal.UnstableApi;
import io.netty.util.internal.logging.InternalLogger;
import io.netty.util.internal.logging.InternalLoggerFactory;
@ -43,12 +42,12 @@ import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
/**
* Abstract base class for {@link OrderedEventExecutor}'s that execute all its submitted tasks in a single thread.
* {@link OrderedEventExecutor}'s implementation that execute all its submitted tasks in a single thread.
*
*/
public abstract class SingleThreadEventExecutor extends AbstractScheduledEventExecutor implements OrderedEventExecutor {
public class SingleThreadEventExecutor extends AbstractScheduledEventExecutor implements OrderedEventExecutor {
static final int DEFAULT_MAX_PENDING_EXECUTOR_TASKS = Math.max(16,
protected static final int DEFAULT_MAX_PENDING_EXECUTOR_TASKS = Math.max(16,
SystemPropertyUtil.getInt("io.netty.eventexecutor.maxPendingTasks", Integer.MAX_VALUE));
private static final InternalLogger logger =
@ -90,7 +89,6 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
private final Semaphore threadLock = new Semaphore(0);
private final Set<Runnable> shutdownHooks = new LinkedHashSet<>();
private final boolean addTaskWakesUp;
private final int maxPendingTasks;
private final RejectedExecutionHandler rejectedExecutionHandler;
private long lastExecutionTime;
@ -106,79 +104,66 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
/**
* Create a new instance
*
* @param parent the {@link EventExecutorGroup} which is the parent of this instance and belongs to it
* @param threadFactory the {@link ThreadFactory} which will be used for the used {@link Thread}
* @param addTaskWakesUp {@code true} if and only if invocation of {@link #addTask(Runnable)} will wake up the
* executor thread
*/
protected SingleThreadEventExecutor(
EventExecutorGroup parent, ThreadFactory threadFactory, boolean addTaskWakesUp) {
this(parent, new ThreadPerTaskExecutor(threadFactory), addTaskWakesUp);
public SingleThreadEventExecutor() {
this(new DefaultThreadFactory(SingleThreadEventExecutor.class));
}
/**
* Create a new instance
*
* @param threadFactory the {@link ThreadFactory} which will be used for the used {@link Thread}
*/
public SingleThreadEventExecutor(ThreadFactory threadFactory) {
this(new ThreadPerTaskExecutor(threadFactory));
}
/**
* Create a new instance
*
* @param parent the {@link EventExecutorGroup} which is the parent of this instance and belongs to it
* @param threadFactory the {@link ThreadFactory} which will be used for the used {@link Thread}
* @param addTaskWakesUp {@code true} if and only if invocation of {@link #addTask(Runnable)} will wake up the
* executor thread
* @param maxPendingTasks the maximum number of pending tasks before new tasks will be rejected.
* @param rejectedHandler the {@link RejectedExecutionHandler} to use.
*/
protected SingleThreadEventExecutor(
EventExecutorGroup parent, ThreadFactory threadFactory,
boolean addTaskWakesUp, int maxPendingTasks, RejectedExecutionHandler rejectedHandler) {
this(parent, new ThreadPerTaskExecutor(threadFactory), addTaskWakesUp, maxPendingTasks, rejectedHandler);
public SingleThreadEventExecutor(ThreadFactory threadFactory,
int maxPendingTasks, RejectedExecutionHandler rejectedHandler) {
this(new ThreadPerTaskExecutor(threadFactory), maxPendingTasks, rejectedHandler);
}
/**
* Create a new instance
*
* @param parent the {@link EventExecutorGroup} which is the parent of this instance and belongs to it
* @param executor the {@link Executor} which will be used for executing
* @param addTaskWakesUp {@code true} if and only if invocation of {@link #addTask(Runnable)} will wake up the
* executor thread
*/
protected SingleThreadEventExecutor(EventExecutorGroup parent, Executor executor, boolean addTaskWakesUp) {
this(parent, executor, addTaskWakesUp, DEFAULT_MAX_PENDING_EXECUTOR_TASKS, RejectedExecutionHandlers.reject());
public SingleThreadEventExecutor(Executor executor) {
this(executor, DEFAULT_MAX_PENDING_EXECUTOR_TASKS, RejectedExecutionHandlers.reject());
}
/**
* Create a new instance
*
* @param parent the {@link EventExecutorGroup} which is the parent of this instance and belongs to it
* @param executor the {@link Executor} which will be used for executing
* @param addTaskWakesUp {@code true} if and only if invocation of {@link #addTask(Runnable)} will wake up the
* executor thread
* @param maxPendingTasks the maximum number of pending tasks before new tasks will be rejected.
* @param rejectedHandler the {@link RejectedExecutionHandler} to use.
*/
protected SingleThreadEventExecutor(EventExecutorGroup parent, Executor executor,
boolean addTaskWakesUp, int maxPendingTasks,
RejectedExecutionHandler rejectedHandler) {
super(parent);
this.addTaskWakesUp = addTaskWakesUp;
this.maxPendingTasks = Math.max(16, maxPendingTasks);
public SingleThreadEventExecutor(Executor executor, int maxPendingTasks, RejectedExecutionHandler rejectedHandler) {
this.executor = ObjectUtil.checkNotNull(executor, "executor");
taskQueue = newTaskQueue(this.maxPendingTasks);
taskQueue = newTaskQueue(Math.max(16, maxPendingTasks));
this.addTaskWakesUp = taskQueue instanceof BlockingQueue;
rejectedExecutionHandler = ObjectUtil.checkNotNull(rejectedHandler, "rejectedHandler");
}
/**
* @deprecated Please use and override {@link #newTaskQueue(int)}.
*/
@Deprecated
protected Queue<Runnable> newTaskQueue() {
return newTaskQueue(maxPendingTasks);
}
/**
* Create a new {@link Queue} which will holds the tasks to execute. This default implementation will return a
* {@link LinkedBlockingQueue} but if your sub-class of {@link SingleThreadEventExecutor} will not do any blocking
* calls on the this {@link Queue} it may make sense to {@code @Override} this and return some more performant
* implementation that does not support blocking operations at all.
*
* Be aware that the implementation of {@link #run()} depends on a {@link BlockingQueue} so you will need to
* override {@link #run()} as well if you return a non {@link BlockingQueue} from this method.
*
* As this method is called from within the constructor you can only use the parameters passed into the method when
* overriding this method.
*/
protected Queue<Runnable> newTaskQueue(int maxPendingTasks) {
return new LinkedBlockingQueue<>(maxPendingTasks);
@ -187,7 +172,7 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
/**
* Interrupt the current running {@link Thread}.
*/
protected void interruptThread() {
protected final void interruptThread() {
Thread currentThread = thread;
if (currentThread == null) {
interrupted = true;
@ -198,8 +183,10 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
/**
* @see Queue#poll()
*
* This method must be called from the {@link EventExecutor} thread.
*/
protected Runnable pollTask() {
protected final Runnable pollTask() {
assert inEventLoop();
for (;;) {
@ -215,12 +202,14 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
* Take the next {@link Runnable} from the task queue and so will block if no task is currently present.
* <p>
* Be aware that this method will throw an {@link UnsupportedOperationException} if the task queue, which was
* created via {@link #newTaskQueue()}, does not implement {@link BlockingQueue}.
* created via {@link #newTaskQueue(int)}, does not implement {@link BlockingQueue}.
* </p>
*
* This method must be called from the {@link EventExecutor} thread.
*
* @return {@code null} if the executor thread has been interrupted or waken up.
*/
protected Runnable takeTask() {
protected final Runnable takeTask() {
assert inEventLoop();
if (!(taskQueue instanceof BlockingQueue)) {
throw new UnsupportedOperationException();
@ -228,7 +217,7 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
BlockingQueue<Runnable> taskQueue = (BlockingQueue<Runnable>) this.taskQueue;
for (;;) {
ScheduledFutureTask<?> scheduledTask = peekScheduledTask();
RunnableScheduledFuture<?> scheduledTask = peekScheduledTask();
if (scheduledTask == null) {
Runnable task = null;
try {
@ -269,11 +258,11 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
private boolean fetchFromScheduledTaskQueue() {
long nanoTime = AbstractScheduledEventExecutor.nanoTime();
Runnable scheduledTask = pollScheduledTask(nanoTime);
RunnableScheduledFuture<?> scheduledTask = pollScheduledTask(nanoTime);
while (scheduledTask != null) {
if (!taskQueue.offer(scheduledTask)) {
// No space left in the task queue add it back to the scheduledTaskQueue so we pick it up again.
scheduledTaskQueue().add((ScheduledFutureTask<?>) scheduledTask);
schedule(scheduledTask);
return false;
}
scheduledTask = pollScheduledTask(nanoTime);
@ -281,29 +270,17 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
return true;
}
/**
* @see Queue#peek()
*/
protected Runnable peekTask() {
assert inEventLoop();
return taskQueue.peek();
}
/**
* @see Queue#isEmpty()
*/
protected boolean hasTasks() {
assert inEventLoop();
protected final boolean hasTasks() {
return !taskQueue.isEmpty();
}
/**
* Return the number of tasks that are pending for processing.
*
* <strong>Be aware that this operation may be expensive as it depends on the internal implementation of the
* SingleThreadEventExecutor. So use it with care!</strong>
* Return the number of tasks that are pending for processing (excluding the scheduled tasks).
*/
public int pendingTasks() {
public final int pendingTasks() {
return taskQueue.size();
}
@ -311,16 +288,19 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
* Add a task to the task queue, or throws a {@link RejectedExecutionException} if this instance was shutdown
* before.
*/
protected void addTask(Runnable task) {
if (task == null) {
throw new NullPointerException("task");
}
private void addTask(Runnable task) {
if (!offerTask(task)) {
rejectedExecutionHandler.rejected(task, this);
}
}
final boolean offerTask(Runnable task) {
/**
* @see Queue#offer(Object)
*/
protected final boolean offerTask(Runnable task) {
if (task == null) {
throw new NullPointerException("task");
}
if (isShutdown()) {
reject();
}
@ -330,19 +310,18 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
/**
* @see Queue#remove(Object)
*/
protected boolean removeTask(Runnable task) {
if (task == null) {
throw new NullPointerException("task");
}
protected final boolean removeTask(Runnable task) {
return taskQueue.remove(task);
}
/**
* Poll all tasks from the task queue and run them via {@link Runnable#run()} method.
*
* This method must be called from the {@link EventExecutor} thread.
*
* @return {@code true} if and only if at least one task was run
*/
protected boolean runAllTasks() {
private boolean runAllTasks() {
boolean fetchedAll;
do {
fetchedAll = fetchFromScheduledTaskQueue();
@ -351,72 +330,61 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
return false;
}
for (;;) {
do {
try {
task.run();
} catch (Throwable t) {
logger.warn("A task raised an exception.", t);
}
task = pollTask();
if (task == null) {
break;
}
}
} while ((task = pollTask()) != null);
} while (!fetchedAll); // keep on processing until we fetched all scheduled tasks.
lastExecutionTime = ScheduledFutureTask.nanoTime();
updateLastExecutionTime();
return true;
}
/**
* Poll all tasks from the task queue and run them via {@link Runnable#run()} method. This method stops running
* the tasks in the task queue and returns if it ran longer than {@code timeoutNanos}.
* Poll all tasks from the task queue and run them via {@link Runnable#run()} method.
*
* This method must be called from the {@link EventExecutor} thread.
*
* @return the number of processed tasks.
*/
protected boolean runAllTasks(long timeoutNanos) {
fetchFromScheduledTaskQueue();
Runnable task = pollTask();
if (task == null) {
return false;
}
final long deadline = ScheduledFutureTask.nanoTime() + timeoutNanos;
long runTasks = 0;
long lastExecutionTime;
for (;;) {
try {
task.run();
} catch (Throwable t) {
logger.warn("A task raised an exception.", t);
}
runTasks ++;
// Check timeout every 64 tasks because nanoTime() is relatively expensive.
// XXX: Hard-coded value - will make it configurable if it is really a problem.
if ((runTasks & 0x3F) == 0) {
lastExecutionTime = ScheduledFutureTask.nanoTime();
if (lastExecutionTime >= deadline) {
protected int runAllTasks(int maxTasks) {
assert inEventLoop();
boolean fetchedAll;
int processedTasks = 0;
do {
fetchedAll = fetchFromScheduledTaskQueue();
for (; processedTasks < maxTasks; processedTasks++) {
Runnable task = pollTask();
if (task == null) {
break;
}
}
task = pollTask();
if (task == null) {
lastExecutionTime = ScheduledFutureTask.nanoTime();
break;
try {
task.run();
} catch (Throwable t) {
logger.warn("A task raised an exception.", t);
}
}
} while (!fetchedAll && processedTasks < maxTasks); // keep on processing until we fetched all scheduled tasks.
if (processedTasks > 0) {
// Only call if we at least executed one task.
updateLastExecutionTime();
}
this.lastExecutionTime = lastExecutionTime;
return true;
return processedTasks;
}
/**
* Returns the amount of time left until the scheduled task with the closest dead line is executed.
*
* This method must be called from the {@link EventExecutor} thread.
*/
protected long delayNanos(long currentTimeNanos) {
ScheduledFutureTask<?> scheduledTask = peekScheduledTask();
protected final long delayNanos(long currentTimeNanos) {
assert inEventLoop();
RunnableScheduledFuture<?> scheduledTask = peekScheduledTask();
if (scheduledTask == null) {
return SCHEDULE_PURGE_INTERVAL;
}
@ -427,10 +395,12 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
/**
* Returns the absolute point in time (relative to {@link #nanoTime()}) at which the the next
* closest scheduled task should run.
*
* This method must be called from the {@link EventExecutor} thread.
*/
@UnstableApi
protected long deadlineNanos() {
ScheduledFutureTask<?> scheduledTask = peekScheduledTask();
protected final long deadlineNanos() {
assert inEventLoop();
RunnableScheduledFuture<?> scheduledTask = peekScheduledTask();
if (scheduledTask == null) {
return nanoTime() + SCHEDULE_PURGE_INTERVAL;
}
@ -439,25 +409,42 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
/**
* Updates the internal timestamp that tells when a submitted task was executed most recently.
* {@link #runAllTasks()} and {@link #runAllTasks(long)} updates this timestamp automatically, and thus there's
* usually no need to call this method. However, if you take the tasks manually using {@link #takeTask()} or
* {@link #pollTask()}, you have to call this method at the end of task execution loop for accurate quiet period
* checks.
* {@link #runAllTasks(int)} updates this timestamp automatically, and thus there's usually no need to call this
* method. However, if you take the tasks manually using {@link #takeTask()} or {@link #pollTask()}, you have to
* call this method at the end of task execution loop if you execute a task for accurate quiet period checks.
*
* This method must be called from the {@link EventExecutor} thread.
*/
protected void updateLastExecutionTime() {
lastExecutionTime = ScheduledFutureTask.nanoTime();
protected final void updateLastExecutionTime() {
assert inEventLoop();
lastExecutionTime = nanoTime();
}
/**
* Run tasks that are submitted to this {@link SingleThreadEventExecutor}.
* The implementation depends on the fact that {@link #newTaskQueue(int)} returns a
* {@link BlockingQueue}. If you change this by overriding {@link #newTaskQueue(int)}
* be aware that you also need to override {@link #run()}.
*
* This method must be called from the {@link EventExecutor} thread.
*/
protected abstract void run();
protected void run() {
assert inEventLoop();
do {
Runnable task = takeTask();
if (task != null) {
task.run();
updateLastExecutionTime();
}
} while (!confirmShutdown());
}
/**
* Do nothing, sub-classes may override
* Do nothing, sub-classes may override.
*/
protected void cleanup() {
// NOOP
assert inEventLoop();
}
protected void wakeup(boolean inEventLoop) {
@ -469,14 +456,14 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
}
@Override
public boolean inEventLoop(Thread thread) {
public final boolean inEventLoop(Thread thread) {
return thread == this.thread;
}
/**
* Add a {@link Runnable} which will be executed on shutdown of this instance
*/
public void addShutdownHook(final Runnable task) {
public final void addShutdownHook(final Runnable task) {
if (inEventLoop()) {
shutdownHooks.add(task);
} else {
@ -492,7 +479,7 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
/**
* Remove a previous added {@link Runnable} as a shutdown hook
*/
public void removeShutdownHook(final Runnable task) {
public final void removeShutdownHook(final Runnable task) {
if (inEventLoop()) {
shutdownHooks.remove(task);
} else {
@ -523,14 +510,14 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
}
if (ran) {
lastExecutionTime = ScheduledFutureTask.nanoTime();
updateLastExecutionTime();
}
return ran;
}
@Override
public Future<?> shutdownGracefully(long quietPeriod, long timeout, TimeUnit unit) {
public final Future<?> shutdownGracefully(long quietPeriod, long timeout, TimeUnit unit) {
if (quietPeriod < 0) {
throw new IllegalArgumentException("quietPeriod: " + quietPeriod + " (expected >= 0)");
}
@ -570,6 +557,7 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
}
}
if (STATE_UPDATER.compareAndSet(this, oldState, newState)) {
//System.err.println(oldState + " " + newState + " " + this);
break;
}
}
@ -588,13 +576,13 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
}
@Override
public Future<?> terminationFuture() {
public final Future<?> terminationFuture() {
return terminationFuture;
}
@Override
@Deprecated
public void shutdown() {
public final void shutdown() {
if (isShutdown()) {
return;
}
@ -638,36 +626,36 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
}
@Override
public boolean isShuttingDown() {
public final boolean isShuttingDown() {
return state >= ST_SHUTTING_DOWN;
}
@Override
public boolean isShutdown() {
public final boolean isShutdown() {
return state >= ST_SHUTDOWN;
}
@Override
public boolean isTerminated() {
public final boolean isTerminated() {
return state == ST_TERMINATED;
}
/**
* Confirm that the shutdown if the instance should be done now!
*
* This method must be called from the {@link EventExecutor} thread.
*/
protected boolean confirmShutdown() {
protected final boolean confirmShutdown() {
assert inEventLoop();
if (!isShuttingDown()) {
return false;
}
if (!inEventLoop()) {
throw new IllegalStateException("must be invoked from an event loop");
}
cancelScheduledTasks();
if (gracefulShutdownStartTime == 0) {
gracefulShutdownStartTime = ScheduledFutureTask.nanoTime();
gracefulShutdownStartTime = nanoTime();
}
if (runAllTasks() || runShutdownHooks()) {
@ -686,7 +674,7 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
return false;
}
final long nanoTime = ScheduledFutureTask.nanoTime();
final long nanoTime = nanoTime();
if (isShutdown() || nanoTime - gracefulShutdownStartTime > gracefulShutdownTimeout) {
return true;
@ -711,7 +699,7 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
}
@Override
public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException {
public final boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException {
if (unit == null) {
throw new NullPointerException("unit");
}
@ -817,8 +805,11 @@ public abstract class SingleThreadEventExecutor extends AbstractScheduledEventEx
return threadProperties;
}
@SuppressWarnings("unused")
protected boolean wakesUpForTask(Runnable task) {
/**
* Returns {@code true} if {@link #wakeup(boolean)} should be called for this {@link Runnable}, {@code false}
* otherwise.
*/
protected boolean wakesUpForTask(@SuppressWarnings("unused") Runnable task) {
return true;
}

View File

@ -82,11 +82,6 @@ public final class UnorderedThreadPoolEventExecutor extends ScheduledThreadPoolE
return this;
}
@Override
public EventExecutorGroup parent() {
return this;
}
@Override
public boolean inEventLoop() {
return false;

View File

@ -15,6 +15,7 @@
*/
package io.netty.util.concurrent;
import org.junit.Assert;
import org.junit.Test;
import java.util.concurrent.Callable;
@ -95,6 +96,11 @@ public class AbstractScheduledEventExecutorTest {
executor.scheduleWithFixedDelay(TEST_RUNNABLE, 0, -1, TimeUnit.DAYS);
}
@Test
public void testDeadlineNanosNotOverflow() {
Assert.assertEquals(Long.MAX_VALUE, AbstractScheduledEventExecutor.deadlineNanos(Long.MAX_VALUE));
}
private static final class TestScheduledEventExecutor extends AbstractScheduledEventExecutor {
@Override
public boolean isShuttingDown() {

View File

@ -88,7 +88,7 @@ public class DefaultPromiseTest {
public void testNoStackOverflowWithDefaultEventExecutorA() throws Exception {
ExecutorService executorService = Executors.newSingleThreadExecutor();
try {
EventExecutor executor = new DefaultEventExecutor(executorService);
EventExecutor executor = new SingleThreadEventExecutor(executorService);
try {
testStackOverFlowChainedFuturesA(stackOverflowTestDepth(), executor, true);
testStackOverFlowChainedFuturesA(stackOverflowTestDepth(), executor, false);
@ -110,7 +110,7 @@ public class DefaultPromiseTest {
public void testNoStackOverflowWithDefaultEventExecutorB() throws Exception {
ExecutorService executorService = Executors.newSingleThreadExecutor();
try {
EventExecutor executor = new DefaultEventExecutor(executorService);
EventExecutor executor = new SingleThreadEventExecutor(executorService);
try {
testStackOverFlowChainedFuturesB(stackOverflowTestDepth(), executor, true);
testStackOverFlowChainedFuturesB(stackOverflowTestDepth(), executor, false);
@ -495,22 +495,7 @@ public class DefaultPromiseTest {
private static final class TestEventExecutor extends SingleThreadEventExecutor {
TestEventExecutor() {
super(null, Executors.defaultThreadFactory(), true);
}
@Override
protected void run() {
for (;;) {
Runnable task = takeTask();
if (task != null) {
task.run();
updateLastExecutionTime();
}
if (confirmShutdown()) {
break;
}
}
super(Executors.defaultThreadFactory());
}
}

View File

@ -34,7 +34,7 @@ public class SingleThreadEventExecutorTest {
public void testWrappedExecutorIsShutdown() {
ExecutorService executorService = Executors.newSingleThreadExecutor();
SingleThreadEventExecutor executor = new SingleThreadEventExecutor(null, executorService, false) {
SingleThreadEventExecutor executor = new SingleThreadEventExecutor(executorService) {
@Override
protected void run() {
while (!confirmShutdown()) {
@ -74,9 +74,8 @@ public class SingleThreadEventExecutorTest {
@Test
public void testThreadProperties() {
final AtomicReference<Thread> threadRef = new AtomicReference<>();
SingleThreadEventExecutor executor = new SingleThreadEventExecutor(
null, new DefaultThreadFactory("test"), false) {
final AtomicReference<Thread> threadRef = new AtomicReference<Thread>();
SingleThreadEventExecutor executor = new SingleThreadEventExecutor(new DefaultThreadFactory("test")) {
@Override
protected void run() {
threadRef.set(Thread.currentThread());
@ -121,8 +120,7 @@ public class SingleThreadEventExecutorTest {
}
private static void testInvokeInEventLoop(final boolean any, final boolean timeout) {
final SingleThreadEventExecutor executor = new SingleThreadEventExecutor(null,
Executors.defaultThreadFactory(), false) {
final SingleThreadEventExecutor executor = new SingleThreadEventExecutor(Executors.defaultThreadFactory()) {
@Override
protected void run() {
while (!confirmShutdown()) {

View File

@ -20,7 +20,8 @@ import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.ssl.SslContext;
@ -47,7 +48,7 @@ public final class DiscardClient {
sslCtx = null;
}
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -20,7 +20,8 @@ import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
@ -47,8 +48,8 @@ public final class DiscardServer {
sslCtx = null;
}
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -21,7 +21,8 @@ import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.ssl.SslContext;
@ -52,7 +53,7 @@ public final class EchoClient {
}
// Configure the client.
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -21,7 +21,8 @@ import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
@ -49,8 +50,8 @@ public final class EchoServer {
}
// Configure the server.
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
final EchoServerHandler serverHandler = new EchoServerHandler();
try {
ServerBootstrap b = new ServerBootstrap();

View File

@ -18,7 +18,8 @@ package io.netty.example.factorial;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.ssl.SslContext;
import io.netty.handler.ssl.SslContextBuilder;
@ -45,7 +46,7 @@ public final class FactorialClient {
sslCtx = null;
}
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -17,7 +17,8 @@ package io.netty.example.factorial;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
@ -44,8 +45,8 @@ public final class FactorialServer {
sslCtx = null;
}
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -21,7 +21,8 @@ import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.codec.LineBasedFrameDecoder;
@ -55,8 +56,8 @@ public final class FileServer {
}
// Configure the server.
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -17,7 +17,8 @@ package io.netty.example.http.cors;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
@ -86,8 +87,8 @@ public final class HttpCorsServer {
sslCtx = null;
}
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -18,7 +18,8 @@ package io.netty.example.http.file;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
@ -43,8 +44,8 @@ public final class HttpStaticFileServer {
sslCtx = null;
}
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -19,7 +19,8 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
@ -47,8 +48,8 @@ public final class HttpHelloWorldServer {
}
// Configure the server.
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.option(ChannelOption.SO_BACKLOG, 1024);

View File

@ -18,7 +18,8 @@ package io.netty.example.http.snoop;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.http.DefaultFullHttpRequest;
import io.netty.handler.codec.http.HttpHeaderNames;
@ -71,7 +72,7 @@ public final class HttpSnoopClient {
}
// Configure the client.
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -18,7 +18,8 @@ package io.netty.example.http.snoop;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
@ -46,8 +47,8 @@ public final class HttpSnoopServer {
}
// Configure the server.
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -19,7 +19,8 @@ import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.http.DefaultHttpRequest;
import io.netty.handler.codec.http.HttpHeaderNames;
@ -101,7 +102,7 @@ public final class HttpUploadClient {
}
// Configure the client.
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
// setup the factory: here using a mixed memory/disk based on size threshold
HttpDataFactory factory = new DefaultHttpDataFactory(DefaultHttpDataFactory.MINSIZE); // Disk if MINSIZE exceed

View File

@ -18,7 +18,8 @@ package io.netty.example.http.upload;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
@ -44,8 +45,8 @@ public final class HttpUploadServer {
sslCtx = null;
}
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup);

View File

@ -18,7 +18,8 @@ package io.netty.example.http.websocketx.benchmarkserver;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.ssl.SslContext;
import io.netty.handler.ssl.SslContextBuilder;
@ -47,8 +48,8 @@ public final class WebSocketServer {
sslCtx = null;
}
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -21,7 +21,8 @@ import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.http.DefaultHttpHeaders;
@ -90,7 +91,7 @@ public final class WebSocketClient {
sslCtx = null;
}
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
// Connect with V13 (RFC 6455 aka HyBi-17). You can change it to V08 or V00.
// If you change it to V00, ping is not supported and remember to change

View File

@ -18,7 +18,8 @@ package io.netty.example.http.websocketx.server;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
@ -60,8 +61,8 @@ public final class WebSocketServer {
sslCtx = null;
}
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -18,7 +18,8 @@ import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.http.DefaultFullHttpRequest;
import io.netty.handler.codec.http.FullHttpRequest;
@ -86,7 +87,7 @@ public final class Http2Client {
sslCtx = null;
}
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
Http2ClientInitializer initializer = new Http2ClientInitializer(sslCtx, Integer.MAX_VALUE);
try {

View File

@ -20,7 +20,8 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.codec.http2.Http2SecurityUtil;
import io.netty.handler.logging.LogLevel;
@ -74,7 +75,7 @@ public final class Http2Server {
sslCtx = null;
}
// Configure the server.
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.option(ChannelOption.SO_BACKLOG, 1024);

View File

@ -20,7 +20,8 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.codec.http2.Http2SecurityUtil;
import io.netty.handler.logging.LogLevel;
@ -74,7 +75,7 @@ public final class Http2Server {
sslCtx = null;
}
// Configure the server.
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.option(ChannelOption.SO_BACKLOG, 1024);

View File

@ -20,7 +20,8 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.codec.http2.Http2SecurityUtil;
import io.netty.handler.logging.LogLevel;
@ -71,7 +72,7 @@ public final class Http2Server {
sslCtx = null;
}
// Configure the server.
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.option(ChannelOption.SO_BACKLOG, 1024);

View File

@ -17,7 +17,8 @@
package io.netty.example.http2.tiles;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
/**
* <p>
@ -38,7 +39,7 @@ import io.netty.channel.nio.NioEventLoopGroup;
public final class Launcher {
public static void main(String[] args) {
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
Http2Server http2 = new Http2Server(group);
HttpServer http = new HttpServer(group);
try {

View File

@ -20,12 +20,12 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.local.LocalServerChannel;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
@ -40,8 +40,8 @@ public final class LocalEcho {
// Address to bind on / connect to.
final LocalAddress addr = new LocalAddress(PORT);
EventLoopGroup serverGroup = new LocalEventLoopGroup();
EventLoopGroup clientGroup = new NioEventLoopGroup(); // NIO event loops are also OK
EventLoopGroup serverGroup = new MultithreadEventLoopGroup(LocalHandler.newFactory());
EventLoopGroup clientGroup = new MultithreadEventLoopGroup(LocalHandler.newFactory());
try {
// Note that we can use any event loop to ensure certain local channels
// are handled by the same event loop thread which drives a certain socket channel

View File

@ -21,7 +21,8 @@ import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.memcache.binary.BinaryMemcacheClientCodec;
@ -52,7 +53,7 @@ public final class MemcacheClient {
sslCtx = null;
}
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -19,7 +19,8 @@ import io.netty.bootstrap.Bootstrap;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.example.echo.EchoClient;
@ -50,7 +51,7 @@ public final class ObjectEchoClient {
sslCtx = null;
}
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -19,7 +19,8 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.example.echo.EchoServer;
@ -50,8 +51,8 @@ public final class ObjectEchoServer {
sslCtx = null;
}
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -21,6 +21,8 @@ import java.math.BigInteger;
import javax.net.ssl.SSLSession;
import javax.security.cert.X509Certificate;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import org.bouncycastle.asn1.ocsp.OCSPResponseStatus;
import org.bouncycastle.cert.ocsp.BasicOCSPResp;
import org.bouncycastle.cert.ocsp.CertificateStatus;
@ -36,7 +38,6 @@ import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.http.DefaultFullHttpRequest;
import io.netty.handler.codec.http.FullHttpRequest;
@ -87,7 +88,7 @@ public class OcspClientExample {
.build();
try {
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Promise<FullHttpResponse> promise = group.next().newPromise();

View File

@ -18,7 +18,8 @@ package io.netty.example.portunification;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
@ -44,8 +45,8 @@ public final class PortUnificationServer {
final SslContext sslCtx = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey())
.build();
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -18,7 +18,8 @@ package io.netty.example.proxy;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
@ -33,8 +34,8 @@ public final class HexDumpProxy {
System.err.println("Proxying *:" + LOCAL_PORT + " to " + REMOTE_HOST + ':' + REMOTE_PORT + " ...");
// Configure the bootstrap.
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -20,7 +20,8 @@ import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.DatagramPacket;
import io.netty.channel.socket.nio.NioDatagramChannel;
import io.netty.util.CharsetUtil;
@ -38,7 +39,7 @@ public final class QuoteOfTheMomentClient {
public static void main(String[] args) throws Exception {
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -18,7 +18,8 @@ package io.netty.example.qotm;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioDatagramChannel;
/**
@ -32,7 +33,7 @@ public final class QuoteOfTheMomentServer {
private static final int PORT = Integer.parseInt(System.getProperty("port", "7686"));
public static void main(String[] args) throws Exception {
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -21,7 +21,8 @@ import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.redis.RedisArrayAggregator;
@ -41,7 +42,7 @@ public class RedisClient {
private static final int PORT = Integer.parseInt(System.getProperty("port", "6379"));
public static void main(String[] args) throws Exception {
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -19,7 +19,8 @@ import io.netty.bootstrap.Bootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.sctp.SctpChannel;
import io.netty.channel.sctp.SctpChannelOption;
import io.netty.channel.sctp.nio.NioSctpChannel;
@ -40,7 +41,7 @@ public final class SctpEchoClient {
public static void main(String[] args) throws Exception {
// Configure the client.
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -20,7 +20,8 @@ import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.sctp.SctpChannel;
import io.netty.channel.sctp.nio.NioSctpServerChannel;
import io.netty.handler.logging.LogLevel;
@ -35,8 +36,8 @@ public final class SctpEchoServer {
public static void main(String[] args) throws Exception {
// Configure the server.
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
final SctpEchoServerHandler serverHandler = new SctpEchoServerHandler();
try {
ServerBootstrap b = new ServerBootstrap();

View File

@ -19,7 +19,8 @@ import io.netty.bootstrap.Bootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.sctp.SctpChannel;
import io.netty.channel.sctp.SctpChannelOption;
import io.netty.channel.sctp.nio.NioSctpChannel;
@ -44,7 +45,7 @@ public final class SctpMultiHomingEchoClient {
public static void main(String[] args) throws Exception {
// Configure the client.
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -20,7 +20,8 @@ import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.sctp.SctpChannel;
import io.netty.channel.sctp.SctpServerChannel;
import io.netty.channel.sctp.nio.NioSctpServerChannel;
@ -44,8 +45,8 @@ public final class SctpMultiHomingEchoServer {
public static void main(String[] args) throws Exception {
// Configure the server.
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -19,7 +19,8 @@ import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.example.telnet.TelnetClient;
import io.netty.handler.ssl.SslContext;
@ -42,7 +43,7 @@ public final class SecureChatClient {
final SslContext sslCtx = SslContextBuilder.forClient()
.trustManager(InsecureTrustManagerFactory.INSTANCE).build();
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -17,7 +17,8 @@ package io.netty.example.securechat;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.example.telnet.TelnetServer;
import io.netty.handler.logging.LogLevel;
@ -38,8 +39,8 @@ public final class SecureChatServer {
SslContext sslCtx = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey())
.build();
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -17,7 +17,8 @@ package io.netty.example.socksproxy;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
@ -27,8 +28,8 @@ public final class SocksServer {
static final int PORT = Integer.parseInt(System.getProperty("port", "1080"));
public static void main(String[] args) throws Exception {
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -19,7 +19,8 @@ import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.http.DefaultFullHttpRequest;
import io.netty.handler.codec.http.HttpHeaderNames;
@ -69,7 +70,7 @@ public final class SpdyClient {
.build();
HttpResponseClientHandler httpResponseHandler = new HttpResponseClientHandler();
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();

View File

@ -19,7 +19,8 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
@ -69,8 +70,8 @@ public final class SpdyServer {
.build();
// Configure the server.
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.option(ChannelOption.SO_BACKLOG, 1024);

View File

@ -19,7 +19,8 @@ import io.netty.bootstrap.Bootstrap;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.stomp.StompSubframeAggregator;
@ -41,7 +42,7 @@ public final class StompClient {
static final String TOPIC = System.getProperty("topic", "jms.topic.exampleTopic");
public static void main(String[] args) throws Exception {
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group).channel(NioSocketChannel.class);

View File

@ -19,7 +19,8 @@ import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.ssl.SslContext;
import io.netty.handler.ssl.SslContextBuilder;
@ -47,7 +48,7 @@ public final class TelnetClient {
sslCtx = null;
}
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -17,7 +17,8 @@ package io.netty.example.telnet;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
@ -43,8 +44,8 @@ public final class TelnetServer {
sslCtx = null;
}
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -20,7 +20,8 @@ import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.timeout.IdleStateHandler;
@ -44,7 +45,7 @@ public final class UptimeClient {
private static final Bootstrap bs = new Bootstrap();
public static void main(String[] args) throws Exception {
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
bs.group(group)
.channel(NioSocketChannel.class)
.remoteAddress(HOST, PORT)

View File

@ -19,7 +19,8 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
@ -38,8 +39,8 @@ public final class UptimeServer {
public static void main(String[] args) throws Exception {
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -18,7 +18,8 @@ package io.netty.example.worldclock;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.ssl.SslContext;
import io.netty.handler.ssl.SslContextBuilder;
@ -49,7 +50,7 @@ public final class WorldClockClient {
sslCtx = null;
}
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)

View File

@ -17,7 +17,8 @@ package io.netty.example.worldclock;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
@ -44,8 +45,8 @@ public final class WorldClockServer {
sslCtx = null;
}
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup bossGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap b = new ServerBootstrap();
b.group(bossGroup, workerGroup)

View File

@ -29,8 +29,9 @@ import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.LineBasedFrameDecoder;
@ -82,7 +83,8 @@ public class ProxyHandlerTest {
private static final String BAD_USERNAME = "badUser";
private static final String BAD_PASSWORD = "badPassword";
static final EventLoopGroup group = new NioEventLoopGroup(3, new DefaultThreadFactory("proxy", true));
static final EventLoopGroup group = new MultithreadEventLoopGroup(3,
new DefaultThreadFactory("proxy", true), NioHandler.newFactory());
static final SslContext serverSslCtx;
static final SslContext clientSslCtx;

View File

@ -29,7 +29,8 @@ import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.ByteToMessageDecoder;
@ -52,7 +53,7 @@ public class FlowControlHandlerTest {
@BeforeClass
public static void init() {
GROUP = new NioEventLoopGroup();
GROUP = new MultithreadEventLoopGroup(NioHandler.newFactory());
}
@AfterClass

View File

@ -24,11 +24,12 @@ import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.local.LocalServerChannel;
import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
import io.netty.handler.ssl.util.SelfSignedCertificate;
@ -75,7 +76,7 @@ public class CipherSuiteCanaryTest {
@BeforeClass
public static void init() throws Exception {
GROUP = new LocalEventLoopGroup();
GROUP = new MultithreadEventLoopGroup(LocalHandler.newFactory());
CERT = new SelfSignedCertificate();
}

View File

@ -26,7 +26,8 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
@ -126,7 +127,7 @@ public class ParameterizedSslHandlerTest {
.trustManager(InsecureTrustManagerFactory.INSTANCE)
.sslProvider(clientProvider).build();
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
Channel sc = null;
Channel cc = null;
try {
@ -306,7 +307,7 @@ public class ParameterizedSslHandlerTest {
ResourcesUtil.getFile(getClass(), "test_unencrypted.pem"))
.sslProvider(clientProvider).build();
NioEventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
Channel sc = null;
Channel cc = null;
try {
@ -397,7 +398,7 @@ public class ParameterizedSslHandlerTest {
.protocols(SslUtils.PROTOCOL_TLS_V1_2)
.build();
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
Channel sc = null;
Channel cc = null;
try {

View File

@ -22,9 +22,10 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.local.LocalServerChannel;
import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
import io.netty.handler.ssl.util.SelfSignedCertificate;
@ -43,7 +44,7 @@ public abstract class RenegotiateTest {
final AtomicReference<Throwable> error = new AtomicReference<>();
final CountDownLatch latch = new CountDownLatch(2);
SelfSignedCertificate cert = new SelfSignedCertificate();
EventLoopGroup group = new LocalEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(LocalHandler.newFactory());
try {
final SslContext context = SslContextBuilder.forServer(cert.key(), cert.cert())
.sslProvider(serverSslProvider()).build();

View File

@ -28,8 +28,9 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
@ -703,7 +704,8 @@ public abstract class SSLEngineTest {
sb = new ServerBootstrap();
cb = new Bootstrap();
sb.group(new NioEventLoopGroup(), new NioEventLoopGroup());
sb.group(new MultithreadEventLoopGroup(NioHandler.newFactory()),
new MultithreadEventLoopGroup(NioHandler.newFactory()));
sb.channel(NioServerSocketChannel.class);
sb.childHandler(new ChannelInitializer<Channel>() {
@Override
@ -747,7 +749,7 @@ public abstract class SSLEngineTest {
}
});
cb.group(new NioEventLoopGroup());
cb.group(new MultithreadEventLoopGroup(NioHandler.newFactory()));
cb.channel(NioSocketChannel.class);
cb.handler(new ChannelInitializer<Channel>() {
@Override
@ -850,7 +852,8 @@ public abstract class SSLEngineTest {
sb = new ServerBootstrap();
cb = new Bootstrap();
sb.group(new NioEventLoopGroup(), new NioEventLoopGroup());
sb.group(new MultithreadEventLoopGroup(NioHandler.newFactory()),
new MultithreadEventLoopGroup(NioHandler.newFactory()));
sb.channel(NioServerSocketChannel.class);
sb.childHandler(new ChannelInitializer<Channel>() {
@Override
@ -889,7 +892,7 @@ public abstract class SSLEngineTest {
}
});
cb.group(new NioEventLoopGroup());
cb.group(new MultithreadEventLoopGroup(NioHandler.newFactory()));
cb.channel(NioSocketChannel.class);
cb.handler(new ChannelInitializer<Channel>() {
@Override
@ -1012,7 +1015,8 @@ public abstract class SSLEngineTest {
sb = new ServerBootstrap();
cb = new Bootstrap();
sb.group(new NioEventLoopGroup(), new NioEventLoopGroup());
sb.group(new MultithreadEventLoopGroup(NioHandler.newFactory()),
new MultithreadEventLoopGroup(NioHandler.newFactory()));
sb.channel(NioServerSocketChannel.class);
sb.childHandler(new ChannelInitializer<Channel>() {
@Override
@ -1054,7 +1058,7 @@ public abstract class SSLEngineTest {
}
});
cb.group(new NioEventLoopGroup());
cb.group(new MultithreadEventLoopGroup(NioHandler.newFactory()));
cb.channel(NioSocketChannel.class);
cb.handler(new ChannelInitializer<Channel>() {
@Override
@ -1248,7 +1252,7 @@ public abstract class SSLEngineTest {
.ciphers(ciphers())
.build();
sb = new ServerBootstrap()
.group(new NioEventLoopGroup(1))
.group(new MultithreadEventLoopGroup(1, NioHandler.newFactory()))
.channel(NioServerSocketChannel.class)
.childHandler(new ChannelInitializer<SocketChannel>() {
@Override
@ -1304,7 +1308,7 @@ public abstract class SSLEngineTest {
.build();
cb = new Bootstrap();
cb.group(new NioEventLoopGroup(1))
cb.group(new MultithreadEventLoopGroup(1, NioHandler.newFactory()))
.channel(NioSocketChannel.class)
.handler(new ChannelInitializer<SocketChannel>() {
@Override
@ -1586,7 +1590,8 @@ public abstract class SSLEngineTest {
sb = new ServerBootstrap();
cb = new Bootstrap();
sb.group(new NioEventLoopGroup(), new NioEventLoopGroup());
sb.group(new MultithreadEventLoopGroup(NioHandler.newFactory()),
new MultithreadEventLoopGroup(NioHandler.newFactory()));
sb.channel(NioServerSocketChannel.class);
sb.childHandler(new ChannelInitializer<Channel>() {
@Override
@ -1611,7 +1616,7 @@ public abstract class SSLEngineTest {
}
});
cb.group(new NioEventLoopGroup());
cb.group(new MultithreadEventLoopGroup(NioHandler.newFactory()));
cb.channel(NioSocketChannel.class);
cb.handler(new ChannelInitializer<Channel>() {
@Override
@ -1660,7 +1665,8 @@ public abstract class SSLEngineTest {
.ciphers(ciphers()).build();
sb = new ServerBootstrap();
sb.group(new NioEventLoopGroup(), new NioEventLoopGroup());
sb.group(new MultithreadEventLoopGroup(NioHandler.newFactory()),
new MultithreadEventLoopGroup(NioHandler.newFactory()));
sb.channel(NioServerSocketChannel.class);
final Promise<String> promise = sb.config().group().next().newPromise();
@ -1717,7 +1723,7 @@ public abstract class SSLEngineTest {
.sslContextProvider(clientSslContextProvider())
.protocols(protocols()).ciphers(ciphers()).build();
cb = new Bootstrap();
cb.group(new NioEventLoopGroup());
cb.group(new MultithreadEventLoopGroup(NioHandler.newFactory()));
cb.channel(NioSocketChannel.class);
clientChannel = cb.handler(new ChannelInitializer<Channel>() {
@Override

View File

@ -22,10 +22,11 @@ import io.netty.channel.Channel;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.local.LocalServerChannel;
import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
import io.netty.handler.ssl.util.SelfSignedCertificate;
@ -79,7 +80,7 @@ final class SniClientJava8TestUtil {
final String sniHost = "sni.netty.io";
SelfSignedCertificate cert = new SelfSignedCertificate();
LocalAddress address = new LocalAddress("test");
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
SslContext sslServerContext = null;
SslContext sslClientContext = null;

View File

@ -20,10 +20,11 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.ByteBufAllocator;
import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.local.LocalServerChannel;
import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
import io.netty.handler.ssl.util.SelfSignedCertificate;
@ -95,7 +96,7 @@ public class SniClientTest {
private static void testSniClient(SslProvider sslServerProvider, SslProvider sslClientProvider) throws Exception {
String sniHostName = "sni.netty.io";
LocalAddress address = new LocalAddress("test");
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
SelfSignedCertificate cert = new SelfSignedCertificate();
SslContext sslServerContext = null;
SslContext sslClientContext = null;

View File

@ -27,13 +27,14 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.embedded.EmbeddedChannel;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.local.LocalServerChannel;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.DecoderException;
@ -346,7 +347,7 @@ public class SniHandlerTest {
.add("*.netty.io", nettyContext)
.add("sni.fake.site", sniContext).build();
final SniHandler handler = new SniHandler(mapping);
EventLoopGroup group = new NioEventLoopGroup(2);
EventLoopGroup group = new MultithreadEventLoopGroup(2, NioHandler.newFactory());
Channel serverChannel = null;
Channel clientChannel = null;
try {
@ -418,7 +419,7 @@ public class SniHandlerTest {
case OPENSSL_REFCNT:
final String sniHost = "sni.netty.io";
LocalAddress address = new LocalAddress("testReplaceHandler-" + Math.random());
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
Channel sc = null;
Channel cc = null;
SslContext sslContext = null;

View File

@ -22,7 +22,8 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.logging.LogLevel;
@ -165,7 +166,7 @@ public class SslErrorTest {
Channel serverChannel = null;
Channel clientChannel = null;
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
serverChannel = new ServerBootstrap().group(group)
.channel(NioServerSocketChannel.class)

View File

@ -32,14 +32,15 @@ import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOutboundHandlerAdapter;
import io.netty.channel.ChannelPromise;
import io.netty.channel.DefaultChannelId;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.embedded.EmbeddedChannel;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.local.LocalServerChannel;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.CodecException;
@ -322,7 +323,7 @@ public class SslHandlerTest {
@Test(timeout = 30000)
public void testRemoval() throws Exception {
NioEventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
Channel sc = null;
Channel cc = null;
try {
@ -451,7 +452,7 @@ public class SslHandlerTest {
final BlockingQueue<Object> events = new LinkedBlockingQueue<>();
Channel serverChannel = null;
Channel clientChannel = null;
EventLoopGroup group = new LocalEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(LocalHandler.newFactory());
try {
ServerBootstrap sb = new ServerBootstrap();
sb.group(group)
@ -544,7 +545,7 @@ public class SslHandlerTest {
final SslContext sslClientCtx = SslContextBuilder.forClient()
.trustManager(InsecureTrustManagerFactory.INSTANCE).build();
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
Channel sc = null;
Channel cc = null;
final CountDownLatch serverReceiveLatch = new CountDownLatch(1);
@ -614,7 +615,7 @@ public class SslHandlerTest {
.trustManager(new SelfSignedCertificate().cert())
.build();
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
Channel sc = null;
Channel cc = null;
try {
@ -687,7 +688,7 @@ public class SslHandlerTest {
.trustManager(InsecureTrustManagerFactory.INSTANCE)
.sslProvider(SslProvider.JDK).build();
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
Channel sc = null;
Channel cc = null;
final CountDownLatch activeLatch = new CountDownLatch(1);
@ -769,7 +770,7 @@ public class SslHandlerTest {
.trustManager(InsecureTrustManagerFactory.INSTANCE)
.sslProvider(SslProvider.JDK).build();
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
Channel sc = null;
Channel cc = null;
final SslHandler sslHandler = sslClientCtx.newHandler(UnpooledByteBufAllocator.DEFAULT);

View File

@ -26,10 +26,11 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.local.LocalServerChannel;
import io.netty.handler.ssl.OpenSsl;
import io.netty.handler.ssl.ReferenceCountedOpenSslEngine;
@ -344,7 +345,7 @@ public class OcspTest {
.build();
try {
EventLoopGroup group = new LocalEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(LocalHandler.newFactory());
try {
LocalAddress address = new LocalAddress("handshake-" + Math.random());
Channel server = newServer(group, address, serverSslContext, response, serverHandler);

View File

@ -23,7 +23,9 @@ import io.netty.channel.ChannelDuplexHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPromise;
import io.netty.channel.epoll.EpollEventLoopGroup;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.epoll.EpollHandler;
import io.netty.channel.epoll.EpollServerSocketChannel;
import io.netty.channel.epoll.EpollSocketChannel;
import io.netty.microbench.util.AbstractMicrobenchmark;
@ -35,7 +37,7 @@ import org.openjdk.jmh.annotations.TearDown;
public class EpollSocketChannelBenchmark extends AbstractMicrobenchmark {
private EpollEventLoopGroup group;
private EventLoopGroup group;
private Channel serverChan;
private Channel chan;
private ByteBuf abyte;
@ -43,7 +45,7 @@ public class EpollSocketChannelBenchmark extends AbstractMicrobenchmark {
@Setup
public void setup() throws Exception {
group = new EpollEventLoopGroup(1);
group = new MultithreadEventLoopGroup(1, EpollHandler.newFactory());
// add an arbitrary timeout to make the timer reschedule
future = group.schedule(new Runnable() {

View File

@ -14,13 +14,17 @@
*/
package io.netty.microbench.concurrent;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.epoll.Epoll;
import io.netty.channel.epoll.EpollEventLoopGroup;
import io.netty.channel.epoll.EpollHandler;
import io.netty.channel.kqueue.KQueue;
import io.netty.channel.kqueue.KQueueEventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.kqueue.KQueueHandler;
import io.netty.channel.nio.NioHandler;
import io.netty.microbench.util.AbstractMicrobenchmark;
import io.netty.util.concurrent.DefaultEventExecutor;
import io.netty.util.concurrent.DefaultThreadFactory;
import io.netty.util.concurrent.RejectedExecutionHandlers;
import io.netty.util.concurrent.SingleThreadEventExecutor;
import io.netty.util.internal.PlatformDependent;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
@ -197,7 +201,7 @@ public class BurstCostExecutorsBenchmark extends AbstractMicrobenchmark {
executorToShutdown = executor;
break;
case defaultEventExecutor:
executor = new DefaultEventExecutor();
executor = new SingleThreadEventExecutor();
executorToShutdown = executor;
break;
case juc:
@ -205,24 +209,27 @@ public class BurstCostExecutorsBenchmark extends AbstractMicrobenchmark {
executorToShutdown = executor;
break;
case nioEventLoop:
NioEventLoopGroup nioEventLoopGroup = new NioEventLoopGroup(1);
nioEventLoopGroup.setIoRatio(1);
EventLoopGroup nioEventLoopGroup = new MultithreadEventLoopGroup(1,
new DefaultThreadFactory(MultithreadEventLoopGroup.class), NioHandler.newFactory(),
Integer.MAX_VALUE, RejectedExecutionHandlers.reject(), Integer.MAX_VALUE);
executor = nioEventLoopGroup.next();
executorToShutdown = nioEventLoopGroup;
break;
case epollEventLoop:
Epoll.ensureAvailability();
EpollEventLoopGroup epollEventLoopGroup = new EpollEventLoopGroup(1);
epollEventLoopGroup.setIoRatio(1);
EventLoopGroup epollEventLoopGroup = new MultithreadEventLoopGroup(1,
new DefaultThreadFactory(MultithreadEventLoopGroup.class), EpollHandler.newFactory(),
Integer.MAX_VALUE, RejectedExecutionHandlers.reject(), Integer.MAX_VALUE);
executor = epollEventLoopGroup.next();
executorToShutdown = epollEventLoopGroup;
break;
case kqueueEventLoop:
KQueue.ensureAvailability();
KQueueEventLoopGroup kQueueEventLoopGroup = new KQueueEventLoopGroup(1);
kQueueEventLoopGroup.setIoRatio(1);
executor = kQueueEventLoopGroup.next();
executorToShutdown = kQueueEventLoopGroup;
EventLoopGroup kqueueEventLoopGroup = new MultithreadEventLoopGroup(1,
new DefaultThreadFactory(MultithreadEventLoopGroup.class), KQueueHandler.newFactory(),
Integer.MAX_VALUE, RejectedExecutionHandlers.reject(), Integer.MAX_VALUE);
executor = kqueueEventLoopGroup.next();
executorToShutdown = kqueueEventLoopGroup;
break;
}
}

View File

@ -16,7 +16,8 @@
package io.netty.microbench.concurrent;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.local.LocalHandler;
import io.netty.microbench.util.AbstractMicrobenchmark;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.ScheduledFuture;
@ -34,9 +35,9 @@ import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
@State(Scope.Benchmark)
public class ScheduledFutureTaskBenchmark extends AbstractMicrobenchmark {
public class RunnableScheduledFutureAdapterBenchmark extends AbstractMicrobenchmark {
static final EventLoopGroup executor = new LocalEventLoopGroup(1);
static final EventLoopGroup executor = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
@State(Scope.Thread)
public static class FuturesHolder {

View File

@ -21,9 +21,11 @@ import io.netty.util.concurrent.AbstractEventExecutor;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.ProgressivePromise;
import io.netty.util.concurrent.Promise;
import io.netty.util.concurrent.ScheduledFuture;
import io.netty.util.internal.logging.InternalLogger;
import io.netty.util.internal.logging.InternalLoggerFactory;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
import org.openjdk.jmh.annotations.Fork;
@ -81,11 +83,6 @@ public class AbstractSharedExecutorMicrobenchmark extends AbstractMicrobenchmark
executor = service;
}
@Override
public boolean inEventLoop() {
return executor.inEventLoop();
}
@Override
public boolean inEventLoop(Thread thread) {
return executor.inEventLoop(thread);
@ -146,6 +143,27 @@ public class AbstractSharedExecutorMicrobenchmark extends AbstractMicrobenchmark
public <V> ProgressivePromise<V> newProgressivePromise() {
return executor.newProgressivePromise();
}
@Override
public ScheduledFuture<?> schedule(Runnable command, long delay, TimeUnit unit) {
return executor.schedule(command, delay, unit);
}
@Override
public <V> ScheduledFuture<V> schedule(Callable<V> callable, long delay, TimeUnit unit) {
return executor.schedule(callable, delay, unit);
}
@Override
public ScheduledFuture<?> scheduleAtFixedRate(Runnable command, long initialDelay, long period, TimeUnit unit) {
return executor.scheduleAtFixedRate(command, initialDelay, period, unit);
}
@Override
public ScheduledFuture<?> scheduleWithFixedDelay(
Runnable command, long initialDelay, long delay, TimeUnit unit) {
return executor.scheduleWithFixedDelay(command, initialDelay, delay, unit);
}
}
@Override

View File

@ -15,10 +15,11 @@
*/
package io.netty.resolver.dns;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.EventLoop;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.nio.NioHandler;
import io.netty.util.NetUtil;
import org.junit.Test;
@ -38,7 +39,7 @@ public class DefaultAuthoritativeDnsServerCacheTest {
InetAddress.getByAddress("ns1", new byte[] { 10, 0, 0, 1 }), 53);
InetSocketAddress resolved2 = new InetSocketAddress(
InetAddress.getByAddress("ns2", new byte[] { 10, 0, 0, 2 }), 53);
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
try {
EventLoop loop = group.next();
@ -73,7 +74,7 @@ public class DefaultAuthoritativeDnsServerCacheTest {
}
private static void testExpireWithTTL0(int days) {
EventLoopGroup group = new NioEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
try {
EventLoop loop = group.next();
@ -90,7 +91,7 @@ public class DefaultAuthoritativeDnsServerCacheTest {
InetAddress.getByAddress("ns1", new byte[] { 10, 0, 0, 1 }), 53);
InetSocketAddress resolved2 = new InetSocketAddress(
InetAddress.getByAddress("ns2", new byte[] { 10, 0, 0, 2 }), 53);
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
try {
EventLoop loop = group.next();
@ -114,7 +115,7 @@ public class DefaultAuthoritativeDnsServerCacheTest {
InetAddress.getByAddress("ns2", new byte[] { 10, 0, 0, 2 }), 53);
InetSocketAddress resolved2 = new InetSocketAddress(
InetAddress.getByAddress("ns1", new byte[] { 10, 0, 0, 1 }), 53);
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
try {
EventLoop loop = group.next();
@ -152,7 +153,7 @@ public class DefaultAuthoritativeDnsServerCacheTest {
InetSocketAddress unresolved = InetSocketAddress.createUnresolved("ns1", 53);
InetSocketAddress resolved = new InetSocketAddress(
InetAddress.getByAddress("ns2", new byte[] { 10, 0, 0, 2 }), 53);
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
try {
EventLoop loop = group.next();

View File

@ -15,11 +15,12 @@
*/
package io.netty.resolver.dns;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.EventLoop;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.nio.NioHandler;
import io.netty.util.NetUtil;
import org.junit.Test;
@ -40,7 +41,7 @@ public class DefaultDnsCacheTest {
public void testExpire() throws Throwable {
InetAddress addr1 = InetAddress.getByAddress(new byte[] { 10, 0, 0, 1 });
InetAddress addr2 = InetAddress.getByAddress(new byte[] { 10, 0, 0, 2 });
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
try {
EventLoop loop = group.next();
@ -75,7 +76,7 @@ public class DefaultDnsCacheTest {
}
private static void testExpireWithTTL0(int days) {
EventLoopGroup group = new NioEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
try {
EventLoop loop = group.next();
@ -88,7 +89,7 @@ public class DefaultDnsCacheTest {
@Test
public void testExpireWithToBigMinTTL() {
EventLoopGroup group = new NioEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
try {
EventLoop loop = group.next();
@ -103,7 +104,7 @@ public class DefaultDnsCacheTest {
public void testAddMultipleAddressesForSameHostname() throws Exception {
InetAddress addr1 = InetAddress.getByAddress(new byte[] { 10, 0, 0, 1 });
InetAddress addr2 = InetAddress.getByAddress(new byte[] { 10, 0, 0, 2 });
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
try {
EventLoop loop = group.next();
@ -123,7 +124,7 @@ public class DefaultDnsCacheTest {
@Test
public void testAddSameAddressForSameHostname() throws Exception {
InetAddress addr1 = InetAddress.getByAddress(new byte[] { 10, 0, 0, 1 });
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
try {
EventLoop loop = group.next();
@ -148,7 +149,7 @@ public class DefaultDnsCacheTest {
public void testCacheFailed() throws Exception {
InetAddress addr1 = InetAddress.getByAddress(new byte[] { 10, 0, 0, 1 });
InetAddress addr2 = InetAddress.getByAddress(new byte[] { 10, 0, 0, 2 });
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
try {
EventLoop loop = group.next();
@ -177,7 +178,7 @@ public class DefaultDnsCacheTest {
public void testDotHandling() throws Exception {
InetAddress addr1 = InetAddress.getByAddress(new byte[] { 10, 0, 0, 1 });
InetAddress addr2 = InetAddress.getByAddress(new byte[] { 10, 0, 0, 2 });
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
try {
EventLoop loop = group.next();

View File

@ -15,9 +15,11 @@
*/
package io.netty.resolver.dns;
import io.netty.channel.local.LocalEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.EventLoop;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.local.LocalHandler;
import io.netty.channel.nio.NioHandler;
import org.junit.Test;
import java.util.concurrent.Callable;
@ -29,7 +31,7 @@ public class DefaultDnsCnameCacheTest {
@Test
public void testExpire() throws Throwable {
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
try {
EventLoop loop = group.next();
@ -63,7 +65,7 @@ public class DefaultDnsCnameCacheTest {
}
private static void testExpireWithTTL0(int days) {
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
try {
EventLoop loop = group.next();
@ -77,7 +79,7 @@ public class DefaultDnsCnameCacheTest {
@Test
public void testMultipleCnamesForSameHostname() throws Exception {
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
try {
EventLoop loop = group.next();
@ -93,7 +95,7 @@ public class DefaultDnsCnameCacheTest {
@Test
public void testAddSameCnameForSameHostname() throws Exception {
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
try {
EventLoop loop = group.next();
@ -109,7 +111,7 @@ public class DefaultDnsCnameCacheTest {
@Test
public void testClear() throws Exception {
EventLoopGroup group = new LocalEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
try {
EventLoop loop = group.next();

View File

@ -16,7 +16,8 @@
package io.netty.resolver.dns;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioDatagramChannel;
import io.netty.handler.codec.dns.DefaultDnsOptEcsRecord;
import io.netty.handler.codec.dns.DnsRecord;
@ -37,7 +38,7 @@ public class DnsNameResolverClientSubnetTest {
@Ignore
@Test
public void testSubnetQuery() throws Exception {
EventLoopGroup group = new NioEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
DnsNameResolver resolver = newResolver(group).build();
try {
// Same as:

View File

@ -22,8 +22,9 @@ import io.netty.channel.ChannelFactory;
import io.netty.channel.ChannelFuture;
import io.netty.channel.EventLoop;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.ReflectiveChannelFactory;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.DatagramChannel;
import io.netty.channel.socket.InternetProtocolFamily;
import io.netty.channel.socket.nio.NioDatagramChannel;
@ -312,7 +313,7 @@ public class DnsNameResolverTest {
}
private static final TestDnsServer dnsServer = new TestDnsServer(DOMAINS_ALL);
private static final EventLoopGroup group = new NioEventLoopGroup(1);
private static final EventLoopGroup group = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
@Rule
public ExpectedException expectedException = ExpectedException.none();
@ -1262,7 +1263,7 @@ public class DnsNameResolverTest {
cache ? new DefaultAuthoritativeDnsServerCache() : NoopAuthoritativeDnsServerCache.INSTANCE);
TestRecursiveCacheDnsQueryLifecycleObserverFactory lifecycleObserverFactory =
new TestRecursiveCacheDnsQueryLifecycleObserverFactory();
EventLoopGroup group = new NioEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
final DnsNameResolver resolver = new DnsNameResolver(
group.next(), new ReflectiveChannelFactory<DatagramChannel>(NioDatagramChannel.class),
NoopDnsCache.INSTANCE, nsCache, lifecycleObserverFactory, 3000, ResolvedAddressTypes.IPV4_ONLY, true,
@ -1423,7 +1424,7 @@ public class DnsNameResolverTest {
}
};
redirectServer.start();
EventLoopGroup group = new NioEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
final DnsNameResolver resolver = new DnsNameResolver(
group.next(), new ReflectiveChannelFactory<DatagramChannel>(NioDatagramChannel.class),
cache, authoritativeDnsServerCache, NoopDnsQueryLifecycleObserverFactory.INSTANCE, 2000,
@ -1559,7 +1560,7 @@ public class DnsNameResolverTest {
}
};
redirectServer.start();
EventLoopGroup group = new NioEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
final List<InetSocketAddress> cached = new CopyOnWriteArrayList<>();
final AuthoritativeDnsServerCache authoritativeDnsServerCache = new AuthoritativeDnsServerCache() {
@ -1689,7 +1690,7 @@ public class DnsNameResolverTest {
}
};
redirectServer.start();
EventLoopGroup group = new NioEventLoopGroup(1);
EventLoopGroup group = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
final List<InetSocketAddress> cached = new CopyOnWriteArrayList<>();
final AuthoritativeDnsServerCache authoritativeDnsServerCache = new AuthoritativeDnsServerCache() {

View File

@ -16,7 +16,8 @@
package io.netty.resolver.dns;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioDatagramChannel;
import io.netty.util.concurrent.Future;
import org.junit.After;
@ -58,7 +59,7 @@ public class SearchDomainTest {
@Before
public void before() {
group = new NioEventLoopGroup(1);
group = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
}
@After

Some files were not shown because too many files have changed in this diff Show More