All *Bootstrap methods that used to return ChannelFuture now return Future<Channel> (#11517)

Bootstrap methods now return Future<Channel> instead of ChannelFuture

Motivation:
In #8516 it was proposed to at some point remove the specialised ChannelFuture and ChannelPromise.
Or at least make them not extend Future and Promise, respectively.
One pain point encountered in this discussion is the need to get access to the channel object after it has been initialised, but without waiting for the channel registration to propagate through the pipeline.

Modification:
Add a Bootstrap.createUnregistered method, which will return a Channel directly.
All other Bootstrap methods that previously returned ChannelFuture now return Future<Channel>

Result:
It's now possible to obtain an initialised but unregistered channel from a bootstrap, without blocking.
And the other bootstrap methods now only release their channels through the result of their futures, preventing racy access to the channels.
This commit is contained in:
Chris Vest 2021-08-03 19:43:38 +02:00 committed by GitHub
parent 4f0f889dbf
commit 6b11f7fbc2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
170 changed files with 1128 additions and 1037 deletions

View File

@ -62,13 +62,13 @@ public class HAProxyIntegrationTest {
});
}
});
Channel serverChannel = sb.bind(localAddress).sync().channel();
Channel serverChannel = sb.bind(localAddress).get();
Bootstrap b = new Bootstrap();
Channel clientChannel = b.channel(LocalChannel.class)
.handler(HAProxyMessageEncoder.INSTANCE)
.group(group)
.connect(localAddress).sync().channel();
.connect(localAddress).get();
try {
HAProxyMessage message = new HAProxyMessage(

View File

@ -20,7 +20,6 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
@ -36,6 +35,7 @@ import io.netty.handler.codec.CodecException;
import io.netty.handler.codec.PrematureChannelClosureException;
import io.netty.util.CharsetUtil;
import io.netty.util.NetUtil;
import io.netty.util.concurrent.Future;
import org.hamcrest.CoreMatchers;
import org.junit.jupiter.api.Test;
@ -44,7 +44,10 @@ import java.util.concurrent.CountDownLatch;
import static io.netty.util.ReferenceCountUtil.release;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.hamcrest.CoreMatchers.*;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.CoreMatchers.nullValue;
import static org.hamcrest.CoreMatchers.sameInstance;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.not;
import static org.junit.jupiter.api.Assertions.assertFalse;
@ -125,7 +128,7 @@ public class HttpClientCodecTest {
}
@Test
public void testServerCloseSocketInputProvidesData() throws InterruptedException {
public void testServerCloseSocketInputProvidesData() throws Exception {
ServerBootstrap sb = new ServerBootstrap();
Bootstrap cb = new Bootstrap();
final CountDownLatch serverChannelLatch = new CountDownLatch(1);
@ -186,12 +189,12 @@ public class HttpClientCodecTest {
}
});
Channel serverChannel = sb.bind(new InetSocketAddress(0)).sync().channel();
Channel serverChannel = sb.bind(new InetSocketAddress(0)).get();
int port = ((InetSocketAddress) serverChannel.localAddress()).getPort();
ChannelFuture ccf = cb.connect(new InetSocketAddress(NetUtil.LOCALHOST, port));
Future<Channel> ccf = cb.connect(new InetSocketAddress(NetUtil.LOCALHOST, port));
assertTrue(ccf.awaitUninterruptibly().isSuccess());
Channel clientChannel = ccf.channel();
Channel clientChannel = ccf.get();
assertTrue(serverChannelLatch.await(5, SECONDS));
clientChannel.writeAndFlush(new DefaultHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "/"));
assertTrue(responseReceivedLatch.await(5, SECONDS));

View File

@ -19,7 +19,6 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
@ -57,8 +56,8 @@ import static org.mockito.Mockito.any;
import static org.mockito.Mockito.anyBoolean;
import static org.mockito.Mockito.anyInt;
import static org.mockito.Mockito.anyShort;
import static org.mockito.Mockito.eq;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.eq;
import static org.mockito.Mockito.verify;
/**
@ -405,6 +404,7 @@ public class DataCompressionHttp2Test {
.codec(decoder, clientEncoder).build();
p.addLast(clientHandler);
p.addLast(new ChannelHandler() {
@Override
public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
if (evt == Http2ConnectionPrefaceAndSettingsFrameWrittenEvent.INSTANCE) {
prefaceWrittenLatch.countDown();
@ -415,12 +415,10 @@ public class DataCompressionHttp2Test {
}
});
serverChannel = sb.bind(new InetSocketAddress(0)).sync().channel();
serverChannel = sb.bind(new InetSocketAddress(0)).get();
int port = ((InetSocketAddress) serverChannel.localAddress()).getPort();
ChannelFuture ccf = cb.connect(new InetSocketAddress(NetUtil.LOCALHOST, port));
assertTrue(ccf.awaitUninterruptibly().isSuccess());
clientChannel = ccf.channel();
clientChannel = cb.connect(new InetSocketAddress(NetUtil.LOCALHOST, port)).get();
assertTrue(prefaceWrittenLatch.await(5, SECONDS));
assertTrue(serverChannelLatch.await(5, SECONDS));
}

View File

@ -18,18 +18,22 @@ package io.netty.handler.codec.http2;
import io.netty.bootstrap.Bootstrap;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandlerContext;
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.channel.socket.nio.NioSocketChannel;
import io.netty.util.CharsetUtil;
import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.GenericFutureListener;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
@ -41,14 +45,14 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
public class DefaultHttp2PushPromiseFrameTest {
private final EventLoopGroup eventLoopGroup = new NioEventLoopGroup(2);
private final EventLoopGroup eventLoopGroup = new MultithreadEventLoopGroup(2, NioHandler.newFactory());
private final ClientHandler clientHandler = new ClientHandler();
private final Map<Integer, String> contentMap = new ConcurrentHashMap<Integer, String>();
private ChannelFuture connectionFuture;
private Future<Channel> connectionFuture;
@BeforeEach
public void setup() throws InterruptedException {
public void setup() throws Exception {
ServerBootstrap serverBootstrap = new ServerBootstrap()
.group(eventLoopGroup)
.channel(NioServerSocketChannel.class)
@ -67,7 +71,7 @@ public class DefaultHttp2PushPromiseFrameTest {
}
});
ChannelFuture channelFuture = serverBootstrap.bind(0).sync();
Channel channel = serverBootstrap.bind(0).get();
final Bootstrap bootstrap = new Bootstrap()
.group(eventLoopGroup)
@ -88,16 +92,13 @@ public class DefaultHttp2PushPromiseFrameTest {
}
});
connectionFuture = bootstrap.connect(channelFuture.channel().localAddress());
connectionFuture = bootstrap.connect(channel.localAddress());
}
@Test
public void send() {
connectionFuture.addListener(new ChannelFutureListener() {
@Override
public void operationComplete(ChannelFuture future) {
clientHandler.write();
}
connectionFuture.addListener((GenericFutureListener<Future<Channel>>) future -> {
clientHandler.write();
});
}

View File

@ -321,7 +321,7 @@ public class Http2ConnectionRoundtripTest {
final CountDownLatch serverSettingsAckLatch2 = new CountDownLatch(2);
final CountDownLatch serverDataLatch = new CountDownLatch(1);
final CountDownLatch clientWriteDataLatch = new CountDownLatch(1);
final byte[] data = new byte[] {1, 2, 3, 4, 5};
final byte[] data = {1, 2, 3, 4, 5};
final ByteArrayOutputStream out = new ByteArrayOutputStream(data.length);
doAnswer((Answer<Void>) invocationOnMock -> {
@ -868,7 +868,7 @@ public class Http2ConnectionRoundtripTest {
final CountDownLatch probeStreamCount = new CountDownLatch(1);
final AtomicBoolean stream3Exists = new AtomicBoolean();
final AtomicInteger streamCount = new AtomicInteger();
runInChannel(this.clientChannel, () -> {
runInChannel(clientChannel, () -> {
stream3Exists.set(http2Client.connection().stream(3) != null);
streamCount.set(http2Client.connection().numActiveStreams());
probeStreamCount.countDown();
@ -1089,11 +1089,9 @@ public class Http2ConnectionRoundtripTest {
}
});
serverChannel = sb.bind(new LocalAddress("Http2ConnectionRoundtripTest")).sync().channel();
serverChannel = sb.bind(new LocalAddress("Http2ConnectionRoundtripTest")).get();
ChannelFuture ccf = cb.connect(serverChannel.localAddress());
assertTrue(ccf.awaitUninterruptibly().isSuccess());
clientChannel = ccf.channel();
clientChannel = cb.connect(serverChannel.localAddress()).get();
assertTrue(prefaceWrittenLatch.await(DEFAULT_AWAIT_TIMEOUT_SECONDS, SECONDS));
http2Client = clientChannel.pipeline().get(Http2ConnectionHandler.class);
assertTrue(serverInitLatch.await(DEFAULT_AWAIT_TIMEOUT_SECONDS, SECONDS));
@ -1126,8 +1124,7 @@ public class Http2ConnectionRoundtripTest {
doAnswer((Answer<Integer>) invocation -> {
ByteBuf buf = (ByteBuf) invocation.getArguments()[2];
int padding = (Integer) invocation.getArguments()[3];
int processedBytes = buf.readableBytes() + padding;
return processedBytes;
return buf.readableBytes() + padding;
}).when(listener).onDataRead(any(ChannelHandlerContext.class), anyInt(),
any(ByteBuf.class), anyInt(), anyBoolean());
}

View File

@ -44,8 +44,8 @@ import java.util.concurrent.CountDownLatch;
import static io.netty.handler.codec.http2.Http2CodecUtil.isStreamIdValid;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
@ -67,7 +67,7 @@ public class Http2MultiplexCodecBuilderTest {
}
@BeforeEach
public void setUp() throws InterruptedException {
public void setUp() throws Exception {
final CountDownLatch serverChannelLatch = new CountDownLatch(1);
LocalAddress serverAddress = new LocalAddress(getClass().getName());
serverLastInboundHandler = new SharableLastInboundHandler();
@ -109,7 +109,7 @@ public class Http2MultiplexCodecBuilderTest {
serverChannelLatch.countDown();
}
});
serverChannel = sb.bind(serverAddress).sync().channel();
serverChannel = sb.bind(serverAddress).get();
Bootstrap cb = new Bootstrap()
.channel(LocalChannel.class)
@ -120,7 +120,7 @@ public class Http2MultiplexCodecBuilderTest {
fail("Should not be called for outbound streams");
}
}).build());
clientChannel = cb.connect(serverAddress).sync().channel();
clientChannel = cb.connect(serverAddress).get();
assertTrue(serverChannelLatch.await(5, SECONDS));
}

View File

@ -25,7 +25,8 @@ import io.netty.channel.ChannelHandlerAdapter;
import io.netty.channel.ChannelHandlerContext;
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.ApplicationProtocolConfig;
@ -45,7 +46,6 @@ import io.netty.util.NetUtil;
import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.FutureListener;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
@ -61,7 +61,6 @@ import java.security.cert.X509Certificate;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
@ -94,7 +93,7 @@ public class Http2MultiplexTransportTest {
@BeforeEach
public void setup() {
eventLoopGroup = new NioEventLoopGroup();
eventLoopGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
}
@AfterEach
@ -113,19 +112,19 @@ public class Http2MultiplexTransportTest {
@Test
@Timeout(value = 10000, unit = MILLISECONDS)
public void asyncSettingsAckWithMultiplexCodec() throws InterruptedException {
public void asyncSettingsAckWithMultiplexCodec() throws Exception {
asyncSettingsAck0(new Http2MultiplexCodecBuilder(true, DISCARD_HANDLER).build(), null);
}
@Test
@Timeout(value = 10000, unit = MILLISECONDS)
public void asyncSettingsAckWithMultiplexHandler() throws InterruptedException {
public void asyncSettingsAckWithMultiplexHandler() throws Exception {
asyncSettingsAck0(new Http2FrameCodecBuilder(true).build(),
new Http2MultiplexHandler(DISCARD_HANDLER));
}
private void asyncSettingsAck0(final Http2FrameCodec codec, final ChannelHandler multiplexer)
throws InterruptedException {
throws Exception {
// The client expects 2 settings frames. One from the connection setup and one from this test.
final CountDownLatch serverAckOneLatch = new CountDownLatch(1);
final CountDownLatch serverAckAllLatch = new CountDownLatch(2);
@ -160,7 +159,7 @@ public class Http2MultiplexTransportTest {
});
}
});
serverChannel = sb.bind(new InetSocketAddress(NetUtil.LOCALHOST, 0)).awaitUninterruptibly().channel();
serverChannel = sb.bind(new InetSocketAddress(NetUtil.LOCALHOST, 0)).get();
Bootstrap bs = new Bootstrap();
bs.group(eventLoopGroup);
@ -181,7 +180,7 @@ public class Http2MultiplexTransportTest {
});
}
});
clientChannel = bs.connect(serverChannel.localAddress()).awaitUninterruptibly().channel();
clientChannel = bs.connect(serverChannel.localAddress()).get();
serverConnectedChannelLatch.await();
serverConnectedChannel = serverConnectedChannelRef.get();
@ -203,8 +202,7 @@ public class Http2MultiplexTransportTest {
@Test
@Timeout(value = 5000L, unit = MILLISECONDS)
public void testFlushNotDiscarded()
throws InterruptedException {
public void testFlushNotDiscarded() throws Exception {
final ScheduledExecutorService executorService = Executors.newScheduledThreadPool(1);
try {
@ -228,14 +226,14 @@ public class Http2MultiplexTransportTest {
CharsetUtil.US_ASCII), true));
ctx.channel().eventLoop().execute(ctx::flush);
});
}, 500, TimeUnit.MILLISECONDS);
}, 500, MILLISECONDS);
}
ReferenceCountUtil.release(msg);
}
}));
}
});
serverChannel = sb.bind(new InetSocketAddress(NetUtil.LOCALHOST, 0)).syncUninterruptibly().channel();
serverChannel = sb.bind(new InetSocketAddress(NetUtil.LOCALHOST, 0)).get();
final CountDownLatch latch = new CountDownLatch(1);
Bootstrap bs = new Bootstrap();
@ -248,7 +246,7 @@ public class Http2MultiplexTransportTest {
ch.pipeline().addLast(new Http2MultiplexHandler(DISCARD_HANDLER));
}
});
clientChannel = bs.connect(serverChannel.localAddress()).syncUninterruptibly().channel();
clientChannel = bs.connect(serverChannel.localAddress()).get();
Http2StreamChannelBootstrap h2Bootstrap = new Http2StreamChannelBootstrap(clientChannel);
h2Bootstrap.handler(new ChannelHandler() {
@Override
@ -347,7 +345,7 @@ public class Http2MultiplexTransportTest {
ch.pipeline().addLast(new Http2MultiplexHandler(DISCARD_HANDLER));
}
});
serverChannel = sb.bind(new InetSocketAddress(NetUtil.LOCALHOST, 0)).syncUninterruptibly().channel();
serverChannel = sb.bind(new InetSocketAddress(NetUtil.LOCALHOST, 0)).get();
final SslContext clientCtx = SslContextBuilder.forClient()
.keyManager(ssc.key(), ssc.cert())
@ -430,7 +428,7 @@ public class Http2MultiplexTransportTest {
});
}
});
clientChannel = bs.connect(serverChannel.localAddress()).syncUninterruptibly().channel();
clientChannel = bs.connect(serverChannel.localAddress()).get();
latch.await();
AssertionError error = errorRef.get();
if (error != null) {
@ -504,7 +502,7 @@ public class Http2MultiplexTransportTest {
});
}
});
serverChannel = sb.bind(new InetSocketAddress(NetUtil.LOCALHOST, 0)).sync().channel();
serverChannel = sb.bind(new InetSocketAddress(NetUtil.LOCALHOST, 0)).get();
final SslContext clientCtx = SslContextBuilder.forClient()
.sslProvider(provider)
@ -561,7 +559,7 @@ public class Http2MultiplexTransportTest {
});
}
});
clientChannel = bs.connect(serverChannel.localAddress()).sync().channel();
clientChannel = bs.connect(serverChannel.localAddress()).get();
latch.await();
} finally {

View File

@ -44,8 +44,8 @@ import static io.netty.handler.codec.http2.Http2FrameCodecBuilder.forClient;
import static io.netty.handler.codec.http2.Http2FrameCodecBuilder.forServer;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.mock;
@ -78,7 +78,7 @@ public class Http2StreamChannelBootstrapTest {
serverChannelLatch.countDown();
}
});
serverChannel = sb.bind(serverAddress).sync().channel();
serverChannel = sb.bind(serverAddress).get();
Bootstrap cb = new Bootstrap()
.channel(LocalChannel.class)
@ -89,7 +89,7 @@ public class Http2StreamChannelBootstrapTest {
ch.pipeline().addLast(forClient().build(), newMultiplexedHandler());
}
});
clientChannel = cb.connect(serverAddress).sync().channel();
clientChannel = cb.connect(serverAddress).get();
assertTrue(serverChannelLatch.await(3, SECONDS));
Http2StreamChannelBootstrap bootstrap = new Http2StreamChannelBootstrap(clientChannel);
@ -145,7 +145,7 @@ public class Http2StreamChannelBootstrapTest {
when(ctx.executor()).thenReturn(executor);
when(ctx.handler()).thenReturn(handler);
Promise<Http2StreamChannel> promise = new DefaultPromise(mock(EventExecutor.class));
Promise<Http2StreamChannel> promise = new DefaultPromise<>(mock(EventExecutor.class));
bootstrap.open0(ctx, promise);
assertThat(promise.isDone(), is(true));
assertThat(promise.cause(), is(instanceOf(IllegalStateException.class)));

View File

@ -43,13 +43,11 @@ import io.netty.handler.codec.http.LastHttpContent;
import io.netty.handler.codec.http2.Http2TestUtil.FrameCountDown;
import io.netty.util.AsciiString;
import io.netty.util.concurrent.Future;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.util.ArrayList;
@ -74,8 +72,8 @@ import static org.mockito.Mockito.any;
import static org.mockito.Mockito.anyBoolean;
import static org.mockito.Mockito.anyInt;
import static org.mockito.Mockito.anyShort;
import static org.mockito.Mockito.eq;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.eq;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
@ -107,7 +105,7 @@ public class HttpToHttp2ConnectionHandlerTest {
}
@AfterEach
public void teardown() throws Exception {
public void tearDown() throws Exception {
if (clientChannel != null) {
clientChannel.close().syncUninterruptibly();
clientChannel = null;
@ -593,11 +591,9 @@ public class HttpToHttp2ConnectionHandlerTest {
}
});
serverChannel = sb.bind(new LocalAddress("HttpToHttp2ConnectionHandlerTest")).sync().channel();
serverChannel = sb.bind(new LocalAddress("HttpToHttp2ConnectionHandlerTest")).get();
ChannelFuture ccf = cb.connect(serverChannel.localAddress());
assertTrue(ccf.awaitUninterruptibly().isSuccess());
clientChannel = ccf.channel();
clientChannel = cb.connect(serverChannel.localAddress()).get();
assertTrue(prefaceWrittenLatch.await(5, SECONDS));
assertTrue(serverChannelLatch.await(WAIT_TIME_SECONDS, SECONDS));
}

View File

@ -19,7 +19,6 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
@ -107,7 +106,7 @@ public class InboundHttp2ToHttpAdapterTest {
}
@AfterEach
public void teardown() throws Exception {
public void tearDown() throws Exception {
cleanupCapturedRequests();
cleanupCapturedResponses();
if (clientChannel != null) {
@ -596,13 +595,12 @@ public class InboundHttp2ToHttpAdapterTest {
assertEquals(settings, settingsCaptor.getValue());
}
private void boostrapEnv(int clientLatchCount, int serverLatchCount, int settingsLatchCount)
throws InterruptedException {
private void boostrapEnv(int clientLatchCount, int serverLatchCount, int settingsLatchCount) throws Exception {
boostrapEnv(clientLatchCount, clientLatchCount, serverLatchCount, serverLatchCount, settingsLatchCount);
}
private void boostrapEnv(int clientLatchCount, int clientLatchCount2, int serverLatchCount, int serverLatchCount2,
int settingsLatchCount) throws InterruptedException {
int settingsLatchCount) throws Exception {
final CountDownLatch prefaceWrittenLatch = new CountDownLatch(1);
clientDelegator = null;
serverDelegator = null;
@ -678,6 +676,7 @@ public class InboundHttp2ToHttpAdapterTest {
}
});
p.addLast(new ChannelHandler() {
@Override
public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
if (evt == Http2ConnectionPrefaceAndSettingsFrameWrittenEvent.INSTANCE) {
prefaceWrittenLatch.countDown();
@ -688,11 +687,9 @@ public class InboundHttp2ToHttpAdapterTest {
}
});
serverChannel = sb.bind(new LocalAddress("InboundHttp2ToHttpAdapterTest")).sync().channel();
serverChannel = sb.bind(new LocalAddress("InboundHttp2ToHttpAdapterTest")).get();
ChannelFuture ccf = cb.connect(serverChannel.localAddress());
assertTrue(ccf.awaitUninterruptibly().isSuccess());
clientChannel = ccf.channel();
clientChannel = cb.connect(serverChannel.localAddress()).get();
assertTrue(prefaceWrittenLatch.await(5, SECONDS));
assertTrue(serverChannelLatch.await(5, SECONDS));
}

View File

@ -22,7 +22,6 @@ import io.netty.buffer.ByteBufAllocator;
import io.netty.buffer.ByteBufInputStream;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
@ -33,7 +32,6 @@ 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;
import java.util.concurrent.TimeUnit;
import net.jpountz.lz4.LZ4BlockInputStream;
import net.jpountz.lz4.LZ4Factory;
import net.jpountz.xxhash.XXHashFactory;
@ -46,6 +44,7 @@ import org.mockito.MockitoAnnotations;
import java.io.InputStream;
import java.net.InetSocketAddress;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.zip.Checksum;
@ -244,7 +243,7 @@ public class Lz4FrameEncoderTest extends AbstractEncoderTest {
@Test
@Timeout(value = 3000, unit = TimeUnit.MILLISECONDS)
public void writingAfterClosedChannelDoesNotNPE() throws InterruptedException {
public void writingAfterClosedChannelDoesNotNPE() throws Exception {
EventLoopGroup group = new MultithreadEventLoopGroup(2, NioHandler.newFactory());
Channel serverChannel = null;
Channel clientChannel = null;
@ -270,8 +269,8 @@ public class Lz4FrameEncoderTest extends AbstractEncoderTest {
}
});
serverChannel = sb.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
clientChannel = bs.connect(serverChannel.localAddress()).syncUninterruptibly().channel();
serverChannel = sb.bind(new InetSocketAddress(0)).get();
clientChannel = bs.connect(serverChannel.localAddress()).get();
final Channel finalClientChannel = clientChannel;
clientChannel.eventLoop().execute(() -> {

View File

@ -19,8 +19,8 @@ import io.netty.util.internal.PromiseNotificationUtil;
import io.netty.util.internal.logging.InternalLogger;
import io.netty.util.internal.logging.InternalLoggerFactory;
import static java.util.Objects.requireNonNull;
import static io.netty.util.internal.ObjectUtil.checkNotNullWithIAE;
import static java.util.Objects.requireNonNull;
/**
* {@link GenericFutureListener} implementation which takes other {@link Promise}s
@ -64,7 +64,7 @@ public class PromiseNotifier<V, F extends Future<V>> implements GenericFutureLis
/**
* Link the {@link Future} and {@link Promise} such that if the {@link Future} completes the {@link Promise}
* will be notified. Cancellation is propagated both ways such that if the {@link Future} is cancelled
* the {@link Promise} is cancelled and vise-versa.
* the {@link Promise} is cancelled and vice-versa.
*
* @param future the {@link Future} which will be used to listen to for notifying the {@link Promise}.
* @param promise the {@link Promise} which will be notified
@ -79,7 +79,7 @@ public class PromiseNotifier<V, F extends Future<V>> implements GenericFutureLis
/**
* Link the {@link Future} and {@link Promise} such that if the {@link Future} completes the {@link Promise}
* will be notified. Cancellation is propagated both ways such that if the {@link Future} is cancelled
* the {@link Promise} is cancelled and vise-versa.
* the {@link Promise} is cancelled and vice-versa.
*
* @param logNotifyFailure {@code true} if logging should be done in case notification fails.
* @param future the {@link Future} which will be used to listen to for notifying the {@link Promise}.
@ -88,20 +88,19 @@ public class PromiseNotifier<V, F extends Future<V>> implements GenericFutureLis
* @param <F> the type of the {@link Future}
* @return the passed in {@link Future}
*/
@SuppressWarnings({"unchecked", "rawtypes"})
public static <V, F extends Future<V>> F cascade(boolean logNotifyFailure, final F future,
final Promise<? super V> promise) {
promise.addListener(new FutureListener() {
promise.addListener(new FutureListener<Object>() {
@Override
public void operationComplete(Future f) {
public void operationComplete(Future<Object> f) {
if (f.isCancelled()) {
future.cancel(false);
}
}
});
future.addListener(new PromiseNotifier(logNotifyFailure, promise) {
future.addListener(new PromiseNotifier<V, F>(logNotifyFailure, promise) {
@Override
public void operationComplete(Future f) throws Exception {
public void operationComplete(F f) throws Exception {
if (promise.isCancelled() && f.isCancelled()) {
// Just return if we propagate a cancel from the promise to the future and both are notified already
return;
@ -112,6 +111,52 @@ public class PromiseNotifier<V, F extends Future<V>> implements GenericFutureLis
return future;
}
/**
* Link the {@link Future} and {@link Promise} such that if the {@link Future} completes the {@link Promise} will be
* notified with the given result.
* Cancellation is propagated both ways such that if the {@link Future} is cancelled the {@link Promise}
* is cancelled and vice-versa.
*
* @param logNotifyFailure {@code true} if logging should be done in case notification fails.
* @param future the {@link Future} which will be used to listen to for notifying the {@link Promise}.
* @param promise the {@link Promise} which will be notified
* @param successResult the result that will be propagated to the promise on success
* @return the passed in {@link Future}
*/
public static <R, F extends Future<Void>> F cascade(boolean logNotifyFailure, F future,
Promise<R> promise, R successResult) {
promise.addListener(new FutureListener<Object>() {
@Override
public void operationComplete(Future<Object> f) {
if (f.isCancelled()) {
future.cancel(false);
}
}
});
future.addListener(new GenericFutureListener<F>() {
@Override
public void operationComplete(F f) throws Exception {
if (promise.isCancelled() && f.isCancelled()) {
// Just return if we propagate a cancel from the promise to the future and both are notified already
return;
}
if (f.isSuccess()) {
promise.setSuccess(successResult);
} else if (f.isCancelled()) {
InternalLogger internalLogger = null;
if (logNotifyFailure) {
internalLogger = InternalLoggerFactory.getInstance(PromiseNotifier.class);
}
PromiseNotificationUtil.tryCancel(promise, internalLogger);
} else {
Throwable cause = future.cause();
promise.tryFailure(cause);
}
}
});
return future;
}
@Override
public void operationComplete(F future) throws Exception {
InternalLogger internalLogger = logNotifyFailure ? logger : null;

View File

@ -16,7 +16,7 @@
package io.netty.example.discard;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
@ -65,10 +65,10 @@ public final class DiscardClient {
});
// Make the connection attempt.
ChannelFuture f = b.connect(HOST, PORT).sync();
Channel channel = b.connect(HOST, PORT).get();
// Wait until the connection is closed.
f.channel().closeFuture().sync();
channel.closeFuture().sync();
} finally {
group.shutdownGracefully();
}

View File

@ -16,6 +16,7 @@
package io.netty.example.discard;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
@ -67,12 +68,12 @@ public final class DiscardServer {
});
// Bind and start to accept incoming connections.
ChannelFuture f = b.bind(PORT).sync();
Channel channel = b.bind(PORT).get();
// Wait until the server socket is closed.
// In this example, this does not happen, but you can do that to gracefully
// shut down your server.
f.channel().closeFuture().sync();
channel.closeFuture().sync();
} finally {
workerGroup.shutdownGracefully();
bossGroup.shutdownGracefully();

View File

@ -17,27 +17,26 @@ package io.netty.example.dns.dot;
import io.netty.bootstrap.Bootstrap;
import io.netty.buffer.ByteBufUtil;
import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelHandlerContext;
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.SimpleChannelInboundHandler;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.handler.codec.dns.DefaultDnsQuery;
import io.netty.handler.codec.dns.DefaultDnsQuestion;
import io.netty.handler.codec.dns.DefaultDnsResponse;
import io.netty.handler.codec.dns.DnsQuestion;
import io.netty.handler.codec.dns.DnsQuery;
import io.netty.handler.codec.dns.DefaultDnsQuery;
import io.netty.handler.codec.dns.DnsOpCode;
import io.netty.handler.codec.dns.DnsRecord;
import io.netty.handler.codec.dns.DnsSection;
import io.netty.handler.codec.dns.DnsRecordType;
import io.netty.handler.codec.dns.DnsQuery;
import io.netty.handler.codec.dns.DnsQuestion;
import io.netty.handler.codec.dns.DnsRawRecord;
import io.netty.handler.codec.dns.DnsRecord;
import io.netty.handler.codec.dns.DnsRecordType;
import io.netty.handler.codec.dns.DnsSection;
import io.netty.handler.codec.dns.TcpDnsQueryEncoder;
import io.netty.handler.codec.dns.TcpDnsResponseDecoder;
import io.netty.handler.ssl.SslContext;
@ -63,7 +62,7 @@ public final class DoTClient {
for (int i = 0, count = msg.count(DnsSection.ANSWER); i < count; i++) {
DnsRecord record = msg.recordAt(DnsSection.ANSWER, i);
if (record.type() == DnsRecordType.A) {
//just print the IP after query
// Just print the IP after query
DnsRawRecord raw = (DnsRawRecord) record;
System.out.println(NetUtil.bytesToIpAddress(ByteBufUtil.getBytes(raw.content())));
}
@ -71,7 +70,7 @@ public final class DoTClient {
}
public static void main(String[] args) throws Exception {
EventLoopGroup group = new NioEventLoopGroup();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
final SslContext sslContext = SslContextBuilder.forClient()
.protocols("TLSv1.3", "TLSv1.2")
@ -100,7 +99,7 @@ public final class DoTClient {
});
}
});
final Channel ch = b.connect(DNS_SERVER_HOST, DNS_SERVER_PORT).sync().channel();
final Channel ch = b.connect(DNS_SERVER_HOST, DNS_SERVER_PORT).get();
int randomID = new Random().nextInt(60000 - 1000) + 1000;
DnsQuery query = new DefaultDnsQuery(randomID, DnsOpCode.QUERY)

View File

@ -17,27 +17,26 @@ package io.netty.example.dns.tcp;
import io.netty.bootstrap.Bootstrap;
import io.netty.buffer.ByteBufUtil;
import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelHandlerContext;
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.SimpleChannelInboundHandler;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.handler.codec.dns.DefaultDnsQuery;
import io.netty.handler.codec.dns.DefaultDnsQuestion;
import io.netty.handler.codec.dns.DefaultDnsResponse;
import io.netty.handler.codec.dns.DnsQuestion;
import io.netty.handler.codec.dns.DnsQuery;
import io.netty.handler.codec.dns.DefaultDnsQuery;
import io.netty.handler.codec.dns.DnsOpCode;
import io.netty.handler.codec.dns.DnsRecord;
import io.netty.handler.codec.dns.DnsSection;
import io.netty.handler.codec.dns.DnsRecordType;
import io.netty.handler.codec.dns.DnsQuery;
import io.netty.handler.codec.dns.DnsQuestion;
import io.netty.handler.codec.dns.DnsRawRecord;
import io.netty.handler.codec.dns.DnsRecord;
import io.netty.handler.codec.dns.DnsRecordType;
import io.netty.handler.codec.dns.DnsSection;
import io.netty.handler.codec.dns.TcpDnsQueryEncoder;
import io.netty.handler.codec.dns.TcpDnsResponseDecoder;
import io.netty.util.NetUtil;
@ -61,7 +60,7 @@ public final class TcpDnsClient {
for (int i = 0, count = msg.count(DnsSection.ANSWER); i < count; i++) {
DnsRecord record = msg.recordAt(DnsSection.ANSWER, i);
if (record.type() == DnsRecordType.A) {
//just print the IP after query
// Just print the IP after query
DnsRawRecord raw = (DnsRawRecord) record;
System.out.println(NetUtil.bytesToIpAddress(ByteBufUtil.getBytes(raw.content())));
}
@ -69,7 +68,7 @@ public final class TcpDnsClient {
}
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)
@ -94,7 +93,7 @@ public final class TcpDnsClient {
}
});
final Channel ch = b.connect(DNS_SERVER_HOST, DNS_SERVER_PORT).sync().channel();
final Channel ch = b.connect(DNS_SERVER_HOST, DNS_SERVER_PORT).get();
int randomID = new Random().nextInt(60000 - 1000) + 1000;
DnsQuery query = new DefaultDnsQuery(randomID, DnsOpCode.QUERY)

View File

@ -22,8 +22,10 @@ import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.IoHandlerFactory;
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;
@ -54,11 +56,13 @@ public final class TcpDnsServer {
private static final String QUERY_DOMAIN = "www.example.com";
private static final int DNS_SERVER_PORT = 53;
private static final String DNS_SERVER_HOST = "127.0.0.1";
private static final byte[] QUERY_RESULT = new byte[]{(byte) 192, (byte) 168, 1, 1};
private static final byte[] QUERY_RESULT = {(byte) 192, (byte) 168, 1, 1};
public static void main(String[] args) throws Exception {
ServerBootstrap bootstrap = new ServerBootstrap().group(new NioEventLoopGroup(1),
new NioEventLoopGroup())
IoHandlerFactory ioHandlerFactory = NioHandler.newFactory();
ServerBootstrap bootstrap = new ServerBootstrap()
.group(new MultithreadEventLoopGroup(1, ioHandlerFactory),
new MultithreadEventLoopGroup(ioHandlerFactory))
.channel(NioServerSocketChannel.class)
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new ChannelInitializer<Channel>() {
@ -93,7 +97,7 @@ public final class TcpDnsServer {
});
}
});
final Channel channel = bootstrap.bind(DNS_SERVER_PORT).channel();
final Channel channel = bootstrap.bind(DNS_SERVER_PORT).get();
Executors.newSingleThreadScheduledExecutor().schedule(new Runnable() {
@Override
public void run() {
@ -110,7 +114,7 @@ public final class TcpDnsServer {
// copy from TcpDnsClient.java
private static void clientQuery() throws Exception {
NioEventLoopGroup group = new NioEventLoopGroup();
MultithreadEventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(group)
@ -134,7 +138,7 @@ public final class TcpDnsServer {
}
});
final Channel ch = b.connect(DNS_SERVER_HOST, DNS_SERVER_PORT).sync().channel();
final Channel ch = b.connect(DNS_SERVER_HOST, DNS_SERVER_PORT).get();
int randomID = new Random().nextInt(60000 - 1000) + 1000;
DnsQuery query = new DefaultDnsQuery(randomID, DnsOpCode.QUERY)

View File

@ -15,9 +15,6 @@
*/
package io.netty.example.dns.udp;
import java.net.InetSocketAddress;
import java.util.concurrent.TimeUnit;
import io.netty.bootstrap.Bootstrap;
import io.netty.buffer.ByteBufUtil;
import io.netty.channel.Channel;
@ -43,6 +40,9 @@ import io.netty.handler.codec.dns.DnsRecordType;
import io.netty.handler.codec.dns.DnsSection;
import io.netty.util.NetUtil;
import java.net.InetSocketAddress;
import java.util.concurrent.TimeUnit;
public final class DnsClient {
private static final String QUERY_DOMAIN = "www.example.com";
@ -59,7 +59,7 @@ public final class DnsClient {
for (int i = 0, count = msg.count(DnsSection.ANSWER); i < count; i++) {
DnsRecord record = msg.recordAt(DnsSection.ANSWER, i);
if (record.type() == DnsRecordType.A) {
//just print the IP after query
// Just print the IP after query
DnsRawRecord raw = (DnsRawRecord) record;
System.out.println(NetUtil.bytesToIpAddress(ByteBufUtil.getBytes(raw.content())));
}
@ -91,13 +91,13 @@ public final class DnsClient {
});
}
});
final Channel ch = b.bind(0).sync().channel();
final Channel ch = b.bind(0).get();
DnsQuery query = new DatagramDnsQuery(null, addr, 1).setRecord(
DnsSection.QUESTION,
new DefaultDnsQuestion(QUERY_DOMAIN, DnsRecordType.A));
ch.writeAndFlush(query).sync();
boolean succ = ch.closeFuture().await(10, TimeUnit.SECONDS);
if (!succ) {
boolean success = ch.closeFuture().await(10, TimeUnit.SECONDS);
if (!success) {
System.err.println("dns query timeout!");
ch.close().sync();
}

View File

@ -16,7 +16,7 @@
package io.netty.example.echo;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline;
@ -72,10 +72,10 @@ public final class EchoClient {
});
// Start the client.
ChannelFuture f = b.connect(HOST, PORT).sync();
Channel channel = b.connect(HOST, PORT).get();
// Wait until the connection is closed.
f.channel().closeFuture().sync();
channel.closeFuture().sync();
} finally {
// Shut down the event loop to terminate all threads.
group.shutdownGracefully();

View File

@ -16,7 +16,7 @@
package io.netty.example.echo;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline;
@ -72,10 +72,10 @@ public final class EchoServer {
});
// Start the server.
ChannelFuture f = b.bind(PORT).sync();
Channel channel = b.bind(PORT).get();
// Wait until the server socket is closed.
f.channel().closeFuture().sync();
channel.closeFuture().sync();
} finally {
// Shut down all event loops to terminate all threads.
bossGroup.shutdownGracefully();

View File

@ -16,7 +16,7 @@
package io.netty.example.factorial;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
@ -54,11 +54,10 @@ public final class FactorialClient {
.handler(new FactorialClientInitializer(sslCtx));
// Make a new connection.
ChannelFuture f = b.connect(HOST, PORT).sync();
Channel channel = b.connect(HOST, PORT).get();
// Get the handler instance to retrieve the answer.
FactorialClientHandler handler =
(FactorialClientHandler) f.channel().pipeline().last();
FactorialClientHandler handler = (FactorialClientHandler) channel.pipeline().last();
// Print out the answer.
System.err.format("Factorial of %,d is: %,d", COUNT, handler.getFactorial());

View File

@ -54,7 +54,7 @@ public final class FactorialServer {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new FactorialServerInitializer(sslCtx));
b.bind(PORT).sync().channel().closeFuture().sync();
b.bind(PORT).get().closeFuture().sync();
} finally {
bossGroup.shutdownGracefully();
workerGroup.shutdownGracefully();

View File

@ -16,7 +16,7 @@
package io.netty.example.file;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline;
@ -81,10 +81,10 @@ public final class FileServer {
});
// Start the server.
ChannelFuture f = b.bind(PORT).sync();
Channel channel = b.bind(PORT).get();
// Wait until the server socket is closed.
f.channel().closeFuture().sync();
channel.closeFuture().sync();
} finally {
// Shut down all event loops to terminate all threads.
bossGroup.shutdownGracefully();

View File

@ -20,7 +20,8 @@ import io.netty.bootstrap.Bootstrap;
import io.netty.buffer.Unpooled;
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.haproxy.HAProxyCommand;
import io.netty.handler.codec.haproxy.HAProxyMessage;
@ -28,14 +29,14 @@ import io.netty.handler.codec.haproxy.HAProxyProtocolVersion;
import io.netty.handler.codec.haproxy.HAProxyProxiedProtocol;
import io.netty.util.CharsetUtil;
import static io.netty.example.haproxy.HAProxyServer.*;
import static io.netty.example.haproxy.HAProxyServer.PORT;
public final class HAProxyClient {
private static final String HOST = System.getProperty("host", "127.0.0.1");
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)
@ -43,7 +44,7 @@ public final class HAProxyClient {
.handler(new HAProxyHandler());
// Start the connection attempt.
Channel ch = b.connect(HOST, PORT).sync().channel();
Channel ch = b.connect(HOST, PORT).get();
HAProxyMessage message = new HAProxyMessage(
HAProxyProtocolVersion.V2, HAProxyCommand.PROXY, HAProxyProxiedProtocol.TCP4,

View File

@ -45,7 +45,7 @@ public final class HAProxyServer {
.channel(NioServerSocketChannel.class)
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new HAProxyServerInitializer());
b.bind(PORT).sync().channel().closeFuture().sync();
b.bind(PORT).get().closeFuture().sync();
} finally {
bossGroup.shutdownGracefully();
workerGroup.shutdownGracefully();

View File

@ -96,7 +96,7 @@ public final class HttpCorsServer {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new HttpCorsServerInitializer(sslCtx));
b.bind(PORT).sync().channel().closeFuture().sync();
b.bind(PORT).get().closeFuture().sync();
} finally {
bossGroup.shutdownGracefully();
workerGroup.shutdownGracefully();

View File

@ -53,7 +53,7 @@ public final class HttpStaticFileServer {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new HttpStaticFileServerInitializer(sslCtx));
Channel ch = b.bind(PORT).sync().channel();
Channel ch = b.bind(PORT).get();
System.err.println("Open your web browser and navigate to " +
(SSL? "https" : "http") + "://127.0.0.1:" + PORT + '/');

View File

@ -58,7 +58,7 @@ public final class HttpHelloWorldServer {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new HttpHelloWorldServerInitializer(sslCtx));
Channel ch = b.bind(PORT).sync().channel();
Channel ch = b.bind(PORT).get();
System.err.println("Open your web browser and navigate to " +
(SSL? "https" : "http") + "://127.0.0.1:" + PORT + '/');

View File

@ -81,7 +81,7 @@ public final class HttpSnoopClient {
.handler(new HttpSnoopClientInitializer(sslCtx));
// Make the connection attempt.
Channel ch = b.connect(host, port).sync().channel();
Channel ch = b.connect(host, port).get();
// Prepare the HTTP request.
HttpRequest request = new DefaultFullHttpRequest(

View File

@ -56,7 +56,7 @@ public final class HttpSnoopServer {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new HttpSnoopServerInitializer(sslCtx));
Channel ch = b.bind(PORT).sync().channel();
Channel ch = b.bind(PORT).get();
System.err.println("Open your web browser and navigate to " +
(SSL? "https" : "http") + "://127.0.0.1:" + PORT + '/');

View File

@ -17,7 +17,6 @@ package io.netty.example.http.upload;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.nio.NioHandler;
@ -41,6 +40,7 @@ import io.netty.handler.codec.http.multipart.InterfaceHttpData;
import io.netty.handler.ssl.SslContext;
import io.netty.handler.ssl.SslContextBuilder;
import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
import io.netty.util.concurrent.Future;
import io.netty.util.internal.SocketUtils;
import java.io.File;
@ -151,7 +151,7 @@ public final class HttpUploadClient {
Bootstrap bootstrap, String host, int port, String get, URI uriSimple) throws Exception {
// XXX /formget
// No use of HttpPostRequestEncoder since not a POST
Channel channel = bootstrap.connect(host, port).sync().channel();
Channel channel = bootstrap.connect(host, port).get();
// Prepare the HTTP request.
QueryStringEncoder encoder = new QueryStringEncoder(get);
@ -208,9 +208,9 @@ public final class HttpUploadClient {
List<Entry<String, String>> headers) throws Exception {
// XXX /formpost
// Start the connection attempt.
ChannelFuture future = bootstrap.connect(SocketUtils.socketAddress(host, port));
Future<Channel> future = bootstrap.connect(SocketUtils.socketAddress(host, port));
// Wait until the connection attempt succeeds or fails.
Channel channel = future.sync().channel();
Channel channel = future.get();
// Prepare the HTTP request.
HttpRequest request = new DefaultHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, uriSimple.toASCIIString());
@ -268,9 +268,9 @@ public final class HttpUploadClient {
Iterable<Entry<String, String>> headers, List<InterfaceHttpData> bodylist) throws Exception {
// XXX /formpostmultipart
// Start the connection attempt.
ChannelFuture future = bootstrap.connect(SocketUtils.socketAddress(host, port));
Future<Channel> future = bootstrap.connect(SocketUtils.socketAddress(host, port));
// Wait until the connection attempt succeeds or fails.
Channel channel = future.sync().channel();
Channel channel = future.get();
// Prepare the HTTP request.
HttpRequest request = new DefaultHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.POST, uriFile.toASCIIString());

View File

@ -54,7 +54,7 @@ public final class HttpUploadServer {
b.handler(new LoggingHandler(LogLevel.INFO));
b.childHandler(new HttpUploadServerInitializer(sslCtx));
Channel ch = b.bind(PORT).sync().channel();
Channel ch = b.bind(PORT).get();
System.err.println("Open your web browser and navigate to " +
(SSL? "https" : "http") + "://127.0.0.1:" + PORT + '/');

View File

@ -56,7 +56,7 @@ public final class WebSocketServer {
.channel(NioServerSocketChannel.class)
.childHandler(new WebSocketServerInitializer(sslCtx));
Channel ch = b.bind(PORT).sync().channel();
Channel ch = b.bind(PORT).get();
System.out.println("Open your web browser and navigate to " +
(SSL? "https" : "http") + "://127.0.0.1:" + PORT + '/');

View File

@ -119,7 +119,7 @@ public final class WebSocketClient {
}
});
Channel ch = b.connect(uri.getHost(), port).sync().channel();
Channel ch = b.connect(uri.getHost(), port).get();
handler.handshakeFuture().sync();
BufferedReader console = new BufferedReader(new InputStreamReader(System.in));
@ -127,11 +127,11 @@ public final class WebSocketClient {
String msg = console.readLine();
if (msg == null) {
break;
} else if ("bye".equals(msg.toLowerCase())) {
} else if ("bye".equalsIgnoreCase(msg)) {
ch.writeAndFlush(new CloseWebSocketFrame());
ch.closeFuture().sync();
break;
} else if ("ping".equals(msg.toLowerCase())) {
} else if ("ping".equalsIgnoreCase(msg)) {
WebSocketFrame frame = new PingWebSocketFrame(Unpooled.wrappedBuffer(new byte[] { 8, 1, 8, 1 }));
ch.writeAndFlush(frame);
} else {

View File

@ -70,7 +70,7 @@ public final class WebSocketServer {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new WebSocketServerInitializer(sslCtx));
Channel ch = b.bind(PORT).sync().channel();
Channel ch = b.bind(PORT).get();
System.out.println("Open your web browser and navigate to " +
(SSL? "https" : "http") + "://127.0.0.1:" + PORT + '/');

View File

@ -34,7 +34,6 @@ import io.netty.handler.ssl.ApplicationProtocolConfig.Protocol;
import io.netty.handler.ssl.ApplicationProtocolConfig.SelectedListenerFailureBehavior;
import io.netty.handler.ssl.ApplicationProtocolConfig.SelectorFailureBehavior;
import io.netty.handler.ssl.ApplicationProtocolNames;
import io.netty.handler.ssl.OpenSsl;
import io.netty.handler.ssl.SslContext;
import io.netty.handler.ssl.SslContextBuilder;
import io.netty.handler.ssl.SslProvider;
@ -49,6 +48,9 @@ import static io.netty.buffer.Unpooled.wrappedBuffer;
import static io.netty.handler.codec.http.HttpMethod.GET;
import static io.netty.handler.codec.http.HttpMethod.POST;
import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
import static io.netty.handler.ssl.SslProvider.JDK;
import static io.netty.handler.ssl.SslProvider.OPENSSL;
import static io.netty.handler.ssl.SslProvider.isAlpnSupported;
/**
* An HTTP2 client that allows you to send HTTP2 frames to a server using HTTP1-style approaches
@ -72,7 +74,7 @@ public final class Http2Client {
// Configure SSL.
final SslContext sslCtx;
if (SSL) {
SslProvider provider = OpenSsl.isAlpnSupported() ? SslProvider.OPENSSL : SslProvider.JDK;
SslProvider provider = isAlpnSupported(OPENSSL) ? OPENSSL : JDK;
sslCtx = SslContextBuilder.forClient()
.sslProvider(provider)
/* NOTE: the cipher filter may not include all ciphers required by the HTTP/2 specification.
@ -105,7 +107,7 @@ public final class Http2Client {
b.handler(initializer);
// Start the client.
Channel channel = b.connect().syncUninterruptibly().channel();
Channel channel = b.connect().get();
System.out.println("Connected to [" + HOST + ':' + PORT + ']');
// Wait for the HTTP/2 upgrade to occur.

View File

@ -89,7 +89,7 @@ public final class Http2FrameClient {
b.handler(new Http2ClientFrameInitializer(sslCtx));
// Start the client.
final Channel channel = b.connect().syncUninterruptibly().channel();
final Channel channel = b.connect().get();
System.out.println("Connected to [" + HOST + ':' + PORT + ']');
final Http2ClientStreamFrameResponseHandler streamFrameResponseHandler =

View File

@ -31,13 +31,14 @@ import io.netty.handler.ssl.ApplicationProtocolConfig.Protocol;
import io.netty.handler.ssl.ApplicationProtocolConfig.SelectedListenerFailureBehavior;
import io.netty.handler.ssl.ApplicationProtocolConfig.SelectorFailureBehavior;
import io.netty.handler.ssl.ApplicationProtocolNames;
import io.netty.handler.ssl.OpenSsl;
import io.netty.handler.ssl.SslContext;
import io.netty.handler.ssl.SslContextBuilder;
import io.netty.handler.ssl.SslProvider;
import io.netty.handler.ssl.SupportedCipherSuiteFilter;
import io.netty.handler.ssl.util.SelfSignedCertificate;
import static io.netty.handler.ssl.SslProvider.isAlpnSupported;
/**
* An HTTP/2 Server that responds to requests with a Hello World. Once started, you can test the
* server with the example client.
@ -55,7 +56,7 @@ public final class Http2Server {
// Configure SSL.
final SslContext sslCtx;
if (SSL) {
SslProvider provider = OpenSsl.isAlpnSupported() ? SslProvider.OPENSSL : SslProvider.JDK;
SslProvider provider = isAlpnSupported(SslProvider.OPENSSL) ? SslProvider.OPENSSL : SslProvider.JDK;
SelfSignedCertificate ssc = new SelfSignedCertificate();
sslCtx = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey())
.sslProvider(provider)
@ -84,7 +85,7 @@ public final class Http2Server {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new Http2ServerInitializer(sslCtx));
Channel ch = b.bind(PORT).sync().channel();
Channel ch = b.bind(PORT).get();
System.err.println("Open your HTTP/2-enabled web browser and navigate to " +
(SSL? "https" : "http") + "://127.0.0.1:" + PORT + '/');

View File

@ -31,13 +31,16 @@ import io.netty.handler.ssl.ApplicationProtocolConfig.Protocol;
import io.netty.handler.ssl.ApplicationProtocolConfig.SelectedListenerFailureBehavior;
import io.netty.handler.ssl.ApplicationProtocolConfig.SelectorFailureBehavior;
import io.netty.handler.ssl.ApplicationProtocolNames;
import io.netty.handler.ssl.OpenSsl;
import io.netty.handler.ssl.SslContext;
import io.netty.handler.ssl.SslContextBuilder;
import io.netty.handler.ssl.SslProvider;
import io.netty.handler.ssl.SupportedCipherSuiteFilter;
import io.netty.handler.ssl.util.SelfSignedCertificate;
import static io.netty.handler.ssl.SslProvider.JDK;
import static io.netty.handler.ssl.SslProvider.OPENSSL;
import static io.netty.handler.ssl.SslProvider.isAlpnSupported;
/**
* An HTTP/2 Server that responds to requests with a Hello World. Once started, you can test the
* server with the example client.
@ -55,7 +58,7 @@ public final class Http2Server {
// Configure SSL.
final SslContext sslCtx;
if (SSL) {
SslProvider provider = OpenSsl.isAlpnSupported() ? SslProvider.OPENSSL : SslProvider.JDK;
SslProvider provider = isAlpnSupported(OPENSSL) ? OPENSSL : JDK;
SelfSignedCertificate ssc = new SelfSignedCertificate();
sslCtx = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey())
.sslProvider(provider)
@ -84,7 +87,7 @@ public final class Http2Server {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new Http2ServerInitializer(sslCtx));
Channel ch = b.bind(PORT).sync().channel();
Channel ch = b.bind(PORT).get();
System.err.println("Open your HTTP/2-enabled web browser and navigate to " +
(SSL? "https" : "http") + "://127.0.0.1:" + PORT + '/');

View File

@ -31,13 +31,14 @@ import io.netty.handler.ssl.ApplicationProtocolConfig.Protocol;
import io.netty.handler.ssl.ApplicationProtocolConfig.SelectedListenerFailureBehavior;
import io.netty.handler.ssl.ApplicationProtocolConfig.SelectorFailureBehavior;
import io.netty.handler.ssl.ApplicationProtocolNames;
import io.netty.handler.ssl.OpenSsl;
import io.netty.handler.ssl.SslContext;
import io.netty.handler.ssl.SslContextBuilder;
import io.netty.handler.ssl.SslProvider;
import io.netty.handler.ssl.SupportedCipherSuiteFilter;
import io.netty.handler.ssl.util.SelfSignedCertificate;
import static io.netty.handler.ssl.SslProvider.isAlpnSupported;
/**
* An HTTP/2 Server that responds to requests with a Hello World. Once started, you can test the
* server with the example client.
@ -52,7 +53,7 @@ public final class Http2Server {
// Configure SSL.
final SslContext sslCtx;
if (SSL) {
SslProvider provider = OpenSsl.isAlpnSupported() ? SslProvider.OPENSSL : SslProvider.JDK;
SslProvider provider = isAlpnSupported(SslProvider.OPENSSL) ? SslProvider.OPENSSL : SslProvider.JDK;
SelfSignedCertificate ssc = new SelfSignedCertificate();
sslCtx = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey())
.sslProvider(provider)
@ -81,7 +82,7 @@ public final class Http2Server {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new Http2ServerInitializer(sslCtx));
Channel ch = b.bind(PORT).sync().channel();
Channel ch = b.bind(PORT).get();
System.err.println("Open your HTTP/2-enabled web browser and navigate to " +
(SSL? "https" : "http") + "://127.0.0.1:" + PORT + '/');

View File

@ -16,7 +16,6 @@
package io.netty.example.http2.tiles;
import static io.netty.handler.codec.http2.Http2SecurityUtil.CIPHERS;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
@ -35,9 +34,10 @@ import io.netty.handler.ssl.SslContextBuilder;
import io.netty.handler.ssl.SupportedCipherSuiteFilter;
import io.netty.handler.ssl.util.SelfSignedCertificate;
import javax.net.ssl.SSLException;
import java.security.cert.CertificateException;
import javax.net.ssl.SSLException;
import static io.netty.handler.codec.http2.Http2SecurityUtil.CIPHERS;
/**
* Demonstrates an Http2 server using Netty to display a bunch of images and
@ -65,7 +65,7 @@ public class Http2Server {
}
});
Channel ch = b.bind(PORT).sync().channel();
Channel ch = b.bind(PORT).get();
return ch.closeFuture();
}

View File

@ -60,7 +60,7 @@ public final class HttpServer {
}
});
Channel ch = b.bind(PORT).sync().channel();
Channel ch = b.bind(PORT).get();
return ch.closeFuture();
}
}

View File

@ -20,8 +20,8 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.MultithreadEventLoopGroup;
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.LocalHandler;
@ -80,7 +80,7 @@ public final class LocalEcho {
sb.bind(addr).sync();
// Start the client.
Channel ch = cb.connect(addr).sync().channel();
Channel ch = cb.connect(addr).get();
// Read commands from the stdin.
System.out.println("Enter text (quit to end)");

View File

@ -72,7 +72,7 @@ public final class MemcacheClient {
});
// Start the connection attempt.
Channel ch = b.connect(HOST, PORT).sync().channel();
Channel ch = b.connect(HOST, PORT).get();
// Read commands from the stdin.
System.out.println("Enter commands (quit to end)");
@ -85,7 +85,7 @@ public final class MemcacheClient {
if (line == null) {
break;
}
if ("quit".equals(line.toLowerCase())) {
if ("quit".equalsIgnoreCase(line)) {
ch.close().sync();
break;
}

View File

@ -16,11 +16,12 @@
package io.netty.example.mqtt.heartBeat;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.Channel;
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.socket.SocketChannel;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.codec.mqtt.MqttDecoder;
@ -35,8 +36,8 @@ public final class MqttHeartBeatBroker {
}
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();
@ -44,6 +45,7 @@ public final class MqttHeartBeatBroker {
b.option(ChannelOption.SO_BACKLOG, 1024);
b.channel(NioServerSocketChannel.class);
b.childHandler(new ChannelInitializer<SocketChannel>() {
@Override
protected void initChannel(SocketChannel ch) throws Exception {
ch.pipeline().addLast("encoder", MqttEncoder.INSTANCE);
ch.pipeline().addLast("decoder", new MqttDecoder());
@ -52,10 +54,10 @@ public final class MqttHeartBeatBroker {
}
});
ChannelFuture f = b.bind(1883).sync();
Channel channel = b.bind(1883).get();
System.out.println("Broker initiated...");
f.channel().closeFuture().sync();
channel.closeFuture().sync();
} finally {
workerGroup.shutdownGracefully();
bossGroup.shutdownGracefully();

View File

@ -16,10 +16,11 @@
package io.netty.example.mqtt.heartBeat;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.Channel;
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.codec.mqtt.MqttDecoder;
@ -39,13 +40,14 @@ public final class MqttHeartBeatClient {
private static final String PASSWORD = System.getProperty("password", "guest");
public static void main(String[] args) throws Exception {
EventLoopGroup workerGroup = new NioEventLoopGroup();
EventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
Bootstrap b = new Bootstrap();
b.group(workerGroup);
b.channel(NioSocketChannel.class);
b.handler(new ChannelInitializer<SocketChannel>() {
@Override
protected void initChannel(SocketChannel ch) throws Exception {
ch.pipeline().addLast("encoder", MqttEncoder.INSTANCE);
ch.pipeline().addLast("decoder", new MqttDecoder());
@ -54,9 +56,9 @@ public final class MqttHeartBeatClient {
}
});
ChannelFuture f = b.connect(HOST, PORT).sync();
Channel channel = b.connect(HOST, PORT).get();
System.out.println("Client connected");
f.channel().closeFuture().sync();
channel.closeFuture().sync();
} finally {
workerGroup.shutdownGracefully();
}

View File

@ -71,7 +71,7 @@ public final class ObjectEchoClient {
});
// Start the connection attempt.
b.connect(HOST, PORT).sync().channel().closeFuture().sync();
b.connect(HOST, PORT).get().closeFuture().sync();
} finally {
group.shutdownGracefully();
}

View File

@ -73,7 +73,7 @@ public final class ObjectEchoServer {
});
// Bind and start to accept incoming connections.
b.bind(PORT).sync().channel().closeFuture().sync();
b.bind(PORT).get().closeFuture().sync();
} finally {
bossGroup.shutdownGracefully();
workerGroup.shutdownGracefully();

View File

@ -16,29 +16,18 @@
package io.netty.example.ocsp;
import java.math.BigInteger;
import javax.net.ssl.SSLSession;
import javax.security.cert.X509Certificate;
import io.netty.buffer.Unpooled;
import io.netty.channel.ChannelHandler;
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;
import org.bouncycastle.cert.ocsp.OCSPResp;
import org.bouncycastle.cert.ocsp.SingleResp;
import io.netty.bootstrap.Bootstrap;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
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.nio.NioHandler;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.http.DefaultFullHttpRequest;
import io.netty.handler.codec.http.FullHttpRequest;
@ -57,6 +46,15 @@ import io.netty.handler.ssl.SslProvider;
import io.netty.handler.ssl.ocsp.OcspClientHandler;
import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.Promise;
import org.bouncycastle.asn1.ocsp.OCSPResponseStatus;
import org.bouncycastle.cert.ocsp.BasicOCSPResp;
import org.bouncycastle.cert.ocsp.CertificateStatus;
import org.bouncycastle.cert.ocsp.OCSPResp;
import org.bouncycastle.cert.ocsp.SingleResp;
import javax.net.ssl.SSLSession;
import javax.security.cert.X509Certificate;
import java.math.BigInteger;
/**
* This is a very simple example for an HTTPS client that uses OCSP stapling.
@ -99,9 +97,7 @@ public class OcspClientExample {
.option(ChannelOption.CONNECT_TIMEOUT_MILLIS, 5 * 1000)
.handler(newClientHandler(context, host, promise));
Channel channel = bootstrap.connect(host, 443)
.syncUninterruptibly()
.channel();
Channel channel = bootstrap.connect(host, 443).get();
try {
FullHttpResponse response = promise.get();

View File

@ -60,7 +60,7 @@ public final class PortUnificationServer {
});
// Bind and start to accept incoming connections.
b.bind(PORT).sync().channel().closeFuture().sync();
b.bind(PORT).get().closeFuture().sync();
} finally {
bossGroup.shutdownGracefully();
workerGroup.shutdownGracefully();

View File

@ -43,7 +43,7 @@ public final class HexDumpProxy {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new HexDumpProxyInitializer(REMOTE_HOST, REMOTE_PORT))
.childOption(ChannelOption.AUTO_READ, false)
.bind(LOCAL_PORT).sync().channel().closeFuture().sync();
.bind(LOCAL_PORT).get().closeFuture().sync();
} finally {
bossGroup.shutdownGracefully();
workerGroup.shutdownGracefully();

View File

@ -18,10 +18,10 @@ package io.netty.example.proxy;
import io.netty.bootstrap.Bootstrap;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
public class HexDumpProxyFrontendHandler implements ChannelHandler {
@ -46,11 +46,16 @@ public class HexDumpProxyFrontendHandler implements ChannelHandler {
Bootstrap b = new Bootstrap();
b.group(inboundChannel.eventLoop())
.channel(ctx.channel().getClass())
.handler(new HexDumpProxyBackendHandler(inboundChannel))
.option(ChannelOption.AUTO_READ, false);
ChannelFuture f = b.connect(remoteHost, remotePort);
outboundChannel = f.channel();
f.addListener((ChannelFutureListener) future -> {
.handler(new ChannelInitializer<Channel>() {
@Override
protected void initChannel(Channel ch) throws Exception {
outboundChannel = ch;
ch.pipeline().addLast(new HexDumpProxyBackendHandler(inboundChannel));
}
})
.option(ChannelOption.AUTO_READ, false)
.connect(remoteHost, remotePort)
.addListener(future -> {
if (future.isSuccess()) {
// connection complete start to read first data
inboundChannel.read();

View File

@ -47,7 +47,7 @@ public final class QuoteOfTheMomentClient {
.option(ChannelOption.SO_BROADCAST, true)
.handler(new QuoteOfTheMomentClientHandler());
Channel ch = b.bind(0).sync().channel();
Channel ch = b.bind(0).get();
// Broadcast the QOTM request to port 8080.
ch.writeAndFlush(new DatagramPacket(

View File

@ -41,7 +41,7 @@ public final class QuoteOfTheMomentServer {
.option(ChannelOption.SO_BROADCAST, true)
.handler(new QuoteOfTheMomentServerHandler());
b.bind(PORT).sync().channel().closeFuture().await();
b.bind(PORT).get().closeFuture().await();
} finally {
group.shutdownGracefully();
}

View File

@ -60,7 +60,7 @@ public class RedisClient {
});
// Start the connection attempt.
Channel ch = b.connect(HOST, PORT).sync().channel();
Channel ch = b.connect(HOST, PORT).get();
// Read commands from the stdin.
System.out.println("Enter Redis commands (quit to end)");
@ -72,7 +72,8 @@ public class RedisClient {
if (line == null || "quit".equalsIgnoreCase(line)) { // EOF or "quit"
ch.close().sync();
break;
} else if (line.isEmpty()) { // skip `enter` or `enter` with spaces.
}
if (line.isEmpty()) { // skip `enter` or `enter` with spaces.
continue;
}
// Sends the received line to the server.

View File

@ -16,7 +16,7 @@
package io.netty.example.sctp;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
@ -57,10 +57,10 @@ public final class SctpEchoClient {
});
// Start the client.
ChannelFuture f = b.connect(HOST, PORT).sync();
Channel channel = b.connect(HOST, PORT).get();
// Wait until the connection is closed.
f.channel().closeFuture().sync();
channel.closeFuture().sync();
} finally {
// Shut down the event loop to terminate all threads.
group.shutdownGracefully();

View File

@ -16,7 +16,7 @@
package io.netty.example.sctp;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
@ -55,10 +55,10 @@ public final class SctpEchoServer {
});
// Start the server.
ChannelFuture f = b.bind(PORT).sync();
Channel channel = b.bind(PORT).get();
// Wait until the server socket is closed.
f.channel().closeFuture().sync();
channel.closeFuture().sync();
} finally {
// Shut down all event loops to terminate all threads.
bossGroup.shutdownGracefully();

View File

@ -16,6 +16,7 @@
package io.netty.example.sctp.multihoming;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
@ -25,6 +26,7 @@ import io.netty.channel.sctp.SctpChannel;
import io.netty.channel.sctp.SctpChannelOption;
import io.netty.channel.sctp.nio.NioSctpChannel;
import io.netty.example.sctp.SctpEchoClientHandler;
import io.netty.util.concurrent.Future;
import io.netty.util.internal.SocketUtils;
import java.net.InetAddress;
@ -66,10 +68,10 @@ public final class SctpMultiHomingEchoClient {
InetSocketAddress remoteAddress = SocketUtils.socketAddress(SERVER_REMOTE_HOST, SERVER_REMOTE_PORT);
// Bind the client channel.
ChannelFuture bindFuture = b.bind(localAddress).sync();
Future<Channel> bindFuture = b.bind(localAddress);
// Get the underlying sctp channel
SctpChannel channel = (SctpChannel) bindFuture.channel();
SctpChannel channel = (SctpChannel) bindFuture.get();
// Bind the secondary address.
// Please note that, bindAddress in the client channel should be done before connecting if you have not

View File

@ -16,6 +16,7 @@
package io.netty.example.sctp.multihoming;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
@ -28,6 +29,7 @@ import io.netty.channel.sctp.nio.NioSctpServerChannel;
import io.netty.example.sctp.SctpEchoServerHandler;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
import io.netty.util.concurrent.Future;
import io.netty.util.internal.SocketUtils;
import java.net.InetAddress;
@ -66,10 +68,10 @@ public final class SctpMultiHomingEchoServer {
InetAddress localSecondaryAddress = SocketUtils.addressByName(SERVER_SECONDARY_HOST);
// Bind the server to primary address.
ChannelFuture bindFuture = b.bind(localAddress).sync();
Future<Channel> bindFuture = b.bind(localAddress);
//Get the underlying sctp channel
SctpServerChannel channel = (SctpServerChannel) bindFuture.channel();
SctpServerChannel channel = (SctpServerChannel) bindFuture.get();
//Bind the secondary address
ChannelFuture connectFuture = channel.bindAddress(localSecondaryAddress).sync();

View File

@ -51,7 +51,7 @@ public final class SecureChatClient {
.handler(new SecureChatClientInitializer(sslCtx));
// Start the connection attempt.
Channel ch = b.connect(HOST, PORT).sync().channel();
Channel ch = b.connect(HOST, PORT).get();
// Read commands from the stdin.
ChannelFuture lastWriteFuture = null;
@ -67,7 +67,7 @@ public final class SecureChatClient {
// If user typed the 'bye' command, wait until the server closes
// the connection.
if ("bye".equals(line.toLowerCase())) {
if ("bye".equalsIgnoreCase(line)) {
ch.closeFuture().sync();
break;
}

View File

@ -48,7 +48,7 @@ public final class SecureChatServer {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new SecureChatServerInitializer(sslCtx));
b.bind(PORT).sync().channel().closeFuture().sync();
b.bind(PORT).get().closeFuture().sync();
} finally {
bossGroup.shutdownGracefully();
workerGroup.shutdownGracefully();

View File

@ -36,7 +36,13 @@ import io.netty.util.concurrent.Future;
import java.util.Base64;
import static io.netty.handler.codec.smtp.SmtpCommand.*;
import static io.netty.handler.codec.smtp.SmtpCommand.AUTH;
import static io.netty.handler.codec.smtp.SmtpCommand.DATA;
import static io.netty.handler.codec.smtp.SmtpCommand.EHLO;
import static io.netty.handler.codec.smtp.SmtpCommand.EMPTY;
import static io.netty.handler.codec.smtp.SmtpCommand.MAIL;
import static io.netty.handler.codec.smtp.SmtpCommand.QUIT;
import static io.netty.handler.codec.smtp.SmtpCommand.RCPT;
/**
* A simple smtp client
@ -92,7 +98,7 @@ public class SmtpClient {
.future();
// Start connect.
Channel ch = b.connect(HOST, PORT).sync().channel();
Channel ch = b.connect(HOST, PORT).get();
f.await();
ch.close();
} finally {

View File

@ -36,7 +36,7 @@ public final class SocksServer {
.channel(NioServerSocketChannel.class)
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new SocksServerInitializer());
b.bind(PORT).sync().channel().closeFuture().sync();
b.bind(PORT).get().closeFuture().sync();
} finally {
bossGroup.shutdownGracefully();
workerGroup.shutdownGracefully();

View File

@ -33,6 +33,7 @@ import io.netty.handler.codec.socksx.v5.Socks5CommandRequest;
import io.netty.handler.codec.socksx.v5.Socks5CommandStatus;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.FutureListener;
import io.netty.util.concurrent.GenericFutureListener;
import io.netty.util.concurrent.Promise;
@ChannelHandler.Sharable
@ -53,7 +54,7 @@ public final class SocksServerConnectHandler extends SimpleChannelInboundHandler
new DefaultSocks4CommandResponse(Socks4CommandStatus.SUCCESS));
responseFuture.addListener((ChannelFutureListener) channelFuture -> {
ctx.pipeline().remove(SocksServerConnectHandler.this);
ctx.pipeline().remove(this);
outboundChannel.pipeline().addLast(new RelayHandler(ctx.channel()));
ctx.pipeline().addLast(new RelayHandler(outboundChannel));
});
@ -71,7 +72,8 @@ public final class SocksServerConnectHandler extends SimpleChannelInboundHandler
.option(ChannelOption.SO_KEEPALIVE, true)
.handler(new DirectClientHandler(promise));
b.connect(request.dstAddr(), request.dstPort()).addListener((ChannelFutureListener) future -> {
b.connect(request.dstAddr(), request.dstPort()).addListener(
(GenericFutureListener<Future<Channel>>) future -> {
if (future.isSuccess()) {
// Connection established use handler provided results
} else {
@ -97,7 +99,7 @@ public final class SocksServerConnectHandler extends SimpleChannelInboundHandler
request.dstPort()));
responseFuture.addListener((ChannelFutureListener) channelFuture -> {
ctx.pipeline().remove(SocksServerConnectHandler.this);
ctx.pipeline().remove(this);
outboundChannel.pipeline().addLast(new RelayHandler(ctx.channel()));
ctx.pipeline().addLast(new RelayHandler(outboundChannel));
});
@ -115,7 +117,8 @@ public final class SocksServerConnectHandler extends SimpleChannelInboundHandler
.option(ChannelOption.SO_KEEPALIVE, true)
.handler(new DirectClientHandler(promise));
b.connect(request.dstAddr(), request.dstPort()).addListener((ChannelFutureListener) future -> {
b.connect(request.dstAddr(), request.dstPort()).addListener(
(GenericFutureListener<Future<Channel>>) future -> {
if (future.isSuccess()) {
// Connection established use handler provided results
} else {

View File

@ -34,7 +34,6 @@ import io.netty.handler.codec.stomp.StompSubframeEncoder;
*/
public final class StompClient {
static final boolean SSL = System.getProperty("ssl") != null;
static final String HOST = System.getProperty("host", "127.0.0.1");
static final int PORT = Integer.parseInt(System.getProperty("port", "61613"));
static final String LOGIN = System.getProperty("login", "guest");
@ -57,7 +56,7 @@ public final class StompClient {
}
});
b.connect(HOST, PORT).sync().channel().closeFuture().sync();
b.connect(HOST, PORT).get().closeFuture().sync();
} finally {
group.shutdownGracefully();
}

View File

@ -16,33 +16,34 @@
package io.netty.example.stomp.websocket;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
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.util.concurrent.Future;
import io.netty.util.concurrent.GenericFutureListener;
public class StompWebSocketChatServer {
static final int PORT = Integer.parseInt(System.getProperty("port", "8080"));
public void start(final int port) throws Exception {
NioEventLoopGroup boosGroup = new NioEventLoopGroup(1);
NioEventLoopGroup workerGroup = new NioEventLoopGroup();
MultithreadEventLoopGroup boosGroup = new MultithreadEventLoopGroup(1, NioHandler.newFactory());
MultithreadEventLoopGroup workerGroup = new MultithreadEventLoopGroup(NioHandler.newFactory());
try {
ServerBootstrap bootstrap = new ServerBootstrap()
.group(boosGroup, workerGroup)
.channel(NioServerSocketChannel.class)
.childHandler(new StompWebSocketChatServerInitializer("/chat"));
bootstrap.bind(port).addListener(new ChannelFutureListener() {
bootstrap.bind(port).addListener(new GenericFutureListener<Future<Object>>() {
@Override
public void operationComplete(ChannelFuture future) {
public void operationComplete(Future<Object> future) {
if (future.isSuccess()) {
System.out.println("Open your web browser and navigate to http://127.0.0.1:" + PORT + '/');
} else {
System.out.println("Cannot start server, follows exception " + future.cause());
}
}
}).channel().closeFuture().sync();
}).get().closeFuture().sync();
} finally {
boosGroup.shutdownGracefully();
workerGroup.shutdownGracefully();

View File

@ -56,7 +56,7 @@ public final class TelnetClient {
.handler(new TelnetClientInitializer(sslCtx));
// Start the connection attempt.
Channel ch = b.connect(HOST, PORT).sync().channel();
Channel ch = b.connect(HOST, PORT).get();
// Read commands from the stdin.
ChannelFuture lastWriteFuture = null;
@ -72,7 +72,7 @@ public final class TelnetClient {
// If user typed the 'bye' command, wait until the server closes
// the connection.
if ("bye".equals(line.toLowerCase())) {
if ("bye".equalsIgnoreCase(line)) {
ch.closeFuture().sync();
break;
}

View File

@ -53,7 +53,7 @@ public final class TelnetServer {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new TelnetServerInitializer(sslCtx));
b.bind(PORT).sync().channel().closeFuture().sync();
b.bind(PORT).get().closeFuture().sync();
} finally {
bossGroup.shutdownGracefully();
workerGroup.shutdownGracefully();

View File

@ -16,8 +16,7 @@
package io.netty.example.uptime;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
@ -25,6 +24,8 @@ 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;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.GenericFutureListener;
/**
@ -59,7 +60,7 @@ public final class UptimeClient {
}
static void connect() {
bs.connect().addListener((ChannelFutureListener) future -> {
bs.connect().addListener((GenericFutureListener<Future<Channel>>) future -> {
if (future.cause() != null) {
handler.startTime = -1;
handler.println("Failed to connect: " + future.cause());

View File

@ -16,6 +16,7 @@
package io.netty.example.uptime;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
@ -54,12 +55,12 @@ public final class UptimeServer {
});
// Bind and start to accept incoming connections.
ChannelFuture f = b.bind(PORT).sync();
Channel channel = b.bind(PORT).get();
// Wait until the server socket is closed.
// In this example, this does not happen, but you can do that to gracefully
// shut down your server.
f.channel().closeFuture().sync();
channel.closeFuture().sync();
} finally {
workerGroup.shutdownGracefully();
bossGroup.shutdownGracefully();

View File

@ -58,7 +58,7 @@ public final class WorldClockClient {
.handler(new WorldClockClientInitializer(sslCtx));
// Make a new connection.
Channel ch = b.connect(HOST, PORT).sync().channel();
Channel ch = b.connect(HOST, PORT).get();
// Get the handler instance to initiate the request.
WorldClockClientHandler handler = ch.pipeline().get(WorldClockClientHandler.class);

View File

@ -54,7 +54,7 @@ public final class WorldClockServer {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new WorldClockServerInitializer(sslCtx));
b.bind(PORT).sync().channel().closeFuture().sync();
b.bind(PORT).get().closeFuture().sync();
} finally {
bossGroup.shutdownGracefully();
workerGroup.shutdownGracefully();

View File

@ -18,7 +18,6 @@ package io.netty.handler.proxy;
import io.netty.bootstrap.Bootstrap;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
@ -41,18 +40,23 @@ import io.netty.handler.codec.http.HttpResponseStatus;
import io.netty.handler.codec.http.HttpVersion;
import io.netty.handler.proxy.HttpProxyHandler.HttpProxyConnectException;
import io.netty.util.NetUtil;
import java.util.concurrent.atomic.AtomicReference;
import io.netty.util.concurrent.Future;
import org.junit.jupiter.api.Test;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.util.concurrent.atomic.AtomicReference;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.*;
import static org.mockito.Mockito.isNull;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.same;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
public class HttpProxyHandlerTest {
@ -185,7 +189,7 @@ public class HttpProxyHandlerTest {
group = new MultithreadEventLoopGroup(1, LocalHandler.newFactory());
final LocalAddress addr = new LocalAddress("a");
final AtomicReference<Throwable> exception = new AtomicReference<Throwable>();
ChannelFuture sf =
Future<Channel> sf =
new ServerBootstrap().channel(LocalServerChannel.class).group(group).childHandler(
new ChannelInitializer<Channel>() {
@ -205,8 +209,8 @@ public class HttpProxyHandlerTest {
});
}
}).bind(addr);
serverChannel = sf.sync().channel();
ChannelFuture cf = new Bootstrap().channel(LocalChannel.class).group(group).handler(
serverChannel = sf.get();
Future<Channel> cf = new Bootstrap().channel(LocalChannel.class).group(group).handler(
new ChannelInitializer<Channel>() {
@Override
protected void initChannel(Channel ch) {
@ -220,7 +224,7 @@ public class HttpProxyHandlerTest {
});
}
}).connect(new InetSocketAddress("localhost", 1234));
clientChannel = cf.sync().channel();
clientChannel = cf.get();
clientChannel.close().sync();
assertTrue(exception.get() instanceof HttpProxyConnectException);

View File

@ -41,10 +41,10 @@ import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
import io.netty.handler.ssl.util.SelfSignedCertificate;
import io.netty.resolver.NoopAddressResolverGroup;
import io.netty.util.CharsetUtil;
import io.netty.util.internal.SocketUtils;
import io.netty.util.concurrent.DefaultThreadFactory;
import io.netty.util.concurrent.Future;
import io.netty.util.internal.EmptyArrays;
import io.netty.util.internal.SocketUtils;
import io.netty.util.internal.StringUtil;
import io.netty.util.internal.logging.InternalLogger;
import io.netty.util.internal.logging.InternalLoggerFactory;
@ -61,12 +61,14 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Queue;
import java.util.Random;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.Random;
import static org.hamcrest.CoreMatchers.*;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.fail;
@ -414,7 +416,7 @@ public class ProxyHandlerTest {
}
// Randomize the execution order to increase the possibility of exposing failure dependencies.
long seed = (reproducibleSeed == 0L) ? System.currentTimeMillis() : reproducibleSeed;
long seed = reproducibleSeed == 0L? System.currentTimeMillis() : reproducibleSeed;
logger.debug("Seed used: {}\n", seed);
Collections.shuffle(params, new Random(seed));
@ -629,7 +631,7 @@ public class ProxyHandlerTest {
Bootstrap b = new Bootstrap();
b.group(group);
b.channel(NioSocketChannel.class);
b.option(ChannelOption.AUTO_READ, this.autoRead);
b.option(ChannelOption.AUTO_READ, autoRead);
b.resolver(NoopAddressResolverGroup.INSTANCE);
b.handler(new ChannelInitializer<SocketChannel>() {
@Override
@ -641,7 +643,8 @@ public class ProxyHandlerTest {
}
});
boolean finished = b.connect(destination).channel().closeFuture().await(10, TimeUnit.SECONDS);
Channel channel = b.connect(destination).get();
boolean finished = channel.closeFuture().await(10, TimeUnit.SECONDS);
logger.debug("Received messages: {}", testHandler.received);
@ -689,7 +692,8 @@ public class ProxyHandlerTest {
}
});
boolean finished = b.connect(destination).channel().closeFuture().await(10, TimeUnit.SECONDS);
Channel channel = b.connect(destination).get();
boolean finished = channel.closeFuture().await(10, TimeUnit.SECONDS);
finished &= testHandler.latch.await(10, TimeUnit.SECONDS);
logger.debug("Recorded exceptions: {}", testHandler.exceptions);
@ -734,7 +738,8 @@ public class ProxyHandlerTest {
}
});
ChannelFuture cf = b.connect(DESTINATION).channel().closeFuture();
Channel channel = b.connect(DESTINATION).get();
ChannelFuture cf = channel.closeFuture();
boolean finished = cf.await(TIMEOUT * 2, TimeUnit.MILLISECONDS);
finished &= testHandler.latch.await(TIMEOUT * 2, TimeUnit.MILLISECONDS);

View File

@ -21,7 +21,6 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
@ -36,6 +35,8 @@ import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.util.CharsetUtil;
import io.netty.util.NetUtil;
import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.GenericFutureListener;
import io.netty.util.internal.PlatformDependent;
import io.netty.util.internal.logging.InternalLogger;
import io.netty.util.internal.logging.InternalLoggerFactory;
@ -105,7 +106,7 @@ abstract class ProxyServer {
}
});
ch = (ServerSocketChannel) b.bind(NetUtil.LOCALHOST, 0).syncUninterruptibly().channel();
ch = (ServerSocketChannel) b.bind(NetUtil.LOCALHOST, 0).syncUninterruptibly().getNow();
}
public final InetSocketAddress address() {
@ -167,13 +168,13 @@ abstract class ProxyServer {
boolean finished = handleProxyProtocol(ctx, msg);
if (finished) {
this.finished = true;
ChannelFuture f = connectToDestination(ctx.channel().eventLoop(), new BackendHandler(ctx));
f.addListener((ChannelFutureListener) future -> {
Future<Channel> f = connectToDestination(ctx.channel().eventLoop(), new BackendHandler(ctx));
f.addListener((GenericFutureListener<Future<Channel>>) future -> {
if (!future.isSuccess()) {
recordException(future.cause());
ctx.close();
} else {
backend = future.channel();
backend = future.getNow();
flush();
}
});
@ -202,7 +203,7 @@ abstract class ProxyServer {
protected abstract SocketAddress intermediaryDestination();
private ChannelFuture connectToDestination(EventLoop loop, ChannelHandler handler) {
private Future<Channel> connectToDestination(EventLoop loop, ChannelHandler handler) {
Bootstrap b = new Bootstrap();
b.channel(NioSocketChannel.class);
b.group(loop);

View File

@ -1476,7 +1476,7 @@ public class SslHandler extends ByteToMessageDecoder {
}
}
private void executeChannelRead(final ChannelHandlerContext ctx, final ByteBuf decodedOut) {
private static void executeChannelRead(final ChannelHandlerContext ctx, final ByteBuf decodedOut) {
try {
ctx.executor().execute(() -> ctx.fireChannelRead(decodedOut));
} catch (RejectedExecutionException e) {
@ -1770,7 +1770,7 @@ public class SslHandler extends ByteToMessageDecoder {
task.run();
runComplete();
}
} catch (final Throwable cause) {
} catch (Throwable cause) {
handleException(cause);
}
}
@ -1805,8 +1805,8 @@ public class SslHandler extends ByteToMessageDecoder {
// Our control flow may invoke this method multiple times for a single FINISHED event. For example
// wrapNonAppData may drain pendingUnencryptedWrites in wrap which transitions to handshake from FINISHED to
// NOT_HANDSHAKING which invokes setHandshakeSuccess, and then wrapNonAppData also directly invokes this method.
final boolean notified;
if (notified = !handshakePromise.isDone() && handshakePromise.trySuccess(ctx.channel())) {
final boolean notified = !handshakePromise.isDone() && handshakePromise.trySuccess(ctx.channel());
if (notified) {
if (logger.isDebugEnabled()) {
SSLSession session = engine.getSession();
logger.debug(

View File

@ -18,7 +18,6 @@ package io.netty.handler.address;
import io.netty.bootstrap.Bootstrap;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.EventLoopGroup;
@ -31,12 +30,12 @@ import io.netty.resolver.AbstractAddressResolver;
import io.netty.resolver.AddressResolver;
import io.netty.resolver.AddressResolverGroup;
import io.netty.util.concurrent.EventExecutor;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.Promise;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import java.net.SocketAddress;
import java.net.UnknownHostException;
import java.util.List;
@ -48,8 +47,8 @@ import static org.junit.jupiter.api.Assertions.fail;
public class ResolveAddressHandlerTest {
private static final LocalAddress UNRESOLVED = new LocalAddress("unresolved-" + UUID.randomUUID().toString());
private static final LocalAddress RESOLVED = new LocalAddress("resolved-" + UUID.randomUUID().toString());
private static final LocalAddress UNRESOLVED = new LocalAddress("unresolved-" + UUID.randomUUID());
private static final LocalAddress RESOLVED = new LocalAddress("resolved-" + UUID.randomUUID());
private static final Exception ERROR = new UnknownHostException();
private static EventLoopGroup group;
@ -67,16 +66,16 @@ public class ResolveAddressHandlerTest {
}
@Test
public void testResolveSuccessful() {
public void testResolveSuccessful() throws Exception {
testResolve(false);
}
@Test
public void testResolveFails() {
public void testResolveFails() throws Exception {
testResolve(true);
}
private static void testResolve(boolean fail) {
private static void testResolve(boolean fail) throws Exception {
AddressResolverGroup<SocketAddress> resolverGroup = new TestResolverGroup(fail);
Bootstrap cb = new Bootstrap();
cb.group(group).channel(LocalChannel.class).handler(new ResolveAddressHandler(resolverGroup));
@ -92,17 +91,16 @@ public class ResolveAddressHandlerTest {
});
// Start server
Channel sc = sb.bind(RESOLVED).syncUninterruptibly().channel();
ChannelFuture future = cb.connect(UNRESOLVED).awaitUninterruptibly();
Channel sc = sb.bind(RESOLVED).get();
Future<Channel> future = cb.connect(UNRESOLVED).awaitUninterruptibly();
try {
if (fail) {
assertSame(ERROR, future.cause());
} else {
assertTrue(future.isSuccess());
future.get().close().syncUninterruptibly();
}
future.channel().close().syncUninterruptibly();
} finally {
future.channel().close().syncUninterruptibly();
sc.close().syncUninterruptibly();
resolverGroup.close();
}
@ -139,5 +137,5 @@ public class ResolveAddressHandlerTest {
}
};
}
};
}
}

View File

@ -27,8 +27,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.embedded.EmbeddedChannel;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.embedded.EmbeddedChannel;
import io.netty.channel.nio.NioHandler;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
@ -47,7 +47,8 @@ import java.util.concurrent.Exchanger;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicReference;
import static java.util.concurrent.TimeUnit.*;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNull;
@ -74,7 +75,7 @@ public class FlowControlHandlerTest {
return Unpooled.wrappedBuffer(new byte[]{ 1 });
}
private static Channel newServer(final boolean autoRead, final ChannelHandler... handlers) {
private static Channel newServer(final boolean autoRead, final ChannelHandler... handlers) throws Exception {
assertTrue(handlers.length >= 1);
ServerBootstrap serverBootstrap = new ServerBootstrap();
@ -90,12 +91,10 @@ public class FlowControlHandlerTest {
}
});
return serverBootstrap.bind(0)
.syncUninterruptibly()
.channel();
return serverBootstrap.bind(0).get();
}
private static Channel newClient(SocketAddress server) {
private static Channel newClient(SocketAddress server) throws Exception {
Bootstrap bootstrap = new Bootstrap();
bootstrap.group(GROUP)
@ -108,9 +107,7 @@ public class FlowControlHandlerTest {
}
});
return bootstrap.connect(server)
.syncUninterruptibly()
.channel();
return bootstrap.connect(server).get();
}
/**

View File

@ -21,7 +21,6 @@ import io.netty.buffer.ByteBufOutputStream;
import io.netty.buffer.ByteBufUtil;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.nio.NioEventLoopGroup;
@ -41,7 +40,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class PcapWriteHandlerTest {
@Test
public void udpV4() throws InterruptedException {
public void udpV4() throws Exception {
ByteBuf byteBuf = Unpooled.buffer();
@ -61,8 +60,7 @@ public class PcapWriteHandlerTest {
}
});
ChannelFuture channelFutureServer = server.bind(srvReqAddr).sync();
assertTrue(channelFutureServer.isSuccess());
Channel channelServer = server.bind(srvReqAddr).get();
// We'll bootstrap a UDP Client for sending UDP Packets to UDP Server.
Bootstrap client = new Bootstrap()
@ -70,11 +68,9 @@ public class PcapWriteHandlerTest {
.channel(NioDatagramChannel.class)
.handler(new PcapWriteHandler(new ByteBufOutputStream(byteBuf)));
ChannelFuture channelFutureClient =
client.connect(channelFutureServer.channel().localAddress(), cltReqAddr).sync();
assertTrue(channelFutureClient.isSuccess());
Channel clientChannel = channelFutureClient.channel();
assertTrue(clientChannel.writeAndFlush(Unpooled.wrappedBuffer("Meow".getBytes())).sync().isSuccess());
Channel channelClient =
client.connect(channelServer.localAddress(), cltReqAddr).get();
assertTrue(channelClient.writeAndFlush(Unpooled.wrappedBuffer("Meow".getBytes())).sync().isSuccess());
assertTrue(eventLoopGroup.shutdownGracefully().sync().isSuccess());
// Verify Pcap Global Headers
@ -111,10 +107,10 @@ public class PcapWriteHandlerTest {
assertEquals((byte) 0xff, ipv4Packet.readByte()); // TTL
assertEquals((byte) 17, ipv4Packet.readByte()); // Protocol
assertEquals(0, ipv4Packet.readShort()); // Checksum
InetSocketAddress localAddr = (InetSocketAddress) clientChannel.remoteAddress();
InetSocketAddress localAddr = (InetSocketAddress) channelClient.remoteAddress();
// Source IPv4 Address
assertEquals(NetUtil.ipv4AddressToInt((Inet4Address) localAddr.getAddress()), ipv4Packet.readInt());
InetSocketAddress remoteAddr = (InetSocketAddress) clientChannel.localAddress();
InetSocketAddress remoteAddr = (InetSocketAddress) channelClient.localAddress();
// Destination IPv4 Address
assertEquals(NetUtil.ipv4AddressToInt((Inet4Address) remoteAddr.getAddress()), ipv4Packet.readInt());

View File

@ -25,8 +25,8 @@ import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
@ -41,6 +41,8 @@ import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.MethodSource;
import javax.net.ssl.SSLContext;
import javax.net.ssl.SSLEngine;
import java.net.SocketAddress;
import java.security.NoSuchAlgorithmException;
import java.util.ArrayList;
@ -52,9 +54,6 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import javax.net.ssl.SSLContext;
import javax.net.ssl.SSLEngine;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assumptions.assumeTrue;
@ -243,7 +242,7 @@ public class CipherSuiteCanaryTest {
.group(GROUP)
.childHandler(handler);
return bootstrap.bind(address).sync().channel();
return bootstrap.bind(address).get();
}
private static Channel client(Channel server, ChannelHandler handler) throws Exception {
@ -254,7 +253,7 @@ public class CipherSuiteCanaryTest {
.group(GROUP)
.handler(handler);
return bootstrap.connect(remoteAddress).sync().channel();
return bootstrap.connect(remoteAddress).get();
}
private static List<Object[]> expand(String rfcCipherName) {

View File

@ -388,7 +388,7 @@ public class OpenSslPrivateKeyMethodTest {
.group(GROUP)
.childHandler(handler);
return bootstrap.bind(address).sync().channel();
return bootstrap.bind(address).get();
}
private static Channel client(Channel server, ChannelHandler handler) throws Exception {
@ -399,7 +399,7 @@ public class OpenSslPrivateKeyMethodTest {
.group(GROUP)
.handler(handler);
return bootstrap.connect(remoteAddress).sync().channel();
return bootstrap.connect(remoteAddress).get();
}
private static final class DelegateThread extends Thread {

View File

@ -104,7 +104,7 @@ public class ParameterizedSslHandlerTest {
@Timeout(value = 48000, unit = TimeUnit.MILLISECONDS)
public void testCompositeBufSizeEstimationGuaranteesSynchronousWrite(
SslProvider clientProvider, SslProvider serverProvider)
throws CertificateException, SSLException, ExecutionException, InterruptedException {
throws Exception {
compositeBufSizeEstimationGuaranteesSynchronousWrite(serverProvider, clientProvider,
true, true, true);
compositeBufSizeEstimationGuaranteesSynchronousWrite(serverProvider, clientProvider,
@ -206,7 +206,7 @@ public class ParameterizedSslHandlerTest {
}
});
}
}).bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
}).bind(new InetSocketAddress(0)).get();
cc = new Bootstrap()
.group(group)
@ -255,7 +255,7 @@ public class ParameterizedSslHandlerTest {
}
});
}
}).connect(sc.localAddress()).syncUninterruptibly().channel();
}).connect(sc.localAddress()).get();
donePromise.get();
} finally {
@ -337,7 +337,7 @@ public class ParameterizedSslHandlerTest {
}
});
}
}).bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
}).bind(new InetSocketAddress(0)).get();
cc = new Bootstrap()
.group(group)
@ -356,7 +356,7 @@ public class ParameterizedSslHandlerTest {
}
});
}
}).connect(sc.localAddress()).syncUninterruptibly().channel();
}).connect(sc.localAddress()).get();
promise.syncUninterruptibly();
} finally {
@ -396,8 +396,8 @@ public class ParameterizedSslHandlerTest {
testCloseNotify(clientProvider, serverProvider, 0, false);
}
private void testCloseNotify(SslProvider clientProvider, SslProvider serverProvider,
final long closeNotifyReadTimeout, final boolean timeout) throws Exception {
private static void testCloseNotify(SslProvider clientProvider, SslProvider serverProvider,
final long closeNotifyReadTimeout, final boolean timeout) throws Exception {
SelfSignedCertificate ssc = new SelfSignedCertificate();
final SslContext sslServerCtx = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey())
@ -442,7 +442,7 @@ public class ParameterizedSslHandlerTest {
});
ch.pipeline().addLast(handler);
}
}).bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
}).bind(new InetSocketAddress(0)).get();
cc = new Bootstrap()
.group(group)
@ -480,7 +480,7 @@ public class ParameterizedSslHandlerTest {
});
ch.pipeline().addLast(handler);
}
}).connect(sc.localAddress()).syncUninterruptibly().channel();
}).connect(sc.localAddress()).get();
serverPromise.awaitUninterruptibly();
clientPromise.awaitUninterruptibly();
@ -554,11 +554,11 @@ public class ParameterizedSslHandlerTest {
}
}
private void reentryOnHandshakeComplete(SslProvider clientProvider, SslProvider serverProvider,
EventLoopGroup group, SocketAddress bindAddress,
Class<? extends ServerChannel> serverClass,
Class<? extends Channel> clientClass, boolean serverAutoRead,
boolean clientAutoRead) throws Exception {
private static void reentryOnHandshakeComplete(SslProvider clientProvider, SslProvider serverProvider,
EventLoopGroup group, SocketAddress bindAddress,
Class<? extends ServerChannel> serverClass,
Class<? extends Channel> clientClass, boolean serverAutoRead,
boolean clientAutoRead) throws Exception {
SelfSignedCertificate ssc = new SelfSignedCertificate();
final SslContext sslServerCtx = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey())
.sslProvider(serverProvider)
@ -589,7 +589,7 @@ public class ParameterizedSslHandlerTest {
ch.pipeline().addLast(new ReentryWriteSslHandshakeHandler(expectedContent, serverQueue,
serverLatch));
}
}).bind(bindAddress).syncUninterruptibly().channel();
}).bind(bindAddress).get();
cc = new Bootstrap()
.group(group)
@ -602,7 +602,7 @@ public class ParameterizedSslHandlerTest {
ch.pipeline().addLast(new ReentryWriteSslHandshakeHandler(expectedContent, clientQueue,
clientLatch));
}
}).connect(sc.localAddress()).syncUninterruptibly().channel();
}).connect(sc.localAddress()).get();
serverLatch.await();
assertEquals(expectedContent, serverQueue.toString());

View File

@ -98,7 +98,7 @@ public abstract class RenegotiateTest {
});
}
});
Channel channel = sb.bind(new LocalAddress("test")).syncUninterruptibly().channel();
Channel channel = sb.bind(new LocalAddress("test")).get();
final SslContext clientContext = SslContextBuilder.forClient()
.trustManager(InsecureTrustManagerFactory.INSTANCE)
@ -131,7 +131,7 @@ public abstract class RenegotiateTest {
}
});
Channel clientChannel = bootstrap.connect(channel.localAddress()).syncUninterruptibly().channel();
Channel clientChannel = bootstrap.connect(channel.localAddress()).get();
latch.await();
clientChannel.close().syncUninterruptibly();
channel.close().syncUninterruptibly();

View File

@ -123,7 +123,6 @@ import java.util.List;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@ -791,7 +790,7 @@ public abstract class SSLEngineTest {
KeyManagerFactory clientKMF, File clientTrustManager,
ClientAuth clientAuth, final boolean failureExpected,
final boolean serverInitEngine)
throws SSLException, InterruptedException {
throws Exception {
serverSslCtx =
wrapContext(param, SslContextBuilder.forServer(serverKMF)
.protocols(param.protocols())
@ -907,12 +906,12 @@ public abstract class SSLEngineTest {
}
});
serverChannel = sb.bind(new InetSocketAddress(0)).sync().channel();
serverChannel = sb.bind(new InetSocketAddress(0)).get();
int port = ((InetSocketAddress) serverChannel.localAddress()).getPort();
ChannelFuture ccf = cb.connect(new InetSocketAddress(NetUtil.LOCALHOST, port));
Future<Channel> ccf = cb.connect(new InetSocketAddress(NetUtil.LOCALHOST, port));
assertTrue(ccf.awaitUninterruptibly().isSuccess());
clientChannel = ccf.channel();
clientChannel = ccf.get();
}
private static void rethrowIfNotNull(Throwable error) {
@ -960,7 +959,7 @@ public abstract class SSLEngineTest {
File serverKeyFile,
File clientTrustCrtFile,
final boolean failureExpected)
throws SSLException, InterruptedException {
throws Exception {
final String expectedHost = "localhost";
serverSslCtx = wrapContext(param, SslContextBuilder.forServer(serverCrtFile, serverKeyFile, null)
.sslProvider(sslServerProvider())
@ -1039,14 +1038,13 @@ public abstract class SSLEngineTest {
protected void initChannel(Channel ch) throws Exception {
ch.config().setAllocator(new TestByteBufAllocator(ch.config().getAllocator(), param.type));
ChannelPipeline p = ch.pipeline();
InetSocketAddress remoteAddress = (InetSocketAddress) serverChannel.localAddress();
SslHandler sslHandler = !param.delegate ?
clientSslCtx.newHandler(ch.alloc(), expectedHost, 0) :
clientSslCtx.newHandler(ch.alloc(), expectedHost, 0, delegatingExecutor);
SSLParameters parameters = sslHandler.engine().getSSLParameters();
if (SslUtils.isValidHostNameForSNI(expectedHost)) {
if (isValidHostNameForSNI(expectedHost)) {
assertEquals(1, parameters.getServerNames().size());
assertEquals(new SNIHostName(expectedHost), parameters.getServerNames().get(0));
}
@ -1093,21 +1091,21 @@ public abstract class SSLEngineTest {
}
});
serverChannel = sb.bind(new InetSocketAddress(expectedHost, 0)).sync().channel();
serverChannel = sb.bind(new InetSocketAddress(expectedHost, 0)).get();
final int port = ((InetSocketAddress) serverChannel.localAddress()).getPort();
ChannelFuture ccf = cb.connect(new InetSocketAddress(expectedHost, port));
Future<Channel> ccf = cb.connect(new InetSocketAddress(expectedHost, port));
assertTrue(ccf.awaitUninterruptibly().isSuccess());
clientChannel = ccf.channel();
clientChannel = ccf.get();
return clientWritePromise;
}
private void mySetupMutualAuth(SSLEngineTestParam param, File keyFile, File crtFile, String keyPassword)
throws SSLException, InterruptedException {
throws Exception {
mySetupMutualAuth(param, crtFile, keyFile, crtFile, keyPassword, crtFile, keyFile, crtFile, keyPassword);
}
private void verifySSLSessionForMutualAuth(
private static void verifySSLSessionForMutualAuth(
SSLEngineTestParam param, SSLSession session, File certFile, String principalName)
throws Exception {
InputStream in = null;
@ -1150,7 +1148,7 @@ public abstract class SSLEngineTest {
private void mySetupMutualAuth(final SSLEngineTestParam param,
File servertTrustCrtFile, File serverKeyFile, final File serverCrtFile, String serverKeyPassword,
File clientTrustCrtFile, File clientKeyFile, final File clientCrtFile, String clientKeyPassword)
throws InterruptedException, SSLException {
throws Exception {
serverSslCtx =
wrapContext(param, SslContextBuilder.forServer(serverCrtFile, serverKeyFile, serverKeyPassword)
.sslProvider(sslServerProvider())
@ -1261,12 +1259,12 @@ public abstract class SSLEngineTest {
}
});
serverChannel = sb.bind(new InetSocketAddress(0)).sync().channel();
serverChannel = sb.bind(new InetSocketAddress(0)).get();
int port = ((InetSocketAddress) serverChannel.localAddress()).getPort();
ChannelFuture ccf = cb.connect(new InetSocketAddress(NetUtil.LOCALHOST, port));
Future<Channel> ccf = cb.connect(new InetSocketAddress(NetUtil.LOCALHOST, port));
assertTrue(ccf.awaitUninterruptibly().isSuccess());
clientChannel = ccf.channel();
clientChannel = ccf.get();
}
protected void runTest(String expectedApplicationProtocol) throws Exception {
@ -1512,7 +1510,7 @@ public abstract class SSLEngineTest {
}
});
serverChannel = sb.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
serverChannel = sb.bind(new InetSocketAddress(0)).get();
clientSslCtx = wrapContext(param, SslContextBuilder.forClient()
// OpenSslEngine doesn't support renegotiation on client side
@ -1570,9 +1568,9 @@ public abstract class SSLEngineTest {
}
});
ChannelFuture ccf = cb.connect(serverChannel.localAddress());
Future<Channel> ccf = cb.connect(serverChannel.localAddress());
assertTrue(ccf.syncUninterruptibly().isSuccess());
clientChannel = ccf.channel();
clientChannel = ccf.get();
serverLatch.await();
ssc.delete();
@ -1769,13 +1767,13 @@ public abstract class SSLEngineTest {
}
protected void setupHandlers(SSLEngineTestParam param, ApplicationProtocolConfig apn)
throws InterruptedException, SSLException, CertificateException {
throws Exception {
setupHandlers(param, apn, apn);
}
protected void setupHandlers(SSLEngineTestParam param,
ApplicationProtocolConfig serverApn, ApplicationProtocolConfig clientApn)
throws InterruptedException, SSLException, CertificateException {
throws Exception {
SelfSignedCertificate ssc = new SelfSignedCertificate();
try {
@ -1816,7 +1814,7 @@ public abstract class SSLEngineTest {
protected void setupHandlers(final BufferType type, final boolean delegate,
SslContext serverCtx, SslContext clientCtx)
throws InterruptedException, SSLException, CertificateException {
throws Exception {
serverSslCtx = serverCtx;
clientSslCtx = clientCtx;
@ -1889,11 +1887,11 @@ public abstract class SSLEngineTest {
}
});
serverChannel = sb.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
serverChannel = sb.bind(new InetSocketAddress(0)).get();
ChannelFuture ccf = cb.connect(serverChannel.localAddress());
Future<Channel> ccf = cb.connect(serverChannel.localAddress());
assertTrue(ccf.syncUninterruptibly().isSuccess());
clientChannel = ccf.channel();
clientChannel = ccf.get();
}
@MethodSource("newTestParams")
@ -1961,8 +1959,8 @@ public abstract class SSLEngineTest {
// See https://bugs.openjdk.java.net/browse/JDK-8241039
assertTrue(PlatformDependent.javaVersion() >= 15);
assertEquals(2, peerCertificates.length);
for (int i = 0; i < peerCertificates.length; i++) {
if (peerCertificates[i] == null) {
for (Certificate peerCertificate : peerCertificates) {
if (peerCertificate == null) {
promise.setFailure(
new IllegalStateException("Certificate in chain is null"));
return;
@ -1978,7 +1976,7 @@ public abstract class SSLEngineTest {
});
serverConnectedChannel = ch;
}
}).bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
}).bind(new InetSocketAddress(0)).get();
// We create a new chain for certificates which contains 2 certificates
ByteArrayOutputStream chainStream = new ByteArrayOutputStream();
@ -2003,7 +2001,7 @@ public abstract class SSLEngineTest {
ch.pipeline().addLast(new SslHandler(wrapEngine(clientSslCtx.newEngine(ch.alloc()))));
}
}).connect(serverChannel.localAddress()).syncUninterruptibly().channel();
}).connect(serverChannel.localAddress()).get();
promise.syncUninterruptibly();
@ -2150,7 +2148,7 @@ public abstract class SSLEngineTest {
}
}
private String[] nonContiguousProtocols(SslProvider provider) {
private static String[] nonContiguousProtocols(SslProvider provider) {
if (provider != null) {
// conscrypt not correctly filters out TLSv1 and TLSv1.1 which is required now by the JDK.
// https://github.com/google/conscrypt/issues/1013
@ -2971,7 +2969,7 @@ public abstract class SSLEngineTest {
// It's not allowed to set only PROTOCOL_SSL_V2_HELLO if using JDK SSLEngine.
return;
}
server.setEnabledProtocols(supported.toArray(new String[0]));
server.setEnabledProtocols(supported.toArray(EmptyArrays.EMPTY_STRINGS));
assertEquals(supported, new HashSet<>(Arrays.asList(server.getEnabledProtocols())));
server.setEnabledProtocols(server.getSupportedProtocols());
}
@ -3069,10 +3067,8 @@ public abstract class SSLEngineTest {
.ciphers(cipherList).build());
server = wrapEngine(serverSslCtx.newEngine(UnpooledByteBufAllocator.DEFAULT));
fail();
} catch (IllegalArgumentException expected) {
// expected when invalid cipher is used.
} catch (SSLException expected) {
// expected when invalid cipher is used.
} catch (IllegalArgumentException | SSLException expected) {
// Expected when invalid cipher is used.
} finally {
cert.delete();
cleanupServerSslEngine(server);
@ -4025,7 +4021,7 @@ public abstract class SSLEngineTest {
});
serverConnectedChannel = ch;
}
}).bind(new InetSocketAddress(0)).sync().channel();
}).bind(new InetSocketAddress(0)).get();
int port = ((InetSocketAddress) serverChannel.localAddress()).getPort();

View File

@ -142,7 +142,7 @@ final class SniClientJava8TestUtil {
}
});
}
}).bind(address).syncUninterruptibly().channel();
}).bind(address).get();
sslClientContext = SslContextBuilder.forClient().trustManager(InsecureTrustManagerFactory.INSTANCE)
.sslProvider(sslClientProvider).build();
@ -150,8 +150,7 @@ final class SniClientJava8TestUtil {
SslHandler sslHandler = new SslHandler(
sslClientContext.newEngine(ByteBufAllocator.DEFAULT, sniHost, -1));
Bootstrap cb = new Bootstrap();
cc = cb.group(group).channel(LocalChannel.class).handler(sslHandler)
.connect(address).syncUninterruptibly().channel();
cc = cb.group(group).channel(LocalChannel.class).handler(sslHandler).connect(address).get();
promise.syncUninterruptibly();
sslHandler.handshakeFuture().syncUninterruptibly();

View File

@ -20,8 +20,8 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.ByteBufAllocator;
import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.MultithreadEventLoopGroup;
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.LocalHandler;
@ -29,7 +29,6 @@ import io.netty.channel.local.LocalServerChannel;
import io.netty.handler.ssl.util.SelfSignedCertificate;
import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.Promise;
import io.netty.util.internal.PlatformDependent;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.api.function.Executable;
@ -39,7 +38,6 @@ import org.junit.jupiter.params.provider.MethodSource;
import javax.net.ssl.KeyManagerFactory;
import javax.net.ssl.SSLException;
import javax.net.ssl.TrustManagerFactory;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@ -131,7 +129,7 @@ public class SniClientTest {
return finalContext;
}));
}
}).bind(address).syncUninterruptibly().channel();
}).bind(address).get();
TrustManagerFactory tmf = SniClientJava8TestUtil.newSniX509TrustmanagerFactory(sniHostName);
sslClientContext = SslContextBuilder.forClient().trustManager(tmf)
@ -140,8 +138,7 @@ public class SniClientTest {
SslHandler handler = new SslHandler(
sslClientContext.newEngine(ByteBufAllocator.DEFAULT, sniHostName, -1));
cc = cb.group(group).channel(LocalChannel.class).handler(handler)
.connect(address).syncUninterruptibly().channel();
cc = cb.group(group).channel(LocalChannel.class).handler(handler).connect(address).get();
assertEquals(sniHostName, promise.syncUninterruptibly().getNow());
// After we are done with handshaking getHandshakeSession() should return null.

View File

@ -16,33 +16,18 @@
package io.netty.handler.ssl;
import java.io.File;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import javax.net.ssl.SSLEngine;
import javax.net.ssl.SSLException;
import io.netty.util.concurrent.Future;
import io.netty.bootstrap.Bootstrap;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.embedded.EmbeddedChannel;
import io.netty.channel.local.LocalAddress;
import io.netty.channel.local.LocalChannel;
@ -59,8 +44,8 @@ import io.netty.util.DomainNameMappingBuilder;
import io.netty.util.Mapping;
import io.netty.util.ReferenceCountUtil;
import io.netty.util.ReferenceCounted;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.Promise;
import io.netty.util.internal.ResourcesUtil;
import io.netty.util.internal.StringUtil;
import org.hamcrest.CoreMatchers;
@ -69,18 +54,28 @@ import org.junit.jupiter.api.function.Executable;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.MethodSource;
import javax.net.ssl.SSLEngine;
import javax.net.ssl.SSLException;
import java.io.File;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import static java.util.Objects.requireNonNull;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.CoreMatchers.nullValue;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assumptions.assumeTrue;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assumptions.assumeTrue;
import static org.mockito.Mockito.mock;
public class SniHandlerTest {
@ -456,11 +451,8 @@ public class SniHandlerTest {
}
});
serverChannel = sb.bind(new InetSocketAddress(0)).sync().channel();
ChannelFuture ccf = cb.connect(serverChannel.localAddress());
assertTrue(ccf.awaitUninterruptibly().isSuccess());
clientChannel = ccf.channel();
serverChannel = sb.bind(new InetSocketAddress(0)).get();
clientChannel = cb.connect(serverChannel.localAddress()).get();
assertTrue(serverApnDoneLatch.await(5, TimeUnit.SECONDS));
assertTrue(clientApnDoneLatch.await(5, TimeUnit.SECONDS));
@ -558,7 +550,7 @@ public class SniHandlerTest {
protected void initChannel(Channel ch) throws Exception {
ch.pipeline().addFirst(handler);
}
}).bind(address).syncUninterruptibly().channel();
}).bind(address).get();
sslContext = SslContextBuilder.forClient().sslProvider(provider)
.trustManager(InsecureTrustManagerFactory.INSTANCE).build();
@ -566,7 +558,7 @@ public class SniHandlerTest {
Bootstrap cb = new Bootstrap();
cc = cb.group(group).channel(LocalChannel.class).handler(new SslHandler(
sslContext.newEngine(ByteBufAllocator.DEFAULT, sniHost, -1)))
.connect(address).syncUninterruptibly().channel();
.connect(address).get();
cc.writeAndFlush(Unpooled.wrappedBuffer("Hello, World!".getBytes()))
.syncUninterruptibly();
@ -643,7 +635,7 @@ public class SniHandlerTest {
testWithFragmentSize(provider, 50);
}
private void testWithFragmentSize(SslProvider provider, final int maxFragmentSize) throws Exception {
private static void testWithFragmentSize(SslProvider provider, final int maxFragmentSize) throws Exception {
final String sni = "netty.io";
SelfSignedCertificate cert = new SelfSignedCertificate();
final SslContext context = SslContextBuilder.forServer(cert.key(), cert.cert())

View File

@ -162,7 +162,7 @@ public class SslErrorTest {
}
});
}
}).bind(0).sync().channel();
}).bind(0).get();
clientChannel = new Bootstrap().group(group)
.channel(NioSocketChannel.class)
@ -183,7 +183,7 @@ public class SslErrorTest {
}
});
}
}).connect(serverChannel.localAddress()).syncUninterruptibly().channel();
}).connect(serverChannel.localAddress()).get();
// Block until we received the correct exception
promise.syncUninterruptibly();
} finally {
@ -298,8 +298,7 @@ public class SslErrorTest {
}
}
Throwable error = new AssertionError("message not contains any of '"
+ Arrays.toString(messageParts) + "': " + message);
error.initCause(cause);
+ Arrays.toString(messageParts) + "': " + message, cause);
promise.setFailure(error);
}

View File

@ -30,8 +30,8 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPromise;
import io.netty.channel.DefaultChannelId;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.embedded.EmbeddedChannel;
import io.netty.channel.local.LocalAddress;
@ -51,27 +51,31 @@ import io.netty.util.AbstractReferenceCounted;
import io.netty.util.IllegalReferenceCountException;
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.GenericFutureListener;
import io.netty.util.concurrent.ImmediateEventExecutor;
import io.netty.util.concurrent.ImmediateExecutor;
import io.netty.util.concurrent.Promise;
import io.netty.util.internal.EmptyArrays;
import org.hamcrest.CoreMatchers;
import org.hamcrest.Matchers;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.api.function.Executable;
import org.junit.jupiter.api.Disabled;
import javax.net.ssl.SSLContext;
import javax.net.ssl.SSLEngine;
import javax.net.ssl.SSLException;
import javax.net.ssl.SSLProtocolException;
import javax.net.ssl.X509ExtendedTrustManager;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.nio.channels.ClosedChannelException;
import java.security.NoSuchAlgorithmException;
import java.security.cert.CertificateException;
import java.security.cert.X509Certificate;
import java.util.Collections;
import java.util.Queue;
import java.util.concurrent.BlockingQueue;
@ -88,14 +92,12 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import javax.net.ssl.SSLContext;
import javax.net.ssl.SSLEngine;
import javax.net.ssl.SSLException;
import javax.net.ssl.SSLProtocolException;
import javax.net.ssl.X509ExtendedTrustManager;
import static io.netty.buffer.Unpooled.wrappedBuffer;
import static org.hamcrest.CoreMatchers.*;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.CoreMatchers.not;
import static org.hamcrest.CoreMatchers.nullValue;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
@ -111,7 +113,7 @@ public class SslHandlerTest {
@Test
@Timeout(value = 5000, unit = TimeUnit.MILLISECONDS)
public void testNonApplicationDataFailureFailsQueuedWrites() throws NoSuchAlgorithmException, InterruptedException {
public void testNonApplicationDataFailureFailsQueuedWrites() throws Exception {
final CountDownLatch writeLatch = new CountDownLatch(1);
final Queue<ChannelPromise> writesToFail = new ConcurrentLinkedQueue<ChannelPromise>();
SSLEngine engine = newClientModeSSLEngine();
@ -139,7 +141,7 @@ public class SslHandlerTest {
try {
final CountDownLatch writeCauseLatch = new CountDownLatch(1);
final AtomicReference<Throwable> failureRef = new AtomicReference<Throwable>();
ch.write(Unpooled.wrappedBuffer(new byte[]{1})).addListener((ChannelFutureListener) future -> {
ch.write(wrappedBuffer(new byte[]{1})).addListener((ChannelFutureListener) future -> {
failureRef.compareAndSet(null, future.cause());
writeCauseLatch.countDown();
});
@ -488,8 +490,8 @@ public class SslHandlerTest {
.channel(NioServerSocketChannel.class)
.childHandler(newHandler(SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey()).build(),
serverPromise));
sc = serverBootstrap.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
cc = bootstrap.connect(sc.localAddress()).syncUninterruptibly().channel();
sc = serverBootstrap.bind(new InetSocketAddress(0)).get();
cc = bootstrap.connect(sc.localAddress()).get();
serverPromise.syncUninterruptibly();
clientPromise.syncUninterruptibly();
@ -645,8 +647,8 @@ public class SslHandlerTest {
}
});
serverChannel = sb.bind(new LocalAddress("SslHandlerTest")).sync().channel();
clientChannel = cb.connect(serverChannel.localAddress()).sync().channel();
serverChannel = sb.bind(new LocalAddress("SslHandlerTest")).get();
clientChannel = cb.connect(serverChannel.localAddress()).get();
latch.await();
SslCompletionEvent evt = (SslCompletionEvent) events.take();
@ -712,7 +714,7 @@ public class SslHandlerTest {
}
});
}
}).bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
}).bind(new InetSocketAddress(0)).get();
cc = new Bootstrap()
.group(group)
@ -722,7 +724,7 @@ public class SslHandlerTest {
protected void initChannel(Channel ch) throws Exception {
ch.pipeline().addLast(sslClientCtx.newHandler(ch.alloc()));
}
}).connect(sc.localAddress()).syncUninterruptibly().channel();
}).connect(sc.localAddress()).get();
// We first write a ReadOnlyBuffer because SslHandler will attempt to take the first buffer and append to it
// until there is no room, or the aggregation size threshold is exceeded. We want to verify that we don't
@ -773,7 +775,7 @@ public class SslHandlerTest {
ch.pipeline().addLast(sslServerCtx.newHandler(ch.alloc()));
}
});
sc = sb.bind(address).syncUninterruptibly().channel();
sc = sb.bind(address).get();
final AtomicReference<SslHandler> sslHandlerRef = new AtomicReference<>();
Bootstrap b = new Bootstrap()
@ -791,7 +793,7 @@ public class SslHandlerTest {
ch.pipeline().addLast(handler);
}
});
cc = b.connect(sc.localAddress()).syncUninterruptibly().channel();
cc = b.connect(sc.localAddress()).get();
SslHandler handler = sslHandlerRef.get();
handler.handshakeFuture().awaitUninterruptibly();
assertFalse(handler.handshakeFuture().isSuccess());
@ -844,7 +846,7 @@ public class SslHandlerTest {
.group(group)
.channel(NioServerSocketChannel.class)
.childHandler(new ChannelHandler() { })
.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
.bind(new InetSocketAddress(0)).get();
cc = new Bootstrap()
.group(group)
@ -868,10 +870,10 @@ public class SslHandlerTest {
}
});
}
}).connect(sc.localAddress()).addListener((ChannelFutureListener) future -> {
}).connect(sc.localAddress()).addListener((GenericFutureListener<Future<Channel>>) future -> {
// Write something to trigger the handshake before fireChannelActive is called.
future.channel().writeAndFlush(wrappedBuffer(new byte [] { 1, 2, 3, 4 }));
}).syncUninterruptibly().channel();
future.get().writeAndFlush(wrappedBuffer(new byte [] { 1, 2, 3, 4 }));
}).get();
// Ensure there is no AssertionError thrown by having the handshake failed by the writeAndFlush(...) before
// channelActive(...) was called. Let's first wait for the activeLatch countdown to happen and after this
@ -881,8 +883,7 @@ public class SslHandlerTest {
if (error != null) {
throw error;
}
assertThat(sslHandler.handshakeFuture().await().cause(),
CoreMatchers.instanceOf(SSLException.class));
assertThat(sslHandler.handshakeFuture().await().cause(), instanceOf(SSLException.class));
} finally {
if (cc != null) {
cc.close().syncUninterruptibly();
@ -925,9 +926,9 @@ public class SslHandlerTest {
.group(group)
.channel(NioServerSocketChannel.class)
.childHandler(new ChannelHandler() { })
.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
.bind(new InetSocketAddress(0)).get();
ChannelFuture future = new Bootstrap()
Future<Channel> future = new Bootstrap()
.group(group)
.channel(NioSocketChannel.class)
.handler(new ChannelInitializer<Channel>() {
@ -945,15 +946,15 @@ public class SslHandlerTest {
}
}).connect(sc.localAddress());
if (!startTls) {
future.addListener((ChannelFutureListener) future1 -> {
future.addListener((GenericFutureListener<Future<Channel>>) future1 -> {
// Write something to trigger the handshake before fireChannelActive is called.
future1.channel().writeAndFlush(wrappedBuffer(new byte [] { 1, 2, 3, 4 }));
future1.getNow().writeAndFlush(wrappedBuffer(new byte [] { 1, 2, 3, 4 }));
});
}
cc = future.syncUninterruptibly().channel();
cc = future.get();
Throwable cause = sslHandler.handshakeFuture().await().cause();
assertThat(cause, CoreMatchers.instanceOf(SSLException.class));
assertThat(cause, instanceOf(SSLException.class));
assertThat(cause.getMessage(), containsString("timed out"));
} finally {
if (cc != null) {
@ -1012,9 +1013,9 @@ public class SslHandlerTest {
.group(group)
.channel(NioServerSocketChannel.class)
.childHandler(serverSslHandler)
.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
.bind(new InetSocketAddress(0)).get();
ChannelFuture future = new Bootstrap()
Future<Channel> future = new Bootstrap()
.group(group)
.channel(NioSocketChannel.class)
.handler(new ChannelInitializer<Channel>() {
@ -1023,7 +1024,7 @@ public class SslHandlerTest {
ch.pipeline().addLast(clientSslHandler);
}
}).connect(sc.localAddress());
cc = future.syncUninterruptibly().channel();
cc = future.get();
assertTrue(clientSslHandler.handshakeFuture().await().isSuccess());
assertTrue(serverSslHandler.handshakeFuture().await().isSuccess());
@ -1084,9 +1085,9 @@ public class SslHandlerTest {
.group(group)
.channel(NioServerSocketChannel.class)
.childHandler(serverSslHandler)
.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
.bind(new InetSocketAddress(0)).get();
ChannelFuture future = new Bootstrap()
Future<Channel> future = new Bootstrap()
.group(group)
.channel(NioSocketChannel.class)
.handler(new ChannelInitializer<Channel>() {
@ -1095,7 +1096,7 @@ public class SslHandlerTest {
ch.pipeline().addLast(clientSslHandler);
}
}).connect(sc.localAddress());
cc = future.syncUninterruptibly().channel();
cc = future.get();
if (client) {
Throwable cause = clientSslHandler.handshakeFuture().await().cause();
@ -1209,13 +1210,13 @@ public class SslHandlerTest {
}
}
ctx.writeAndFlush(Unpooled.wrappedBuffer(bytes));
ctx.writeAndFlush(wrappedBuffer(bytes));
}
}
});
}
})
.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
.bind(new InetSocketAddress(0)).get();
InetSocketAddress serverAddr = (InetSocketAddress) sc.localAddress();
testSessionTickets(serverAddr, group, sslClientCtx, bytes, false);
@ -1242,7 +1243,7 @@ public class SslHandlerTest {
final SslHandler clientSslHandler = sslClientCtx.newHandler(UnpooledByteBufAllocator.DEFAULT,
serverAddress.getAddress().getHostAddress(), serverAddress.getPort());
ChannelFuture future = new Bootstrap()
Future<Channel> future = new Bootstrap()
.group(group)
.channel(NioSocketChannel.class)
.handler(new ChannelInitializer<Channel>() {
@ -1265,7 +1266,7 @@ public class SslHandlerTest {
});
}
}).connect(serverAddress);
cc = future.syncUninterruptibly().channel();
cc = future.get();
assertTrue(clientSslHandler.handshakeFuture().sync().isSuccess());
@ -1279,7 +1280,7 @@ public class SslHandlerTest {
Object obj = queue.take();
if (obj instanceof ByteBuf) {
ByteBuf buffer = (ByteBuf) obj;
ByteBuf expected = Unpooled.wrappedBuffer(bytes);
ByteBuf expected = wrappedBuffer(bytes);
try {
assertEquals(expected, buffer);
} finally {
@ -1380,8 +1381,8 @@ public class SslHandlerTest {
});
}
})
.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
final ChannelFuture future = new Bootstrap()
.bind(new InetSocketAddress(0)).get();
Channel channel = new Bootstrap()
.group(group)
.channel(NioSocketChannel.class)
.handler(new ChannelInitializer<Channel>() {
@ -1389,10 +1390,9 @@ public class SslHandlerTest {
protected void initChannel(Channel ch) {
ch.pipeline().addLast(clientSslHandler);
}
}).connect(sc.localAddress());
future.syncUninterruptibly();
}).connect(sc.localAddress()).get();
clientSslHandler.handshakeFuture().addListener((FutureListener<Channel>) f -> {
future.channel().close();
channel.close();
});
assertFalse(clientSslHandler.handshakeFuture().await().isSuccess());
assertFalse(serverSslHandler.handshakeFuture().await().isSuccess());
@ -1465,8 +1465,8 @@ public class SslHandlerTest {
.sslProvider(provider).build();
EventLoopGroup group = new MultithreadEventLoopGroup(NioHandler.newFactory());
Channel sc = null;
Channel cc = null;
Channel sc;
Channel cc;
final SslHandler clientSslHandler = sslClientCtx.newHandler(UnpooledByteBufAllocator.DEFAULT);
final SslHandler serverSslHandler = sslServerCtx.newHandler(UnpooledByteBufAllocator.DEFAULT);
@ -1500,9 +1500,9 @@ public class SslHandlerTest {
ch.pipeline().addLast(new SslEventHandler(serverEvent));
}
})
.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
.bind(new InetSocketAddress(0)).get();
ChannelFuture future = new Bootstrap()
cc = new Bootstrap()
.group(group)
.channel(NioSocketChannel.class)
.handler(new ChannelInitializer<Channel>() {
@ -1511,8 +1511,7 @@ public class SslHandlerTest {
ch.pipeline().addLast(clientSslHandler);
ch.pipeline().addLast(new SslEventHandler(clientEvent));
}
}).connect(sc.localAddress());
cc = future.syncUninterruptibly().channel();
}).connect(sc.localAddress()).get();
Throwable clientCause = clientSslHandler.handshakeFuture().await().cause();
assertThat(clientCause, CoreMatchers.<Throwable>instanceOf(SSLException.class));
@ -1562,7 +1561,7 @@ public class SslHandlerTest {
testHandshakeEvents(SslProvider.OPENSSL, SslProtocols.TLS_v1_3);
}
private void testHandshakeEvents(SslProvider provider, String protocol) throws Exception {
private static void testHandshakeEvents(SslProvider provider, String protocol) throws Exception {
final SslContext sslClientCtx = SslContextBuilder.forClient()
.trustManager(InsecureTrustManagerFactory.INSTANCE)
.protocols(protocol)
@ -1591,7 +1590,7 @@ public class SslHandlerTest {
ch.pipeline().addLast(new SslHandshakeCompletionEventHandler(serverCompletionEvents));
}
})
.bind(new InetSocketAddress(0)).syncUninterruptibly().channel();
.bind(new InetSocketAddress(0)).get();
Bootstrap bs = new Bootstrap()
.group(group)
@ -1606,8 +1605,8 @@ public class SslHandlerTest {
})
.remoteAddress(sc.localAddress());
Channel cc1 = bs.connect().sync().channel();
Channel cc2 = bs.connect().sync().channel();
Channel cc1 = bs.connect().get();
Channel cc2 = bs.connect().get();
// We expect 4 events as we have 2 connections and for each connection there should be one event
// on the server-side and one on the client-side.
@ -1645,5 +1644,5 @@ public class SslHandlerTest {
completionEvents.add((SslHandshakeCompletionEvent) evt);
}
}
};
}
}

View File

@ -25,8 +25,8 @@ import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.MultithreadEventLoopGroup;
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.LocalHandler;
@ -47,14 +47,13 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.api.function.Executable;
import javax.net.ssl.SSLHandshakeException;
import java.net.SocketAddress;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
import javax.net.ssl.SSLHandshakeException;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
@ -399,29 +398,25 @@ public class OcspTest {
}
private static Channel newServer(EventLoopGroup group, SocketAddress address,
SslContext context, byte[] response, ChannelHandler handler) {
SslContext context, byte[] response, ChannelHandler handler) throws Exception {
ServerBootstrap bootstrap = new ServerBootstrap()
.channel(LocalServerChannel.class)
.group(group)
.childHandler(newServerHandler(context, response, handler));
return bootstrap.bind(address)
.syncUninterruptibly()
.channel();
return bootstrap.bind(address).get();
}
private static Channel newClient(EventLoopGroup group, SocketAddress address,
SslContext context, OcspClientCallback callback, ChannelHandler handler) {
SslContext context, OcspClientCallback callback, ChannelHandler handler) throws Exception {
Bootstrap bootstrap = new Bootstrap()
.channel(LocalChannel.class)
.group(group)
.handler(newClientHandler(context, callback, handler));
return bootstrap.connect(address)
.syncUninterruptibly()
.channel();
return bootstrap.connect(address).get();
}
private static ChannelHandler newServerHandler(final SslContext context,

View File

@ -20,7 +20,6 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
@ -98,8 +97,8 @@ public class FileRegionThrottleTest {
ch.pipeline().addLast(gtsh);
}
});
Channel sc = bs.bind(0).sync().channel();
Channel cc = clientConnect(sc.localAddress(), new ReadHandler(latch)).channel();
Channel sc = bs.bind(0).get();
Channel cc = clientConnect(sc.localAddress(), new ReadHandler(latch));
long start = TrafficCounter.milliSecondFromNano();
cc.writeAndFlush(Unpooled.copiedBuffer("send-file\n", CharsetUtil.US_ASCII)).sync();
@ -110,7 +109,7 @@ public class FileRegionThrottleTest {
cc.close().sync();
}
private ChannelFuture clientConnect(final SocketAddress server, final ReadHandler readHandler) throws Exception {
private Channel clientConnect(final SocketAddress server, final ReadHandler readHandler) throws Exception {
Bootstrap bc = new Bootstrap();
bc.group(group).channel(NioSocketChannel.class).handler(new ChannelInitializer<SocketChannel>() {
@Override
@ -118,7 +117,7 @@ public class FileRegionThrottleTest {
ch.pipeline().addLast(readHandler);
}
});
return bc.connect(server).sync();
return bc.connect(server).get();
}
private static final class MessageDecoder implements ChannelHandler {
@ -128,7 +127,7 @@ public class FileRegionThrottleTest {
ByteBuf buf = (ByteBuf) msg;
String message = buf.toString(Charset.defaultCharset());
buf.release();
if (message.equals("send-file")) {
if ("send-file".equals(message)) {
RandomAccessFile raf = new RandomAccessFile(tmp, "r");
ctx.channel().writeAndFlush(new DefaultFileRegion(raf.getChannel(), 0, tmp.length()));
}
@ -138,7 +137,7 @@ public class FileRegionThrottleTest {
private static final class ReadHandler implements ChannelHandler {
private long bytesTransferred;
private CountDownLatch latch;
private final CountDownLatch latch;
ReadHandler(CountDownLatch latch) {
this.latch = latch;

View File

@ -16,26 +16,26 @@
package io.netty.handler.traffic;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import io.netty.channel.ChannelHandler;
import io.netty.channel.MultithreadEventLoopGroup;
import io.netty.channel.local.LocalHandler;
import io.netty.bootstrap.Bootstrap;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
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.util.Attribute;
import io.netty.util.CharsetUtil;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Test;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
@ -72,7 +72,7 @@ public class TrafficShapingHandlerTest {
}
}
private void testHandlerRemove0(final AbstractTrafficShapingHandler trafficHandler)
private static void testHandlerRemove0(final AbstractTrafficShapingHandler trafficHandler)
throws Exception {
Channel svrChannel = null;
Channel ch = null;
@ -91,7 +91,7 @@ public class TrafficShapingHandlerTest {
}
});
final LocalAddress svrAddr = new LocalAddress("foo");
svrChannel = serverBootstrap.bind(svrAddr).sync().channel();
svrChannel = serverBootstrap.bind(svrAddr).get();
Bootstrap bootstrap = new Bootstrap();
bootstrap.channel(LocalChannel.class).group(GROUP).handler(new ChannelInitializer<Channel>() {
@Override
@ -99,7 +99,7 @@ public class TrafficShapingHandlerTest {
ch.pipeline().addLast("traffic-shaping", trafficHandler);
}
});
ch = bootstrap.connect(svrAddr).sync().channel();
ch = bootstrap.connect(svrAddr).get();
Attribute<Runnable> attr = ch.attr(AbstractTrafficShapingHandler.REOPEN_TASK);
assertNull(attr.get());
ch.writeAndFlush(Unpooled.wrappedBuffer("foo".getBytes(CharsetUtil.UTF_8)));

View File

@ -72,8 +72,7 @@ public class EpollSocketChannelBenchmark extends AbstractMicrobenchmark {
}
})
.bind(0)
.sync()
.channel();
.get();
chan = new Bootstrap()
.channel(EpollSocketChannel.class)
.handler(new ChannelInitializer<Channel>() {
@ -116,8 +115,7 @@ public class EpollSocketChannelBenchmark extends AbstractMicrobenchmark {
})
.group(group)
.connect(serverChan.localAddress())
.sync()
.channel();
.get();
abyte = chan.alloc().directBuffer(1);
abyte.writeByte('a');

View File

@ -56,6 +56,7 @@ import io.netty.util.concurrent.EventExecutor;
import io.netty.util.concurrent.FastThreadLocal;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.FutureListener;
import io.netty.util.concurrent.GenericFutureListener;
import io.netty.util.concurrent.Promise;
import io.netty.util.internal.EmptyArrays;
import io.netty.util.internal.PlatformDependent;
@ -145,7 +146,7 @@ public class DnsNameResolver extends InetNameResolver {
List<String> list = PlatformDependent.isWindows()
? getSearchDomainsHack()
: UnixResolverDnsServerAddressStreamProvider.parseEtcResolverSearchDomains();
searchDomains = list.toArray(new String[0]);
searchDomains = list.toArray(EmptyArrays.EMPTY_STRINGS);
} catch (Exception ignore) {
// Failed to get the system name search domain list.
searchDomains = EmptyArrays.EMPTY_STRINGS;
@ -478,34 +479,27 @@ public class DnsNameResolver extends InetNameResolver {
@Override
protected void initChannel(DatagramChannel ch) {
ch.pipeline().addLast(DATAGRAM_ENCODER, DATAGRAM_DECODER, responseHandler);
ch.closeFuture().addListener(closeFuture -> {
resolveCache.clear();
cnameCache.clear();
authoritativeDnsServerCache.clear();
});
}
});
b.option(ChannelOption.RCVBUF_ALLOCATOR, new FixedRecvByteBufAllocator(maxPayloadSize));
channelFuture = responseHandler.channelActivePromise;
final ChannelFuture future;
if (localAddress == null) {
future = b.register();
} else {
future = b.bind(localAddress);
}
Throwable cause = future.cause();
if (cause != null) {
if (cause instanceof RuntimeException) {
throw (RuntimeException) cause;
}
if (cause instanceof Error) {
throw (Error) cause;
try {
ch = b.createUnregistered();
ChannelFuture future = localAddress == null ? ch.register() : ch.bind(localAddress);
if (future.cause() != null) {
throw future.cause();
}
} catch (Error | RuntimeException e) {
throw e;
} catch (Throwable cause) {
throw new IllegalStateException("Unable to create / register Channel", cause);
}
ch = future.channel();
ch.config().setRecvByteBufAllocator(new FixedRecvByteBufAllocator(maxPayloadSize));
ch.closeFuture().addListener((ChannelFutureListener) future1 -> {
resolveCache.clear();
cnameCache.clear();
authoritativeDnsServerCache.clear();
});
}
static InternetProtocolFamily preferredAddressType(ResolvedAddressTypes resolvedAddressTypes) {
@ -712,7 +706,7 @@ public class DnsNameResolver extends InetNameResolver {
private static boolean isLocalWindowsHost(String hostname) {
return PlatformDependent.isWindows() &&
(LOCALHOST.equalsIgnoreCase(hostname) ||
(WINDOWS_HOST_NAME != null && WINDOWS_HOST_NAME.equalsIgnoreCase(hostname)));
WINDOWS_HOST_NAME != null && WINDOWS_HOST_NAME.equalsIgnoreCase(hostname));
}
/**
@ -1286,7 +1280,7 @@ public class DnsNameResolver extends InetNameResolver {
.group(executor())
.channelFactory(socketChannelFactory)
.handler(TCP_ENCODER);
bs.connect(res.sender()).addListener((ChannelFutureListener) future -> {
bs.connect(res.sender()).addListener((GenericFutureListener<Future<Channel>>) future -> {
if (!future.isSuccess()) {
if (logger.isDebugEnabled()) {
logger.debug("{} Unable to fallback to TCP [{}]", queryId, future.cause());
@ -1296,7 +1290,7 @@ public class DnsNameResolver extends InetNameResolver {
qCtx.finish(res);
return;
}
final Channel channel = future.channel();
final Channel channel = future.getNow();
Promise<AddressedEnvelope<DnsResponse, InetSocketAddress>> promise =
channel.eventLoop().newPromise();
@ -1341,8 +1335,7 @@ public class DnsNameResolver extends InetNameResolver {
}
});
promise.addListener(
new FutureListener<AddressedEnvelope<DnsResponse, InetSocketAddress>>() {
promise.addListener(new FutureListener<AddressedEnvelope<DnsResponse, InetSocketAddress>>() {
@Override
public void operationComplete(
Future<AddressedEnvelope<DnsResponse, InetSocketAddress>> future) {
@ -1357,7 +1350,7 @@ public class DnsNameResolver extends InetNameResolver {
}
}
});
tcpCtx.query(true, future.channel().newPromise());
tcpCtx.query(true, channel.newPromise());
});
}

View File

@ -17,7 +17,7 @@ package io.netty.testsuite.autobahn;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.PooledByteBufAllocator;
import io.netty.channel.ChannelFuture;
import io.netty.channel.Channel;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.MultithreadEventLoopGroup;
@ -46,9 +46,9 @@ public class AutobahnServer {
.childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
.childHandler(new AutobahnServerInitializer());
ChannelFuture f = b.bind(port).sync();
Channel channel = b.bind(port).get();
System.out.println("Web Socket Server started at port " + port);
f.channel().closeFuture().sync();
channel.closeFuture().sync();
} finally {
bossGroup.shutdownGracefully();
workerGroup.shutdownGracefully();

View File

@ -49,7 +49,7 @@ public final class Http2Server {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new Http2ServerInitializer());
Channel ch = b.bind(port).sync().channel();
Channel ch = b.bind(port).get();
ch.closeFuture().sync();
} finally {

View File

@ -50,7 +50,7 @@ public final class HttpNativeServer {
.handler(new LoggingHandler(LogLevel.INFO))
.childHandler(new HttpNativeServerInitializer());
Channel channel = b.bind(0).sync().channel();
Channel channel = b.bind(0).get();
System.err.println("Server started, will shutdown now.");
channel.close().sync();
serverStartSucess = true;

View File

@ -16,7 +16,7 @@
package io.netty.testsuite.transport;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.ChannelFuture;
import io.netty.channel.Channel;
import io.netty.channel.ChannelHandler;
import io.netty.channel.EventLoop;
import io.netty.channel.EventLoopGroup;
@ -66,9 +66,9 @@ public abstract class AbstractSingleThreadEventLoopTest {
.childHandler(new ChannelHandler() { });
// Not close the Channel to ensure the EventLoop is still shutdown in time.
ChannelFuture cf = serverChannelClass() == LocalServerChannel.class
Future<Channel> cf = serverChannelClass() == LocalServerChannel.class
? b.bind(new LocalAddress("local")) : b.bind(0);
cf.sync().channel();
cf.sync();
Future<?> f = loop.shutdownGracefully(0, 1, TimeUnit.MINUTES);
assertTrue(loop.awaitTermination(600, TimeUnit.MILLISECONDS));

View File

@ -29,11 +29,11 @@ import io.netty.handler.codec.sctp.SctpMessageCompletionHandler;
import io.netty.handler.codec.sctp.SctpOutboundByteStreamHandler;
import io.netty.testsuite.util.TestUtils;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo;
import java.io.IOException;
import java.util.Random;
import java.util.concurrent.atomic.AtomicReference;
import org.junit.jupiter.api.TestInfo;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assumptions.assumeTrue;
@ -92,8 +92,8 @@ public class SctpEchoTest extends AbstractSctpTest {
}
});
Channel sc = sb.bind().sync().channel();
Channel cc = cb.connect(sc.localAddress()).sync().channel();
Channel sc = sb.bind().get();
Channel cc = cb.connect(sc.localAddress()).get();
for (int i = 0; i < data.length;) {
int length = Math.min(random.nextInt(1024 * 64), data.length - i);

View File

@ -20,13 +20,14 @@ import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.testsuite.transport.TestsuitePermutation;
import io.netty.util.CharsetUtil;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.GenericFutureListener;
import io.netty.util.concurrent.ImmediateEventExecutor;
import io.netty.util.concurrent.Promise;
import org.junit.jupiter.api.Test;
@ -90,13 +91,13 @@ public abstract class AbstractSocketReuseFdTest extends AbstractSocketTest {
}
});
ChannelFutureListener listener = future -> {
GenericFutureListener<Future<? super Channel>> listener = future -> {
if (!future.isSuccess()) {
clientDonePromise.tryFailure(future.cause());
}
};
Channel sc = sb.bind().sync().channel();
Channel sc = sb.bind().get();
for (int i = 0; i < numChannels; i++) {
cb.connect(sc.localAddress()).addListener(listener);
}

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