Make Promise not extend Future (#11634)

Motivation:
We wish to separate these two into clearer write/read interfaces.
In particular, we don't want to be able to add listeners to promises, because it makes it easy to add them out of order.
We can't prevent it entirely, because any promise can be freely converted to a future where listeners can be added.
We can, however, discourage this in the API.

Modification:
The Promise interface no longer extends the Future interface.
Numerous changes to make the project compile and its tests run.

Result:
Clearer separation of concerns in the code.
This commit is contained in:
Chris Vest 2021-09-02 10:46:54 +02:00 committed by GitHub
parent 23601902ab
commit 0cb4cc4e49
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
112 changed files with 629 additions and 610 deletions

View File

@ -268,7 +268,7 @@ public abstract class WebSocketClientHandshaker {
promise.setFailure(future.cause());
}
});
return promise;
return promise.asFuture();
}
/**
@ -432,7 +432,7 @@ public abstract class WebSocketClientHandshaker {
} catch (Throwable cause) {
promise.setFailure(cause);
}
return promise;
return promise.asFuture();
}
}

View File

@ -117,7 +117,7 @@ class WebSocketClientProtocolHandshakeHandler implements ChannelHandler {
}, handshakeTimeoutMillis, TimeUnit.MILLISECONDS);
// Cancel the handshake timeout when handshake is finished.
localHandshakePromise.addListener(f -> timeoutFuture.cancel(false));
localHandshakePromise.asFuture().addListener(f -> timeoutFuture.cancel(false));
}
/**
@ -126,6 +126,6 @@ class WebSocketClientProtocolHandshakeHandler implements ChannelHandler {
* @return current handshake future
*/
Future<Void> getHandshakeFuture() {
return handshakePromise;
return handshakePromise.asFuture();
}
}

View File

