Support using an Executor to offload blocking / long-running tasks wh… (#8847)
Motivation: The SSLEngine does provide a way to signal to the caller that it may need to execute a blocking / long-running task which then can be offloaded to an Executor to ensure the I/O thread is not blocked. Currently how we handle this in SslHandler is not really optimal as while we offload to the Executor we still block the I/O Thread. Modifications: - Correctly support offloading the task to the Executor while suspending processing of SSL in the I/O Thread - Add new methods to SslContext to specify the Executor when creating a SslHandler - Remove @deprecated annotations from SslHandler constructor that takes an Executor - Adjust tests to also run with the Executor to ensure all works as expected. Result: Be able to offload long running tasks to an Executor when using SslHandler. Partly fixes https://github.com/netty/netty/issues/7862 and https://github.com/netty/netty/issues/7020.
This commit is contained in:
parent
c6a90d90a6
commit
fa6a8cb09c
@ -21,6 +21,7 @@ import io.netty.util.internal.ObjectUtil;
|
||||
import javax.net.ssl.SSLEngine;
|
||||
import javax.net.ssl.SSLSessionContext;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
/**
|
||||
* Adapter class which allows to wrap another {@link SslContext} and init {@link SSLEngine} instances.
|
||||
@ -86,6 +87,21 @@ public abstract class DelegatingSslContext extends SslContext {
|
||||
return handler;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SslHandler newHandler(ByteBufAllocator alloc, boolean startTls, Executor executor) {
|
||||
SslHandler handler = ctx.newHandler(alloc, startTls, executor);
|
||||
initHandler(handler);
|
||||
return handler;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SslHandler newHandler(ByteBufAllocator alloc, String peerHost, int peerPort,
|
||||
boolean startTls, Executor executor) {
|
||||
SslHandler handler = ctx.newHandler(alloc, peerHost, peerPort, startTls, executor);
|
||||
initHandler(handler);
|
||||
return handler;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final SSLSessionContext sessionContext() {
|
||||
return ctx.sessionContext();
|
||||
|
@ -44,6 +44,7 @@ import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
@ -400,6 +401,17 @@ public abstract class ReferenceCountedOpenSslContext extends SslContext implemen
|
||||
return new SslHandler(newEngine0(alloc, peerHost, peerPort, false), startTls);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SslHandler newHandler(ByteBufAllocator alloc, boolean startTls, Executor executor) {
|
||||
return new SslHandler(newEngine0(alloc, null, -1, false), startTls, executor);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SslHandler newHandler(ByteBufAllocator alloc, String peerHost, int peerPort,
|
||||
boolean startTls, Executor executor) {
|
||||
return new SslHandler(newEngine0(alloc, peerHost, peerPort, false), executor);
|
||||
}
|
||||
|
||||
SSLEngine newEngine0(ByteBufAllocator alloc, String peerHost, int peerPort, boolean jdkCompatibilityMode) {
|
||||
return new ReferenceCountedOpenSslEngine(this, alloc, peerHost, peerPort, jdkCompatibilityMode, true);
|
||||
}
|
||||
|
@ -15,6 +15,7 @@
|
||||
*/
|
||||
package io.netty.handler.ssl;
|
||||
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.handler.codec.DecoderException;
|
||||
import io.netty.util.AsyncMapping;
|
||||
@ -129,7 +130,7 @@ public class SniHandler extends AbstractSniHandler<SslContext> {
|
||||
protected void replaceHandler(ChannelHandlerContext ctx, String hostname, SslContext sslContext) throws Exception {
|
||||
SslHandler sslHandler = null;
|
||||
try {
|
||||
sslHandler = sslContext.newHandler(ctx.alloc());
|
||||
sslHandler = newSslHandler(sslContext, ctx.alloc());
|
||||
ctx.pipeline().replace(this, SslHandler.class.getName(), sslHandler);
|
||||
sslHandler = null;
|
||||
} finally {
|
||||
@ -142,6 +143,14 @@ public class SniHandler extends AbstractSniHandler<SslContext> {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a new {@link SslHandler} using the given {@link SslContext} and {@link ByteBufAllocator}.
|
||||
* Users may override this method to implement custom behavior.
|
||||
*/
|
||||
protected SslHandler newSslHandler(SslContext context, ByteBufAllocator allocator) {
|
||||
return context.newHandler(allocator);
|
||||
}
|
||||
|
||||
private static final class AsyncMappingAdapter implements AsyncMapping<String, SslContext> {
|
||||
private final Mapping<? super String, ? extends SslContext> mapping;
|
||||
|
||||
|
@ -60,6 +60,7 @@ import java.security.cert.X509Certificate;
|
||||
import java.security.spec.InvalidKeySpecException;
|
||||
import java.security.spec.PKCS8EncodedKeySpec;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
/**
|
||||
* A secure socket protocol implementation which acts as a factory for {@link SSLEngine} and {@link SslHandler}.
|
||||
@ -879,6 +880,22 @@ public abstract class SslContext {
|
||||
*/
|
||||
public abstract SSLSessionContext sessionContext();
|
||||
|
||||
/**
|
||||
* Create a new SslHandler.
|
||||
* @see #newHandler(ByteBufAllocator, Executor)
|
||||
*/
|
||||
public final SslHandler newHandler(ByteBufAllocator alloc) {
|
||||
return newHandler(alloc, startTls);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new SslHandler.
|
||||
* @see #newHandler(ByteBufAllocator)
|
||||
*/
|
||||
protected SslHandler newHandler(ByteBufAllocator alloc, boolean startTls) {
|
||||
return new SslHandler(newEngine(alloc), startTls);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link SslHandler}.
|
||||
* <p>If {@link SslProvider#OPENSSL_REFCNT} is used then the returned {@link SslHandler} will release the engine
|
||||
@ -900,18 +917,37 @@ public abstract class SslContext {
|
||||
* <a href="https://docs.oracle.com/javase/7/docs/api/javax/net/ssl/SSLEngine.html">SSLEngine javadocs</a> which
|
||||
* limits wrap/unwrap to operate on a single SSL/TLS packet.
|
||||
* @param alloc If supported by the SSLEngine then the SSLEngine will use this to allocate ByteBuf objects.
|
||||
* @param delegatedTaskExecutor the {@link Executor} that will be used to execute tasks that are returned by
|
||||
* {@link SSLEngine#getDelegatedTask()}.
|
||||
* @return a new {@link SslHandler}
|
||||
*/
|
||||
public final SslHandler newHandler(ByteBufAllocator alloc) {
|
||||
return newHandler(alloc, startTls);
|
||||
public SslHandler newHandler(ByteBufAllocator alloc, Executor delegatedTaskExecutor) {
|
||||
return newHandler(alloc, startTls, delegatedTaskExecutor);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new SslHandler.
|
||||
* @see #newHandler(ByteBufAllocator)
|
||||
* @see #newHandler(ByteBufAllocator, String, int, boolean, Executor)
|
||||
*/
|
||||
protected SslHandler newHandler(ByteBufAllocator alloc, boolean startTls) {
|
||||
return new SslHandler(newEngine(alloc), startTls);
|
||||
protected SslHandler newHandler(ByteBufAllocator alloc, boolean startTls, Executor executor) {
|
||||
return new SslHandler(newEngine(alloc), startTls, executor);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link SslHandler}
|
||||
*
|
||||
* @see #newHandler(ByteBufAllocator, String, int, Executor)
|
||||
*/
|
||||
public final SslHandler newHandler(ByteBufAllocator alloc, String peerHost, int peerPort) {
|
||||
return newHandler(alloc, peerHost, peerPort, startTls);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new SslHandler.
|
||||
* @see #newHandler(ByteBufAllocator, String, int, boolean, Executor)
|
||||
*/
|
||||
protected SslHandler newHandler(ByteBufAllocator alloc, String peerHost, int peerPort, boolean startTls) {
|
||||
return new SslHandler(newEngine(alloc, peerHost, peerPort), startTls);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -937,19 +973,19 @@ public abstract class SslContext {
|
||||
* @param alloc If supported by the SSLEngine then the SSLEngine will use this to allocate ByteBuf objects.
|
||||
* @param peerHost the non-authoritative name of the host
|
||||
* @param peerPort the non-authoritative port
|
||||
* @param delegatedTaskExecutor the {@link Executor} that will be used to execute tasks that are returned by
|
||||
* {@link SSLEngine#getDelegatedTask()}.
|
||||
*
|
||||
* @return a new {@link SslHandler}
|
||||
*/
|
||||
public final SslHandler newHandler(ByteBufAllocator alloc, String peerHost, int peerPort) {
|
||||
return newHandler(alloc, peerHost, peerPort, startTls);
|
||||
public SslHandler newHandler(ByteBufAllocator alloc, String peerHost, int peerPort,
|
||||
Executor delegatedTaskExecutor) {
|
||||
return newHandler(alloc, peerHost, peerPort, startTls, delegatedTaskExecutor);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new SslHandler.
|
||||
* @see #newHandler(ByteBufAllocator, String, int, boolean)
|
||||
*/
|
||||
protected SslHandler newHandler(ByteBufAllocator alloc, String peerHost, int peerPort, boolean startTls) {
|
||||
return new SslHandler(newEngine(alloc, peerHost, peerPort), startTls);
|
||||
protected SslHandler newHandler(ByteBufAllocator alloc, String peerHost, int peerPort, boolean startTls,
|
||||
Executor delegatedTaskExecutor) {
|
||||
return new SslHandler(newEngine(alloc, peerHost, peerPort), startTls, delegatedTaskExecutor);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -33,6 +33,7 @@ import io.netty.channel.ChannelPipeline;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
import io.netty.channel.ChannelPromiseNotifier;
|
||||
import io.netty.handler.codec.ByteToMessageDecoder;
|
||||
import io.netty.handler.codec.DecoderException;
|
||||
import io.netty.handler.codec.UnsupportedMessageTypeException;
|
||||
import io.netty.util.ReferenceCountUtil;
|
||||
import io.netty.util.ReferenceCounted;
|
||||
@ -55,10 +56,9 @@ import java.nio.ByteBuffer;
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
import java.nio.channels.DatagramChannel;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.regex.Pattern;
|
||||
@ -390,6 +390,7 @@ public class SslHandler extends ByteToMessageDecoder implements ChannelOutboundH
|
||||
private boolean flushedBeforeHandshake;
|
||||
private boolean readDuringHandshake;
|
||||
private boolean handshakeStarted;
|
||||
|
||||
private SslHandlerCoalescingBufferQueue pendingUnencryptedWrites;
|
||||
private Promise<Channel> handshakePromise = new LazyChannelPromise();
|
||||
private final LazyChannelPromise sslClosePromise = new LazyChannelPromise();
|
||||
@ -402,6 +403,7 @@ public class SslHandler extends ByteToMessageDecoder implements ChannelOutboundH
|
||||
|
||||
private boolean outboundClosed;
|
||||
private boolean closeNotify;
|
||||
private boolean processTask;
|
||||
|
||||
private int packetLength;
|
||||
|
||||
@ -417,7 +419,7 @@ public class SslHandler extends ByteToMessageDecoder implements ChannelOutboundH
|
||||
volatile int wrapDataSize = MAX_PLAINTEXT_LENGTH;
|
||||
|
||||
/**
|
||||
* Creates a new instance.
|
||||
* Creates a new instance which runs all delegated tasks directly on the {@link EventExecutor}.
|
||||
*
|
||||
* @param engine the {@link SSLEngine} this handler will use
|
||||
*/
|
||||
@ -426,29 +428,36 @@ public class SslHandler extends ByteToMessageDecoder implements ChannelOutboundH
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new instance.
|
||||
* Creates a new instance which runs all delegated tasks directly on the {@link EventExecutor}.
|
||||
*
|
||||
* @param engine the {@link SSLEngine} this handler will use
|
||||
* @param startTls {@code true} if the first write request shouldn't be
|
||||
* encrypted by the {@link SSLEngine}
|
||||
*/
|
||||
@SuppressWarnings("deprecation")
|
||||
public SslHandler(SSLEngine engine, boolean startTls) {
|
||||
this(engine, startTls, ImmediateExecutor.INSTANCE);
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #SslHandler(SSLEngine)} instead.
|
||||
* Creates a new instance.
|
||||
*
|
||||
* @param engine the {@link SSLEngine} this handler will use
|
||||
* @param delegatedTaskExecutor the {@link Executor} that will be used to execute tasks that are returned by
|
||||
* {@link SSLEngine#getDelegatedTask()}.
|
||||
*/
|
||||
@Deprecated
|
||||
public SslHandler(SSLEngine engine, Executor delegatedTaskExecutor) {
|
||||
this(engine, false, delegatedTaskExecutor);
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #SslHandler(SSLEngine, boolean)} instead.
|
||||
* Creates a new instance.
|
||||
*
|
||||
* @param engine the {@link SSLEngine} this handler will use
|
||||
* @param startTls {@code true} if the first write request shouldn't be
|
||||
* encrypted by the {@link SSLEngine}
|
||||
* @param delegatedTaskExecutor the {@link Executor} that will be used to execute tasks that are returned by
|
||||
* {@link SSLEngine#getDelegatedTask()}.
|
||||
*/
|
||||
@Deprecated
|
||||
public SslHandler(SSLEngine engine, boolean startTls, Executor delegatedTaskExecutor) {
|
||||
if (engine == null) {
|
||||
throw new NullPointerException("engine");
|
||||
@ -774,6 +783,10 @@ public class SslHandler extends ByteToMessageDecoder implements ChannelOutboundH
|
||||
return;
|
||||
}
|
||||
|
||||
if (processTask) {
|
||||
return;
|
||||
}
|
||||
|
||||
try {
|
||||
wrapAndFlush(ctx);
|
||||
} catch (Throwable cause) {
|
||||
@ -813,7 +826,7 @@ public class SslHandler extends ByteToMessageDecoder implements ChannelOutboundH
|
||||
final int wrapDataSize = this.wrapDataSize;
|
||||
// Only continue to loop if the handler was not removed in the meantime.
|
||||
// See https://github.com/netty/netty/issues/5860
|
||||
while (!ctx.isRemoved()) {
|
||||
outer: while (!ctx.isRemoved()) {
|
||||
promise = ctx.newPromise();
|
||||
buf = wrapDataSize > 0 ?
|
||||
pendingUnencryptedWrites.remove(alloc, wrapDataSize, promise) :
|
||||
@ -850,7 +863,11 @@ public class SslHandler extends ByteToMessageDecoder implements ChannelOutboundH
|
||||
|
||||
switch (result.getHandshakeStatus()) {
|
||||
case NEED_TASK:
|
||||
runDelegatedTasks();
|
||||
if (!runDelegatedTasks(inUnwrap)) {
|
||||
// We scheduled a task on the delegatingTaskExecutor, so stop processing as we will
|
||||
// resume once the task completes.
|
||||
break outer;
|
||||
}
|
||||
break;
|
||||
case FINISHED:
|
||||
setHandshakeSuccess();
|
||||
@ -919,7 +936,7 @@ public class SslHandler extends ByteToMessageDecoder implements ChannelOutboundH
|
||||
try {
|
||||
// Only continue to loop if the handler was not removed in the meantime.
|
||||
// See https://github.com/netty/netty/issues/5860
|
||||
while (!ctx.isRemoved()) {
|
||||
outer: while (!ctx.isRemoved()) {
|
||||
if (out == null) {
|
||||
// As this is called for the handshake we have no real idea how big the buffer needs to be.
|
||||
// That said 2048 should give us enough room to include everything like ALPN / NPN data.
|
||||
@ -941,7 +958,11 @@ public class SslHandler extends ByteToMessageDecoder implements ChannelOutboundH
|
||||
setHandshakeSuccess();
|
||||
return false;
|
||||
case NEED_TASK:
|
||||
runDelegatedTasks();
|
||||
if (!runDelegatedTasks(inUnwrap)) {
|
||||
// We scheduled a task on the delegatingTaskExecutor, so stop processing as we will
|
||||
// resume once the task completes.
|
||||
break outer;
|
||||
}
|
||||
break;
|
||||
case NEED_UNWRAP:
|
||||
if (inUnwrap) {
|
||||
@ -1243,6 +1264,9 @@ public class SslHandler extends ByteToMessageDecoder implements ChannelOutboundH
|
||||
|
||||
@Override
|
||||
protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) throws SSLException {
|
||||
if (processTask) {
|
||||
return;
|
||||
}
|
||||
if (jdkCompatibilityMode) {
|
||||
decodeJdkCompatible(ctx, in);
|
||||
} else {
|
||||
@ -1252,6 +1276,10 @@ public class SslHandler extends ByteToMessageDecoder implements ChannelOutboundH
|
||||
|
||||
@Override
|
||||
public void channelReadComplete(ChannelHandlerContext ctx) throws Exception {
|
||||
channelReadComplete0(ctx);
|
||||
}
|
||||
|
||||
private void channelReadComplete0(ChannelHandlerContext ctx) {
|
||||
// Discard bytes of the cumulation buffer if needed.
|
||||
discardSomeReadBytes();
|
||||
|
||||
@ -1366,7 +1394,16 @@ public class SslHandler extends ByteToMessageDecoder implements ChannelOutboundH
|
||||
}
|
||||
break;
|
||||
case NEED_TASK:
|
||||
runDelegatedTasks();
|
||||
if (!runDelegatedTasks(true)) {
|
||||
// We scheduled a task on the delegatingTaskExecutor, so stop processing as we will
|
||||
// resume once the task completes.
|
||||
//
|
||||
// We break out of the loop only and do NOT return here as we still may need to notify
|
||||
// about the closure of the SSLEngine.
|
||||
//
|
||||
wrapLater = false;
|
||||
break unwrapLoop;
|
||||
}
|
||||
break;
|
||||
case FINISHED:
|
||||
setHandshakeSuccess();
|
||||
@ -1447,65 +1484,223 @@ public class SslHandler extends ByteToMessageDecoder implements ChannelOutboundH
|
||||
out.nioBuffer(index, len);
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches all delegated tasks from the {@link SSLEngine} and runs them via the {@link #delegatedTaskExecutor}.
|
||||
* If the {@link #delegatedTaskExecutor} is {@link ImmediateExecutor}, just call {@link Runnable#run()} directly
|
||||
* instead of using {@link Executor#execute(Runnable)}. Otherwise, run the tasks via
|
||||
* the {@link #delegatedTaskExecutor} and wait until the tasks are finished.
|
||||
*/
|
||||
private void runDelegatedTasks() {
|
||||
if (delegatedTaskExecutor == ImmediateExecutor.INSTANCE) {
|
||||
private static boolean inEventLoop(Executor executor) {
|
||||
return executor instanceof EventExecutor && ((EventExecutor) executor).inEventLoop();
|
||||
}
|
||||
|
||||
private static void runAllDelegatedTasks(SSLEngine engine) {
|
||||
for (;;) {
|
||||
Runnable task = engine.getDelegatedTask();
|
||||
if (task == null) {
|
||||
break;
|
||||
return;
|
||||
}
|
||||
|
||||
task.run();
|
||||
}
|
||||
} else {
|
||||
final List<Runnable> tasks = new ArrayList<Runnable>(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Will either run the delegated task directly calling {@link Runnable#run()} and return {@code true} or will
|
||||
* offload the delegated task using {@link Executor#execute(Runnable)} and return {@code false}.
|
||||
*
|
||||
* If the task is offloaded it will take care to resume its work on the {@link EventExecutor} once there are no
|
||||
* more tasks to process.
|
||||
*/
|
||||
private boolean runDelegatedTasks(boolean inUnwrap) {
|
||||
if (delegatedTaskExecutor == ImmediateExecutor.INSTANCE || inEventLoop(delegatedTaskExecutor)) {
|
||||
// We should run the task directly in the EventExecutor thread and not offload at all.
|
||||
for (;;) {
|
||||
final Runnable task = engine.getDelegatedTask();
|
||||
if (task == null) {
|
||||
runAllDelegatedTasks(engine);
|
||||
return true;
|
||||
}
|
||||
} else {
|
||||
executeDelegatedTasks(inUnwrap);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private void executeDelegatedTasks(boolean inUnwrap) {
|
||||
processTask = true;
|
||||
try {
|
||||
delegatedTaskExecutor.execute(new SslTasksRunner(inUnwrap));
|
||||
} catch (RejectedExecutionException e) {
|
||||
processTask = false;
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link Runnable} that will be scheduled on the {@code delegatedTaskExecutor} and will take care
|
||||
* of resume work on the {@link EventExecutor} once the task was executed.
|
||||
*/
|
||||
private final class SslTasksRunner implements Runnable {
|
||||
private final boolean inUnwrap;
|
||||
|
||||
SslTasksRunner(boolean inUnwrap) {
|
||||
this.inUnwrap = inUnwrap;
|
||||
}
|
||||
|
||||
// Handle errors which happened during task processing.
|
||||
private void taskError(Throwable e) {
|
||||
if (inUnwrap) {
|
||||
// As the error happened while the task was scheduled as part of unwrap(...) we also need to ensure
|
||||
// we fire it through the pipeline as inbound error to be consistent with what we do in decode(...).
|
||||
//
|
||||
// This will also ensure we fail the handshake future and flush all produced data.
|
||||
try {
|
||||
handleUnwrapThrowable(ctx, e);
|
||||
} catch (Throwable cause) {
|
||||
safeExceptionCaught(cause);
|
||||
}
|
||||
} else {
|
||||
setHandshakeFailure(ctx, e);
|
||||
forceFlush(ctx);
|
||||
}
|
||||
}
|
||||
|
||||
// Try to call exceptionCaught(...)
|
||||
private void safeExceptionCaught(Throwable cause) {
|
||||
try {
|
||||
exceptionCaught(ctx, wrapIfNeeded(cause));
|
||||
} catch (Throwable error) {
|
||||
ctx.fireExceptionCaught(error);
|
||||
}
|
||||
}
|
||||
|
||||
private Throwable wrapIfNeeded(Throwable cause) {
|
||||
if (!inUnwrap) {
|
||||
// If we are not in unwrap(...) we can just rethrow without wrapping at all.
|
||||
return cause;
|
||||
}
|
||||
// As the exception would have been triggered by an inbound operation we will need to wrap it in a
|
||||
// DecoderException to mimic what a decoder would do when decode(...) throws.
|
||||
return cause instanceof DecoderException ? cause : new DecoderException(cause);
|
||||
}
|
||||
|
||||
private void tryDecodeAgain() {
|
||||
try {
|
||||
channelRead(ctx, Unpooled.EMPTY_BUFFER);
|
||||
} catch (Throwable cause) {
|
||||
safeExceptionCaught(cause);
|
||||
} finally {
|
||||
// As we called channelRead(...) we also need to call channelReadComplete(...) which
|
||||
// will ensure we either call ctx.fireChannelReadComplete() or will trigger a ctx.read() if
|
||||
// more data is needed.
|
||||
channelReadComplete0(ctx);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Executed after the wrapped {@code task} was executed via {@code delegatedTaskExecutor} to resume work
|
||||
* on the {@link EventExecutor}.
|
||||
*/
|
||||
private void resumeOnEventExecutor() {
|
||||
assert ctx.executor().inEventLoop();
|
||||
|
||||
processTask = false;
|
||||
|
||||
try {
|
||||
HandshakeStatus status = engine.getHandshakeStatus();
|
||||
switch (status) {
|
||||
// There is another task that needs to be executed and offloaded to the delegatingTaskExecutor.
|
||||
case NEED_TASK:
|
||||
executeDelegatedTasks(inUnwrap);
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
tasks.add(task);
|
||||
}
|
||||
// The handshake finished, lets notify about the completion of it and resume processing.
|
||||
case FINISHED:
|
||||
setHandshakeSuccess();
|
||||
|
||||
if (tasks.isEmpty()) {
|
||||
// deliberate fall-through
|
||||
|
||||
// Not handshaking anymore, lets notify about the completion if not done yet and resume processing.
|
||||
case NOT_HANDSHAKING:
|
||||
setHandshakeSuccessIfStillHandshaking();
|
||||
try {
|
||||
// Lets call wrap to ensure we produce the alert if there is any pending and also to
|
||||
// ensure we flush any queued data..
|
||||
wrap(ctx, inUnwrap);
|
||||
} catch (Throwable e) {
|
||||
taskError(e);
|
||||
return;
|
||||
}
|
||||
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
delegatedTaskExecutor.execute(new Runnable() {
|
||||
// Flush now as we may have written some data as part of the wrap call.
|
||||
forceFlush(ctx);
|
||||
|
||||
tryDecodeAgain();
|
||||
break;
|
||||
|
||||
// We need more data so lets try to unwrap first and then call decode again which will feed us
|
||||
// with buffered data (if there is any).
|
||||
case NEED_UNWRAP:
|
||||
unwrapNonAppData(ctx);
|
||||
tryDecodeAgain();
|
||||
break;
|
||||
|
||||
// To make progress we need to call SSLEngine.wrap(...) which may produce more output data
|
||||
// that will be written to the Channel.
|
||||
case NEED_WRAP:
|
||||
try {
|
||||
wrapNonAppData(ctx, inUnwrap);
|
||||
} catch (Throwable e) {
|
||||
taskError(e);
|
||||
return;
|
||||
}
|
||||
// Flush now as we may have written some data as part of the wrap call.
|
||||
forceFlush(ctx);
|
||||
|
||||
// Now try to feed in more data that we have buffered.
|
||||
tryDecodeAgain();
|
||||
break;
|
||||
default:
|
||||
// Should never reach here as we handle all cases.
|
||||
throw new AssertionError();
|
||||
}
|
||||
} catch (Throwable cause) {
|
||||
safeExceptionCaught(cause);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
for (Runnable task: tasks) {
|
||||
task.run();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
ctx.fireExceptionCaught(e);
|
||||
} finally {
|
||||
latch.countDown();
|
||||
}
|
||||
runAllDelegatedTasks(engine);
|
||||
|
||||
// All tasks were processed.
|
||||
assert engine.getHandshakeStatus() != HandshakeStatus.NEED_TASK;
|
||||
|
||||
// Jump back on the EventExecutor.
|
||||
ctx.executor().execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
resumeOnEventExecutor();
|
||||
}
|
||||
});
|
||||
} catch (final Throwable cause) {
|
||||
handleException(cause);
|
||||
}
|
||||
}
|
||||
|
||||
boolean interrupted = false;
|
||||
while (latch.getCount() != 0) {
|
||||
private void handleException(final Throwable cause) {
|
||||
if (ctx.executor().inEventLoop()) {
|
||||
processTask = false;
|
||||
safeExceptionCaught(cause);
|
||||
} else {
|
||||
try {
|
||||
latch.await();
|
||||
} catch (InterruptedException e) {
|
||||
// Interrupt later.
|
||||
interrupted = true;
|
||||
ctx.executor().execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
processTask = false;
|
||||
safeExceptionCaught(cause);
|
||||
}
|
||||
});
|
||||
} catch (RejectedExecutionException ignore) {
|
||||
processTask = false;
|
||||
// the context itself will handle the rejected exception when try to schedule the operation so
|
||||
// ignore the RejectedExecutionException
|
||||
ctx.fireExceptionCaught(cause);
|
||||
}
|
||||
|
||||
if (interrupted) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -18,6 +18,7 @@ package io.netty.handler.ssl;
|
||||
|
||||
import io.netty.bootstrap.Bootstrap;
|
||||
import io.netty.bootstrap.ServerBootstrap;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.channel.Channel;
|
||||
import io.netty.channel.ChannelHandler;
|
||||
@ -41,6 +42,9 @@ import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.junit.AfterClass;
|
||||
@ -66,7 +70,7 @@ public class CipherSuiteCanaryTest {
|
||||
|
||||
private static SelfSignedCertificate CERT;
|
||||
|
||||
@Parameters(name = "{index}: serverSslProvider = {0}, clientSslProvider = {1}, rfcCipherName = {2}")
|
||||
@Parameters(name = "{index}: serverSslProvider = {0}, clientSslProvider = {1}, rfcCipherName = {2}, delegate = {3}")
|
||||
public static Collection<Object[]> parameters() {
|
||||
List<Object[]> dst = new ArrayList<Object[]>();
|
||||
dst.addAll(expand("TLS_DHE_RSA_WITH_AES_128_GCM_SHA256")); // DHE-RSA-AES128-GCM-SHA256
|
||||
@ -80,7 +84,7 @@ public class CipherSuiteCanaryTest {
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void destory() {
|
||||
public static void destroy() {
|
||||
GROUP.shutdownGracefully();
|
||||
CERT.delete();
|
||||
}
|
||||
@ -90,11 +94,14 @@ public class CipherSuiteCanaryTest {
|
||||
private final SslProvider clientSslProvider;
|
||||
|
||||
private final String rfcCipherName;
|
||||
private final boolean delegate;
|
||||
|
||||
public CipherSuiteCanaryTest(SslProvider serverSslProvider, SslProvider clientSslProvider, String rfcCipherName) {
|
||||
public CipherSuiteCanaryTest(SslProvider serverSslProvider, SslProvider clientSslProvider,
|
||||
String rfcCipherName, boolean delegate) {
|
||||
this.serverSslProvider = serverSslProvider;
|
||||
this.clientSslProvider = clientSslProvider;
|
||||
this.rfcCipherName = rfcCipherName;
|
||||
this.delegate = delegate;
|
||||
}
|
||||
|
||||
private static void assumeCipherAvailable(SslProvider provider, String cipher) throws NoSuchAlgorithmException {
|
||||
@ -113,6 +120,14 @@ public class CipherSuiteCanaryTest {
|
||||
Assume.assumeTrue("Unsupported cipher: " + cipher, cipherSupported);
|
||||
}
|
||||
|
||||
private static SslHandler newSslHandler(SslContext sslCtx, ByteBufAllocator allocator, Executor executor) {
|
||||
if (executor == null) {
|
||||
return sslCtx.newHandler(allocator);
|
||||
} else {
|
||||
return sslCtx.newHandler(allocator, executor);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHandshake() throws Exception {
|
||||
// Check if the cipher is supported at all which may not be the case for various JDK versions and OpenSSL API
|
||||
@ -129,6 +144,8 @@ public class CipherSuiteCanaryTest {
|
||||
.protocols(SslUtils.PROTOCOL_TLS_V1_2)
|
||||
.build();
|
||||
|
||||
final ExecutorService executorService = delegate ? Executors.newCachedThreadPool() : null;
|
||||
|
||||
try {
|
||||
final SslContext sslClientContext = SslContextBuilder.forClient()
|
||||
.sslProvider(clientSslProvider)
|
||||
@ -146,7 +163,7 @@ public class CipherSuiteCanaryTest {
|
||||
@Override
|
||||
protected void initChannel(Channel ch) throws Exception {
|
||||
ChannelPipeline pipeline = ch.pipeline();
|
||||
pipeline.addLast(sslServerContext.newHandler(ch.alloc()));
|
||||
pipeline.addLast(newSslHandler(sslServerContext, ch.alloc(), executorService));
|
||||
|
||||
pipeline.addLast(new SimpleChannelInboundHandler<Object>() {
|
||||
@Override
|
||||
@ -182,7 +199,7 @@ public class CipherSuiteCanaryTest {
|
||||
@Override
|
||||
protected void initChannel(Channel ch) throws Exception {
|
||||
ChannelPipeline pipeline = ch.pipeline();
|
||||
pipeline.addLast(sslClientContext.newHandler(ch.alloc()));
|
||||
pipeline.addLast(newSslHandler(sslClientContext, ch.alloc(), executorService));
|
||||
|
||||
pipeline.addLast(new SimpleChannelInboundHandler<Object>() {
|
||||
@Override
|
||||
@ -229,6 +246,10 @@ public class CipherSuiteCanaryTest {
|
||||
}
|
||||
} finally {
|
||||
ReferenceCountUtil.release(sslServerContext);
|
||||
|
||||
if (executorService != null) {
|
||||
executorService.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -267,7 +288,8 @@ public class CipherSuiteCanaryTest {
|
||||
continue;
|
||||
}
|
||||
|
||||
dst.add(new Object[]{serverSslProvider, clientSslProvider, rfcCipherName});
|
||||
dst.add(new Object[]{serverSslProvider, clientSslProvider, rfcCipherName, true});
|
||||
dst.add(new Object[]{serverSslProvider, clientSslProvider, rfcCipherName, false});
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -31,17 +31,18 @@ import static org.junit.Assume.assumeTrue;
|
||||
@RunWith(Parameterized.class)
|
||||
public class ConscryptJdkSslEngineInteropTest extends SSLEngineTest {
|
||||
|
||||
@Parameterized.Parameters(name = "{index}: bufferType = {0}, combo = {1}")
|
||||
@Parameterized.Parameters(name = "{index}: bufferType = {0}, combo = {1}, delegate = {2}")
|
||||
public static Collection<Object[]> data() {
|
||||
List<Object[]> params = new ArrayList<Object[]>();
|
||||
for (BufferType type: BufferType.values()) {
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12()});
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12(), false });
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12(), true });
|
||||
}
|
||||
return params;
|
||||
}
|
||||
|
||||
public ConscryptJdkSslEngineInteropTest(BufferType type, ProtocolCipherCombo combo) {
|
||||
super(type, combo);
|
||||
public ConscryptJdkSslEngineInteropTest(BufferType type, ProtocolCipherCombo combo, boolean delegate) {
|
||||
super(type, combo, delegate);
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
|
@ -30,17 +30,18 @@ import static org.junit.Assume.assumeTrue;
|
||||
@RunWith(Parameterized.class)
|
||||
public class ConscryptSslEngineTest extends SSLEngineTest {
|
||||
|
||||
@Parameterized.Parameters(name = "{index}: bufferType = {0}, combo = {1}")
|
||||
@Parameterized.Parameters(name = "{index}: bufferType = {0}, combo = {1}, delegate = {2}")
|
||||
public static Collection<Object[]> data() {
|
||||
List<Object[]> params = new ArrayList<Object[]>();
|
||||
for (BufferType type: BufferType.values()) {
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12()});
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12(), false });
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12(), true });
|
||||
}
|
||||
return params;
|
||||
}
|
||||
|
||||
public ConscryptSslEngineTest(BufferType type, ProtocolCipherCombo combo) {
|
||||
super(type, combo);
|
||||
public ConscryptSslEngineTest(BufferType type, ProtocolCipherCombo combo, boolean delegate) {
|
||||
super(type, combo, delegate);
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
|
@ -32,17 +32,18 @@ import static org.junit.Assume.assumeTrue;
|
||||
@RunWith(Parameterized.class)
|
||||
public class JdkConscryptSslEngineInteropTest extends SSLEngineTest {
|
||||
|
||||
@Parameterized.Parameters(name = "{index}: bufferType = {0}, combo = {1}")
|
||||
@Parameterized.Parameters(name = "{index}: bufferType = {0}, combo = {1}, delegate = {2}")
|
||||
public static Collection<Object[]> data() {
|
||||
List<Object[]> params = new ArrayList<Object[]>();
|
||||
for (BufferType type: BufferType.values()) {
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12()});
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12(), false });
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12(), true });
|
||||
}
|
||||
return params;
|
||||
}
|
||||
|
||||
public JdkConscryptSslEngineInteropTest(BufferType type, ProtocolCipherCombo combo) {
|
||||
super(type, combo);
|
||||
public JdkConscryptSslEngineInteropTest(BufferType type, ProtocolCipherCombo combo, boolean delegate) {
|
||||
super(type, combo, delegate);
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
|
@ -33,21 +33,23 @@ import static org.junit.Assume.assumeTrue;
|
||||
@RunWith(Parameterized.class)
|
||||
public class JdkOpenSslEngineInteroptTest extends SSLEngineTest {
|
||||
|
||||
@Parameterized.Parameters(name = "{index}: bufferType = {0}, combo = {1}")
|
||||
@Parameterized.Parameters(name = "{index}: bufferType = {0}, combo = {1}, delegate = {2}")
|
||||
public static Collection<Object[]> data() {
|
||||
List<Object[]> params = new ArrayList<Object[]>();
|
||||
for (BufferType type: BufferType.values()) {
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12()});
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12(), false });
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12(), true });
|
||||
|
||||
if (PlatformDependent.javaVersion() >= 11 && OpenSsl.isTlsv13Supported()) {
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv13() });
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv13(), false });
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv13(), true });
|
||||
}
|
||||
}
|
||||
return params;
|
||||
}
|
||||
|
||||
public JdkOpenSslEngineInteroptTest(BufferType type, ProtocolCipherCombo protocolCipherCombo) {
|
||||
super(type, protocolCipherCombo);
|
||||
public JdkOpenSslEngineInteroptTest(BufferType type, ProtocolCipherCombo protocolCipherCombo, boolean delegate) {
|
||||
super(type, protocolCipherCombo, delegate);
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
|
@ -142,14 +142,17 @@ public class JdkSslEngineTest extends SSLEngineTest {
|
||||
private static final String FALLBACK_APPLICATION_LEVEL_PROTOCOL = "my-protocol-http1_1";
|
||||
private static final String APPLICATION_LEVEL_PROTOCOL_NOT_COMPATIBLE = "my-protocol-FOO";
|
||||
|
||||
@Parameterized.Parameters(name = "{index}: providerType = {0}, bufferType = {1}, combo = {2}")
|
||||
@Parameterized.Parameters(name = "{index}: providerType = {0}, bufferType = {1}, combo = {2}, delegate = {3}")
|
||||
public static Collection<Object[]> data() {
|
||||
List<Object[]> params = new ArrayList<Object[]>();
|
||||
for (ProviderType providerType : ProviderType.values()) {
|
||||
for (BufferType bufferType : BufferType.values()) {
|
||||
params.add(new Object[]{ providerType, bufferType, ProtocolCipherCombo.tlsv12()});
|
||||
params.add(new Object[]{ providerType, bufferType, ProtocolCipherCombo.tlsv12(), true });
|
||||
params.add(new Object[]{ providerType, bufferType, ProtocolCipherCombo.tlsv12(), false });
|
||||
|
||||
if (PlatformDependent.javaVersion() >= 11) {
|
||||
params.add(new Object[] { providerType, bufferType, ProtocolCipherCombo.tlsv13() });
|
||||
params.add(new Object[] { providerType, bufferType, ProtocolCipherCombo.tlsv13(), true });
|
||||
params.add(new Object[] { providerType, bufferType, ProtocolCipherCombo.tlsv13(), false });
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -160,8 +163,9 @@ public class JdkSslEngineTest extends SSLEngineTest {
|
||||
|
||||
private Provider provider;
|
||||
|
||||
public JdkSslEngineTest(ProviderType providerType, BufferType bufferType, ProtocolCipherCombo protocolCipherCombo) {
|
||||
super(bufferType, protocolCipherCombo);
|
||||
public JdkSslEngineTest(ProviderType providerType, BufferType bufferType,
|
||||
ProtocolCipherCombo protocolCipherCombo, boolean delegate) {
|
||||
super(bufferType, protocolCipherCombo, delegate);
|
||||
this.providerType = providerType;
|
||||
}
|
||||
|
||||
|
@ -67,21 +67,23 @@ public class OpenSslEngineTest extends SSLEngineTest {
|
||||
private static final String PREFERRED_APPLICATION_LEVEL_PROTOCOL = "my-protocol-http2";
|
||||
private static final String FALLBACK_APPLICATION_LEVEL_PROTOCOL = "my-protocol-http1_1";
|
||||
|
||||
@Parameterized.Parameters(name = "{index}: bufferType = {0}, combo = {1}")
|
||||
@Parameterized.Parameters(name = "{index}: bufferType = {0}, combo = {1}, delegate = {2}")
|
||||
public static Collection<Object[]> data() {
|
||||
List<Object[]> params = new ArrayList<Object[]>();
|
||||
for (BufferType type: BufferType.values()) {
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12()});
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12(), false });
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12(), true });
|
||||
|
||||
if (OpenSsl.isTlsv13Supported()) {
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv13() });
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv13(), false });
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv13(), true });
|
||||
}
|
||||
}
|
||||
return params;
|
||||
}
|
||||
|
||||
public OpenSslEngineTest(BufferType type, ProtocolCipherCombo cipherCombo) {
|
||||
super(type, cipherCombo);
|
||||
public OpenSslEngineTest(BufferType type, ProtocolCipherCombo cipherCombo, boolean delegate) {
|
||||
super(type, cipherCombo, delegate);
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
|
@ -35,21 +35,23 @@ import static org.junit.Assume.assumeTrue;
|
||||
@RunWith(Parameterized.class)
|
||||
public class OpenSslJdkSslEngineInteroptTest extends SSLEngineTest {
|
||||
|
||||
@Parameterized.Parameters(name = "{index}: bufferType = {0}, combo = {1}")
|
||||
@Parameterized.Parameters(name = "{index}: bufferType = {0}, combo = {1}, delegate = {2}")
|
||||
public static Collection<Object[]> data() {
|
||||
List<Object[]> params = new ArrayList<Object[]>();
|
||||
for (BufferType type: BufferType.values()) {
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12()});
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12(), false });
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv12(), true });
|
||||
|
||||
if (PlatformDependent.javaVersion() >= 11 && OpenSsl.isTlsv13Supported()) {
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv13() });
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv13(), false });
|
||||
params.add(new Object[] { type, ProtocolCipherCombo.tlsv13(), true });
|
||||
}
|
||||
}
|
||||
return params;
|
||||
}
|
||||
|
||||
public OpenSslJdkSslEngineInteroptTest(BufferType type, ProtocolCipherCombo combo) {
|
||||
super(type, combo);
|
||||
public OpenSslJdkSslEngineInteroptTest(BufferType type, ProtocolCipherCombo combo, boolean delegate) {
|
||||
super(type, combo, delegate);
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
|
@ -23,8 +23,8 @@ import javax.net.ssl.SSLEngine;
|
||||
|
||||
public class ReferenceCountedOpenSslEngineTest extends OpenSslEngineTest {
|
||||
|
||||
public ReferenceCountedOpenSslEngineTest(BufferType type, ProtocolCipherCombo combo) {
|
||||
super(type, combo);
|
||||
public ReferenceCountedOpenSslEngineTest(BufferType type, ProtocolCipherCombo combo, boolean delegate) {
|
||||
super(type, combo, delegate);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -73,6 +73,8 @@ import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import javax.net.ssl.KeyManagerFactory;
|
||||
@ -252,10 +254,13 @@ public abstract class SSLEngineTest {
|
||||
|
||||
private final BufferType type;
|
||||
private final ProtocolCipherCombo protocolCipherCombo;
|
||||
private final boolean delegate;
|
||||
private ExecutorService delegatingExecutor;
|
||||
|
||||
protected SSLEngineTest(BufferType type, ProtocolCipherCombo protocolCipherCombo) {
|
||||
protected SSLEngineTest(BufferType type, ProtocolCipherCombo protocolCipherCombo, boolean delegate) {
|
||||
this.type = type;
|
||||
this.protocolCipherCombo = protocolCipherCombo;
|
||||
this.delegate = delegate;
|
||||
}
|
||||
|
||||
protected ByteBuffer allocateBuffer(int len) {
|
||||
@ -441,6 +446,9 @@ public abstract class SSLEngineTest {
|
||||
MockitoAnnotations.initMocks(this);
|
||||
serverLatch = new CountDownLatch(1);
|
||||
clientLatch = new CountDownLatch(1);
|
||||
if (delegate) {
|
||||
delegatingExecutor = Executors.newCachedThreadPool();
|
||||
}
|
||||
}
|
||||
|
||||
@After
|
||||
@ -500,6 +508,10 @@ public abstract class SSLEngineTest {
|
||||
clientGroupShutdownFuture.sync();
|
||||
}
|
||||
serverException = null;
|
||||
|
||||
if (delegatingExecutor != null) {
|
||||
delegatingExecutor.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@ -703,7 +715,8 @@ public abstract class SSLEngineTest {
|
||||
ch.config().setAllocator(new TestByteBufAllocator(ch.config().getAllocator(), type));
|
||||
|
||||
ChannelPipeline p = ch.pipeline();
|
||||
SslHandler handler = serverSslCtx.newHandler(ch.alloc());
|
||||
SslHandler handler = delegatingExecutor == null ? serverSslCtx.newHandler(ch.alloc()) :
|
||||
serverSslCtx.newHandler(ch.alloc(), delegatingExecutor);
|
||||
if (serverInitEngine) {
|
||||
mySetupMutualAuthServerInitSslHandler(handler);
|
||||
}
|
||||
@ -746,7 +759,10 @@ public abstract class SSLEngineTest {
|
||||
protected void initChannel(Channel ch) throws Exception {
|
||||
ch.config().setAllocator(new TestByteBufAllocator(ch.config().getAllocator(), type));
|
||||
ChannelPipeline p = ch.pipeline();
|
||||
p.addLast(clientSslCtx.newHandler(ch.alloc()));
|
||||
|
||||
SslHandler handler = delegatingExecutor == null ? clientSslCtx.newHandler(ch.alloc()) :
|
||||
clientSslCtx.newHandler(ch.alloc(), delegatingExecutor);
|
||||
p.addLast(handler);
|
||||
p.addLast(new MessageDelegatorChannelHandler(clientReceiver, clientLatch));
|
||||
p.addLast(new ChannelInboundHandlerAdapter() {
|
||||
@Override
|
||||
@ -849,7 +865,10 @@ public abstract class SSLEngineTest {
|
||||
protected void initChannel(Channel ch) throws Exception {
|
||||
ch.config().setAllocator(new TestByteBufAllocator(ch.config().getAllocator(), type));
|
||||
ChannelPipeline p = ch.pipeline();
|
||||
p.addLast(serverSslCtx.newHandler(ch.alloc()));
|
||||
|
||||
SslHandler handler = delegatingExecutor == null ? serverSslCtx.newHandler(ch.alloc()) :
|
||||
serverSslCtx.newHandler(ch.alloc(), delegatingExecutor);
|
||||
p.addLast(handler);
|
||||
p.addLast(new MessageDelegatorChannelHandler(serverReceiver, serverLatch));
|
||||
p.addLast(new ChannelInboundHandlerAdapter() {
|
||||
@Override
|
||||
@ -889,7 +908,11 @@ public abstract class SSLEngineTest {
|
||||
ch.config().setAllocator(new TestByteBufAllocator(ch.config().getAllocator(), type));
|
||||
ChannelPipeline p = ch.pipeline();
|
||||
InetSocketAddress remoteAddress = (InetSocketAddress) serverChannel.localAddress();
|
||||
SslHandler sslHandler = clientSslCtx.newHandler(ch.alloc(), expectedHost, 0);
|
||||
|
||||
SslHandler sslHandler = delegatingExecutor == null ?
|
||||
clientSslCtx.newHandler(ch.alloc(), expectedHost, 0) :
|
||||
clientSslCtx.newHandler(ch.alloc(), expectedHost, 0, delegatingExecutor);
|
||||
|
||||
SSLParameters parameters = sslHandler.engine().getSSLParameters();
|
||||
if (SslUtils.isValidHostNameForSNI(expectedHost)) {
|
||||
assertEquals(1, parameters.getServerNames().size());
|
||||
@ -1053,7 +1076,10 @@ public abstract class SSLEngineTest {
|
||||
protected void initChannel(Channel ch) throws Exception {
|
||||
ch.config().setAllocator(new TestByteBufAllocator(ch.config().getAllocator(), type));
|
||||
|
||||
final SslHandler handler = clientSslCtx.newHandler(ch.alloc());
|
||||
final SslHandler handler = delegatingExecutor == null ?
|
||||
clientSslCtx.newHandler(ch.alloc()) :
|
||||
clientSslCtx.newHandler(ch.alloc(), delegatingExecutor);
|
||||
|
||||
handler.engine().setNeedClientAuth(true);
|
||||
ChannelPipeline p = ch.pipeline();
|
||||
p.addLast(handler);
|
||||
@ -1125,7 +1151,7 @@ public abstract class SSLEngineTest {
|
||||
MessageReceiver receiver) throws Exception {
|
||||
List<ByteBuf> dataCapture = null;
|
||||
try {
|
||||
assertTrue(sendChannel.writeAndFlush(message).await(5, TimeUnit.SECONDS));
|
||||
assertTrue(sendChannel.writeAndFlush(message).await(50, TimeUnit.SECONDS));
|
||||
receiverLatch.await(5, TimeUnit.SECONDS);
|
||||
message.resetReaderIndex();
|
||||
ArgumentCaptor<ByteBuf> captor = ArgumentCaptor.forClass(ByteBuf.class);
|
||||
@ -1248,7 +1274,12 @@ public abstract class SSLEngineTest {
|
||||
ch.config().setAllocator(new TestByteBufAllocator(ch.config().getAllocator(), type));
|
||||
|
||||
ChannelPipeline p = ch.pipeline();
|
||||
p.addLast(serverSslCtx.newHandler(ch.alloc()));
|
||||
|
||||
SslHandler handler = delegatingExecutor == null ?
|
||||
serverSslCtx.newHandler(ch.alloc()) :
|
||||
serverSslCtx.newHandler(ch.alloc(), delegatingExecutor);
|
||||
|
||||
p.addLast(handler);
|
||||
p.addLast(new ChannelInboundHandlerAdapter() {
|
||||
@Override
|
||||
public void userEventTriggered(ChannelHandlerContext ctx, Object evt) {
|
||||
@ -1304,7 +1335,11 @@ public abstract class SSLEngineTest {
|
||||
ch.config().setAllocator(new TestByteBufAllocator(ch.config().getAllocator(), type));
|
||||
|
||||
ChannelPipeline p = ch.pipeline();
|
||||
SslHandler sslHandler = clientSslCtx.newHandler(ch.alloc());
|
||||
|
||||
SslHandler sslHandler = delegatingExecutor == null ?
|
||||
clientSslCtx.newHandler(ch.alloc()) :
|
||||
clientSslCtx.newHandler(ch.alloc(), delegatingExecutor);
|
||||
|
||||
// The renegotiate is not expected to succeed, so we should stop trying in a timely manner so
|
||||
// the unit test can terminate relativley quicly.
|
||||
sslHandler.setHandshakeTimeout(1, TimeUnit.SECONDS);
|
||||
@ -1384,7 +1419,7 @@ public abstract class SSLEngineTest {
|
||||
}
|
||||
}
|
||||
|
||||
protected void handshake(SSLEngine clientEngine, SSLEngine serverEngine) throws SSLException {
|
||||
protected void handshake(SSLEngine clientEngine, SSLEngine serverEngine) throws Exception {
|
||||
ByteBuffer cTOs = allocateBuffer(clientEngine.getSession().getPacketBufferSize());
|
||||
ByteBuffer sTOc = allocateBuffer(serverEngine.getSession().getPacketBufferSize());
|
||||
|
||||
@ -1477,14 +1512,18 @@ public abstract class SSLEngineTest {
|
||||
return result.getHandshakeStatus() == SSLEngineResult.HandshakeStatus.FINISHED;
|
||||
}
|
||||
|
||||
private static void runDelegatedTasks(SSLEngineResult result, SSLEngine engine) {
|
||||
private void runDelegatedTasks(SSLEngineResult result, SSLEngine engine) throws Exception {
|
||||
if (result.getHandshakeStatus() == SSLEngineResult.HandshakeStatus.NEED_TASK) {
|
||||
for (;;) {
|
||||
Runnable task = engine.getDelegatedTask();
|
||||
if (task == null) {
|
||||
break;
|
||||
}
|
||||
if (delegatingExecutor == null) {
|
||||
task.run();
|
||||
} else {
|
||||
delegatingExecutor.submit(task).get();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1586,7 +1625,12 @@ public abstract class SSLEngineTest {
|
||||
ch.config().setAllocator(new TestByteBufAllocator(ch.config().getAllocator(), type));
|
||||
|
||||
ChannelPipeline p = ch.pipeline();
|
||||
p.addLast(serverSslCtx.newHandler(ch.alloc()));
|
||||
|
||||
SslHandler sslHandler = delegatingExecutor == null ?
|
||||
serverSslCtx.newHandler(ch.alloc()) :
|
||||
serverSslCtx.newHandler(ch.alloc(), delegatingExecutor);
|
||||
|
||||
p.addLast(sslHandler);
|
||||
p.addLast(new MessageDelegatorChannelHandler(serverReceiver, serverLatch));
|
||||
p.addLast(new ChannelInboundHandlerAdapter() {
|
||||
@Override
|
||||
@ -1611,7 +1655,12 @@ public abstract class SSLEngineTest {
|
||||
ch.config().setAllocator(new TestByteBufAllocator(ch.config().getAllocator(), type));
|
||||
|
||||
ChannelPipeline p = ch.pipeline();
|
||||
p.addLast(clientSslCtx.newHandler(ch.alloc()));
|
||||
|
||||
SslHandler sslHandler = delegatingExecutor == null ?
|
||||
clientSslCtx.newHandler(ch.alloc()) :
|
||||
clientSslCtx.newHandler(ch.alloc(), delegatingExecutor);
|
||||
|
||||
p.addLast(sslHandler);
|
||||
p.addLast(new MessageDelegatorChannelHandler(clientReceiver, clientLatch));
|
||||
p.addLast(new ChannelInboundHandlerAdapter() {
|
||||
@Override
|
||||
@ -1661,7 +1710,11 @@ public abstract class SSLEngineTest {
|
||||
protected void initChannel(Channel ch) throws Exception {
|
||||
ch.config().setAllocator(new TestByteBufAllocator(ch.config().getAllocator(), type));
|
||||
|
||||
ch.pipeline().addFirst(serverSslCtx.newHandler(ch.alloc()));
|
||||
SslHandler sslHandler = delegatingExecutor == null ?
|
||||
serverSslCtx.newHandler(ch.alloc()) :
|
||||
serverSslCtx.newHandler(ch.alloc(), delegatingExecutor);
|
||||
|
||||
ch.pipeline().addFirst(sslHandler);
|
||||
ch.pipeline().addLast(new ChannelInboundHandlerAdapter() {
|
||||
@Override
|
||||
public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
|
||||
|
@ -53,6 +53,8 @@ import io.netty.util.ReferenceCountUtil;
|
||||
import io.netty.util.ReferenceCounted;
|
||||
import io.netty.util.concurrent.Future;
|
||||
import io.netty.util.concurrent.FutureListener;
|
||||
import io.netty.util.concurrent.ImmediateEventExecutor;
|
||||
import io.netty.util.concurrent.ImmediateExecutor;
|
||||
import io.netty.util.concurrent.Promise;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
import org.junit.Test;
|
||||
@ -63,6 +65,9 @@ import java.security.NoSuchAlgorithmException;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
@ -824,4 +829,168 @@ public class SslHandlerTest {
|
||||
ReferenceCountUtil.release(sslClientCtx);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHandshakeWithExecutorThatExecuteDirecty() throws Exception {
|
||||
testHandshakeWithExecutor(new Executor() {
|
||||
@Override
|
||||
public void execute(Runnable command) {
|
||||
command.run();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHandshakeWithImmediateExecutor() throws Exception {
|
||||
testHandshakeWithExecutor(ImmediateExecutor.INSTANCE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHandshakeWithImmediateEventExecutor() throws Exception {
|
||||
testHandshakeWithExecutor(ImmediateEventExecutor.INSTANCE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHandshakeWithExecutor() throws Exception {
|
||||
ExecutorService executorService = Executors.newCachedThreadPool();
|
||||
try {
|
||||
testHandshakeWithExecutor(executorService);
|
||||
} finally {
|
||||
executorService.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
private void testHandshakeWithExecutor(Executor executor) throws Exception {
|
||||
final SslContext sslClientCtx = SslContextBuilder.forClient()
|
||||
.trustManager(InsecureTrustManagerFactory.INSTANCE)
|
||||
.sslProvider(SslProvider.JDK).build();
|
||||
|
||||
final SelfSignedCertificate cert = new SelfSignedCertificate();
|
||||
final SslContext sslServerCtx = SslContextBuilder.forServer(cert.key(), cert.cert())
|
||||
.sslProvider(SslProvider.JDK).build();
|
||||
|
||||
EventLoopGroup group = new NioEventLoopGroup();
|
||||
Channel sc = null;
|
||||
Channel cc = null;
|
||||
final SslHandler clientSslHandler = sslClientCtx.newHandler(UnpooledByteBufAllocator.DEFAULT, executor);
|
||||
final SslHandler serverSslHandler = sslServerCtx.newHandler(UnpooledByteBufAllocator.DEFAULT, executor);
|
||||
|
||||
try {
|
||||
sc = new ServerBootstrap()
|
||||
.group(group)
|
||||
.channel(NioServerSocketChannel.class)
|
||||
.childHandler(serverSslHandler)
|
||||
.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
|
||||
|
||||
ChannelFuture future = new Bootstrap()
|
||||
.group(group)
|
||||
.channel(NioSocketChannel.class)
|
||||
.handler(new ChannelInitializer<Channel>() {
|
||||
@Override
|
||||
protected void initChannel(Channel ch) {
|
||||
ch.pipeline().addLast(clientSslHandler);
|
||||
}
|
||||
}).connect(sc.localAddress());
|
||||
cc = future.syncUninterruptibly().channel();
|
||||
|
||||
assertTrue(clientSslHandler.handshakeFuture().await().isSuccess());
|
||||
assertTrue(serverSslHandler.handshakeFuture().await().isSuccess());
|
||||
} finally {
|
||||
if (cc != null) {
|
||||
cc.close().syncUninterruptibly();
|
||||
}
|
||||
if (sc != null) {
|
||||
sc.close().syncUninterruptibly();
|
||||
}
|
||||
group.shutdownGracefully();
|
||||
ReferenceCountUtil.release(sslClientCtx);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClientHandshakeTimeoutBecauseExecutorNotExecute() throws Exception {
|
||||
testHandshakeTimeoutBecauseExecutorNotExecute(true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testServerHandshakeTimeoutBecauseExecutorNotExecute() throws Exception {
|
||||
testHandshakeTimeoutBecauseExecutorNotExecute(false);
|
||||
}
|
||||
|
||||
private void testHandshakeTimeoutBecauseExecutorNotExecute(final boolean client) throws Exception {
|
||||
final SslContext sslClientCtx = SslContextBuilder.forClient()
|
||||
.trustManager(InsecureTrustManagerFactory.INSTANCE)
|
||||
.sslProvider(SslProvider.JDK).build();
|
||||
|
||||
final SelfSignedCertificate cert = new SelfSignedCertificate();
|
||||
final SslContext sslServerCtx = SslContextBuilder.forServer(cert.key(), cert.cert())
|
||||
.sslProvider(SslProvider.JDK).build();
|
||||
|
||||
EventLoopGroup group = new NioEventLoopGroup();
|
||||
Channel sc = null;
|
||||
Channel cc = null;
|
||||
final SslHandler clientSslHandler = sslClientCtx.newHandler(UnpooledByteBufAllocator.DEFAULT, new Executor() {
|
||||
@Override
|
||||
public void execute(Runnable command) {
|
||||
if (!client) {
|
||||
command.run();
|
||||
}
|
||||
// Do nothing to simulate slow execution.
|
||||
}
|
||||
});
|
||||
if (client) {
|
||||
clientSslHandler.setHandshakeTimeout(100, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
final SslHandler serverSslHandler = sslServerCtx.newHandler(UnpooledByteBufAllocator.DEFAULT, new Executor() {
|
||||
@Override
|
||||
public void execute(Runnable command) {
|
||||
if (client) {
|
||||
command.run();
|
||||
}
|
||||
// Do nothing to simulate slow execution.
|
||||
}
|
||||
});
|
||||
if (!client) {
|
||||
serverSslHandler.setHandshakeTimeout(100, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
try {
|
||||
sc = new ServerBootstrap()
|
||||
.group(group)
|
||||
.channel(NioServerSocketChannel.class)
|
||||
.childHandler(serverSslHandler)
|
||||
.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
|
||||
|
||||
ChannelFuture future = new Bootstrap()
|
||||
.group(group)
|
||||
.channel(NioSocketChannel.class)
|
||||
.handler(new ChannelInitializer<Channel>() {
|
||||
@Override
|
||||
protected void initChannel(Channel ch) {
|
||||
ch.pipeline().addLast(clientSslHandler);
|
||||
}
|
||||
}).connect(sc.localAddress());
|
||||
cc = future.syncUninterruptibly().channel();
|
||||
|
||||
if (client) {
|
||||
Throwable cause = clientSslHandler.handshakeFuture().await().cause();
|
||||
assertThat(cause, CoreMatchers.<Throwable>instanceOf(SSLException.class));
|
||||
assertThat(cause.getMessage(), containsString("timed out"));
|
||||
assertFalse(serverSslHandler.handshakeFuture().await().isSuccess());
|
||||
} else {
|
||||
Throwable cause = serverSslHandler.handshakeFuture().await().cause();
|
||||
assertThat(cause, CoreMatchers.<Throwable>instanceOf(SSLException.class));
|
||||
assertThat(cause.getMessage(), containsString("timed out"));
|
||||
assertFalse(clientSslHandler.handshakeFuture().await().isSuccess());
|
||||
}
|
||||
} finally {
|
||||
if (cc != null) {
|
||||
cc.close().syncUninterruptibly();
|
||||
}
|
||||
if (sc != null) {
|
||||
sc.close().syncUninterruptibly();
|
||||
}
|
||||
group.shutdownGracefully();
|
||||
ReferenceCountUtil.release(sslClientCtx);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -18,6 +18,7 @@ package io.netty.testsuite.transport.socket;
|
||||
import io.netty.bootstrap.Bootstrap;
|
||||
import io.netty.bootstrap.ServerBootstrap;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.channel.Channel;
|
||||
import io.netty.channel.ChannelHandler.Sharable;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
@ -46,6 +47,9 @@ import java.security.cert.CertificateException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import javax.net.ssl.SSLHandshakeException;
|
||||
@ -73,7 +77,7 @@ public class SocketSslClientRenegotiateTest extends AbstractSocketTest {
|
||||
KEY_FILE = ssc.privateKey();
|
||||
}
|
||||
|
||||
@Parameters(name = "{index}: serverEngine = {0}, clientEngine = {1}")
|
||||
@Parameters(name = "{index}: serverEngine = {0}, clientEngine = {1}, delegate = {2}")
|
||||
public static Collection<Object[]> data() throws Exception {
|
||||
List<SslContext> serverContexts = new ArrayList<SslContext>();
|
||||
List<SslContext> clientContexts = new ArrayList<SslContext>();
|
||||
@ -91,7 +95,8 @@ public class SocketSslClientRenegotiateTest extends AbstractSocketTest {
|
||||
for (SslContext sc: serverContexts) {
|
||||
for (SslContext cc: clientContexts) {
|
||||
for (int i = 0; i < 32; i++) {
|
||||
params.add(new Object[] { sc, cc});
|
||||
params.add(new Object[] { sc, cc, true});
|
||||
params.add(new Object[] { sc, cc, false});
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -101,6 +106,7 @@ public class SocketSslClientRenegotiateTest extends AbstractSocketTest {
|
||||
|
||||
private final SslContext serverCtx;
|
||||
private final SslContext clientCtx;
|
||||
private final boolean delegate;
|
||||
|
||||
private final AtomicReference<Throwable> clientException = new AtomicReference<Throwable>();
|
||||
private final AtomicReference<Throwable> serverException = new AtomicReference<Throwable>();
|
||||
@ -116,9 +122,10 @@ public class SocketSslClientRenegotiateTest extends AbstractSocketTest {
|
||||
private final TestHandler serverHandler = new TestHandler(serverException);
|
||||
|
||||
public SocketSslClientRenegotiateTest(
|
||||
SslContext serverCtx, SslContext clientCtx) {
|
||||
SslContext serverCtx, SslContext clientCtx, boolean delegate) {
|
||||
this.serverCtx = serverCtx;
|
||||
this.clientCtx = clientCtx;
|
||||
this.delegate = delegate;
|
||||
}
|
||||
|
||||
@Test(timeout = 30000)
|
||||
@ -129,15 +136,26 @@ public class SocketSslClientRenegotiateTest extends AbstractSocketTest {
|
||||
run();
|
||||
}
|
||||
|
||||
private static SslHandler newSslHandler(SslContext sslCtx, ByteBufAllocator allocator, Executor executor) {
|
||||
if (executor == null) {
|
||||
return sslCtx.newHandler(allocator);
|
||||
} else {
|
||||
return sslCtx.newHandler(allocator, executor);
|
||||
}
|
||||
}
|
||||
|
||||
public void testSslRenegotiationRejected(ServerBootstrap sb, Bootstrap cb) throws Throwable {
|
||||
reset();
|
||||
|
||||
final ExecutorService executorService = delegate ? Executors.newCachedThreadPool() : null;
|
||||
|
||||
try {
|
||||
sb.childHandler(new ChannelInitializer<Channel>() {
|
||||
@Override
|
||||
@SuppressWarnings("deprecation")
|
||||
public void initChannel(Channel sch) throws Exception {
|
||||
serverChannel = sch;
|
||||
serverSslHandler = serverCtx.newHandler(sch.alloc());
|
||||
serverSslHandler = newSslHandler(serverCtx, sch.alloc(), executorService);
|
||||
// As we test renegotiation we should use a protocol that support it.
|
||||
serverSslHandler.engine().setEnabledProtocols(new String[]{"TLSv1.2"});
|
||||
sch.pipeline().addLast("ssl", serverSslHandler);
|
||||
@ -150,7 +168,7 @@ public class SocketSslClientRenegotiateTest extends AbstractSocketTest {
|
||||
@SuppressWarnings("deprecation")
|
||||
public void initChannel(Channel sch) throws Exception {
|
||||
clientChannel = sch;
|
||||
clientSslHandler = clientCtx.newHandler(sch.alloc());
|
||||
clientSslHandler = newSslHandler(clientCtx, sch.alloc(), executorService);
|
||||
// As we test renegotiation we should use a protocol that support it.
|
||||
clientSslHandler.engine().setEnabledProtocols(new String[]{"TLSv1.2"});
|
||||
sch.pipeline().addLast("ssl", clientSslHandler);
|
||||
@ -182,6 +200,11 @@ public class SocketSslClientRenegotiateTest extends AbstractSocketTest {
|
||||
if (clientException.get() != null) {
|
||||
throw clientException.get();
|
||||
}
|
||||
} finally {
|
||||
if (executorService != null) {
|
||||
executorService.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void reset() {
|
||||
|
@ -18,6 +18,7 @@ package io.netty.testsuite.transport.socket;
|
||||
import io.netty.bootstrap.Bootstrap;
|
||||
import io.netty.bootstrap.ServerBootstrap;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.channel.Channel;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelInitializer;
|
||||
@ -48,6 +49,9 @@ import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
@ -74,7 +78,7 @@ public class SocketSslGreetingTest extends AbstractSocketTest {
|
||||
KEY_FILE = ssc.privateKey();
|
||||
}
|
||||
|
||||
@Parameters(name = "{index}: serverEngine = {0}, clientEngine = {1}")
|
||||
@Parameters(name = "{index}: serverEngine = {0}, clientEngine = {1}, delegate = {2}")
|
||||
public static Collection<Object[]> data() throws Exception {
|
||||
List<SslContext> serverContexts = new ArrayList<SslContext>();
|
||||
serverContexts.add(SslContextBuilder.forServer(CERT_FILE, KEY_FILE).sslProvider(SslProvider.JDK).build());
|
||||
@ -95,7 +99,8 @@ public class SocketSslGreetingTest extends AbstractSocketTest {
|
||||
List<Object[]> params = new ArrayList<Object[]>();
|
||||
for (SslContext sc: serverContexts) {
|
||||
for (SslContext cc: clientContexts) {
|
||||
params.add(new Object[] { sc, cc });
|
||||
params.add(new Object[] { sc, cc, true });
|
||||
params.add(new Object[] { sc, cc, false });
|
||||
}
|
||||
}
|
||||
return params;
|
||||
@ -103,10 +108,20 @@ public class SocketSslGreetingTest extends AbstractSocketTest {
|
||||
|
||||
private final SslContext serverCtx;
|
||||
private final SslContext clientCtx;
|
||||
private final boolean delegate;
|
||||
|
||||
public SocketSslGreetingTest(SslContext serverCtx, SslContext clientCtx) {
|
||||
public SocketSslGreetingTest(SslContext serverCtx, SslContext clientCtx, boolean delegate) {
|
||||
this.serverCtx = serverCtx;
|
||||
this.clientCtx = clientCtx;
|
||||
this.delegate = delegate;
|
||||
}
|
||||
|
||||
private static SslHandler newSslHandler(SslContext sslCtx, ByteBufAllocator allocator, Executor executor) {
|
||||
if (executor == null) {
|
||||
return sslCtx.newHandler(allocator);
|
||||
} else {
|
||||
return sslCtx.newHandler(allocator, executor);
|
||||
}
|
||||
}
|
||||
|
||||
// Test for https://github.com/netty/netty/pull/2437
|
||||
@ -119,11 +134,13 @@ public class SocketSslGreetingTest extends AbstractSocketTest {
|
||||
final ServerHandler sh = new ServerHandler();
|
||||
final ClientHandler ch = new ClientHandler();
|
||||
|
||||
final ExecutorService executorService = delegate ? Executors.newCachedThreadPool() : null;
|
||||
try {
|
||||
sb.childHandler(new ChannelInitializer<Channel>() {
|
||||
@Override
|
||||
public void initChannel(Channel sch) throws Exception {
|
||||
ChannelPipeline p = sch.pipeline();
|
||||
p.addLast(serverCtx.newHandler(sch.alloc()));
|
||||
p.addLast(newSslHandler(serverCtx, sch.alloc(), executorService));
|
||||
p.addLast(new LoggingHandler(LOG_LEVEL));
|
||||
p.addLast(sh);
|
||||
}
|
||||
@ -133,7 +150,7 @@ public class SocketSslGreetingTest extends AbstractSocketTest {
|
||||
@Override
|
||||
public void initChannel(Channel sch) throws Exception {
|
||||
ChannelPipeline p = sch.pipeline();
|
||||
p.addLast(clientCtx.newHandler(sch.alloc()));
|
||||
p.addLast(newSslHandler(clientCtx, sch.alloc(), executorService));
|
||||
p.addLast(new LoggingHandler(LOG_LEVEL));
|
||||
p.addLast(ch);
|
||||
}
|
||||
@ -160,6 +177,11 @@ public class SocketSslGreetingTest extends AbstractSocketTest {
|
||||
if (ch.exception.get() != null) {
|
||||
throw ch.exception.get();
|
||||
}
|
||||
} finally {
|
||||
if (executorService != null) {
|
||||
executorService.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static class ClientHandler extends SimpleChannelInboundHandler<ByteBuf> {
|
||||
|
@ -26,8 +26,8 @@ import java.util.List;
|
||||
|
||||
public class EpollDomainSocketSslClientRenegotiateTest extends SocketSslClientRenegotiateTest {
|
||||
|
||||
public EpollDomainSocketSslClientRenegotiateTest(SslContext serverCtx, SslContext clientCtx) {
|
||||
super(serverCtx, clientCtx);
|
||||
public EpollDomainSocketSslClientRenegotiateTest(SslContext serverCtx, SslContext clientCtx, boolean delegate) {
|
||||
super(serverCtx, clientCtx, delegate);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -26,8 +26,8 @@ import java.util.List;
|
||||
|
||||
public class EpollDomainSocketSslGreetingTest extends SocketSslGreetingTest {
|
||||
|
||||
public EpollDomainSocketSslGreetingTest(SslContext serverCtx, SslContext clientCtx) {
|
||||
super(serverCtx, clientCtx);
|
||||
public EpollDomainSocketSslGreetingTest(SslContext serverCtx, SslContext clientCtx, boolean delegate) {
|
||||
super(serverCtx, clientCtx, delegate);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -25,8 +25,8 @@ import java.util.List;
|
||||
|
||||
public class EpollSocketSslClientRenegotiateTest extends SocketSslClientRenegotiateTest {
|
||||
|
||||
public EpollSocketSslClientRenegotiateTest(SslContext serverCtx, SslContext clientCtx) {
|
||||
super(serverCtx, clientCtx);
|
||||
public EpollSocketSslClientRenegotiateTest(SslContext serverCtx, SslContext clientCtx, boolean delegate) {
|
||||
super(serverCtx, clientCtx, delegate);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -25,8 +25,8 @@ import java.util.List;
|
||||
|
||||
public class EpollSocketSslGreetingTest extends SocketSslGreetingTest {
|
||||
|
||||
public EpollSocketSslGreetingTest(SslContext serverCtx, SslContext clientCtx) {
|
||||
super(serverCtx, clientCtx);
|
||||
public EpollSocketSslGreetingTest(SslContext serverCtx, SslContext clientCtx, boolean delegate) {
|
||||
super(serverCtx, clientCtx, delegate);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -26,8 +26,8 @@ import java.util.List;
|
||||
|
||||
public class KQueueDomainSocketSslClientRenegotiateTest extends SocketSslClientRenegotiateTest {
|
||||
|
||||
public KQueueDomainSocketSslClientRenegotiateTest(SslContext serverCtx, SslContext clientCtx) {
|
||||
super(serverCtx, clientCtx);
|
||||
public KQueueDomainSocketSslClientRenegotiateTest(SslContext serverCtx, SslContext clientCtx, boolean delegate) {
|
||||
super(serverCtx, clientCtx, delegate);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -26,8 +26,8 @@ import java.util.List;
|
||||
|
||||
public class KQueueDomainSocketSslGreetingTest extends SocketSslGreetingTest {
|
||||
|
||||
public KQueueDomainSocketSslGreetingTest(SslContext serverCtx, SslContext clientCtx) {
|
||||
super(serverCtx, clientCtx);
|
||||
public KQueueDomainSocketSslGreetingTest(SslContext serverCtx, SslContext clientCtx, boolean delegate) {
|
||||
super(serverCtx, clientCtx, delegate);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -25,8 +25,8 @@ import java.util.List;
|
||||
|
||||
public class KQueueSocketSslClientRenegotiateTest extends SocketSslClientRenegotiateTest {
|
||||
|
||||
public KQueueSocketSslClientRenegotiateTest(SslContext serverCtx, SslContext clientCtx) {
|
||||
super(serverCtx, clientCtx);
|
||||
public KQueueSocketSslClientRenegotiateTest(SslContext serverCtx, SslContext clientCtx, boolean delegate) {
|
||||
super(serverCtx, clientCtx, delegate);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -25,8 +25,8 @@ import java.util.List;
|
||||
|
||||
public class KQueueSocketSslGreetingTest extends SocketSslGreetingTest {
|
||||
|
||||
public KQueueSocketSslGreetingTest(SslContext serverCtx, SslContext clientCtx) {
|
||||
super(serverCtx, clientCtx);
|
||||
public KQueueSocketSslGreetingTest(SslContext serverCtx, SslContext clientCtx, boolean delegate) {
|
||||
super(serverCtx, clientCtx, delegate);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
Loading…
Reference in New Issue
Block a user