Skip to content

Commit e490273

Browse files
authored
netty: Handle write queue promise failures (#11016)
Handles Netty write frame failures caused by issues in the Netty itself. Normally we don't need to do anything on frame write failures because the cause of a failed future would be an IO error that resulted in the stream closure. Prior to this PR we treated these issues as a noop, except the initial headers write on the client side. However, a case like netty/netty#13805 (a bug in generating next stream id) resulted in an unclosed stream on our side. This PR adds write frame future failure handlers that ensures the stream is cancelled, and the cause is propagated via Status. Fixes #10849
1 parent 497e155 commit e490273

File tree

7 files changed

+241
-50
lines changed

7 files changed

+241
-50
lines changed

core/src/main/java/io/grpc/internal/AbstractServerStream.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -293,6 +293,7 @@ public void inboundDataReceived(ReadableBuffer frame, boolean endOfStream) {
293293
*/
294294
public final void transportReportStatus(final Status status) {
295295
Preconditions.checkArgument(!status.isOk(), "status must not be OK");
296+
onStreamDeallocated();
296297
if (deframerClosed) {
297298
deframerClosedTask = null;
298299
closeListener(status);
@@ -315,6 +316,7 @@ public void run() {
315316
* #transportReportStatus}.
316317
*/
317318
public void complete() {
319+
onStreamDeallocated();
318320
if (deframerClosed) {
319321
deframerClosedTask = null;
320322
closeListener(Status.OK);
@@ -350,7 +352,6 @@ private void closeListener(Status newStatus) {
350352
getTransportTracer().reportStreamClosed(closedStatus.isOk());
351353
}
352354
listenerClosed = true;
353-
onStreamDeallocated();
354355
listener().closed(newStatus);
355356
}
356357
}

core/src/main/java/io/grpc/internal/AbstractStream.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -322,6 +322,12 @@ protected final void onStreamDeallocated() {
322322
}
323323
}
324324

325+
protected boolean isStreamDeallocated() {
326+
synchronized (onReadyLock) {
327+
return deallocated;
328+
}
329+
}
330+
325331
/**
326332
* Event handler to be called by the subclass when a number of bytes are being queued for
327333
* sending to the remote endpoint.

netty/src/main/java/io/grpc/netty/NettyClientStream.java

Lines changed: 26 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -182,20 +182,10 @@ private void writeFrameInternal(
182182
if (numBytes > 0) {
183183
// Add the bytes to outbound flow control.
184184
onSendingBytes(numBytes);
185+
ChannelFutureListener failureListener =
186+
future -> transportState().onWriteFrameData(future, numMessages, numBytes);
185187
writeQueue.enqueue(new SendGrpcFrameCommand(transportState(), bytebuf, endOfStream), flush)
186-
.addListener(new ChannelFutureListener() {
187-
@Override
188-
public void operationComplete(ChannelFuture future) throws Exception {
189-
// If the future succeeds when http2stream is null, the stream has been cancelled
190-
// before it began and Netty is purging pending writes from the flow-controller.
191-
if (future.isSuccess() && transportState().http2Stream() != null) {
192-
// Remove the bytes from outbound flow control, optionally notifying
193-
// the client that they can send more bytes.
194-
transportState().onSentBytes(numBytes);
195-
NettyClientStream.this.getTransportTracer().reportMessageSent(numMessages);
196-
}
197-
}
198-
});
188+
.addListener(failureListener);
199189
} else {
200190
// The frame is empty and will not impact outbound flow control. Just send it.
201191
writeQueue.enqueue(
@@ -307,6 +297,29 @@ protected void http2ProcessingFailed(Status status, boolean stopDelivery, Metada
307297
handler.getWriteQueue().enqueue(new CancelClientStreamCommand(this, status), true);
308298
}
309299

300+
private void onWriteFrameData(ChannelFuture future, int numMessages, int numBytes) {
301+
// If the future succeeds when http2stream is null, the stream has been cancelled
302+
// before it began and Netty is purging pending writes from the flow-controller.
303+
if (future.isSuccess() && http2Stream() == null) {
304+
return;
305+
}
306+
307+
if (future.isSuccess()) {
308+
// Remove the bytes from outbound flow control, optionally notifying
309+
// the client that they can send more bytes.
310+
onSentBytes(numBytes);
311+
getTransportTracer().reportMessageSent(numMessages);
312+
} else if (!isStreamDeallocated()) {
313+
// Future failed, fail RPC.
314+
// Normally we don't need to do anything here because the cause of a failed future
315+
// while writing DATA frames would be an IO error and the stream is already closed.
316+
// However, we still need handle any unexpected failures raised in Netty.
317+
// Note: isStreamDeallocated() protects from spamming stream resets by scheduling multiple
318+
// CancelClientStreamCommand commands.
319+
http2ProcessingFailed(statusFromFailedFuture(future), true, new Metadata());
320+
}
321+
}
322+
310323
@Override
311324
public void runOnTransportThread(final Runnable r) {
312325
if (eventLoop.inEventLoop()) {

netty/src/main/java/io/grpc/netty/NettyServerHandler.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -502,8 +502,7 @@ private void onHeadersRead(ChannelHandlerContext ctx, int streamId, Http2Headers
502502
state,
503503
attributes,
504504
authority,
505-
statsTraceCtx,
506-
transportTracer);
505+
statsTraceCtx);
507506
transportListener.streamCreated(stream, method, metadata);
508507
state.onStreamAllocated();
509508
http2Stream.setProperty(streamKey, state);

netty/src/main/java/io/grpc/netty/NettyServerStream.java

Lines changed: 52 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -52,22 +52,19 @@ class NettyServerStream extends AbstractServerStream {
5252
private final WriteQueue writeQueue;
5353
private final Attributes attributes;
5454
private final String authority;
55-
private final TransportTracer transportTracer;
5655
private final int streamId;
5756

5857
public NettyServerStream(
5958
Channel channel,
6059
TransportState state,
6160
Attributes transportAttrs,
6261
String authority,
63-
StatsTraceContext statsTraceCtx,
64-
TransportTracer transportTracer) {
62+
StatsTraceContext statsTraceCtx) {
6563
super(new NettyWritableBufferAllocator(channel.alloc()), statsTraceCtx);
6664
this.state = checkNotNull(state, "transportState");
6765
this.writeQueue = state.handler.getWriteQueue();
6866
this.attributes = checkNotNull(transportAttrs);
6967
this.authority = authority;
70-
this.transportTracer = checkNotNull(transportTracer, "transportTracer");
7168
// Read the id early to avoid reading transportState later.
7269
this.streamId = transportState().id();
7370
}
@@ -96,48 +93,37 @@ private class Sink implements AbstractServerStream.Sink {
9693
@Override
9794
public void writeHeaders(Metadata headers, boolean flush) {
9895
try (TaskCloseable ignore = PerfMark.traceTask("NettyServerStream$Sink.writeHeaders")) {
99-
writeQueue.enqueue(
100-
SendResponseHeadersCommand.createHeaders(
101-
transportState(),
102-
Utils.convertServerHeaders(headers)),
103-
flush);
96+
Http2Headers http2headers = Utils.convertServerHeaders(headers);
97+
SendResponseHeadersCommand headersCommand =
98+
SendResponseHeadersCommand.createHeaders(transportState(), http2headers);
99+
writeQueue.enqueue(headersCommand, flush)
100+
.addListener((ChannelFutureListener) transportState()::handleWriteFutureFailures);
104101
}
105102
}
106103

107-
private void writeFrameInternal(WritableBuffer frame, boolean flush, final int numMessages) {
108-
Preconditions.checkArgument(numMessages >= 0);
109-
ByteBuf bytebuf = ((NettyWritableBuffer) frame).bytebuf().touch();
110-
final int numBytes = bytebuf.readableBytes();
111-
// Add the bytes to outbound flow control.
112-
onSendingBytes(numBytes);
113-
writeQueue.enqueue(new SendGrpcFrameCommand(transportState(), bytebuf, false), flush)
114-
.addListener(new ChannelFutureListener() {
115-
@Override
116-
public void operationComplete(ChannelFuture future) throws Exception {
117-
// Remove the bytes from outbound flow control, optionally notifying
118-
// the client that they can send more bytes.
119-
transportState().onSentBytes(numBytes);
120-
if (future.isSuccess()) {
121-
transportTracer.reportMessageSent(numMessages);
122-
}
123-
}
124-
});
125-
}
126-
127104
@Override
128105
public void writeFrame(WritableBuffer frame, boolean flush, final int numMessages) {
129106
try (TaskCloseable ignore = PerfMark.traceTask("NettyServerStream$Sink.writeFrame")) {
130-
writeFrameInternal(frame, flush, numMessages);
107+
Preconditions.checkArgument(numMessages >= 0);
108+
ByteBuf bytebuf = ((NettyWritableBuffer) frame).bytebuf().touch();
109+
final int numBytes = bytebuf.readableBytes();
110+
// Add the bytes to outbound flow control.
111+
onSendingBytes(numBytes);
112+
ChannelFutureListener failureListener =
113+
future -> transportState().onWriteFrameData(future, numMessages, numBytes);
114+
writeQueue.enqueue(new SendGrpcFrameCommand(transportState(), bytebuf, false), flush)
115+
.addListener(failureListener);
131116
}
132117
}
133118

134119
@Override
135120
public void writeTrailers(Metadata trailers, boolean headersSent, Status status) {
136121
try (TaskCloseable ignore = PerfMark.traceTask("NettyServerStream$Sink.writeTrailers")) {
137122
Http2Headers http2Trailers = Utils.convertTrailers(trailers, headersSent);
138-
writeQueue.enqueue(
139-
SendResponseHeadersCommand.createTrailers(transportState(), http2Trailers, status),
140-
true);
123+
SendResponseHeadersCommand trailersCommand =
124+
SendResponseHeadersCommand.createTrailers(transportState(), http2Trailers, status);
125+
writeQueue.enqueue(trailersCommand, true)
126+
.addListener((ChannelFutureListener) transportState()::handleWriteFutureFailures);
141127
}
142128
}
143129

@@ -206,6 +192,39 @@ public void deframeFailed(Throwable cause) {
206192
handler.getWriteQueue().enqueue(new CancelServerStreamCommand(this, status), true);
207193
}
208194

195+
private void onWriteFrameData(ChannelFuture future, int numMessages, int numBytes) {
196+
// Remove the bytes from outbound flow control, optionally notifying
197+
// the client that they can send more bytes.
198+
if (future.isSuccess()) {
199+
onSentBytes(numBytes);
200+
getTransportTracer().reportMessageSent(numMessages);
201+
} else {
202+
handleWriteFutureFailures(future);
203+
}
204+
}
205+
206+
private void handleWriteFutureFailures(ChannelFuture future) {
207+
// isStreamDeallocated() check protects from spamming stream resets by scheduling multiple
208+
// CancelServerStreamCommand commands.
209+
if (future.isSuccess() || isStreamDeallocated()) {
210+
return;
211+
}
212+
213+
// Future failed, fail RPC.
214+
// Normally we don't need to do anything on frame write failures because the cause of
215+
// the failed future would be an IO error that closed the stream.
216+
// However, we still need handle any unexpected failures raised in Netty.
217+
http2ProcessingFailed(Utils.statusFromThrowable(future.cause()));
218+
}
219+
220+
/**
221+
* Called to process a failure in HTTP/2 processing.
222+
*/
223+
protected void http2ProcessingFailed(Status status) {
224+
transportReportStatus(status);
225+
handler.getWriteQueue().enqueue(new CancelServerStreamCommand(this, status), true);
226+
}
227+
209228
void inboundDataReceived(ByteBuf frame, boolean endOfStream) {
210229
super.inboundDataReceived(new NettyReadableBuffer(frame.retain()), endOfStream);
211230
}

netty/src/test/java/io/grpc/netty/NettyClientStreamTest.java

Lines changed: 49 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,8 @@
2323
import static io.grpc.netty.Utils.CONTENT_TYPE_GRPC;
2424
import static io.grpc.netty.Utils.CONTENT_TYPE_HEADER;
2525
import static io.grpc.netty.Utils.STATUS_OK;
26+
import static io.netty.handler.codec.http2.Http2Error.PROTOCOL_ERROR;
27+
import static io.netty.handler.codec.http2.Http2Exception.connectionError;
2628
import static io.netty.util.CharsetUtil.UTF_8;
2729
import static org.junit.Assert.assertEquals;
2830
import static org.junit.Assert.assertFalse;
@@ -34,6 +36,7 @@
3436
import static org.mockito.ArgumentMatchers.eq;
3537
import static org.mockito.ArgumentMatchers.isA;
3638
import static org.mockito.ArgumentMatchers.same;
39+
import static org.mockito.Mockito.atLeast;
3740
import static org.mockito.Mockito.doAnswer;
3841
import static org.mockito.Mockito.mock;
3942
import static org.mockito.Mockito.never;
@@ -62,6 +65,7 @@
6265
import io.netty.channel.ChannelPromise;
6366
import io.netty.channel.DefaultChannelPromise;
6467
import io.netty.handler.codec.http2.DefaultHttp2Headers;
68+
import io.netty.handler.codec.http2.Http2Exception;
6569
import io.netty.handler.codec.http2.Http2Headers;
6670
import io.netty.util.AsciiString;
6771
import java.io.BufferedInputStream;
@@ -75,6 +79,7 @@
7579
import org.junit.runners.JUnit4;
7680
import org.mockito.ArgumentCaptor;
7781
import org.mockito.ArgumentMatchers;
82+
import org.mockito.InOrder;
7883
import org.mockito.Mock;
7984
import org.mockito.Mockito;
8085
import org.mockito.invocation.InvocationOnMock;
@@ -205,6 +210,50 @@ public void writeMessageShouldSendRequestUnknownLength() throws Exception {
205210
eq(true));
206211
}
207212

213+
@Test
214+
public void writeFrameFutureFailedShouldCancelRpc() {
215+
Http2Exception h2Error = connectionError(PROTOCOL_ERROR, "Stream does not exist %d", STREAM_ID);
216+
// Fail all SendGrpcFrameCommands command sent to the queue.
217+
when(writeQueue.enqueue(any(SendGrpcFrameCommand.class), anyBoolean())).thenReturn(
218+
new DefaultChannelPromise(channel).setFailure(h2Error));
219+
220+
// Write multiple messages to ensure multiple SendGrpcFrameCommand are enqueued. We set up all
221+
// of them to fail, which allows us to assert that only a single cancel is sent, and the stream
222+
// isn't spammed with multiple RST_STREAM.
223+
stream().transportState().setId(STREAM_ID);
224+
stream.writeMessage(new ByteArrayInputStream(smallMessage()));
225+
stream.writeMessage(new ByteArrayInputStream(largeMessage()));
226+
stream.flush();
227+
228+
InOrder inOrder = Mockito.inOrder(writeQueue);
229+
// Normal stream create and write frame.
230+
inOrder.verify(writeQueue).enqueue(any(CreateStreamCommand.class), eq(false));
231+
inOrder.verify(writeQueue).enqueue(any(SendGrpcFrameCommand.class), eq(false));
232+
// Verify that failed SendGrpcFrameCommand results in immediate CancelClientStreamCommand.
233+
inOrder.verify(writeQueue).enqueue(any(CancelClientStreamCommand.class), eq(true));
234+
// Verify that any other failures do not produce another CancelClientStreamCommand in the queue.
235+
inOrder.verify(writeQueue, atLeast(1)).enqueue(any(SendGrpcFrameCommand.class), eq(false));
236+
inOrder.verify(writeQueue).enqueue(any(SendGrpcFrameCommand.class), eq(true));
237+
inOrder.verifyNoMoreInteractions();
238+
239+
// Get the CancelClientStreamCommand written to the queue. Above we verified that there is
240+
// only one CancelClientStreamCommand enqueued, and is the third enqueued command (create,
241+
// frame write failure, cancel).
242+
CancelClientStreamCommand cancelCommand = Mockito.mockingDetails(writeQueue).getInvocations()
243+
// Get enqueue() innovations only
244+
.stream().filter(invocation -> invocation.getMethod().getName().equals("enqueue"))
245+
// Get the third invocation of enqueue()
246+
.skip(2).findFirst().get()
247+
// Get the first argument (QueuedCommand command)
248+
.getArgument(0);
249+
250+
Status cancelReason = cancelCommand.reason();
251+
assertThat(cancelReason.getCode()).isEqualTo(Status.INTERNAL.getCode());
252+
assertThat(cancelReason.getCause()).isEqualTo(h2Error);
253+
// Verify listener closed.
254+
verify(listener).closed(same(cancelReason), eq(PROCESSED), any(Metadata.class));
255+
}
256+
208257
@Test
209258
public void setStatusWithOkShouldCloseStream() {
210259
stream().transportState().setId(STREAM_ID);

0 commit comments

Comments
 (0)