@ -85,13 +85,14 @@ abstract class WebSocketProtocolHandler extends MessageToMessageDecoder<WebSocke
if (closeStatus == null || !ctx.channel().isActive()) {
return ctx.close();
}
final Future<Void> future = closeSent == null ? write(ctx, new CloseWebSocketFrame(closeStatus)) : closeSent;
final Future<Void> future = closeSent == null ?
write(ctx, new CloseWebSocketFrame(closeStatus)) : closeSent.asFuture();
flush(ctx);
applyCloseSentTimeout(ctx);
Promise<Void> promise = ctx.newPromise();
future.addListener(f -> ctx.close().cascadeTo(promise));
return promise;
return promise.asFuture();
}
@Override
@ -104,7 +105,7 @@ abstract class WebSocketProtocolHandler extends MessageToMessageDecoder<WebSocke
Promise<Void> promise = ctx.newPromise();
closeSent(promise);
ctx.write(msg).cascadeTo(closeSent);
return promise;
return promise.asFuture();
}
return ctx.write(msg);
}
@ -124,7 +125,7 @@ abstract class WebSocketProtocolHandler extends MessageToMessageDecoder<WebSocke
}
}, forceCloseTimeoutMillis, TimeUnit.MILLISECONDS);
closeSent.addListener(future -> timeoutTask.cancel(false));
closeSent.asFuture().addListener(future -> timeoutTask.cancel(false));
}
/**

View File

@ -311,7 +311,7 @@ public abstract class WebSocketServerHandshaker {
} catch (Throwable cause) {
promise.setFailure(cause);
}
return promise;
return promise.asFuture();
}
/**

View File

@ -27,7 +27,6 @@ import io.netty.handler.codec.http.HttpResponse;
import io.netty.handler.codec.http.websocketx.WebSocketServerProtocolHandler.ServerHandshakeStateEvent;
import io.netty.handler.ssl.SslHandler;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.FutureListener;
import io.netty.util.concurrent.Promise;
import java.util.Objects;
@ -158,6 +157,6 @@ class WebSocketServerProtocolHandshakeHandler implements ChannelHandler {
}, handshakeTimeoutMillis, TimeUnit.MILLISECONDS);
// Cancel the handshake timeout when handshake is finished.
localHandshakePromise.addListener(f -> timeoutFuture.cancel(false));
localHandshakePromise.asFuture().addListener(f -> timeoutFuture.cancel(false));
}
}

View File

@ -101,8 +101,8 @@ public class HttpServerUpgradeHandlerTest {
writeUpgradeMessage = true;
Promise<Void> promise = ctx.newPromise();
ctx.channel().executor().execute(() -> ctx.write(msg).cascadeTo(promise));
promise.addListener(f -> writeFlushed = true);
return promise;
Future<Void> future = promise.asFuture();
return future.addListener(f -> writeFlushed = true);
}
};

View File

@ -158,7 +158,7 @@ public class WebSocketProtocolHandlerTest {
EmbeddedChannel channel = new EmbeddedChannel(new ChannelHandler() {
@Override
public Future<Void> write(ChannelHandlerContext ctx, Object msg) {
Future<Void> future = ctx.newPromise();
Future<Void> future = ctx.newPromise().asFuture();
ref.set(future);
ReferenceCountUtil.release(msg);
return future;

View File

@ -334,7 +334,7 @@ abstract class AbstractHttp2StreamChannel extends DefaultAttributeMap implements
@Override
public Future<Void> closeFuture() {
return closePromise;
return closePromise.asFuture();
}
@Override
@ -514,7 +514,7 @@ abstract class AbstractHttp2StreamChannel extends DefaultAttributeMap implements
promise.setSuccess(null);
} else {
// This means close() was called before so we just register a listener and return
closePromise.addListener(promise, (p, future) -> p.setSuccess(null));
closeFuture().addListener(promise, (p, future) -> p.setSuccess(null));
}
return;
}

View File

@ -187,7 +187,7 @@ public class CompressorHttp2ConnectionEncoder extends DecoratingHttp2ConnectionE
buf = nextBuf;
}
combiner.finish(promise);
return promise;
return promise.asFuture();
} catch (Throwable cause) {
return ctx.newFailedFuture(cause);
} finally {

View File

@ -123,7 +123,7 @@ public class DefaultHttp2Connection implements Http2Connection {
if (closePromise == promise) {
// Do nothing
} else {
closePromise.addListener(new UnaryPromiseNotifier<>(promise));
closePromise.asFuture().addListener(new UnaryPromiseNotifier<>(promise));
}
} else {
closePromise = promise;

View File

@ -143,7 +143,7 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder, Ht
// Hand control of the frame to the flow controller.
flowController().addFlowControlled(stream,
new FlowControlledData(stream, data, padding, endOfStream, promise, ctx.channel()));
return promise;
return promise.asFuture();
}
@Override
@ -266,7 +266,7 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder, Ht
flowController.addFlowControlled(stream,
new FlowControlledHeaders(stream, headers, hasPriority, streamDependency,
weight, exclusive, padding, true, promise));
return promise;
return promise.asFuture();
}
} catch (Throwable t) {
lifecycleManager.onError(ctx, true, t);
@ -479,7 +479,7 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder, Ht
// queue and it is not end of stream yet. Just complete their promises by getting the buffer
// corresponding to 0 bytes and writing it to the channel (to preserve notification order).
Promise<Void> writePromise = ctx.newPromise();
writePromise.addListener(this);
writePromise.asFuture().addListener(this);
ctx.write(queue.remove(0, writePromise)).cascadeTo(writePromise);
}
return;
@ -493,7 +493,7 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder, Ht
// Determine how much data to write.
int writableData = min(queuedData, allowedBytes);
Promise<Void> writePromise = ctx.newPromise();
writePromise.addListener(this);
writePromise.asFuture().addListener(this);
ByteBuf toWrite = queue.remove(writableData, writePromise);
dataSize = queue.readableBytes();
@ -572,7 +572,7 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder, Ht
boolean isInformational = validateHeadersSentState(stream, headers, connection.isServer(), endOfStream);
// The code is currently requiring adding this listener before writing, in order to call onError() before
// closeStreamLocal().
promise.addListener(this);
promise.asFuture().addListener(this);
Future<Void> f = sendHeaders(frameWriter, ctx, stream.id(), headers, hasPriority, streamDependency,
weight, exclusive, padding, endOfStream);
@ -612,7 +612,7 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder, Ht
@Override
public void writeComplete() {
if (endOfStream) {
lifecycleManager.closeStreamLocal(stream, promise);
lifecycleManager.closeStreamLocal(stream, promise.asFuture());
}
}

View File

@ -23,6 +23,7 @@ import io.netty.handler.ssl.ApplicationProtocolNames;
import io.netty.util.AsciiString;
import io.netty.util.concurrent.DefaultPromise;
import io.netty.util.concurrent.EventExecutor;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.Promise;
import io.netty.util.internal.UnstableApi;
@ -284,13 +285,13 @@ public final class Http2CodecUtil {
/**
* Signify that no more {@link #newPromise()} allocations will be made.
* The aggregation can not be successful until this method is called.
* @return The promise that is the aggregation of all promises allocated with {@link #newPromise()}.
* @return The {@link Future} that is the aggregation of all promises allocated with {@link #newPromise()}.
*/
public Promise<Void> doneAllocatingPromises() {
public Future<Void> doneAllocatingPromises() {
if (!doneAllocating) {
doneAllocating = true;
if (doneCount == expectedCount || expectedCount == 0) {
return setPromise();
return setPromise().asFuture();
}
}
return this;
@ -323,7 +324,7 @@ public final class Http2CodecUtil {
++doneCount;
setAggregateFailure(cause);
if (allPromisesDone()) {
return setPromise();
setPromise();
}
}
return this;
@ -369,11 +370,12 @@ public final class Http2CodecUtil {
private Promise<Void> setPromise() {
if (aggregateFailure == null) {
promise.setSuccess(null);
return super.setSuccess(null);
super.setSuccess(null);
} else {
promise.setFailure(aggregateFailure);
return super.setFailure(aggregateFailure);
super.setFailure(aggregateFailure);
}
return this;
}
private boolean tryPromise() {

View File

@ -454,7 +454,7 @@ public class Http2ConnectionHandler extends ByteToMessageDecoder implements Http
ctx.flush();
Promise<Void> promise = ctx.newPromise();
doGracefulShutdown(ctx, f, promise);
return promise;
return promise.asFuture();
}
private FutureListener<Object> newClosingChannelFutureListener(

View File

@ -128,7 +128,7 @@ public final class Http2StreamChannelBootstrap {
} catch (Throwable cause) {
promise.setFailure(cause);
}
return promise;
return promise.asFuture();
}
@SuppressWarnings("deprecation")
@ -189,7 +189,7 @@ public final class Http2StreamChannelBootstrap {
if (future1.isSuccess()) {
promise.setSuccess(streamChannel);
} else if (future1.isCancelled()) {
promise.cancel(false);
promise.cancel();
} else {
if (streamChannel.isRegistered()) {
streamChannel.close();

View File

@ -144,7 +144,7 @@ public class HttpToHttp2ConnectionHandler extends Http2ConnectionHandler {
}
promiseAggregator.doneAllocatingPromises();
}
return promise;
return promise.asFuture();
}
private static Future<Void> writeHeaders(ChannelHandlerContext ctx, Http2ConnectionEncoder encoder, int streamId,

View File

@ -182,7 +182,7 @@ public class StreamBufferingEncoder extends DecoratingHttp2ConnectionEncoder {
pendingStream.frames.add(new HeadersFrame(headers, streamDependency, weight, exclusive,
padding, endOfStream, promise));
return promise;
return promise.asFuture();
}
@Override
@ -215,7 +215,7 @@ public class StreamBufferingEncoder extends DecoratingHttp2ConnectionEncoder {
if (pendingStream != null) {
Promise<Void> promise = ctx.newPromise();
pendingStream.frames.add(new DataFrame(data, padding, endOfStream, promise));
return promise;
return promise.asFuture();
} else {
ReferenceCountUtil.safeRelease(data);
return ctx.newFailedFuture(connectionError(PROTOCOL_ERROR, "Stream does not exist %d", streamId));

View File

@ -155,9 +155,10 @@ public class DefaultHttp2ConnectionTest {
final Promise<Void> promise = group.next().newPromise();
final CountDownLatch latch = new CountDownLatch(client.numActiveStreams());
client.forEachActiveStream(stream -> {
client.close(promise.addListener(future -> {
promise.asFuture().addListener(future -> {
latch.countDown();
}));
});
client.close(promise);
return true;
});
assertTrue(latch.await(5, TimeUnit.SECONDS));
@ -183,9 +184,10 @@ public class DefaultHttp2ConnectionTest {
return true;
});
} catch (Http2Exception ignored) {
client.close(promise.addListener(future -> {
promise.asFuture().addListener(future -> {
latch.countDown();
}));
});
client.close(promise);
}
assertTrue(latch.await(5, TimeUnit.SECONDS));
}
@ -632,9 +634,10 @@ public class DefaultHttp2ConnectionTest {
private void testRemoveAllStreams() throws InterruptedException {
final CountDownLatch latch = new CountDownLatch(1);
final Promise<Void> promise = group.next().newPromise();
client.close(promise.addListener(future -> {
promise.asFuture().addListener(future -> {
latch.countDown();
}));
});
client.close(promise);
assertTrue(latch.await(5, TimeUnit.SECONDS));
}

View File

@ -54,8 +54,8 @@ import static io.netty.handler.codec.http2.Http2Stream.State.CLOSED;
import static io.netty.handler.codec.http2.Http2Stream.State.IDLE;
import static io.netty.util.CharsetUtil.US_ASCII;
import static io.netty.util.CharsetUtil.UTF_8;
import static io.netty.util.concurrent.DefaultPromise.newSuccessfulPromise;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertSame;
@ -194,7 +194,7 @@ public class Http2ConnectionHandlerTest {
when(ctx.newFailedFuture(any(Throwable.class)))
.thenAnswer(invocationOnMock ->
DefaultPromise.newFailedPromise(executor, invocationOnMock.getArgument(0)));
when(ctx.newSucceededFuture()).thenReturn(DefaultPromise.newSuccessfulPromise(executor, null));
when(ctx.newSucceededFuture()).thenReturn(newSuccessfulPromise(executor, (Void) null).asFuture());
when(ctx.newPromise()).thenReturn(promise);
when(ctx.write(any())).thenReturn(future);
when(ctx.executor()).thenReturn(executor);
@ -204,9 +204,9 @@ public class Http2ConnectionHandlerTest {
return null;
}).when(ctx).fireChannelRead(any());
doAnswer((Answer<Future<Void>>) in ->
DefaultPromise.newSuccessfulPromise(executor, null)).when(ctx).write(any());
newSuccessfulPromise(executor, (Void) null).asFuture()).when(ctx).write(any());
doAnswer((Answer<Future<Void>>) in ->
DefaultPromise.newSuccessfulPromise(executor, null)).when(ctx).close();
newSuccessfulPromise(executor, (Void) null).asFuture()).when(ctx).close();
}
private Http2ConnectionHandler newHandler() throws Exception {

View File

@ -150,8 +150,8 @@ public class Http2ConnectionRoundtripTest {
doAnswer(invocationOnMock -> {
ChannelHandlerContext ctx = invocationOnMock.getArgument(0);
http2Server.encoder().writeHeaders(ctx,
(Integer) invocationOnMock.getArgument(1),
(Http2Headers) invocationOnMock.getArgument(2),
invocationOnMock.getArgument(1),
invocationOnMock.getArgument(2),
0,
false);
http2Server.flush(ctx);
@ -615,7 +615,7 @@ public class Http2ConnectionRoundtripTest {
ExecutionException e = assertThrows(ExecutionException.class, new Executable() {
@Override
public void execute() throws Throwable {
promise.get();
promise.asFuture().get();
}
});
assertThat(e.getCause(), is(instanceOf(IllegalReferenceCountException.class)));
@ -668,11 +668,11 @@ public class Http2ConnectionRoundtripTest {
ExecutionException e = assertThrows(ExecutionException.class, new Executable() {
@Override
public void execute() throws Throwable {
dataPromise.get();
dataPromise.asFuture().get();
}
});
assertThat(e.getCause(), is(instanceOf(IllegalStateException.class)));
assertPromise.sync();
assertPromise.asFuture().sync();
}
@Test
@ -1094,10 +1094,6 @@ public class Http2ConnectionRoundtripTest {
return ctx().newPromise();
}
private Promise<Void> serverNewPromise() {
return serverCtx().newPromise();
}
private static Http2Headers dummyHeaders() {
return new DefaultHttp2Headers(false).method(new AsciiString("GET")).scheme(new AsciiString("https"))
.authority(new AsciiString("example.org")).path(new AsciiString("/some/path/resource2"))

View File

@ -100,23 +100,23 @@ public class Http2ControlFrameLimitEncoderTest {
when(frameSizePolicy.maxFrameSize()).thenReturn(DEFAULT_MAX_FRAME_SIZE);
when(writer.writeRstStream(eq(ctx), anyInt(), anyLong()))
.thenAnswer((Answer<Future<Void>>) invocationOnMock -> handlePromise());
.thenAnswer((Answer<Future<Void>>) invocationOnMock -> handlePromise().asFuture());
when(writer.writeSettingsAck(any(ChannelHandlerContext.class)))
.thenAnswer((Answer<Future<Void>>) invocationOnMock -> handlePromise());
.thenAnswer((Answer<Future<Void>>) invocationOnMock -> handlePromise().asFuture());
when(writer.writePing(any(ChannelHandlerContext.class), anyBoolean(), anyLong()))
.thenAnswer((Answer<Future<Void>>) invocationOnMock -> {
Promise<Void> promise = handlePromise();
if (invocationOnMock.getArgument(1) == Boolean.FALSE) {
promise.trySuccess(null);
}
return promise;
return promise.asFuture();
});
when(writer.writeGoAway(any(ChannelHandlerContext.class), anyInt(), anyLong(), any(ByteBuf.class)))
.thenAnswer((Answer<Future<Void>>) invocationOnMock -> {
ReferenceCountUtil.release(invocationOnMock.getArgument(3));
Promise<Void> promise = ImmediateEventExecutor.INSTANCE.newPromise();
goAwayPromises.offer(promise);
return promise;
return promise.asFuture();
});
Http2Connection connection = new DefaultHttp2Connection(false);
connection.remote().flowController(new DefaultHttp2RemoteFlowController(connection));
@ -136,8 +136,8 @@ public class Http2ControlFrameLimitEncoderTest {
when(ctx.alloc()).thenReturn(UnpooledByteBufAllocator.DEFAULT);
when(channel.alloc()).thenReturn(UnpooledByteBufAllocator.DEFAULT);
when(executor.inEventLoop()).thenReturn(true);
doAnswer((Answer<Promise>) invocation -> newPromise()).when(ctx).newPromise();
doAnswer((Answer<Future>) invocation ->
doAnswer((Answer<Promise<Void>>) invocation -> newPromise()).when(ctx).newPromise();
doAnswer((Answer<Future<Void>>) invocation ->
ImmediateEventExecutor.INSTANCE.newFailedFuture(invocation.getArgument(0)))
.when(ctx).newFailedFuture(any(Throwable.class));
@ -164,7 +164,7 @@ public class Http2ControlFrameLimitEncoderTest {
}
@AfterEach
public void teardown() {
public void tearDown() {
// Close and release any buffered frames.
encoder.close();

View File

@ -626,7 +626,7 @@ public class Http2FrameCodecTest {
Future<Void> f = channel.writeAndFlush(new DefaultHttp2DataFrame(data).stream(stream));
assertTrue(f.isSuccess());
listenerExecuted.syncUninterruptibly();
listenerExecuted.asFuture().syncUninterruptibly();
assertTrue(listenerExecuted.isSuccess());
}

View File

@ -99,7 +99,7 @@ public class Http2FrameRoundtripTest {
when(ctx.executor()).thenReturn(executor);
when(ctx.channel()).thenReturn(channel);
doAnswer((Answer<Future<Void>>) in ->
DefaultPromise.newSuccessfulPromise(executor, null)).when(ctx).write(any());
DefaultPromise.newSuccessfulPromise(executor, (Void) null).asFuture()).when(ctx).write(any());
doAnswer((Answer<ByteBuf>) in -> Unpooled.buffer()).when(alloc).buffer();
doAnswer((Answer<ByteBuf>) in -> Unpooled.buffer((Integer) in.getArguments()[0])).when(alloc).buffer(anyInt());
doAnswer((Answer<Promise<Void>>) invocation ->
@ -110,7 +110,7 @@ public class Http2FrameRoundtripTest {
}
@AfterEach
public void teardown() {
public void tearDown() {
try {
// Release all of the buffers.
for (ByteBuf buf : needReleasing) {

View File

@ -72,7 +72,6 @@ import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@SuppressWarnings("unchecked")
public abstract class Http2MultiplexTest<C extends Http2FrameCodec> {
private final Http2Headers request = new DefaultHttp2Headers()
.method(HttpMethod.GET.asciiName()).scheme(HttpScheme.HTTPS.name())
@ -729,7 +728,7 @@ public abstract class Http2MultiplexTest<C extends Http2FrameCodec> {
// Create a promise before actually doing the close, because otherwise we would be adding a listener to a future
// that is already completed because we are using EmbeddedChannel which executes code in the JUnit thread.
Promise<Void> p = childChannel.newPromise();
p.addListener(childChannel, (channel, future) -> {
p.asFuture().addListener(childChannel, (channel, future) -> {
channelOpen.set(channel.isOpen());
channelActive.set(channel.isActive());
});
@ -972,7 +971,7 @@ public abstract class Http2MultiplexTest<C extends Http2FrameCodec> {
Promise<Void> promise = childChannel.newPromise();
childChannel.unsafe().close(promise);
promise.syncUninterruptibly();
promise.asFuture().syncUninterruptibly();
childChannel.closeFuture().syncUninterruptibly();
}

View File

@ -101,10 +101,10 @@ public class Http2StreamChannelBootstrapTest {
ExecutionException exception = assertThrows(ExecutionException.class, new Executable() {
@Override
public void execute() throws Throwable {
promise.get(3, SECONDS);
promise.asFuture().get(3, SECONDS);
}
});
assertThat(exception.getCause(), IsInstanceOf.<Throwable>instanceOf(ClosedChannelException.class));
assertThat(exception.getCause(), IsInstanceOf.instanceOf(ClosedChannelException.class));
} finally {
safeClose(clientChannel);
safeClose(serverConnectedChannel);

View File

@ -25,7 +25,6 @@ import io.netty.channel.ChannelMetadata;
import io.netty.channel.DefaultMessageSizeEstimator;
import io.netty.handler.codec.http2.StreamBufferingEncoder.Http2GoAwayException;
import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.DefaultPromise;
import io.netty.util.concurrent.EventExecutor;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.ImmediateEventExecutor;
@ -545,9 +544,7 @@ public class StreamBufferingEncoderTest {
ReferenceCountUtil.safeRelease(a);
}
Promise<Void> future = newPromise();
future.setSuccess(null);
return future;
return ImmediateEventExecutor.INSTANCE.newSucceededFuture(null);
};
}
@ -557,16 +554,16 @@ public class StreamBufferingEncoderTest {
public Future<Void> answer(InvocationOnMock invocation) throws Throwable {
for (Object a : invocation.getArguments()) {
if (a instanceof Promise) {
return (Future<Void>) a;
return ((Promise<Void>) a).asFuture();
}
}
return newPromise();
return newPromise().asFuture();
}
};
}
private static Promise<Void> newPromise() {
return new DefaultPromise<Void>(ImmediateEventExecutor.INSTANCE);
return ImmediateEventExecutor.INSTANCE.newPromise();
}
private static ByteBuf data() {

View File

@ -106,7 +106,7 @@ public abstract class MessageToMessageEncoder<I> extends ChannelHandlerAdapter {
writePromiseCombiner(ctx, out, promise);
}
}
return promise;
return promise.asFuture();
} else {
return ctx.write(msg);
}

View File

@ -183,7 +183,7 @@ public class Bzip2Encoder extends MessageToByteEncoder<ByteBuf> {
Future<Void> f = finishEncode(ctx());
f.cascadeTo(promise);
});
return promise;
return promise.asFuture();
}
}
@ -198,7 +198,7 @@ public class Bzip2Encoder extends MessageToByteEncoder<ByteBuf> {
// Ensure the channel is closed even if the write operation completes in time.
ctx.executor().schedule(() -> ctx.close().cascadeTo(promise),
10, TimeUnit.SECONDS); // FIXME: Magic number
return promise;
return promise.asFuture();
}
private Future<Void> finishEncode(final ChannelHandlerContext ctx) {

View File

@ -165,7 +165,7 @@ public class JdkZlibEncoder extends ZlibEncoder {
Future<Void> f = finishEncode(ctx());
f.cascadeTo(p);
});
return p;
return p.asFuture();
}
}
@ -265,7 +265,7 @@ public class JdkZlibEncoder extends ZlibEncoder {
// Ensure the channel is closed even if the write operation completes in time.
ctx.executor().schedule(() -> ctx.close().cascadeTo(promise),
10, TimeUnit.SECONDS); // FIXME: Magic number
return promise;
return promise.asFuture();
}
private Future<Void> finishEncode(final ChannelHandlerContext ctx) {

View File

@ -349,7 +349,7 @@ public class Lz4FrameEncoder extends MessageToByteEncoder<ByteBuf> {
Future<Void> f = finishEncode(ctx());
f.cascadeTo(promise);
});
return promise;
return promise.asFuture();
}
}
@ -364,7 +364,7 @@ public class Lz4FrameEncoder extends MessageToByteEncoder<ByteBuf> {
// Ensure the channel is closed even if the write operation completes in time.
ctx.executor().schedule(() -> ctx.close().cascadeTo(promise),
10, TimeUnit.SECONDS); // FIXME: Magic number
return promise;
return promise.asFuture();
}
private ChannelHandlerContext ctx() {

View File

@ -36,7 +36,7 @@ public abstract class AbstractEventExecutor extends AbstractExecutorService impl
static final long DEFAULT_SHUTDOWN_TIMEOUT = 15;
private final Collection<EventExecutor> selfCollection = Collections.singleton(this);
private final Future<?> successfulVoidFuture = DefaultPromise.newSuccessfulPromise(this, null);
private final Future<?> successfulVoidFuture = DefaultPromise.newSuccessfulPromise(this, null).asFuture();
@Override
public EventExecutor next() {
@ -87,12 +87,13 @@ public abstract class AbstractEventExecutor extends AbstractExecutorService impl
Future<V> f = (Future<V>) successfulVoidFuture;
return f;
}
return DefaultPromise.newSuccessfulPromise(this, result);
return DefaultPromise.newSuccessfulPromise(this, result).asFuture();
}
@Override
public <V> Future<V> newFailedFuture(Throwable cause) {
return DefaultPromise.newFailedPromise(this, cause);
Promise<V> promise = DefaultPromise.newFailedPromise(this, cause);
return promise.asFuture();
}
@Override

View File

@ -393,6 +393,11 @@ public abstract class AbstractScheduledEventExecutor extends AbstractEventExecut
return this;
}
@Override
public boolean cancel() {
return cancel(false);
}
@Override
public boolean isSuccess() {
return future.isSuccess();

View File

@ -0,0 +1,96 @@
/*
* Copyright 2021 The Netty Project
*
* The Netty Project licenses this file to you under the Apache License,
* version 2.0 (the "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at:
*
* https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations
* under the License.
*/
package io.netty.util.concurrent;
import java.util.concurrent.CancellationException;
/**
* A result of an asynchronous operation.
* <p>
* This interface is used as the super-interface of {@link Promise} and {@link Future}, and these should generally be
* used instead.
*/
interface AsynchronousResult<V> {
/**
* Cancel this asynchronous operation, unless it has already been completed.
* <p>
* A cancelled operation is considered to be {@linkplain #isFailed() failed}.
* <p>
* If the cancellation was successful, the result of this operation will be that it has failed with a
* {@link CancellationException}.
* <p>
* Cancellation will not cause any threads working on the operation to be {@linkplain Thread#interrupt()
* interrupted}.
*
* @return {@code true} if the operation was cancelled by this call, otherwise {@code false}.
*/
boolean cancel();
/**
* Returns {@code true} if and only if the operation was completed successfully.
*/
boolean isSuccess();
/**
* Returns {@code true} if and only if the operation was completed and failed.
*/
boolean isFailed();
/**
* Return {@code true} if this operation has been {@linkplain #cancel() cancelled}.
*
* @return {@code true} if this operation has been cancelled, otherwise {@code false}.
*/
boolean isCancelled();
/**
* Return {@code true} if this operation has been completed either {@linkplain Promise#setSuccess(Object)
* successfully}, {@linkplain Promise#setFailure(Throwable) unsuccessfully}, or through
* {@linkplain #cancel() cancellation}.
*
* @return {@code true} if this operation has completed, otherwise {@code false}.
*/
boolean isDone();
/**
* returns {@code true} if and only if the operation can be cancelled via {@link #cancel()}.
*/
boolean isCancellable();
/**
* Return the successful result of this asynchronous operation, if any.
* If the operation has not yet been completed, then this will throw {@link IllegalStateException}.
* If the operation has been cancelled or failed with an exception, then this returns {@code null}.
* Note that asynchronous operations can also be completed successfully with a {@code null} result.
*
* @return the result of this operation, if completed successfully.
* @throws IllegalStateException if this {@code Future} or {@link Promise} has not completed yet.
*/
V getNow();
/**
* Returns the cause of the failed operation if the operation has failed.
*
* @return The cause of the failure, if any. Otherwise {@code null} if succeeded.
* @throws IllegalStateException if this {@code Promise} has not completed yet.
*/
Throwable cause();
/**
* Returns the {@link EventExecutor} that is tied to this {@link Promise} or {@link Future}.
*/
EventExecutor executor();
}

View File

@ -201,7 +201,7 @@ final class DefaultFutureCompletionStage<V> implements FutureCompletionStage<V>
promise.setFailure(cause);
}
});
return promise.asStage();
return promise.asFuture().asStage();
}
private static <U, V> void thenApplyAsync0(Promise<U> promise, V value, Function<? super V, ? extends U> fn) {
@ -234,7 +234,7 @@ final class DefaultFutureCompletionStage<V> implements FutureCompletionStage<V>
promise.setFailure(cause);
}
});
return promise.asStage();
return promise.asFuture().asStage();
}
private static <U, V> void thenAcceptAsync0(Promise<U> promise, V value, Consumer<? super V> action) {
@ -299,7 +299,7 @@ final class DefaultFutureCompletionStage<V> implements FutureCompletionStage<V>
applyAndNotify(promise, value2, value1, fn);
}
}, otherExecutor(executor));
return promise.asStage();
return promise.asFuture().asStage();
}
private Executor otherExecutor(Executor executor) {
@ -342,7 +342,7 @@ final class DefaultFutureCompletionStage<V> implements FutureCompletionStage<V>
};
whenCompleteAsync(consumer, executor);
other.whenCompleteAsync(consumer, otherExecutor(executor));
return promise.asStage();
return promise.asFuture().asStage();
}
@Override
@ -363,7 +363,7 @@ final class DefaultFutureCompletionStage<V> implements FutureCompletionStage<V>
};
whenCompleteAsync(consumer, executor);
other.whenCompleteAsync(consumer, otherExecutor(executor));
return promise.asStage();
return promise.asFuture().asStage();
}
@Override
@ -384,7 +384,7 @@ final class DefaultFutureCompletionStage<V> implements FutureCompletionStage<V>
};
whenCompleteAsync(consumer, executor);
other.whenCompleteAsync(consumer, otherExecutor(executor));
return promise.asStage();
return promise.asFuture().asStage();
}
@Override
@ -407,7 +407,7 @@ final class DefaultFutureCompletionStage<V> implements FutureCompletionStage<V>
promise.setFailure(cause);
}
});
return promise.asStage();
return promise.asFuture().asStage();
}
private static <V, U> void thenComposeAsync0(
@ -449,7 +449,7 @@ final class DefaultFutureCompletionStage<V> implements FutureCompletionStage<V>
promise.setSuccess(result);
}
});
return promise.asStage();
return promise.asFuture().asStage();
}
@Override
@ -466,7 +466,7 @@ final class DefaultFutureCompletionStage<V> implements FutureCompletionStage<V>
safeExecute(executor, () -> whenCompleteAsync0(promise, f, action), promise);
}
});
return promise.asStage();
return promise.asFuture().asStage();
}
private static <V> void whenCompleteAsync0(
@ -501,7 +501,7 @@ final class DefaultFutureCompletionStage<V> implements FutureCompletionStage<V>
safeExecute(executor, () -> handleAsync0(promise, f, fn), promise);
}
});
return promise.asStage();
return promise.asFuture().asStage();
}
@SuppressWarnings("unchecked")

View File

@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
public class DefaultPromise<V> implements Promise<V> {
public class DefaultPromise<V> implements Promise<V>, Future<V> {
private static final InternalLogger logger = InternalLoggerFactory.getInstance(DefaultPromise.class);
private static final InternalLogger rejectedExecutionLogger =
InternalLoggerFactory.getInstance(DefaultPromise.class.getName() + ".rejectedExecution");
@ -157,6 +157,11 @@ public class DefaultPromise<V> implements Promise<V> {
return !isDone0(result) || !isCancelled0(result);
}
@Override
public Future<V> asFuture() {
return this;
}
@Override
public boolean isSuccess() {
Object result = this.result;
@ -212,7 +217,7 @@ public class DefaultPromise<V> implements Promise<V> {
}
@Override
public Promise<V> addListener(FutureListener<? super V> listener) {
public Future<V> addListener(FutureListener<? super V> listener) {
requireNonNull(listener, "listener");
addListener0(listener, null);
@ -224,7 +229,7 @@ public class DefaultPromise<V> implements Promise<V> {
}
@Override
public <C> Promise<V> addListener(C context, FutureContextListener<? super C, ? super V> listener) {
public <C> Future<V> addListener(C context, FutureContextListener<? super C, ? super V> listener) {
requireNonNull(listener, "listener");
addListener0(listener, context == null ? NULL_CONTEXT : context);
@ -377,6 +382,11 @@ public class DefaultPromise<V> implements Promise<V> {
throw new ExecutionException(cause);
}
@Override
public boolean cancel() {
return cancel(false);
}
/**
* @param mayInterruptIfRunning this value has no effect in this implementation.
*/

View File

@ -151,30 +151,7 @@ import java.util.function.Function;
* </pre>
*/
@SuppressWarnings("ClassNameSameAsAncestorName")
public interface Future<V> extends java.util.concurrent.Future<V> {
/**
* Returns {@code true} if and only if the operation was completed successfully.
*/
boolean isSuccess();
/**
* Returns {@code true} if and only if the operation was completed and failed.
*/
boolean isFailed();
/**
* returns {@code true} if and only if the operation can be cancelled via {@link #cancel(boolean)}.
*/
boolean isCancellable();
/**
* Returns the cause of the failed I/O operation if the I/O operation has failed.
*
* @return the cause of the failure. {@code null} if succeeded.
* @throws IllegalStateException if this {@code Future} has not completed yet.
*/
Throwable cause();
public interface Future<V> extends java.util.concurrent.Future<V>, AsynchronousResult<V> {
/**
* Adds the specified listener to this future. The specified listener is notified when this future is {@linkplain
* #isDone() done}. If this future is already completed, the specified listener is notified immediately.
@ -258,14 +235,6 @@ public interface Future<V> extends java.util.concurrent.Future<V> {
*/
boolean awaitUninterruptibly(long timeoutMillis);
/**
* Return the result without blocking. If the future is not done yet this will throw {@link IllegalStateException}.
* <p>
*
* @throws IllegalStateException if this {@code Future} has not completed yet.
*/
V getNow();
/**
* {@inheritDoc}
* <p>
@ -274,11 +243,6 @@ public interface Future<V> extends java.util.concurrent.Future<V> {
@Override
boolean cancel(boolean mayInterruptIfRunning);
/**
* Returns the {@link EventExecutor} that is tied to this {@link Future}.
*/
EventExecutor executor();
@Override
default V get() throws InterruptedException, ExecutionException {
await();

View File

@ -28,7 +28,7 @@ import java.util.function.Function;
/**
* A {@link CompletionStage} that provides the same threading semantics and guarantees as the underlying
* {@link io.netty.util.concurrent.Future}, which means that all the callbacks will be executed by {@link #executor()}
* {@link Future}, which means that all the callbacks will be executed by {@link #executor()}
* if not specified otherwise (by calling the corresponding *Async methods).
*
* Please be aware that {@link FutureCompletionStage#toCompletableFuture()} is not supported and so will throw
@ -188,7 +188,7 @@ public interface FutureCompletionStage<V> extends CompletionStage<V> {
static <U> FutureCompletionStage<U> toFutureCompletionStage(CompletionStage<U> stage, EventExecutor executor) {
Objects.requireNonNull(stage, "stage");
Objects.requireNonNull(executor, "executor");
if (stage instanceof FutureCompletionStage && ((FutureCompletionStage) stage).executor() == executor) {
if (stage instanceof FutureCompletionStage && ((FutureCompletionStage<?>) stage).executor() == executor) {
return (FutureCompletionStage<U>) stage;
}
@ -208,6 +208,6 @@ public interface FutureCompletionStage<V> extends CompletionStage<V> {
promise.setSuccess(v);
}
});
return promise.asStage();
return promise.asFuture().asStage();
}
}

View File

@ -63,8 +63,9 @@ final class Futures {
}
Promise<R> promise = future.executor().newPromise();
future.addListener(new Mapper<>(promise, mapper));
promise.addListener(future, propagateCancel());
return promise;
Future<R> mappedFuture = promise.asFuture();
mappedFuture.addListener(future, propagateCancel());
return mappedFuture;
}
/**
@ -98,12 +99,13 @@ final class Futures {
requireNonNull(mapper, "mapper");
Promise<R> promise = future.executor().newPromise();
future.addListener(new FlatMapper<>(promise, mapper));
Future<R> mappedFuture = promise.asFuture();
if (!future.isSuccess()) {
// Propagate cancellation if future is either incomplete or failed.
// Failed means it could be cancelled, so that needs to be propagated.
promise.addListener(future, propagateCancel());
mappedFuture.addListener(future, propagateCancel());
}
return promise;
return mappedFuture;
}
@SuppressWarnings("unchecked")
@ -120,7 +122,7 @@ final class Futures {
if (completed.isCancelled()) {
// Don't check or log if cancellation propagation fails.
// Propagation goes both ways, which means at least one future will already be cancelled here.
recipient.cancel(false);
recipient.cancel();
} else {
Throwable cause = completed.cause();
recipient.tryFailure(cause);
@ -219,7 +221,7 @@ final class Futures {
propagateUncommonCompletion(future, recipient);
} else {
future.addListener(recipient, passThrough());
recipient.addListener(future, propagateCancel());
recipient.asFuture().addListener(future, propagateCancel());
}
} catch (Throwable e) {
tryFailure(recipient, e, logger);
@ -246,7 +248,7 @@ final class Futures {
if (!future.isSuccess()) {
// Propagate cancellation if future is either incomplete or failed.
// Failed means it could be cancelled, so that needs to be propagated.
promise.addListener(future, propagateCancel());
promise.asFuture().addListener(future, propagateCancel());
}
future.addListener(promise, passThrough());
}

View File

@ -68,7 +68,7 @@ public final class GlobalEventExecutor extends AbstractScheduledEventExecutor im
volatile Thread thread;
private final Future<?> terminationFuture = DefaultPromise.newFailedPromise(
this, new UnsupportedOperationException());
this, new UnsupportedOperationException()).asFuture();
private GlobalEventExecutor() {
threadFactory = ThreadExecutorMap.apply(new DefaultThreadFactory(

View File

@ -55,7 +55,7 @@ public final class ImmediateEventExecutor extends AbstractEventExecutor {
};
private final Future<?> terminationFuture = DefaultPromise.newFailedPromise(
GlobalEventExecutor.INSTANCE, new UnsupportedOperationException());
GlobalEventExecutor.INSTANCE, new UnsupportedOperationException()).asFuture();
private ImmediateEventExecutor() { }

View File

@ -232,7 +232,7 @@ public class MultithreadEventExecutorGroup extends AbstractEventExecutorGroup {
@Override
public final Future<?> terminationFuture() {
return terminationFuture;
return terminationFuture.asFuture();
}
@Override

View File

@ -15,14 +15,12 @@
*/
package io.netty.util.concurrent;
import java.util.concurrent.CancellationException;
/**
* Special {@link Future} which is writable.
*/
public interface Promise<V> extends Future<V> {
public interface Promise<V> extends AsynchronousResult<V> {
/**
* Marks this future as a success and notifies all listeners.
* Marks this promise as a success and notifies all listeners attached to the {@linkplain #asFuture() future}.
* <p>
* If it is success or failed already it will throw an {@link IllegalStateException}.
*/
@ -31,37 +29,39 @@ public interface Promise<V> extends Future<V> {
/**
* Marks this future as a success and notifies all listeners.
*
* @return {@code true} if and only if successfully marked this future as a success. Otherwise {@code false} because
* this future is already marked as either a success or a failure.
* @return {@code true} if and only if successfully marked this promise as a success.
* Otherwise {@code false} because this promise is already marked as either a success or a failure.
*/
boolean trySuccess(V result);
/**
* Marks this future as a failure and notifies all listeners.
* Marks this promise as a failure and notifies all listeners attached to the {@linkplain #asFuture() future}.
* <p>
* If it is success or failed already it will throw an {@link IllegalStateException}.
*/
Promise<V> setFailure(Throwable cause);
/**
* Marks this future as a failure and notifies all listeners.
* Marks this promise as a failure and notifies all listeners.
*
* @return {@code true} if and only if successfully marked this future as a failure. Otherwise {@code false} because
* this future is already marked as either a success or a failure.
* @return {@code true} if and only if successfully marked this promise as a failure.
* Otherwise {@code false} because this promise is already marked as either a success or a failure.
*/
boolean tryFailure(Throwable cause);
/**
* Make this future impossible to cancel.
* Make this promise impossible to cancel.
*
* @return {@code true} if and only if successfully marked this future as uncancellable, or it is already done
* without being cancelled. Otherwise {@code false} if this future has been cancelled already.
* @return {@code true} if and only if successfully marked this promise as uncancellable, or it is already done
* without being cancelled. Otherwise {@code false} if this promise has been cancelled already.
*/
boolean setUncancellable();
@Override
Promise<V> addListener(FutureListener<? super V> listener);
@Override
<C> Promise<V> addListener(C context, FutureContextListener<? super C, ? super V> listener);
/**
* Return the {@link Future} instance is associated with this promise.
* This future will be completed upon completion of this promise.
*
* @return A future instance associated with this promise.
*/
Future<V> asFuture();
}

View File

@ -27,11 +27,13 @@ import java.util.concurrent.TimeoutException;
final class RunnableFutureAdapter<V> implements RunnableFuture<V> {
private final Promise<V> promise;
private final Future<V> future;
private final Callable<V> task;
RunnableFutureAdapter(Promise<V> promise, Callable<V> task) {
this.promise = requireNonNull(promise, "promise");
this.task = requireNonNull(task, "task");
future = promise.asFuture();
}
@Override
@ -61,58 +63,58 @@ final class RunnableFutureAdapter<V> implements RunnableFuture<V> {
@Override
public RunnableFuture<V> addListener(FutureListener<? super V> listener) {
promise.addListener(listener);
future.addListener(listener);
return this;
}
@Override
public <C> RunnableFuture<V> addListener(C context, FutureContextListener<? super C, ? super V> listener) {
promise.addListener(context, listener);
future.addListener(context, listener);
return this;
}
@Override
public RunnableFuture<V> sync() throws InterruptedException {
promise.sync();
future.sync();
return this;
}
@Override
public RunnableFuture<V> syncUninterruptibly() {
promise.syncUninterruptibly();
future.syncUninterruptibly();
return this;
}
@Override
public RunnableFuture<V> await() throws InterruptedException {
promise.await();
future.await();
return this;
}
@Override
public RunnableFuture<V> awaitUninterruptibly() {
promise.awaitUninterruptibly();
future.awaitUninterruptibly();
return this;
}
@Override
public boolean await(long timeout, TimeUnit unit) throws InterruptedException {
return promise.await(timeout, unit);
return future.await(timeout, unit);
}
@Override
public boolean await(long timeoutMillis) throws InterruptedException {
return promise.await(timeoutMillis);
return future.await(timeoutMillis);
}
@Override
public boolean awaitUninterruptibly(long timeout, TimeUnit unit) {
return promise.awaitUninterruptibly(timeout, unit);
return future.awaitUninterruptibly(timeout, unit);
}
@Override
public boolean awaitUninterruptibly(long timeoutMillis) {
return promise.awaitUninterruptibly(timeoutMillis);
return future.awaitUninterruptibly(timeoutMillis);
}
@Override
@ -134,7 +136,12 @@ final class RunnableFutureAdapter<V> implements RunnableFuture<V> {
@Override
public boolean cancel(boolean mayInterruptIfRunning) {
return promise.cancel(mayInterruptIfRunning);
return future.cancel(mayInterruptIfRunning);
}
@Override
public boolean cancel() {
return cancel(false);
}
@Override
@ -149,12 +156,12 @@ final class RunnableFutureAdapter<V> implements RunnableFuture<V> {
@Override
public V get() throws InterruptedException, ExecutionException {
return promise.get();
return future.get();
}
@Override
public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
return promise.get(timeout, unit);
return future.get(timeout, unit);
}
@Override

View File

@ -41,12 +41,14 @@ final class RunnableScheduledFutureAdapter<V> implements AbstractScheduledEventE
private final AbstractScheduledEventExecutor executor;
private final Promise<V> promise;
private final Future<V> future;
private final Callable<V> callable;
RunnableScheduledFutureAdapter(AbstractScheduledEventExecutor executor, Promise<V> promise, Callable<V> callable,
long deadlineNanos, long periodNanos) {
this.executor = requireNonNull(executor, "executor");
this.promise = requireNonNull(promise, "promise");
future = promise.asFuture();
this.callable = requireNonNull(callable, "callable");
this.deadlineNanos = deadlineNanos;
this.periodNanos = periodNanos;
@ -133,13 +135,18 @@ final class RunnableScheduledFutureAdapter<V> implements AbstractScheduledEventE
*/
@Override
public boolean cancel(boolean mayInterruptIfRunning) {
boolean canceled = promise.cancel(mayInterruptIfRunning);
boolean canceled = future.cancel(mayInterruptIfRunning);
if (canceled) {
executor.removeScheduled(this);
}
return canceled;
}
@Override
public boolean cancel() {
return cancel(false);
}
@Override
public boolean isSuccess() {
return promise.isSuccess();
@ -162,58 +169,58 @@ final class RunnableScheduledFutureAdapter<V> implements AbstractScheduledEventE
@Override
public RunnableScheduledFuture<V> addListener(FutureListener<? super V> listener) {
promise.addListener(listener);
future.addListener(listener);
return this;
}
@Override
public <C> RunnableScheduledFuture<V> addListener(C context, FutureContextListener<? super C, ? super V> listener) {
promise.addListener(context, listener);
future.addListener(context, listener);
return this;
}
@Override
public RunnableScheduledFuture<V> sync() throws InterruptedException {
promise.sync();
future.sync();
return this;
}
@Override
public RunnableScheduledFuture<V> syncUninterruptibly() {
promise.syncUninterruptibly();
future.syncUninterruptibly();
return this;
}
@Override
public RunnableScheduledFuture<V> await() throws InterruptedException {
promise.await();
future.await();
return this;
}
@Override
public RunnableScheduledFuture<V> awaitUninterruptibly() {
promise.awaitUninterruptibly();
future.awaitUninterruptibly();
return this;
}
@Override
public boolean await(long timeout, TimeUnit unit) throws InterruptedException {
return promise.await(timeout, unit);
return future.await(timeout, unit);
}
@Override
public boolean await(long timeoutMillis) throws InterruptedException {
return promise.await(timeoutMillis);
return future.await(timeoutMillis);
}
@Override
public boolean awaitUninterruptibly(long timeout, TimeUnit unit) {
return promise.awaitUninterruptibly(timeout, unit);
return future.awaitUninterruptibly(timeout, unit);
}
@Override
public boolean awaitUninterruptibly(long timeoutMillis) {
return promise.awaitUninterruptibly(timeoutMillis);
return future.awaitUninterruptibly(timeoutMillis);
}
@Override
@ -228,7 +235,7 @@ final class RunnableScheduledFutureAdapter<V> implements AbstractScheduledEventE
@Override
public boolean isCancelled() {
return promise.isCancelled();
return future.isCancelled();
}
@Override
@ -238,12 +245,12 @@ final class RunnableScheduledFutureAdapter<V> implements AbstractScheduledEventE
@Override
public V get() throws InterruptedException, ExecutionException {
return promise.get();
return future.get();
}
@Override
public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
return promise.get(timeout, unit);
return future.get(timeout, unit);
}
@Override

View File

@ -546,7 +546,7 @@ public class SingleThreadEventExecutor extends AbstractScheduledEventExecutor im
gracefulShutdownTimeout = unit.toNanos(timeout);
if (ensureThreadStarted(oldState)) {
return terminationFuture;
return terminationFuture.asFuture();
}
if (wakeup) {
@ -561,7 +561,7 @@ public class SingleThreadEventExecutor extends AbstractScheduledEventExecutor im
@Override
public final Future<?> terminationFuture() {
return terminationFuture;
return terminationFuture.asFuture();
}
@Override

View File

@ -39,7 +39,7 @@ public final class UnaryPromiseNotifier<T> implements FutureListener<T> {
logger.warn("Failed to mark a promise as success because it is done already: {}", promise);
}
} else if (completedFuture.isCancelled()) {
if (!promise.cancel(false)) {
if (!promise.cancel()) {
logger.warn("Failed to cancel a promise because it is done already: {}", promise);
}
} else {

View File

@ -99,12 +99,13 @@ public final class UnorderedThreadPoolEventExecutor extends ScheduledThreadPoolE
@Override
public <V> Future<V> newSucceededFuture(V result) {
return DefaultPromise.newSuccessfulPromise(this, result);
return DefaultPromise.newSuccessfulPromise(this, result).asFuture();
}
@Override
public <V> Future<V> newFailedFuture(Throwable cause) {
return DefaultPromise.newFailedPromise(this, cause);
Promise<V> promise = DefaultPromise.newFailedPromise(this, cause);
return promise.asFuture();
}
@Override
@ -140,7 +141,7 @@ public final class UnorderedThreadPoolEventExecutor extends ScheduledThreadPoolE
@Override
public Future<?> terminationFuture() {
return terminationFuture;
return terminationFuture.asFuture();
}
@Override

View File

@ -29,7 +29,7 @@ public final class PromiseNotificationUtil {
* Try to cancel the {@link Promise} and log if {@code logger} is not {@code null} in case this fails.
*/
public static void tryCancel(Promise<?> p, InternalLogger logger) {
if (!p.cancel(false) && logger != null) {
if (!p.cancel() && logger != null) {
Throwable err = p.cause();
if (err == null) {
logger.warn("Failed to cancel promise because it has succeeded already: {}", p);

View File

@ -79,7 +79,7 @@ public class DefaultFutureCompletionStageTest {
public void testSameExecutorAndFuture() {
EventExecutor executor = executor();
Promise<Boolean> promise = executor.newPromise();
FutureCompletionStage<Boolean> stage = new DefaultFutureCompletionStage<>(promise);
FutureCompletionStage<Boolean> stage = new DefaultFutureCompletionStage<>(promise.asFuture());
assertSame(executor, stage.executor());
assertSame(promise, stage.future());
}
@ -88,7 +88,7 @@ public class DefaultFutureCompletionStageTest {
public void testThrowsUnsupportedOperationException() {
EventExecutor executor = executor();
Promise<Boolean> promise = executor.newPromise();
FutureCompletionStage<Boolean> stage = new DefaultFutureCompletionStage<>(promise);
FutureCompletionStage<Boolean> stage = new DefaultFutureCompletionStage<>(promise.asFuture());
assertThrows(UnsupportedOperationException.class, () -> stage.toCompletableFuture());
}
@ -766,7 +766,7 @@ public class DefaultFutureCompletionStageTest {
// may use different threads.
for (int i = 0; i < 1000; i++) {
Promise<Boolean> promise = executor.newPromise();
FutureCompletionStage<Boolean> stage = new DefaultFutureCompletionStage<>(promise);
FutureCompletionStage<Boolean> stage = new DefaultFutureCompletionStage<>(promise.asFuture());
CompletableFuture<Boolean> completableFuture = new CompletableFuture<>();
Future<Integer> f = fn.apply(stage, completableFuture).future();

View File

@ -20,7 +20,6 @@ import org.junit.jupiter.api.Test;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import static io.netty.util.concurrent.DefaultPromise.newSuccessfulPromise;
import static io.netty.util.concurrent.ImmediateEventExecutor.INSTANCE;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.assertj.core.api.Assertions.assertThat;
@ -39,7 +38,7 @@ class FuturesTest {
}
@Test
public void mapMustApplyMapperFunctionOnSuccededFuture() {
public void mapMustApplyMapperFunctionOnSucceededFuture() {
DefaultPromise<Integer> promise = new DefaultPromise<>(INSTANCE);
promise.setSuccess(42);
assertThat(promise.map(i -> i.toString()).getNow()).isEqualTo("42");
@ -108,16 +107,16 @@ class FuturesTest {
@Test
public void flatMapMustApplyMapperFunctionWhenFutureSucceeds() {
DefaultPromise<Integer> promise = new DefaultPromise<>(INSTANCE);
Future<String> strFut = promise.flatMap(i -> newSuccessfulPromise(INSTANCE, i.toString()));
Future<String> strFut = promise.flatMap(i -> INSTANCE.newSucceededFuture(i.toString()));
promise.setSuccess(42);
assertThat(strFut.getNow()).isEqualTo("42");
}
@Test
public void flatMapMustApplyMapperFunctionOnSuccededFuture() {
public void flatMapMustApplyMapperFunctionOnSucceededFuture() {
DefaultPromise<Integer> promise = new DefaultPromise<>(INSTANCE);
promise.setSuccess(42);
assertThat(promise.flatMap(i -> newSuccessfulPromise(INSTANCE, i.toString())).getNow()).isEqualTo("42");
assertThat(promise.flatMap(i -> INSTANCE.newSucceededFuture(i.toString())).getNow()).isEqualTo("42");
}
@Test
@ -125,7 +124,7 @@ class FuturesTest {
DefaultPromise<Integer> promise = new DefaultPromise<>(INSTANCE);
Exception cause = new Exception("boom");
promise.setFailure(cause);
assertThat(promise.flatMap(i -> newSuccessfulPromise(INSTANCE, i.toString())).cause()).isSameAs(cause);
assertThat(promise.flatMap(i -> INSTANCE.newSucceededFuture(i.toString())).cause()).isSameAs(cause);
}
@Test
@ -136,7 +135,7 @@ class FuturesTest {
AtomicInteger counter = new AtomicInteger();
assertThat(promise.flatMap(i -> {
counter.getAndIncrement();
return newSuccessfulPromise(INSTANCE, i.toString());
return INSTANCE.newSucceededFuture(i.toString());
}).cause()).isSameAs(cause);
assertThat(counter.get()).isZero();
}
@ -165,7 +164,7 @@ class FuturesTest {
@Test
public void cancelOnFutureFromFlatMapMustCancelOriginalFuture() {
DefaultPromise<Integer> promise = new DefaultPromise<>(INSTANCE);
Future<String> strFut = promise.flatMap(i -> newSuccessfulPromise(INSTANCE, i.toString()));
Future<String> strFut = promise.flatMap(i -> INSTANCE.newSucceededFuture(i.toString()));
strFut.cancel(false);
assertTrue(promise.isCancelled());
assertTrue(strFut.isCancelled());
@ -174,7 +173,7 @@ class FuturesTest {
@Test
public void cancelOnOriginalFutureMustCancelFutureFromFlatMap() {
DefaultPromise<Integer> promise = new DefaultPromise<>(INSTANCE);
Future<String> strFut = promise.flatMap(i -> newSuccessfulPromise(INSTANCE, i.toString()));
Future<String> strFut = promise.flatMap(i -> INSTANCE.newSucceededFuture(i.toString()));
promise.cancel(false);
assertTrue(promise.isCancelled());
assertTrue(strFut.isCancelled());

View File

@ -17,46 +17,30 @@ package io.netty.util.concurrent;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
import static io.netty.util.concurrent.ImmediateEventExecutor.INSTANCE;
import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
public class PromiseCombinerTest {
@Mock
private Promise<Void> p1;
private FutureListener<Void> l1;
private final FutureListenerConsumer l1Consumer = new FutureListenerConsumer() {
@Override
public void accept(FutureListener<Void> listener) {
l1 = listener;
}
};
@Mock
private Future<Void> f1;
private Promise<Void> p2;
private FutureListener<Void> l2;
private final FutureListenerConsumer l2Consumer = new FutureListenerConsumer() {
@Override
public void accept(FutureListener<Void> listener) {
l2 = listener;
}
};
@Mock
private Future<Void> f2;
private Promise<Void> p3;
private PromiseCombiner combiner;
@BeforeEach
public void setup() {
MockitoAnnotations.initMocks(this);
combiner = new PromiseCombiner(ImmediateEventExecutor.INSTANCE);
p1 = INSTANCE.newPromise();
p2 = INSTANCE.newPromise();
p3 = INSTANCE.newPromise();
f1 = p1.asFuture();
f2 = p2.asFuture();
combiner = new PromiseCombiner(INSTANCE);
}
@Test
@ -68,13 +52,15 @@ public class PromiseCombinerTest {
// expected
}
combiner.finish(p1);
verify(p1).trySuccess(null);
assertTrue(p1.isSuccess());
assertThat(p1.getNow()).isNull();
}
@Test
public void testNullAggregatePromise() {
combiner.finish(p1);
verify(p1).trySuccess(null);
assertTrue(p1.isSuccess());
assertThat(p1.getNow()).isNull();
}
@Test
@ -90,13 +76,13 @@ public class PromiseCombinerTest {
@Test
public void testAddAfterFinish() {
combiner.finish(p1);
assertThrows(IllegalStateException.class, () -> combiner.add(p2));
assertThrows(IllegalStateException.class, () -> combiner.add(f2));
}
@Test
public void testAddAllAfterFinish() {
combiner.finish(p1);
assertThrows(IllegalStateException.class, () -> combiner.addAll(p2));
assertThrows(IllegalStateException.class, () -> combiner.addAll(f2));
}
@Test
@ -107,65 +93,59 @@ public class PromiseCombinerTest {
@Test
public void testAddAllSuccess() throws Exception {
mockSuccessPromise(p1, l1Consumer);
mockSuccessPromise(p2, l2Consumer);
combiner.addAll(p1, p2);
p1.setSuccess(null);
combiner.addAll(f1, f2);
combiner.finish(p3);
l1.operationComplete(p1);
verifyNotCompleted(p3);
l2.operationComplete(p2);
verifySuccess(p3);
assertFalse(p3.isDone());
p2.setSuccess(null);
assertTrue(p3.isDone());
assertTrue(p3.isSuccess());
}
@Test
public void testAddSuccess() throws Exception {
mockSuccessPromise(p1, l1Consumer);
mockSuccessPromise(p2, l2Consumer);
combiner.add(p1);
l1.operationComplete(p1);
combiner.add(p2);
l2.operationComplete(p2);
verifyNotCompleted(p3);
p1.setSuccess(null);
p2.setSuccess(null);
combiner.add(f1);
combiner.add(f2);
assertFalse(p3.isDone());
combiner.finish(p3);
verifySuccess(p3);
assertTrue(p3.isSuccess());
}
@Test
public void testAddAllFail() throws Exception {
RuntimeException e1 = new RuntimeException("fake exception 1");
RuntimeException e2 = new RuntimeException("fake exception 2");
mockFailedPromise(p1, e1, l1Consumer);
mockFailedPromise(p2, e2, l2Consumer);
combiner.addAll(p1, p2);
combiner.addAll(f1, f2);
combiner.finish(p3);
l1.operationComplete(p1);
verifyNotCompleted(p3);
l2.operationComplete(p2);
verifyFail(p3, e1);
p1.setFailure(e1);
assertFalse(p3.isDone());
p2.setFailure(e2);
assertTrue(p3.isFailed());
assertThat(p3.cause()).isSameAs(e1);
}
@Test
public void testAddFail() throws Exception {
RuntimeException e1 = new RuntimeException("fake exception 1");
RuntimeException e2 = new RuntimeException("fake exception 2");
mockFailedPromise(p1, e1, l1Consumer);
mockFailedPromise(p2, e2, l2Consumer);
combiner.add(p1);
l1.operationComplete(p1);
combiner.add(p2);
l2.operationComplete(p2);
verifyNotCompleted(p3);
combiner.add(f1);
p1.setFailure(e1);
combiner.add(f2);
p2.setFailure(e2);
assertFalse(p3.isDone());
combiner.finish(p3);
verifyFail(p3, e1);
assertTrue(p3.isFailed());
assertThat(p3.cause()).isSameAs(e1);
}
@Test
public void testEventExecutor() {
EventExecutor executor = mock(EventExecutor.class);
when(executor.inEventLoop()).thenReturn(false);
EventExecutor executor = new SingleThreadEventExecutor();
combiner = new PromiseCombiner(executor);
Future<?> future = mock(Future.class);
Future<?> future = executor.newPromise().asFuture();
try {
combiner.add(future);
@ -181,8 +161,7 @@ public class PromiseCombinerTest {
// expected
}
@SuppressWarnings("unchecked")
Promise<Void> promise = (Promise<Void>) mock(Promise.class);
Promise<Void> promise = executor.newPromise();
try {
combiner.finish(promise);
fail();
@ -190,45 +169,4 @@ public class PromiseCombinerTest {
// expected
}
}
private static void verifyFail(Promise<Void> p, Throwable cause) {
verify(p).tryFailure(eq(cause));
}
private static void verifySuccess(Promise<Void> p) {
verify(p).trySuccess(null);
}
private static void verifyNotCompleted(Promise<Void> p) {
verify(p, never()).trySuccess(any(Void.class));
verify(p, never()).tryFailure(any(Throwable.class));
verify(p, never()).setSuccess(any(Void.class));
verify(p, never()).setFailure(any(Throwable.class));
}
private static void mockSuccessPromise(Promise<Void> p, FutureListenerConsumer consumer) {
when(p.isDone()).thenReturn(true);
when(p.isSuccess()).thenReturn(true);
mockListener(p, consumer);
}
private static void mockFailedPromise(Promise<Void> p, Throwable cause, FutureListenerConsumer consumer) {
when(p.isDone()).thenReturn(true);
when(p.isSuccess()).thenReturn(false);
when(p.isFailed()).thenReturn(true);
when(p.cause()).thenReturn(cause);
mockListener(p, consumer);
}
@SuppressWarnings("unchecked")
private static void mockListener(final Promise<Void> p, final FutureListenerConsumer consumer) {
doAnswer(invocation -> {
consumer.accept(invocation.getArgument(0));
return p;
}).when(p).addListener(any(FutureListener.class));
}
interface FutureListenerConsumer {
void accept(FutureListener<Void> listener);
}
}

View File

@ -169,7 +169,7 @@ public class SingleThreadEventExecutorTest {
promise.setFailure(cause);
}
});
promise.syncUninterruptibly();
promise.asFuture().syncUninterruptibly();
} finally {
executor.shutdownGracefully(0, 0, TimeUnit.MILLISECONDS);
}

View File

@ -61,7 +61,7 @@ public class WebSocketClientHandler extends SimpleChannelInboundHandler<Object>
}
public Future<Void> handshakeFuture() {
return handshakeFuture;
return handshakeFuture.asFuture();
}
@Override

View File

@ -45,7 +45,7 @@ public class Http2SettingsHandler extends SimpleChannelInboundHandler<Http2Setti
* @throws Exception if timeout or other failure occurs
*/
public void awaitSettings(long timeout, TimeUnit unit) throws Exception {
if (!promise.awaitUninterruptibly(timeout, unit)) {
if (!promise.asFuture().awaitUninterruptibly(timeout, unit)) {
throw new IllegalStateException("Timed out waiting for settings");
}
if (promise.isFailed()) {

View File

@ -75,7 +75,7 @@ public class HttpResponseHandler extends SimpleChannelInboundHandler<FullHttpRes
throw new RuntimeException(writeFuture.cause());
}
Promise<Void> promise = entry.getValue().getValue();
if (!promise.awaitUninterruptibly(timeout, unit)) {
if (!promise.asFuture().awaitUninterruptibly(timeout, unit)) {
throw new IllegalStateException("Timed out waiting for response on stream id " + entry.getKey());
}
if (promise.isFailed()) {

View File

@ -100,7 +100,7 @@ public class OcspClientExample {
Channel channel = bootstrap.connect(host, 443).get();
try {
FullHttpResponse response = promise.get();
FullHttpResponse response = promise.asFuture().get();
ReferenceCountUtil.release(response);
} finally {
channel.close();

View File

@ -85,6 +85,6 @@ public class SmtpClientHandler extends SimpleChannelInboundHandler<SmtpResponse>
throw new RuntimeException("Still waiting for the past response");
}
}
return promise.asStage();
return promise.asFuture().asStage();
}
}

View File

@ -42,23 +42,23 @@ public final class SocksServerConnectHandler extends SimpleChannelInboundHandler
if (message instanceof Socks4CommandRequest) {
final Socks4CommandRequest request = (Socks4CommandRequest) message;
Promise<Channel> promise = ctx.executor().newPromise();
promise.addListener(future -> {
final Channel outboundChannel = future.getNow();
if (future.isSuccess()) {
Future<Void> responseFuture = ctx.channel().writeAndFlush(
new DefaultSocks4CommandResponse(Socks4CommandStatus.SUCCESS));
promise.asFuture().addListener(future -> {
final Channel outboundChannel = future.getNow();
if (future.isSuccess()) {
Future<Void> responseFuture = ctx.channel().writeAndFlush(
new DefaultSocks4CommandResponse(Socks4CommandStatus.SUCCESS));
responseFuture.addListener(fut -> {
ctx.pipeline().remove(this);
outboundChannel.pipeline().addLast(new RelayHandler(ctx.channel()));
ctx.pipeline().addLast(new RelayHandler(outboundChannel));
});
} else {
ctx.channel().writeAndFlush(
new DefaultSocks4CommandResponse(Socks4CommandStatus.REJECTED_OR_FAILED));
SocksServerUtils.closeOnFlush(ctx.channel());
}
responseFuture.addListener(fut -> {
ctx.pipeline().remove(this);
outboundChannel.pipeline().addLast(new RelayHandler(ctx.channel()));
ctx.pipeline().addLast(new RelayHandler(outboundChannel));
});
} else {
ctx.channel().writeAndFlush(
new DefaultSocks4CommandResponse(Socks4CommandStatus.REJECTED_OR_FAILED));
SocksServerUtils.closeOnFlush(ctx.channel());
}
});
final Channel inboundChannel = ctx.channel();
b.group(inboundChannel.executor())
@ -81,27 +81,27 @@ public final class SocksServerConnectHandler extends SimpleChannelInboundHandler
} else if (message instanceof Socks5CommandRequest) {
final Socks5CommandRequest request = (Socks5CommandRequest) message;
Promise<Channel> promise = ctx.executor().newPromise();
promise.addListener(future -> {
final Channel outboundChannel = future.getNow();
if (future.isSuccess()) {
Future<Void> responseFuture =
ctx.channel().writeAndFlush(new DefaultSocks5CommandResponse(
Socks5CommandStatus.SUCCESS,
request.dstAddrType(),
request.dstAddr(),
request.dstPort()));
responseFuture.addListener(fut -> {
ctx.pipeline().remove(this);
outboundChannel.pipeline().addLast(new RelayHandler(ctx.channel()));
ctx.pipeline().addLast(new RelayHandler(outboundChannel));
});
} else {
promise.asFuture().addListener(future -> {
final Channel outboundChannel = future.getNow();
if (future.isSuccess()) {
Future<Void> responseFuture =
ctx.channel().writeAndFlush(new DefaultSocks5CommandResponse(
Socks5CommandStatus.FAILURE, request.dstAddrType()));
SocksServerUtils.closeOnFlush(ctx.channel());
}
Socks5CommandStatus.SUCCESS,
request.dstAddrType(),
request.dstAddr(),
request.dstPort()));
responseFuture.addListener(fut -> {
ctx.pipeline().remove(this);
outboundChannel.pipeline().addLast(new RelayHandler(ctx.channel()));
ctx.pipeline().addLast(new RelayHandler(outboundChannel));
});
} else {
ctx.channel().writeAndFlush(new DefaultSocks5CommandResponse(
Socks5CommandStatus.FAILURE, request.dstAddrType()));
SocksServerUtils.closeOnFlush(ctx.channel());
}
});
final Channel inboundChannel = ctx.channel();
b.group(inboundChannel.executor())

View File

@ -109,7 +109,7 @@ public abstract class ProxyHandler implements ChannelHandler {
* or the connection attempt has failed.
*/
public final Future<Channel> connectFuture() {
return connectPromise;
return connectPromise.asFuture();
}
/**
@ -396,7 +396,7 @@ public abstract class ProxyHandler implements ChannelHandler {
}
Promise<Void> promise = ctx.newPromise();
addPendingWrite(ctx, msg, promise);
return promise;
return promise.asFuture();
}
@Override

View File

@ -40,7 +40,6 @@ import io.netty.handler.codec.http.HttpVersion;
import io.netty.handler.proxy.HttpProxyHandler.HttpProxyConnectException;
import io.netty.util.NetUtil;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.Promise;
import org.junit.jupiter.api.Test;
import java.net.InetAddress;
@ -55,7 +54,6 @@ 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 {
@ -252,11 +250,10 @@ public class HttpProxyHandlerTest {
boolean ignoreDefaultPortsInConnectHostHeader) throws Exception {
InetSocketAddress proxyAddress = new InetSocketAddress(NetUtil.LOCALHOST, 8080);
Promise<Void> promise = mock(Promise.class);
verifyNoMoreInteractions(promise);
Future<Void> future = mock(Future.class);
ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
when(ctx.connect(same(proxyAddress), isNull(InetSocketAddress.class))).thenReturn(promise);
when(ctx.connect(same(proxyAddress), isNull(InetSocketAddress.class))).thenReturn(future);
HttpProxyHandler handler = new HttpProxyHandler(
new InetSocketAddress(NetUtil.LOCALHOST, 8080),

View File

@ -23,9 +23,9 @@ import io.netty.resolver.AddressResolverGroup;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.FutureListener;
import io.netty.util.concurrent.Promise;
import io.netty.util.internal.ObjectUtil;
import java.net.SocketAddress;
import java.util.Objects;
/**
* {@link ChannelHandler} which will resolve the {@link SocketAddress} that is passed to
@ -38,7 +38,7 @@ public class ResolveAddressHandler implements ChannelHandler {
private final AddressResolverGroup<? extends SocketAddress> resolverGroup;
public ResolveAddressHandler(AddressResolverGroup<? extends SocketAddress> resolverGroup) {
this.resolverGroup = ObjectUtil.checkNotNull(resolverGroup, "resolverGroup");
this.resolverGroup = Objects.requireNonNull(resolverGroup, "resolverGroup");
}
@Override
@ -54,9 +54,9 @@ public class ResolveAddressHandler implements ChannelHandler {
} else {
ctx.connect(future.getNow(), localAddress).cascadeTo(promise);
}
ctx.pipeline().remove(ResolveAddressHandler.this);
ctx.pipeline().remove(this);
});
return promise;
return promise.asFuture();
} else {
Future<Void> f = ctx.connect(remoteAddress, localAddress);
ctx.pipeline().remove(this);

View File

@ -164,9 +164,9 @@ public class SniHandler extends AbstractSniHandler<SslContext> {
try {
context = mapping.map(input);
} catch (Throwable cause) {
return promise.setFailure(cause);
return promise.setFailure(cause).asFuture();
}
return promise.setSuccess(context);
return promise.setSuccess(context).asFuture();
}
}

View File

@ -591,7 +591,7 @@ public class SslHandler extends ByteToMessageDecoder {
* The {@link Future} for the most recent {@linkplain #renegotiate() TLS renegotiation} otherwise.
*/
public Future<Channel> handshakeFuture() {
return handshakePromise;
return handshakePromise.asFuture();
}
/**
@ -608,7 +608,7 @@ public class SslHandler extends ByteToMessageDecoder {
} else {
ctx.executor().execute(() -> closeOutbound0(promise));
}
return promise;
return promise.asFuture();
}
private void closeOutbound0(Promise<Void> promise) {
@ -631,7 +631,7 @@ public class SslHandler extends ByteToMessageDecoder {
* @see SSLEngine
*/
public Future<Channel> sslCloseFuture() {
return sslClosePromise;
return sslClosePromise.asFuture();
}
@Override
@ -700,7 +700,7 @@ public class SslHandler extends ByteToMessageDecoder {
} else {
Promise<Void> promise = ctx.newPromise();
pendingUnencryptedWrites.add(((ByteBufConvertible) msg).asByteBuf(), promise);
return promise;
return promise.asFuture();
}
}
@ -1874,14 +1874,14 @@ public class SslHandler extends ByteToMessageDecoder {
//
// See https://github.com/netty/netty/issues/5931
Promise<Void> cascade = ctx.newPromise();
cascade.cascadeTo(promise);
safeClose(ctx, closeNotifyPromise, cascade);
cascade.asFuture().cascadeTo(promise);
safeClose(ctx, closeNotifyPromise.asFuture(), cascade);
} else {
/// We already handling the close_notify so just attach the promise to the sslClosePromise.
sslClosePromise.addListener(future -> promise.setSuccess(null));
sslCloseFuture().addListener(future -> promise.setSuccess(null));
}
}
return promise;
return promise.asFuture();
}
private void flush(ChannelHandlerContext ctx, Promise<Void> promise) {
@ -1957,15 +1957,15 @@ public class SslHandler extends ByteToMessageDecoder {
EventExecutor executor = ctx.executor();
if (!executor.inEventLoop()) {
executor.execute(() -> renegotiateOnEventLoop(promise));
return promise;
return promise.asFuture();
}
renegotiateOnEventLoop(promise);
return promise;
return promise.asFuture();
}
private void renegotiateOnEventLoop(final Promise<Channel> newHandshakePromise) {
final Promise<Channel> oldHandshakePromise = handshakePromise;
Future<Channel> oldHandshakePromise = handshakeFuture();
if (!oldHandshakePromise.isDone()) {
// There's no need to handshake because handshake is in progress already.
// Merge the new promise into the old one.
@ -2038,7 +2038,7 @@ public class SslHandler extends ByteToMessageDecoder {
}, handshakeTimeoutMillis, TimeUnit.MILLISECONDS);
// Cancel the handshake timeout when handshake is finished.
localHandshakePromise.addListener(f -> timeoutFuture.cancel(false));
localHandshakePromise.asFuture().addListener(f -> timeoutFuture.cancel(false));
}
private void forceFlush(ChannelHandlerContext ctx) {
@ -2102,9 +2102,11 @@ public class SslHandler extends ByteToMessageDecoder {
} else {
Future<?> closeNotifyReadTimeoutFuture;
if (!sslClosePromise.isDone()) {
Future<Channel> closeFuture = sslCloseFuture();
if (!closeFuture.isDone()) {
closeNotifyReadTimeoutFuture = ctx.executor().schedule(() -> {
if (!sslClosePromise.isDone()) {
if (!closeFuture.isDone()) {
logger.debug(
"{} did not receive close_notify in {}ms; force-closing the connection.",
ctx.channel(), closeNotifyReadTimeout);
@ -2117,8 +2119,8 @@ public class SslHandler extends ByteToMessageDecoder {
closeNotifyReadTimeoutFuture = null;
}
// Do the close once the we received the close_notify.
sslClosePromise.addListener(future -> {
// Do the close once we received the close_notify.
closeFuture.addListener(future -> {
if (closeNotifyReadTimeoutFuture != null) {
closeNotifyReadTimeoutFuture.cancel(false);
}

View File

@ -116,7 +116,7 @@ public class ChunkedWriteHandler implements ChannelHandler {
public Future<Void> write(ChannelHandlerContext ctx, Object msg) {
Promise<Void> promise = ctx.newPromise();
queue.add(new PendingWrite(msg, promise));
return promise;
return promise.asFuture();
}
@Override
@ -150,10 +150,8 @@ public class ChunkedWriteHandler implements ChannelHandler {
if (message instanceof ChunkedInput) {
ChunkedInput<?> in = (ChunkedInput<?>) message;
boolean endOfInput;
long inputLength;
try {
endOfInput = in.isEndOfInput();
inputLength = in.length();
closeInput(in);
} catch (Exception e) {
closeInput(in);
@ -299,14 +297,10 @@ public class ChunkedWriteHandler implements ChannelHandler {
private static void handleEndOfInputFuture(Future<?> future, PendingWrite currentWrite) {
ChunkedInput<?> input = (ChunkedInput<?>) currentWrite.msg;
closeInput(input);
if (future.isFailed()) {
closeInput(input);
currentWrite.fail(future.cause());
} else {
// read state of the input in local variables before closing it
long inputProgress = input.progress();
long inputLength = input.length();
closeInput(input);
currentWrite.success();
}
}

View File

@ -563,12 +563,12 @@ public abstract class AbstractTrafficShapingHandler implements ChannelHandler {
+ isHandlerActive(ctx));
}
submitWrite(ctx, msg, size, wait, now, promise);
return promise;
return promise.asFuture();
}
}
// to maintain order of write
submitWrite(ctx, msg, size, 0, now, promise);
return promise;
return promise.asFuture();
}
@Deprecated

View File

@ -684,13 +684,13 @@ public class GlobalChannelTrafficShapingHandler extends AbstractTrafficShapingHa
}
Promise<Void> promise = ctx.newPromise();
submitWrite(ctx, msg, size, wait, now, promise);
return promise;
return promise.asFuture();
}
}
Promise<Void> promise = ctx.newPromise();
// to maintain order of write
submitWrite(ctx, msg, size, 0, now, promise);
return promise;
return promise.asFuture();
}
@Override

View File

@ -35,6 +35,7 @@ import io.netty.channel.local.LocalServerChannel;
import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
import io.netty.handler.ssl.util.SelfSignedCertificate;
import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.Promise;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
@ -151,7 +152,7 @@ public class CipherSuiteCanaryTest {
pipeline.addLast(new SimpleChannelInboundHandler<Object>() {
@Override
public void channelInactive(ChannelHandlerContext ctx) throws Exception {
serverPromise.cancel(true);
serverPromise.cancel();
ctx.fireChannelInactive();
}
@ -187,7 +188,7 @@ public class CipherSuiteCanaryTest {
pipeline.addLast(new SimpleChannelInboundHandler<Object>() {
@Override
public void channelInactive(ChannelHandlerContext ctx) throws Exception {
clientPromise.cancel(true);
clientPromise.cancel();
ctx.fireChannelInactive();
}
@ -213,11 +214,14 @@ public class CipherSuiteCanaryTest {
client.writeAndFlush(Unpooled.wrappedBuffer(new byte[] {'P', 'I', 'N', 'G'}))
.syncUninterruptibly();
assertTrue(clientPromise.await(5L, TimeUnit.SECONDS), "client timeout");
assertTrue(serverPromise.await(5L, TimeUnit.SECONDS), "server timeout");
Future<Object> clientFuture = clientPromise.asFuture();
Future<Object> serverFuture = serverPromise.asFuture();
clientPromise.sync();
serverPromise.sync();
assertTrue(clientFuture.await(5L, TimeUnit.SECONDS), "client timeout");
assertTrue(serverFuture.await(5L, TimeUnit.SECONDS), "server timeout");
clientFuture.sync();
serverFuture.sync();
} finally {
client.close().sync();
}
@ -260,19 +264,15 @@ public class CipherSuiteCanaryTest {
List<Object[]> dst = new ArrayList<>();
SslProvider[] sslProviders = SslProvider.values();
for (int i = 0; i < sslProviders.length; i++) {
SslProvider serverSslProvider = sslProviders[i];
for (int j = 0; j < sslProviders.length; j++) {
SslProvider clientSslProvider = sslProviders[j];
for (SslProvider serverSslProvider : sslProviders) {
for (SslProvider clientSslProvider : sslProviders) {
if ((serverSslProvider != SslProvider.JDK || clientSslProvider != SslProvider.JDK)
&& !OpenSsl.isAvailable()) {
&& !OpenSsl.isAvailable()) {
continue;
}
dst.add(new Object[]{serverSslProvider, clientSslProvider, rfcCipherName, true});
dst.add(new Object[]{serverSslProvider, clientSslProvider, rfcCipherName, false});
dst.add(new Object[] { serverSslProvider, clientSslProvider, rfcCipherName, true });
dst.add(new Object[] { serverSslProvider, clientSslProvider, rfcCipherName, false });
}
}

View File

@ -239,7 +239,7 @@ public class OpenSslPrivateKeyMethodTest {
pipeline.addLast(new SimpleChannelInboundHandler<Object>() {
@Override
public void channelInactive(ChannelHandlerContext ctx) {
serverPromise.cancel(true);
serverPromise.cancel();
ctx.fireChannelInactive();
}
@ -275,7 +275,7 @@ public class OpenSslPrivateKeyMethodTest {
pipeline.addLast(new SimpleChannelInboundHandler<Object>() {
@Override
public void channelInactive(ChannelHandlerContext ctx) {
clientPromise.cancel(true);
clientPromise.cancel();
ctx.fireChannelInactive();
}
@ -300,11 +300,13 @@ public class OpenSslPrivateKeyMethodTest {
client.writeAndFlush(Unpooled.wrappedBuffer(new byte[] {'P', 'I', 'N', 'G'}))
.syncUninterruptibly();
assertTrue(clientPromise.await(5L, TimeUnit.SECONDS), "client timeout");
assertTrue(serverPromise.await(5L, TimeUnit.SECONDS), "server timeout");
Future<Object> clientFuture = clientPromise.asFuture();
Future<Object> serverFuture = serverPromise.asFuture();
assertTrue(clientFuture.await(5L, TimeUnit.SECONDS), "client timeout");
assertTrue(serverFuture.await(5L, TimeUnit.SECONDS), "server timeout");
clientPromise.sync();
serverPromise.sync();
clientFuture.sync();
serverFuture.sync();
assertTrue(signCalled.get());
} finally {
client.close().sync();
@ -442,7 +444,7 @@ public class OpenSslPrivateKeyMethodTest {
} catch (Throwable cause) {
promise.setFailure(cause);
}
return promise;
return promise.asFuture();
}
@Override
@ -470,7 +472,7 @@ public class OpenSslPrivateKeyMethodTest {
} catch (Throwable cause) {
promise.setFailure(cause);
}
return promise;
return promise.asFuture();
}
}
}

View File

@ -254,7 +254,7 @@ public class ParameterizedSslHandlerTest {
}
}).connect(sc.localAddress()).get();
donePromise.get();
donePromise.asFuture().sync();
} finally {
if (cc != null) {
cc.close().syncUninterruptibly();
@ -355,7 +355,7 @@ public class ParameterizedSslHandlerTest {
}
}).connect(sc.localAddress()).get();
promise.syncUninterruptibly();
promise.asFuture().syncUninterruptibly();
} finally {
if (cc != null) {
cc.close().syncUninterruptibly();
@ -479,8 +479,8 @@ public class ParameterizedSslHandlerTest {
}
}).connect(sc.localAddress()).get();
serverPromise.awaitUninterruptibly();
clientPromise.awaitUninterruptibly();
serverPromise.asFuture().awaitUninterruptibly();
clientPromise.asFuture().awaitUninterruptibly();
// Server always received the close_notify as the client triggers the close sequence.
assertTrue(serverPromise.isSuccess());

View File

@ -1094,7 +1094,7 @@ public abstract class SSLEngineTest {
Future<Channel> ccf = cb.connect(new InetSocketAddress(expectedHost, port));
assertTrue(ccf.awaitUninterruptibly().isSuccess());
clientChannel = ccf.get();
return clientWritePromise;
return clientWritePromise.asFuture();
}
private void mySetupMutualAuth(SSLEngineTestParam param, File keyFile, File crtFile, String keyPassword)
@ -2000,7 +2000,7 @@ public abstract class SSLEngineTest {
}).connect(serverChannel.localAddress()).get();
promise.syncUninterruptibly();
promise.asFuture().syncUninterruptibly();
serverCert.delete();
clientCert.delete();
@ -4029,8 +4029,9 @@ public abstract class SSLEngineTest {
out.write(1);
out.flush();
assertTrue(promise.await(10, TimeUnit.SECONDS));
SecretKey key = promise.get();
Future<SecretKey> future = promise.asFuture();
assertTrue(future.await(10, TimeUnit.SECONDS));
SecretKey key = future.get();
assertEquals(48, key.getEncoded().length, "AES secret key must be 48 bytes");
} finally {
closeQuietly(socket);

View File

@ -152,7 +152,7 @@ final class SniClientJava8TestUtil {
Bootstrap cb = new Bootstrap();
cc = cb.group(group).channel(LocalChannel.class).handler(sslHandler).connect(address).get();
promise.syncUninterruptibly();
promise.asFuture().syncUninterruptibly();
sslHandler.handshakeFuture().syncUninterruptibly();
} catch (CompletionException e) {
throw e.getCause();

View File

@ -139,7 +139,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).get();
assertEquals(sniHostName, promise.syncUninterruptibly().getNow());
assertEquals(sniHostName, promise.asFuture().syncUninterruptibly().getNow());
// After we are done with handshaking getHandshakeSession() should return null.
handler.handshakeFuture().syncUninterruptibly();

View File

@ -537,7 +537,7 @@ public class SniHandlerTest {
} finally {
if (!success) {
ReferenceCountUtil.safeRelease(sslContext);
releasePromise.cancel(true);
releasePromise.cancel();
}
}
}
@ -569,7 +569,7 @@ public class SniHandlerTest {
// The client disconnects
cc.close().syncUninterruptibly();
if (!releasePromise.awaitUninterruptibly(10L, TimeUnit.SECONDS)) {
if (!releasePromise.asFuture().awaitUninterruptibly(10L, TimeUnit.SECONDS)) {
throw new IllegalStateException("It doesn't seem #replaceHandler() got called.");
}

View File

@ -185,7 +185,7 @@ public class SslErrorTest {
}
}).connect(serverChannel.localAddress()).get();
// Block until we received the correct exception
promise.syncUninterruptibly();
promise.asFuture().syncUninterruptibly();
} finally {
if (clientChannel != null) {
clientChannel.close().syncUninterruptibly();

View File

@ -129,7 +129,7 @@ public class SslHandlerTest {
if (((ByteBuf) msg).isReadable()) {
Promise<Void> promise = ctx.newPromise();
writesToFail.add(promise);
return promise;
return promise.asFuture();
}
}
return ctx.newSucceededFuture();
@ -493,8 +493,8 @@ public class SslHandlerTest {
sc = serverBootstrap.bind(new InetSocketAddress(0)).get();
cc = bootstrap.connect(sc.localAddress()).get();
serverPromise.syncUninterruptibly();
clientPromise.syncUninterruptibly();
serverPromise.asFuture().syncUninterruptibly();
clientPromise.asFuture().syncUninterruptibly();
} finally {
if (cc != null) {
cc.close().syncUninterruptibly();

View File

@ -51,6 +51,11 @@ abstract class Cache<E> {
AtomicReferenceFieldUpdater.newUpdater(Cache.Entries.class, FutureAndDelay.class, "expirationFuture");
private static final Future<?> CANCELLED_FUTURE = new Future<Object>() {
@Override
public boolean cancel() {
return cancel(false);
}
@Override
public boolean isSuccess() {
return false;

View File

@ -483,7 +483,7 @@ public class DnsNameResolver extends InetNameResolver {
});
b.option(ChannelOption.RCVBUF_ALLOCATOR, new FixedRecvByteBufAllocator(maxPayloadSize));
channelFuture = responseHandler.channelActivePromise;
channelFuture = responseHandler.channelActivePromise.asFuture();
try {
ch = b.createUnregistered();
Future<Void> future = localAddress == null ? ch.register() : ch.bind(localAddress);
@ -731,10 +731,10 @@ public class DnsNameResolver extends InetNameResolver {
DnsRecord[] additionalsArray = toArray(additionals, true);
try {
doResolve(inetHost, additionalsArray, promise, resolveCache);
return promise;
} catch (Exception e) {
return promise.setFailure(e);
promise.setFailure(e);
}
return promise.asFuture();
}
/**
@ -764,10 +764,10 @@ public class DnsNameResolver extends InetNameResolver {
DnsRecord[] additionalsArray = toArray(additionals, true);
try {
doResolveAll(inetHost, additionalsArray, promise, resolveCache);
return promise;
} catch (Exception e) {
return promise.setFailure(e);
promise.setFailure(e);
}
return promise.asFuture();
}
@Override
@ -858,7 +858,7 @@ public class DnsNameResolver extends InetNameResolver {
if (!result.isEmpty()) {
trySuccess(promise, result);
return promise;
return promise.asFuture();
}
}
}
@ -868,7 +868,7 @@ public class DnsNameResolver extends InetNameResolver {
dnsServerAddressStreamProvider.nameServerAddressStream(hostname);
new DnsRecordResolveContext(this, promise, question, additionals, nameServerAddrs, maxQueriesPerResolve)
.resolve(promise);
return promise;
return promise.asFuture();
}
private static DnsRecord[] toArray(Iterable<DnsRecord> additionals, boolean validateType) {
@ -994,7 +994,7 @@ public class DnsNameResolver extends InetNameResolver {
DnsCache resolveCache, boolean completeEarlyIfPossible) {
final Promise<List<InetAddress>> allPromise = executor().newPromise();
doResolveAllUncached(hostname, additionals, promise, allPromise, resolveCache, true);
allPromise.addListener(future -> {
allPromise.asFuture().addListener(future -> {
if (future.isSuccess()) {
trySuccess(promise, future.getNow().get(0));
} else {
@ -1225,10 +1225,10 @@ public class DnsNameResolver extends InetNameResolver {
try {
new DatagramDnsQueryContext(this, nameServerAddr, question, additionals, castPromise)
.query(flush, writePromise);
return castPromise;
} catch (Exception e) {
return castPromise.setFailure(e);
castPromise.setFailure(e);
}
return castPromise.asFuture();
}
@SuppressWarnings("unchecked")
@ -1330,7 +1330,7 @@ public class DnsNameResolver extends InetNameResolver {
}
});
promise.addListener(addressEnvelopeFuture -> {
promise.asFuture().addListener(addressEnvelopeFuture -> {
channel.close();
if (addressEnvelopeFuture.isSuccess()) {

View File

@ -64,7 +64,7 @@ abstract class DnsQueryContext implements FutureListener<AddressedEnvelope<DnsRe
id = parent.queryContextManager.add(this);
// Ensure we remove the id from the QueryContextManager once the query completes.
promise.addListener(this);
promise.asFuture().addListener(this);
if (parent.isOptResourceEnabled()) {
optResource = new AbstractDnsOptPseudoRrRecord(parent.maxPayloadSize(), 0, 0) {

View File

@ -203,7 +203,7 @@ abstract class DnsResolveContext<T> {
final int initialSearchDomainIdx = startWithoutSearchDomain ? 0 : 1;
final Promise<List<T>> searchDomainPromise = parent.executor().newPromise();
searchDomainPromise.addListener(new FutureListener<List<T>>() {
searchDomainPromise.asFuture().addListener(new FutureListener<List<T>>() {
private int searchDomainIdx = initialSearchDomainIdx;
@Override
public void operationComplete(Future<? extends List<T>> future) {
@ -220,7 +220,7 @@ abstract class DnsResolveContext<T> {
promise.tryFailure(new SearchDomainUnknownHostException(cause, hostname));
} else if (searchDomainIdx < searchDomains.length) {
Promise<List<T>> newPromise = parent.executor().newPromise();
newPromise.addListener(this);
newPromise.asFuture().addListener(this);
doSearchDomainQuery(hostname + '.' + searchDomains[searchDomainIdx++], newPromise);
} else if (!startWithoutSearchDomain) {
internalResolve(hostname, promise);
@ -421,7 +421,7 @@ abstract class DnsResolveContext<T> {
queriesInProgress.add(f);
queryLifecycleObserver.queryWritten(nameServerAddr, writePromise);
queryLifecycleObserver.queryWritten(nameServerAddr, writePromise.asFuture());
f.addListener(future -> {
queriesInProgress.remove(future);
@ -475,7 +475,7 @@ abstract class DnsResolveContext<T> {
queriesInProgress.add(resolveFuture);
Promise<List<InetAddress>> resolverPromise = parent.executor().newPromise();
resolverPromise.addListener(future -> {
resolverPromise.asFuture().addListener(future -> {
// Remove placeholder.
queriesInProgress.remove(resolveFuture);

View File

@ -61,26 +61,27 @@ final class InflightNameResolver<T> implements NameResolver<T> {
}
@Override
public Promise<T> resolve(String inetHost, Promise<T> promise) {
public Future<T> resolve(String inetHost, Promise<T> promise) {
return resolve(resolvesInProgress, inetHost, promise, false);
}
@Override
public Promise<List<T>> resolveAll(String inetHost, Promise<List<T>> promise) {
public Future<List<T>> resolveAll(String inetHost, Promise<List<T>> promise) {
return resolve(resolveAllsInProgress, inetHost, promise, true);
}
private <U> Promise<U> resolve(
private <U> Future<U> resolve(
final ConcurrentMap<String, Promise<U>> resolveMap,
final String inetHost, final Promise<U> promise, boolean resolveAll) {
final Promise<U> earlyPromise = resolveMap.putIfAbsent(inetHost, promise);
if (earlyPromise != null) {
// Name resolution for the specified inetHost is in progress already.
if (earlyPromise.isDone()) {
transferResult(earlyPromise, promise);
Future<U> earlyFuture = promise.asFuture();
if (earlyFuture.isDone()) {
transferResult(earlyFuture, promise);
} else {
earlyPromise.addListener(f -> transferResult(f, promise));
earlyFuture.addListener(f -> transferResult(f, promise));
}
} else {
try {
@ -97,12 +98,12 @@ final class InflightNameResolver<T> implements NameResolver<T> {
if (promise.isDone()) {
resolveMap.remove(inetHost);
} else {
promise.addListener(f -> resolveMap.remove(inetHost));
promise.asFuture().addListener(f -> resolveMap.remove(inetHost));
}
}
}
return promise;
return promise.asFuture();
}
private static <T> void transferResult(Future<? extends T> src, Promise<T> dst) {

View File

@ -55,7 +55,7 @@ public class DnsAddressResolverGroupTest {
promise.setFailure(cause);
}
}).await();
promise.sync();
promise.asFuture().sync();
} finally {
resolverGroup.close();
group.shutdownGracefully();

View File

@ -3138,7 +3138,7 @@ public class DnsNameResolverTest {
final TestDnsServer dnsServer1 = new TestDnsServer(Collections.emptySet()) {
@Override
protected DnsMessage filterMessage(DnsMessage message) {
promise.cancel(true);
promise.cancel();
return message;
}
};

View File

@ -104,7 +104,7 @@ public abstract class AbstractAddressResolver<T extends SocketAddress> implement
final T cast = (T) address;
final Promise<T> promise = executor().newPromise();
doResolve(cast, promise);
return promise;
return promise.asFuture();
} catch (Exception e) {
return executor().newFailedFuture(e);
}
@ -117,24 +117,26 @@ public abstract class AbstractAddressResolver<T extends SocketAddress> implement
if (!isSupported(address)) {
// Address type not supported by the resolver
return promise.setFailure(new UnsupportedAddressTypeException());
promise.setFailure(new UnsupportedAddressTypeException());
return promise.asFuture();
}
if (isResolved(address)) {
// Resolved already; no need to perform a lookup
@SuppressWarnings("unchecked")
final T cast = (T) address;
return promise.setSuccess(cast);
promise.setSuccess(cast);
return promise.asFuture();
}
try {
@SuppressWarnings("unchecked")
final T cast = (T) address;
doResolve(cast, promise);
return promise;
} catch (Exception e) {
return promise.setFailure(e);
promise.setFailure(e);
}
return promise.asFuture();
}
@Override
@ -156,7 +158,7 @@ public abstract class AbstractAddressResolver<T extends SocketAddress> implement
final T cast = (T) address;
final Promise<List<T>> promise = executor().newPromise();
doResolveAll(cast, promise);
return promise;
return promise.asFuture();
} catch (Exception e) {
return executor().newFailedFuture(e);
}
@ -169,24 +171,26 @@ public abstract class AbstractAddressResolver<T extends SocketAddress> implement
if (!isSupported(address)) {
// Address type not supported by the resolver
return promise.setFailure(new UnsupportedAddressTypeException());
promise.setFailure(new UnsupportedAddressTypeException());
return promise.asFuture();
}
if (isResolved(address)) {
// Resolved already; no need to perform a lookup
@SuppressWarnings("unchecked")
final T cast = (T) address;
return promise.setSuccess(Collections.singletonList(cast));
promise.setSuccess(Collections.singletonList(cast));
return promise.asFuture();
}
try {
@SuppressWarnings("unchecked")
final T cast = (T) address;
doResolveAll(cast, promise);
return promise;
} catch (Exception e) {
return promise.setFailure(e);
promise.setFailure(e);
}
return promise.asFuture();
}
/**

View File

@ -59,10 +59,10 @@ public abstract class SimpleNameResolver<T> implements NameResolver<T> {
try {
doResolve(inetHost, promise);
return promise;
} catch (Exception e) {
return promise.setFailure(e);
promise.setFailure(e);
}
return promise.asFuture();
}
@Override
@ -77,10 +77,10 @@ public abstract class SimpleNameResolver<T> implements NameResolver<T> {
try {
doResolveAll(inetHost, promise);
return promise;
} catch (Exception e) {
return promise.setFailure(e);
promise.setFailure(e);
}
return promise.asFuture();
}
/**

View File

@ -101,8 +101,8 @@ public abstract class AbstractSocketReuseFdTest extends AbstractSocketTest {
cb.connect(sc.localAddress()).addListener(listener);
}
clientDonePromise.sync();
serverDonePromise.sync();
clientDonePromise.asFuture().sync();
serverDonePromise.asFuture().sync();
sc.close().sync();
if (globalException.get() != null && !(globalException.get() instanceof IOException)) {

View File

@ -69,7 +69,7 @@ public class DatagramConnectNotExistsTest extends AbstractClientSocketTest {
assertTrue(datagramChannel.isActive());
datagramChannel.writeAndFlush(
Unpooled.copiedBuffer("test", CharsetUtil.US_ASCII)).syncUninterruptibly();
assertTrue(promise.syncUninterruptibly().getNow() instanceof PortUnreachableException);
assertTrue(promise.asFuture().syncUninterruptibly().getNow() instanceof PortUnreachableException);
} finally {
if (datagramChannel != null) {
datagramChannel.close();

View File

@ -22,7 +22,6 @@ import io.netty.channel.Channel;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.util.concurrent.DefaultEventExecutorGroup;
import io.netty.util.concurrent.DefaultPromise;
import io.netty.util.concurrent.DefaultThreadFactory;
import io.netty.util.concurrent.EventExecutor;
import io.netty.util.concurrent.Promise;
@ -55,7 +54,7 @@ public class SocketBufReleaseTest extends AbstractSocketTest {
Channel cc = cb.connect(sc.localAddress()).get();
// Ensure the server socket accepted the client connection *and* initialized pipeline successfully.
serverHandler.channelFuture.sync();
serverHandler.channelFuture.asFuture().sync();
// and then close all sockets.
sc.close().sync();
@ -73,7 +72,7 @@ public class SocketBufReleaseTest extends AbstractSocketTest {
private final Random random = new Random();
private final CountDownLatch latch = new CountDownLatch(1);
private ByteBuf buf;
private final Promise<Channel> channelFuture = new DefaultPromise<>(executor);
private final Promise<Channel> channelFuture = executor.newPromise();
@Override
public void handlerAdded(ChannelHandlerContext ctx) throws Exception {

View File

@ -78,7 +78,7 @@ public class SocketConnectTest extends AbstractSocketTest {
assertLocalAddress((InetSocketAddress) clientChannel.localAddress());
assertNotNull(clientChannel.remoteAddress());
assertLocalAddress(localAddressPromise.get());
assertLocalAddress(localAddressPromise.asFuture().get());
} finally {
if (clientChannel != null) {
clientChannel.close().syncUninterruptibly();

View File

@ -151,7 +151,7 @@ public class SocketGatheringWriteTest extends AbstractSocketTest {
throw t;
}
serverDonePromise.sync();
serverDonePromise.asFuture().sync();
sh.channel.close().sync();
ch.channel.close().sync();
sc.close().sync();

View File

@ -111,8 +111,8 @@ public class SocketStringEchoTest extends AbstractSocketTest {
cc.writeAndFlush(element + delimiter);
}
ch.donePromise.sync();
sh.donePromise.sync();
ch.donePromise.asFuture().sync();
sh.donePromise.asFuture().sync();
sh.channel.close().sync();
ch.channel.close().sync();
sc.close().sync();

View File

@ -348,7 +348,7 @@ public class TrafficShapingHandlerTest extends AbstractSocketTest {
}
cc.flush();
promise.await();
promise.asFuture().await();
Long stop = TrafficCounter.milliSecondFromNano();
assertTrue(promise.isSuccess(), "Error during execution of TrafficShapping: " + promise.cause());

View File

@ -609,7 +609,7 @@ abstract class AbstractEpollChannel extends AbstractChannel implements UnixChann
}, connectTimeoutMillis, TimeUnit.MILLISECONDS);
}
promise.addListener(future -> {
promise.asFuture().addListener(future -> {
if (future.isCancelled()) {
if (connectTimeoutFuture != null) {
connectTimeoutFuture.cancel(false);

View File

@ -242,7 +242,8 @@ public abstract class AbstractEpollStreamChannel extends AbstractEpollChannel im
in.remove();
}
return 1;
} else if (flushedAmount == 0) {
}
if (flushedAmount == 0) {
validateFileRegion(region, offset);
}
return WRITE_STATUS_SNDBUF_FULL;
@ -437,7 +438,7 @@ public abstract class AbstractEpollStreamChannel extends AbstractEpollChannel im
loop.execute(() -> ((AbstractUnsafe) unsafe()).shutdownOutput(promise));
}
return promise;
return promise.asFuture();
}
@Override
@ -458,7 +459,7 @@ public abstract class AbstractEpollStreamChannel extends AbstractEpollChannel im
loop.execute(() -> shutdownInput0(promise));
}
}
return promise;
return promise.asFuture();
}
@Override
@ -474,7 +475,7 @@ public abstract class AbstractEpollStreamChannel extends AbstractEpollChannel im
} else {
shutdownOutputFuture.addListener(promise, this::shutdownOutputDone);
}
return promise;
return promise.asFuture();
}
private void shutdownOutputDone(Promise<Void> promise, Future<?> shutdownOutputFuture) {

View File

@ -149,9 +149,8 @@ public final class EpollDatagramChannel extends AbstractEpollChannel implements
}
return joinGroup(multicastAddress, iface, null, promise);
} catch (IOException e) {
promise.setFailure(e);
return promise.setFailure(e).asFuture();
}
return promise;
}
@Override
@ -186,7 +185,7 @@ public final class EpollDatagramChannel extends AbstractEpollChannel implements
} catch (IOException e) {
promise.setFailure(e);
}
return promise;
return promise.asFuture();
}
@Override
@ -200,9 +199,8 @@ public final class EpollDatagramChannel extends AbstractEpollChannel implements
return leaveGroup(
multicastAddress, NetworkInterface.getByInetAddress(localAddress().getAddress()), null, promise);
} catch (IOException e) {
promise.setFailure(e);
return promise.setFailure(e).asFuture();
}
return promise;
}
@Override
@ -237,7 +235,7 @@ public final class EpollDatagramChannel extends AbstractEpollChannel implements
} catch (IOException e) {
promise.setFailure(e);
}
return promise;
return promise.asFuture();
}
@Override
@ -255,7 +253,7 @@ public final class EpollDatagramChannel extends AbstractEpollChannel implements
requireNonNull(sourceToBlock, "sourceToBlock");
requireNonNull(networkInterface, "networkInterface");
promise.setFailure(new UnsupportedOperationException("Multicast not supported"));
return promise;
return promise.asFuture();
}
@Override
@ -274,7 +272,7 @@ public final class EpollDatagramChannel extends AbstractEpollChannel implements
} catch (Throwable e) {
promise.setFailure(e);
}
return promise;
return promise.asFuture();
}
@Override

View File

@ -560,7 +560,7 @@ abstract class AbstractKQueueChannel extends AbstractChannel implements UnixChan
}, connectTimeoutMillis, TimeUnit.MILLISECONDS);
}
promise.addListener(future -> {
promise.asFuture().addListener(future -> {
if (future.isCancelled()) {
if (connectTimeoutFuture != null) {
connectTimeoutFuture.cancel(false);

View File

@ -224,7 +224,8 @@ public abstract class AbstractKQueueStreamChannel extends AbstractKQueueChannel
in.remove();
}
return 1;
} else if (flushedAmount == 0) {
}
if (flushedAmount == 0) {
validateFileRegion(region, offset);
}
return WRITE_STATUS_SNDBUF_FULL;
@ -409,7 +410,7 @@ public abstract class AbstractKQueueStreamChannel extends AbstractKQueueChannel
} else {
loop.execute(() -> ((AbstractUnsafe) unsafe()).shutdownOutput(promise));
}
return promise;
return promise.asFuture();
}
@Override
@ -425,7 +426,7 @@ public abstract class AbstractKQueueStreamChannel extends AbstractKQueueChannel
} else {
loop.execute(() -> shutdownInput0(promise));
}
return promise;
return promise.asFuture();
}
private void shutdownInput0(Promise<Void> promise) {
@ -451,7 +452,7 @@ public abstract class AbstractKQueueStreamChannel extends AbstractKQueueChannel
} else {
shutdownOutputFuture.addListener(promise, this::shutdownOutputDone);
}
return promise;
return promise.asFuture();
}
private void shutdownOutputDone(Promise<Void> promise, Future<?> shutdownOutputFuture) {

View File

@ -106,9 +106,8 @@ public final class KQueueDatagramChannel extends AbstractKQueueDatagramChannel i
}
return joinGroup(multicastAddress, iface, null, promise);
} catch (SocketException e) {
promise.setFailure(e);
return promise.setFailure(e).asFuture();
}
return promise;
}
@Override
@ -138,7 +137,7 @@ public final class KQueueDatagramChannel extends AbstractKQueueDatagramChannel i
requireNonNull(networkInterface, "networkInterface");
promise.setFailure(new UnsupportedOperationException("Multicast not supported"));
return promise;
return promise.asFuture();
}
@Override
@ -152,9 +151,8 @@ public final class KQueueDatagramChannel extends AbstractKQueueDatagramChannel i
return leaveGroup(
multicastAddress, NetworkInterface.getByInetAddress(localAddress().getAddress()), null, promise);
} catch (SocketException e) {
promise.setFailure(e);
return promise.setFailure(e).asFuture();
}
return promise;
}
@Override
@ -184,8 +182,7 @@ public final class KQueueDatagramChannel extends AbstractKQueueDatagramChannel i
requireNonNull(networkInterface, "networkInterface");
promise.setFailure(new UnsupportedOperationException("Multicast not supported"));
return promise;
return promise.asFuture();
}
@Override
@ -203,7 +200,7 @@ public final class KQueueDatagramChannel extends AbstractKQueueDatagramChannel i
requireNonNull(sourceToBlock, "sourceToBlock");
requireNonNull(networkInterface, "networkInterface");
promise.setFailure(new UnsupportedOperationException("Multicast not supported"));
return promise;
return promise.asFuture();
}
@Override
@ -220,9 +217,8 @@ public final class KQueueDatagramChannel extends AbstractKQueueDatagramChannel i
NetworkInterface.getByInetAddress(localAddress().getAddress()),
sourceToBlock, promise);
} catch (Throwable e) {
promise.setFailure(e);
return promise.setFailure(e).asFuture();
}
return promise;
}
@Override

View File

@ -352,7 +352,7 @@ public class NioSctpChannel extends AbstractNioMessageChannel implements io.nett
} else {
executor().execute(() -> bindAddress(localAddress, promise));
}
return promise;
return promise.asFuture();
}
@Override
@ -372,7 +372,7 @@ public class NioSctpChannel extends AbstractNioMessageChannel implements io.nett
} else {
executor().execute(() -> unbindAddress(localAddress, promise));
}
return promise;
return promise.asFuture();
}
private final class NioSctpChannelConfig extends DefaultSctpChannelConfig {

View File

@ -172,7 +172,7 @@ public class NioSctpServerChannel extends AbstractNioMessageChannel
} else {
executor().execute(() -> bindAddress(localAddress, promise));
}
return promise;
return promise.asFuture();
}
@Override
@ -192,7 +192,7 @@ public class NioSctpServerChannel extends AbstractNioMessageChannel
} else {
executor().execute(() -> unbindAddress(localAddress, promise));
}
return promise;
return promise.asFuture();
}
// Unnecessary stuff

View File

@ -23,7 +23,6 @@ import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoop;
import io.netty.channel.EventLoopGroup;
import io.netty.util.AttributeKey;
import io.netty.util.concurrent.DefaultPromise;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.Promise;
import io.netty.util.internal.SocketUtils;
@ -243,12 +242,12 @@ public abstract class AbstractBootstrap<B extends AbstractBootstrap<B, C, F>, C
return regFuture;
}
Promise<Channel> bindPromise = new DefaultPromise<>(loop);
Promise<Channel> bindPromise = loop.newPromise();
if (regFuture.isDone()) {
// At this point we know that the registration was complete and successful.
Channel channel = regFuture.getNow();
Promise<Void> promise = channel.newPromise();
promise.map(v -> channel).cascadeTo(bindPromise);
promise.asFuture().map(v -> channel).cascadeTo(bindPromise);
doBind0(regFuture, channel, localAddress, promise);
} else {
// Registration future is almost always fulfilled already, but just in case it's not.
@ -261,12 +260,12 @@ public abstract class AbstractBootstrap<B extends AbstractBootstrap<B, C, F>, C
} else {
Channel channel = future.getNow();
Promise<Void> promise = channel.newPromise();
promise.map(v -> channel).cascadeTo(bindPromise);
promise.asFuture().map(v -> channel).cascadeTo(bindPromise);
doBind0(regFuture, channel, localAddress, promise);
}
});
}
return bindPromise;
return bindPromise.asFuture();
}
final Future<Channel> initAndRegister(EventLoop loop) {
@ -274,10 +273,10 @@ public abstract class AbstractBootstrap<B extends AbstractBootstrap<B, C, F>, C
try {
channel = newChannel(loop);
} catch (Throwable t) {
return DefaultPromise.newFailedPromise(loop, t);
return loop.newFailedFuture(t);
}
Promise<Channel> promise = new DefaultPromise<>(loop);
Promise<Channel> promise = loop.newPromise();
loop.execute(() -> init(channel).addListener(future -> {
if (future.isSuccess()) {
// TODO eventually I think we'd like to be able to either pass the generic promise down,
@ -291,7 +290,7 @@ public abstract class AbstractBootstrap<B extends AbstractBootstrap<B, C, F>, C
}
}));
return promise;
return promise.asFuture();
}
final Channel initWithoutRegister() throws Exception {

View File

@ -26,7 +26,6 @@ import io.netty.resolver.AddressResolver;
import io.netty.resolver.AddressResolverGroup;
import io.netty.resolver.DefaultAddressResolverGroup;
import io.netty.resolver.NameResolver;
import io.netty.util.concurrent.DefaultPromise;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.Promise;
import io.netty.util.internal.logging.InternalLogger;
@ -187,7 +186,7 @@ public class Bootstrap extends AbstractBootstrap<Bootstrap, Channel, ChannelFact
EventLoop loop = group.next();
final Future<Channel> regFuture = initAndRegister(loop);
Promise<Channel> resolveAndConnectPromise = new DefaultPromise<>(loop);
Promise<Channel> resolveAndConnectPromise = loop.newPromise();
if (regFuture.isDone()) {
if (regFuture.isFailed()) {
return regFuture;
@ -210,7 +209,7 @@ public class Bootstrap extends AbstractBootstrap<Bootstrap, Channel, ChannelFact
}
});
}
return resolveAndConnectPromise;
return resolveAndConnectPromise.asFuture();
}
private void doResolveAndConnect0(final Channel channel, SocketAddress remoteAddress,
@ -280,7 +279,7 @@ public class Bootstrap extends AbstractBootstrap<Bootstrap, Channel, ChannelFact
p.addLast(config.handler());
return DefaultPromise.newSuccessfulPromise(channel.executor(), channel);
return channel.executor().newSucceededFuture(channel);
}
@Override

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