mirror of https://github.com/grpc/grpc-java.git
Context deadline propagation should cascade. Fixes #1205
A call's timeout as specified in its metadata should be set depending on the deadline of the call's context. If a call has an explicit deadline set (through CallOptions), then the smaller deadline (from context and call options) should be used to compute the timeout. Also, a new method Contexts.statusFromCancelled(Context) was introduced that attempts to map a canceled context to a gRPC status.
This commit is contained in:
parent
363e0f6cfc
commit
fd8fd517d2
|
|
@ -651,14 +651,20 @@ public class Context {
|
|||
ScheduledExecutorService scheduler) {
|
||||
super(parent, deriveDeadline(parent, deadline), true);
|
||||
if (DEADLINE_KEY.get(this) == deadline) {
|
||||
// The parent deadline was after the new deadline so we need to install a listener
|
||||
// on the new earlier deadline to trigger expiration for this context.
|
||||
pendingDeadline = deadline.runOnExpiration(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
cancel(new TimeoutException("context timed out"));
|
||||
}
|
||||
}, scheduler);
|
||||
final TimeoutException cause = new TimeoutException("context timed out");
|
||||
if (!deadline.isExpired()) {
|
||||
// The parent deadline was after the new deadline so we need to install a listener
|
||||
// on the new earlier deadline to trigger expiration for this context.
|
||||
pendingDeadline = deadline.runOnExpiration(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
cancel(cause);
|
||||
}
|
||||
}, scheduler);
|
||||
} else {
|
||||
// Cancel immediately if the deadline is already expired.
|
||||
cancel(cause);
|
||||
}
|
||||
}
|
||||
uncancellableSurrogate = new Context(this, EMPTY_ENTRIES);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -31,6 +31,10 @@
|
|||
|
||||
package io.grpc;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
/**
|
||||
* Utility methods for working with {@link Context}s in GRPC.
|
||||
*/
|
||||
|
|
@ -130,4 +134,33 @@ public class Contexts {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link Status} of a cancelled context or {@code null} if the context
|
||||
* is not cancelled.
|
||||
*/
|
||||
public static Status statusFromCancelled(Context context) {
|
||||
Preconditions.checkNotNull(context, "context must not be null");
|
||||
if (!context.isCancelled()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
Throwable cancellationCause = context.cancellationCause();
|
||||
if (cancellationCause == null) {
|
||||
return Status.CANCELLED;
|
||||
}
|
||||
if (cancellationCause instanceof TimeoutException) {
|
||||
return Status.DEADLINE_EXCEEDED
|
||||
.withDescription(cancellationCause.getMessage())
|
||||
.withCause(cancellationCause);
|
||||
}
|
||||
Status status = Status.fromThrowable(cancellationCause);
|
||||
if (Status.Code.UNKNOWN.equals(status.getCode())
|
||||
&& status.getCause() == cancellationCause) {
|
||||
// If fromThrowable could not determine a status, then
|
||||
// just return CANCELLED.
|
||||
return Status.CANCELLED.withCause(cancellationCause);
|
||||
}
|
||||
return status.withCause(cancellationCause);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -35,12 +35,14 @@ import static com.google.common.base.Preconditions.checkArgument;
|
|||
import static com.google.common.base.Preconditions.checkNotNull;
|
||||
import static com.google.common.base.Preconditions.checkState;
|
||||
import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
|
||||
import static io.grpc.Contexts.statusFromCancelled;
|
||||
import static io.grpc.Status.DEADLINE_EXCEEDED;
|
||||
import static io.grpc.internal.GrpcUtil.ACCEPT_ENCODING_JOINER;
|
||||
import static io.grpc.internal.GrpcUtil.MESSAGE_ACCEPT_ENCODING_KEY;
|
||||
import static io.grpc.internal.GrpcUtil.MESSAGE_ENCODING_KEY;
|
||||
import static io.grpc.internal.GrpcUtil.TIMEOUT_KEY;
|
||||
import static io.grpc.internal.GrpcUtil.USER_AGENT_KEY;
|
||||
import static java.util.concurrent.TimeUnit.NANOSECONDS;
|
||||
import static java.lang.Math.max;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
|
@ -62,7 +64,9 @@ import io.grpc.Status;
|
|||
import java.io.InputStream;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.logging.Level;
|
||||
import java.util.logging.Logger;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
|
|
@ -71,20 +75,22 @@ import javax.annotation.Nullable;
|
|||
*/
|
||||
final class ClientCallImpl<ReqT, RespT> extends ClientCall<ReqT, RespT>
|
||||
implements Context.CancellationListener {
|
||||
|
||||
private static final Logger log = Logger.getLogger(ClientCallImpl.class.getName());
|
||||
|
||||
private final MethodDescriptor<ReqT, RespT> method;
|
||||
private final Executor callExecutor;
|
||||
private final Context context;
|
||||
private final Context parentContext;
|
||||
private volatile Context context;
|
||||
private final boolean unaryRequest;
|
||||
private final CallOptions callOptions;
|
||||
private ClientStream stream;
|
||||
private volatile ScheduledFuture<?> deadlineCancellationFuture;
|
||||
private volatile boolean deadlineCancellationFutureShouldBeCancelled;
|
||||
private volatile boolean contextListenerShouldBeRemoved;
|
||||
private boolean cancelCalled;
|
||||
private boolean halfCloseCalled;
|
||||
private final ClientTransportProvider clientTransportProvider;
|
||||
private String userAgent;
|
||||
private ScheduledExecutorService deadlineCancellationExecutor;
|
||||
private Compressor compressor;
|
||||
private DecompressorRegistry decompressorRegistry = DecompressorRegistry.getDefaultInstance();
|
||||
private CompressorRegistry compressorRegistry = CompressorRegistry.getDefaultInstance();
|
||||
|
||||
|
|
@ -99,7 +105,7 @@ final class ClientCallImpl<ReqT, RespT> extends ClientCall<ReqT, RespT>
|
|||
? new SerializeReentrantCallsDirectExecutor()
|
||||
: new SerializingExecutor(executor);
|
||||
// Propagate the context from the thread which initiated the call to all callbacks.
|
||||
this.context = Context.current();
|
||||
this.parentContext = Context.current();
|
||||
this.unaryRequest = method.getType() == MethodType.UNARY
|
||||
|| method.getType() == MethodType.SERVER_STREAMING;
|
||||
this.callOptions = callOptions;
|
||||
|
|
@ -109,7 +115,7 @@ final class ClientCallImpl<ReqT, RespT> extends ClientCall<ReqT, RespT>
|
|||
|
||||
@Override
|
||||
public void cancelled(Context context) {
|
||||
stream.cancel(Status.CANCELLED.withCause(context.cancellationCause()));
|
||||
stream.cancel(statusFromCancelled(context));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -165,6 +171,14 @@ final class ClientCallImpl<ReqT, RespT> extends ClientCall<ReqT, RespT>
|
|||
checkNotNull(observer, "observer");
|
||||
checkNotNull(headers, "headers");
|
||||
|
||||
// Create the context
|
||||
final Deadline effectiveDeadline = min(callOptions.getDeadline(), parentContext.getDeadline());
|
||||
if (effectiveDeadline != parentContext.getDeadline()) {
|
||||
context = parentContext.withDeadline(effectiveDeadline, deadlineCancellationExecutor);
|
||||
} else {
|
||||
context = parentContext.withCancellation();
|
||||
}
|
||||
|
||||
if (context.isCancelled()) {
|
||||
// Context is already cancelled so no need to create a real stream, just notify the observer
|
||||
// of cancellation via callback on the executor
|
||||
|
|
@ -172,12 +186,13 @@ final class ClientCallImpl<ReqT, RespT> extends ClientCall<ReqT, RespT>
|
|||
callExecutor.execute(new ContextRunnable(context) {
|
||||
@Override
|
||||
public void runInContext() {
|
||||
observer.onClose(Status.CANCELLED.withCause(context.cancellationCause()), new Metadata());
|
||||
observer.onClose(statusFromCancelled(context), new Metadata());
|
||||
}
|
||||
});
|
||||
return;
|
||||
}
|
||||
final String compressorName = callOptions.getCompressor();
|
||||
Compressor compressor = null;
|
||||
if (compressorName != null) {
|
||||
compressor = compressorRegistry.lookupCompressor(compressorName);
|
||||
if (compressor == null) {
|
||||
|
|
@ -199,11 +214,14 @@ final class ClientCallImpl<ReqT, RespT> extends ClientCall<ReqT, RespT>
|
|||
|
||||
prepareHeaders(headers, callOptions, userAgent, decompressorRegistry, compressor);
|
||||
|
||||
if (updateTimeoutHeader(callOptions.getDeadline(), headers)) {
|
||||
final boolean deadlineExceeded = effectiveDeadline != null && effectiveDeadline.isExpired();
|
||||
if (!deadlineExceeded) {
|
||||
updateTimeoutHeaders(effectiveDeadline, callOptions.getDeadline(),
|
||||
parentContext.getDeadline(), headers);
|
||||
ClientTransport transport = clientTransportProvider.get(callOptions);
|
||||
stream = transport.newStream(method, headers);
|
||||
} else {
|
||||
stream = new FailingClientStream(Status.DEADLINE_EXCEEDED);
|
||||
stream = new FailingClientStream(DEADLINE_EXCEEDED);
|
||||
}
|
||||
|
||||
if (callOptions.getAuthority() != null) {
|
||||
|
|
@ -215,45 +233,66 @@ final class ClientCallImpl<ReqT, RespT> extends ClientCall<ReqT, RespT>
|
|||
if (compressor != Codec.Identity.NONE) {
|
||||
stream.setMessageCompression(true);
|
||||
}
|
||||
|
||||
// Delay any sources of cancellation after start(), because most of the transports are broken if
|
||||
// they receive cancel before start. Issue #1343 has more details
|
||||
|
||||
// Start the deadline timer after stream creation because it will close the stream
|
||||
if (callOptions.getDeadline() != null) {
|
||||
long timeoutNanos = callOptions.getDeadline().timeRemaining(NANOSECONDS);
|
||||
deadlineCancellationFuture = startDeadlineTimer(timeoutNanos);
|
||||
if (deadlineCancellationFutureShouldBeCancelled) {
|
||||
// Race detected! ClientStreamListener.closed may have been called before
|
||||
// deadlineCancellationFuture was set, thereby preventing the future from being cancelled.
|
||||
// Go ahead and cancel again, just to be sure it was cancelled.
|
||||
deadlineCancellationFuture.cancel(false);
|
||||
}
|
||||
}
|
||||
// Propagate later Context cancellation to the remote side.
|
||||
this.context.addListener(this, directExecutor());
|
||||
context.addListener(this, directExecutor());
|
||||
if (contextListenerShouldBeRemoved) {
|
||||
// Race detected! ClientStreamListener.closed may have been called before
|
||||
// deadlineCancellationFuture was set, thereby preventing the future from being cancelled.
|
||||
// Go ahead and cancel again, just to be sure it was cancelled.
|
||||
context.removeListener(this);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Based on the deadline, calculate and set the timeout to the given headers.
|
||||
*
|
||||
* @return {@code false} if deadline already exceeded
|
||||
*/
|
||||
static boolean updateTimeoutHeader(@Nullable Deadline deadline, Metadata headers) {
|
||||
// Fill out timeout on the headers
|
||||
// TODO(carl-mastrangelo): Find out if this should always remove the timeout,
|
||||
// even when returning false.
|
||||
private static void updateTimeoutHeaders(@Nullable Deadline effectiveDeadline,
|
||||
@Nullable Deadline callDeadline, @Nullable Deadline outerCallDeadline, Metadata headers) {
|
||||
headers.removeAll(TIMEOUT_KEY);
|
||||
|
||||
if (deadline != null) {
|
||||
// Convert the deadline to timeout. Timeout is more favorable than deadline on the wire
|
||||
// because timeout tolerates the clock difference between machines.
|
||||
long timeoutNanos = deadline.timeRemaining(NANOSECONDS);
|
||||
if (timeoutNanos <= 0) {
|
||||
return false;
|
||||
}
|
||||
headers.put(TIMEOUT_KEY, timeoutNanos);
|
||||
if (effectiveDeadline == null) {
|
||||
return;
|
||||
}
|
||||
return true;
|
||||
|
||||
long effectiveTimeout = max(0, effectiveDeadline.timeRemaining(TimeUnit.NANOSECONDS));
|
||||
headers.put(TIMEOUT_KEY, effectiveTimeout);
|
||||
|
||||
logIfContextNarrowedTimeout(effectiveTimeout, effectiveDeadline, outerCallDeadline,
|
||||
callDeadline);
|
||||
}
|
||||
|
||||
private static void logIfContextNarrowedTimeout(long effectiveTimeout,
|
||||
Deadline effectiveDeadline, @Nullable Deadline outerCallDeadline,
|
||||
@Nullable Deadline callDeadline) {
|
||||
if (!log.isLoggable(Level.INFO) || outerCallDeadline != effectiveDeadline) {
|
||||
return;
|
||||
}
|
||||
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append(String.format("Call timeout set to '%d' ns, due to context deadline.",
|
||||
effectiveTimeout));
|
||||
if (callDeadline == null) {
|
||||
builder.append(" Explicit call timeout was not set.");
|
||||
} else {
|
||||
long callTimeout = callDeadline.timeRemaining(TimeUnit.NANOSECONDS);
|
||||
builder.append(String.format(" Explicit call timeout was '%d' ns.", callTimeout));
|
||||
}
|
||||
|
||||
log.info(builder.toString());
|
||||
}
|
||||
|
||||
private static Deadline min(@Nullable Deadline deadline0, @Nullable Deadline deadline1) {
|
||||
if (deadline0 == null) {
|
||||
return deadline1;
|
||||
}
|
||||
if (deadline1 == null) {
|
||||
return deadline0;
|
||||
}
|
||||
return deadline0.minimum(deadline1);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -276,7 +315,9 @@ final class ClientCallImpl<ReqT, RespT> extends ClientCall<ReqT, RespT>
|
|||
stream.cancel(Status.CANCELLED);
|
||||
}
|
||||
} finally {
|
||||
context.removeListener(ClientCallImpl.this);
|
||||
if (context != null) {
|
||||
context.removeListener(ClientCallImpl.this);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -321,15 +362,6 @@ final class ClientCallImpl<ReqT, RespT> extends ClientCall<ReqT, RespT>
|
|||
return stream.isReady();
|
||||
}
|
||||
|
||||
private ScheduledFuture<?> startDeadlineTimer(long timeoutNanos) {
|
||||
return deadlineCancellationExecutor.schedule(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
stream.cancel(Status.DEADLINE_EXCEEDED);
|
||||
}
|
||||
}, timeoutNanos, NANOSECONDS);
|
||||
}
|
||||
|
||||
private class ClientStreamListenerImpl implements ClientStreamListener {
|
||||
private final Listener<RespT> observer;
|
||||
private boolean closed;
|
||||
|
|
@ -394,13 +426,13 @@ final class ClientCallImpl<ReqT, RespT> extends ClientCall<ReqT, RespT>
|
|||
|
||||
@Override
|
||||
public void closed(Status status, Metadata trailers) {
|
||||
if (status.getCode() == Status.Code.CANCELLED && callOptions.getDeadline() != null) {
|
||||
Deadline deadline = context.getDeadline();
|
||||
if (status.getCode() == Status.Code.CANCELLED && deadline != null) {
|
||||
// When the server's deadline expires, it can only reset the stream with CANCEL and no
|
||||
// description. Since our timer may be delayed in firing, we double-check the deadline and
|
||||
// turn the failure into the likely more helpful DEADLINE_EXCEEDED status.
|
||||
long timeoutNanos = callOptions.getDeadline().timeRemaining(NANOSECONDS);
|
||||
if (timeoutNanos <= 0) {
|
||||
status = Status.DEADLINE_EXCEEDED;
|
||||
if (deadline.isExpired()) {
|
||||
status = DEADLINE_EXCEEDED;
|
||||
// Replace trailers to prevent mixing sources of status and trailers.
|
||||
trailers = new Metadata();
|
||||
}
|
||||
|
|
@ -412,12 +444,7 @@ final class ClientCallImpl<ReqT, RespT> extends ClientCall<ReqT, RespT>
|
|||
public final void runInContext() {
|
||||
try {
|
||||
closed = true;
|
||||
deadlineCancellationFutureShouldBeCancelled = true;
|
||||
// manually optimize the volatile read
|
||||
ScheduledFuture<?> future = deadlineCancellationFuture;
|
||||
if (future != null) {
|
||||
future.cancel(false);
|
||||
}
|
||||
contextListenerShouldBeRemoved = true;
|
||||
observer.onClose(savedStatus, savedTrailers);
|
||||
} finally {
|
||||
context.removeListener(ClientCallImpl.this);
|
||||
|
|
|
|||
|
|
@ -58,7 +58,6 @@ import java.io.IOException;
|
|||
import java.io.InputStream;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Future;
|
||||
|
||||
final class ServerCallImpl<ReqT, RespT> extends ServerCall<RespT> {
|
||||
private final ServerStream stream;
|
||||
|
|
@ -198,9 +197,8 @@ final class ServerCallImpl<ReqT, RespT> extends ServerCall<RespT> {
|
|||
return cancelled;
|
||||
}
|
||||
|
||||
ServerStreamListener newServerStreamListener(ServerCall.Listener<ReqT> listener,
|
||||
Future<?> timeout) {
|
||||
return new ServerStreamListenerImpl<ReqT>(this, listener, timeout, context);
|
||||
ServerStreamListener newServerStreamListener(ServerCall.Listener<ReqT> listener) {
|
||||
return new ServerStreamListenerImpl<ReqT>(this, listener, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -216,16 +214,14 @@ final class ServerCallImpl<ReqT, RespT> extends ServerCall<RespT> {
|
|||
static final class ServerStreamListenerImpl<ReqT> implements ServerStreamListener {
|
||||
private final ServerCallImpl<ReqT, ?> call;
|
||||
private final ServerCall.Listener<ReqT> listener;
|
||||
private final Future<?> timeout;
|
||||
private final Context.CancellableContext context;
|
||||
private boolean messageReceived;
|
||||
|
||||
public ServerStreamListenerImpl(
|
||||
ServerCallImpl<ReqT, ?> call, ServerCall.Listener<ReqT> listener, Future<?> timeout,
|
||||
ServerCallImpl<ReqT, ?> call, ServerCall.Listener<ReqT> listener,
|
||||
Context.CancellableContext context) {
|
||||
this.call = checkNotNull(call, "call");
|
||||
this.listener = checkNotNull(listener, "listener must not be null");
|
||||
this.timeout = checkNotNull(timeout, "timeout");
|
||||
this.context = checkNotNull(context, "context");
|
||||
}
|
||||
|
||||
|
|
@ -265,7 +261,6 @@ final class ServerCallImpl<ReqT, RespT> extends ServerCall<RespT> {
|
|||
|
||||
@Override
|
||||
public void closed(Status status) {
|
||||
timeout.cancel(true);
|
||||
try {
|
||||
if (status.isOk()) {
|
||||
listener.onComplete();
|
||||
|
|
|
|||
|
|
@ -32,12 +32,14 @@
|
|||
package io.grpc.internal;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkState;
|
||||
import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
|
||||
import static io.grpc.Contexts.statusFromCancelled;
|
||||
import static io.grpc.Status.DEADLINE_EXCEEDED;
|
||||
import static io.grpc.internal.GrpcUtil.TIMEOUT_KEY;
|
||||
import static io.grpc.internal.GrpcUtil.TIMER_SERVICE;
|
||||
import static java.util.concurrent.TimeUnit.NANOSECONDS;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
|
||||
import io.grpc.CompressorRegistry;
|
||||
import io.grpc.Context;
|
||||
|
|
@ -55,10 +57,8 @@ import java.util.Collection;
|
|||
import java.util.HashSet;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
/**
|
||||
* Default implementation of {@link io.grpc.Server}, for creation by transports.
|
||||
|
|
@ -77,17 +77,6 @@ import java.util.concurrent.TimeoutException;
|
|||
public final class ServerImpl extends io.grpc.Server {
|
||||
private static final ServerStreamListener NOOP_LISTENER = new NoopListener();
|
||||
|
||||
private static final Future<?> DEFAULT_TIMEOUT_FUTURE = Futures.immediateCancelledFuture();
|
||||
|
||||
private static final TimeoutException TIMEOUT_EXCEPTION =
|
||||
new TimeoutException("request timed out") {
|
||||
@Override
|
||||
public synchronized Throwable fillInStackTrace() {
|
||||
// Suppress the stack trace as it would be confusing.
|
||||
return this;
|
||||
}
|
||||
};
|
||||
|
||||
/** Executor for application processing. */
|
||||
private Executor executor;
|
||||
private boolean usingSharedExecutor;
|
||||
|
|
@ -200,7 +189,7 @@ public final class ServerImpl extends io.grpc.Server {
|
|||
long timeoutNanos = unit.toNanos(timeout);
|
||||
long endTimeNanos = System.nanoTime() + timeoutNanos;
|
||||
while (!terminated && (timeoutNanos = endTimeNanos - System.nanoTime()) > 0) {
|
||||
TimeUnit.NANOSECONDS.timedWait(lock, timeoutNanos);
|
||||
NANOSECONDS.timedWait(lock, timeoutNanos);
|
||||
}
|
||||
return terminated;
|
||||
}
|
||||
|
|
@ -293,12 +282,11 @@ public final class ServerImpl extends io.grpc.Server {
|
|||
@Override
|
||||
public ServerStreamListener streamCreated(final ServerStream stream, final String methodName,
|
||||
final Metadata headers) {
|
||||
final Context.CancellableContext context = rootContext.withCancellation();
|
||||
final Future<?> timeout = scheduleTimeout(stream, headers, context);
|
||||
final Context.CancellableContext context = createContext(stream, headers);
|
||||
final Executor wrappedExecutor;
|
||||
// This is a performance optimization that avoids the synchronization and queuing overhead
|
||||
// that comes with SerializingExecutor.
|
||||
if (executor == MoreExecutors.directExecutor()) {
|
||||
if (executor == directExecutor()) {
|
||||
wrappedExecutor = new SerializeReentrantCallsDirectExecutor();
|
||||
} else {
|
||||
wrappedExecutor = new SerializingExecutor(executor);
|
||||
|
|
@ -319,17 +307,17 @@ public final class ServerImpl extends io.grpc.Server {
|
|||
stream.close(
|
||||
Status.UNIMPLEMENTED.withDescription("Method not found: " + methodName),
|
||||
new Metadata());
|
||||
timeout.cancel(true);
|
||||
context.cancel(null);
|
||||
return;
|
||||
}
|
||||
listener = startCall(stream, methodName, method, timeout, headers, context);
|
||||
listener = startCall(stream, methodName, method, headers, context);
|
||||
} catch (RuntimeException e) {
|
||||
stream.close(Status.fromThrowable(e), new Metadata());
|
||||
timeout.cancel(true);
|
||||
context.cancel(null);
|
||||
throw e;
|
||||
} catch (Throwable t) {
|
||||
stream.close(Status.fromThrowable(t), new Metadata());
|
||||
timeout.cancel(true);
|
||||
context.cancel(null);
|
||||
throw new RuntimeException(t);
|
||||
} finally {
|
||||
jumpListener.setListener(listener);
|
||||
|
|
@ -339,31 +327,34 @@ public final class ServerImpl extends io.grpc.Server {
|
|||
return jumpListener;
|
||||
}
|
||||
|
||||
private Future<?> scheduleTimeout(final ServerStream stream, Metadata headers,
|
||||
final Context.CancellableContext context) {
|
||||
private Context.CancellableContext createContext(final ServerStream stream, Metadata headers) {
|
||||
Long timeoutNanos = headers.get(TIMEOUT_KEY);
|
||||
|
||||
if (timeoutNanos == null) {
|
||||
return DEFAULT_TIMEOUT_FUTURE;
|
||||
return rootContext.withCancellation();
|
||||
}
|
||||
return timeoutService.schedule(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
|
||||
Context.CancellableContext context =
|
||||
rootContext.withDeadlineAfter(timeoutNanos, NANOSECONDS, timeoutService);
|
||||
context.addListener(new Context.CancellationListener() {
|
||||
@Override
|
||||
public void cancelled(Context context) {
|
||||
Status status = statusFromCancelled(context);
|
||||
if (DEADLINE_EXCEEDED.getCode().equals(status.getCode())) {
|
||||
// This should rarely get run, since the client will likely cancel the stream before
|
||||
// the timeout is reached.
|
||||
stream.cancel(Status.DEADLINE_EXCEEDED);
|
||||
// Cancel the context using a statically created exception as this event may occur
|
||||
// often enough that stack trace alloc impacts performance.
|
||||
context.cancel(TIMEOUT_EXCEPTION);
|
||||
stream.cancel(status);
|
||||
}
|
||||
},
|
||||
timeoutNanos,
|
||||
TimeUnit.NANOSECONDS);
|
||||
}
|
||||
}, directExecutor());
|
||||
|
||||
return context;
|
||||
}
|
||||
|
||||
/** Never returns {@code null}. */
|
||||
private <ReqT, RespT> ServerStreamListener startCall(ServerStream stream, String fullMethodName,
|
||||
ServerMethodDefinition<ReqT, RespT> methodDef, Future<?> timeout,
|
||||
Metadata headers, Context.CancellableContext context) {
|
||||
ServerMethodDefinition<ReqT, RespT> methodDef, Metadata headers,
|
||||
Context.CancellableContext context) {
|
||||
// TODO(ejona86): should we update fullMethodName to have the canonical path of the method?
|
||||
ServerCallImpl<ReqT, RespT> call = new ServerCallImpl<ReqT, RespT>(
|
||||
stream, methodDef.getMethodDescriptor(), headers, context, decompressorRegistry,
|
||||
|
|
@ -374,7 +365,7 @@ public final class ServerImpl extends io.grpc.Server {
|
|||
throw new NullPointerException(
|
||||
"startCall() returned a null listener for method " + fullMethodName);
|
||||
}
|
||||
return call.newServerStreamListener(listener, timeout);
|
||||
return call.newServerStreamListener(listener);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -31,12 +31,14 @@
|
|||
|
||||
package io.grpc;
|
||||
|
||||
import static org.hamcrest.core.IsInstanceOf.instanceOf;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNotSame;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertSame;
|
||||
import static org.junit.Assert.assertThat;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
|
|
@ -734,4 +736,21 @@ public class ContextTest {
|
|||
assertTrue(parentCalled.get());
|
||||
assertTrue(childAfterParent.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void expiredDeadlineShouldCancelContextImmediately() {
|
||||
Context parent = Context.current();
|
||||
assertFalse(parent.isCancelled());
|
||||
|
||||
Context.CancellableContext context = parent.withDeadlineAfter(0, TimeUnit.SECONDS, scheduler);
|
||||
assertTrue(context.isCancelled());
|
||||
assertThat(context.cancellationCause(), instanceOf(TimeoutException.class));
|
||||
|
||||
assertFalse(parent.isCancelled());
|
||||
Deadline deadline = Deadline.after(-10, TimeUnit.SECONDS);
|
||||
assertTrue(deadline.isExpired());
|
||||
context = parent.withDeadline(deadline, scheduler);
|
||||
assertTrue(context.isCancelled());
|
||||
assertThat(context.cancellationCause(), instanceOf(TimeoutException.class));
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,138 @@
|
|||
/*
|
||||
* Copyright 2016, Google Inc. All rights reserved.
|
||||
*
|
||||
* Redistribution and use in source and binary forms, with or without
|
||||
* modification, are permitted provided that the following conditions are
|
||||
* met:
|
||||
*
|
||||
* * Redistributions of source code must retain the above copyright
|
||||
* notice, this list of conditions and the following disclaimer.
|
||||
* * Redistributions in binary form must reproduce the above
|
||||
* copyright notice, this list of conditions and the following disclaimer
|
||||
* in the documentation and/or other materials provided with the
|
||||
* distribution.
|
||||
*
|
||||
* * Neither the name of Google Inc. nor the names of its
|
||||
* contributors may be used to endorse or promote products derived from
|
||||
* this software without specific prior written permission.
|
||||
*
|
||||
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
|
||||
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
|
||||
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
|
||||
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
*/
|
||||
|
||||
package io.grpc;
|
||||
|
||||
import static io.grpc.Contexts.statusFromCancelled;
|
||||
import static org.hamcrest.core.IsInstanceOf.instanceOf;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertSame;
|
||||
import static org.junit.Assert.assertThat;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import io.grpc.internal.FakeClock;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.JUnit4;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
/**
|
||||
* Tests for {@link Contexts}.
|
||||
*/
|
||||
@RunWith(JUnit4.class)
|
||||
public class ContextsTest {
|
||||
|
||||
@Test
|
||||
public void statusFromCancelled_returnNullIfCtxNotCancelled() {
|
||||
Context context = Context.current();
|
||||
assertFalse(context.isCancelled());
|
||||
assertNull(statusFromCancelled(context));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void statusFromCancelled_returnStatusAsSetOnCtx() {
|
||||
Context.CancellableContext cancellableContext = Context.current().fork();
|
||||
cancellableContext.cancel(Status.DEADLINE_EXCEEDED.withDescription("foo bar").asException());
|
||||
Status status = statusFromCancelled(cancellableContext);
|
||||
assertNotNull(status);
|
||||
assertEquals(Status.Code.DEADLINE_EXCEEDED, status.getCode());
|
||||
assertEquals("foo bar", status.getDescription());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void statusFromCancelled_shouldReturnStatusWithCauseAttached() {
|
||||
Context.CancellableContext cancellableContext = Context.current().fork();
|
||||
Throwable t = new Throwable();
|
||||
cancellableContext.cancel(t);
|
||||
Status status = statusFromCancelled(cancellableContext);
|
||||
assertNotNull(status);
|
||||
assertEquals(Status.Code.CANCELLED, status.getCode());
|
||||
assertSame(t, status.getCause());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void statusFromCancelled_TimeoutExceptionShouldMapToDeadlineExceeded() {
|
||||
FakeClock fakeClock = new FakeClock();
|
||||
Context.CancellableContext cancellableContext = Context.current()
|
||||
.withDeadlineAfter(100, TimeUnit.MILLISECONDS, fakeClock.scheduledExecutorService);
|
||||
fakeClock.forwardTime(System.nanoTime(), TimeUnit.NANOSECONDS);
|
||||
fakeClock.forwardMillis(100);
|
||||
|
||||
assertTrue(cancellableContext.isCancelled());
|
||||
assertThat(cancellableContext.cancellationCause(), instanceOf(TimeoutException.class));
|
||||
|
||||
Status status = statusFromCancelled(cancellableContext);
|
||||
assertNotNull(status);
|
||||
assertEquals(Status.Code.DEADLINE_EXCEEDED, status.getCode());
|
||||
assertEquals("context timed out", status.getDescription());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void statusFromCancelled_returnCancelledIfCauseIsNull() {
|
||||
Context.CancellableContext cancellableContext = Context.current().fork();
|
||||
cancellableContext.cancel(null);
|
||||
assertTrue(cancellableContext.isCancelled());
|
||||
Status status = statusFromCancelled(cancellableContext);
|
||||
assertNotNull(status);
|
||||
assertEquals(Status.Code.CANCELLED, status.getCode());
|
||||
}
|
||||
|
||||
/** This is a whitebox test, to verify a special case of the implementation. */
|
||||
@Test
|
||||
public void statusFromCancelled_StatusUnknownShouldWork() {
|
||||
Context.CancellableContext cancellableContext = Context.current().fork();
|
||||
Exception e = Status.UNKNOWN.asException();
|
||||
cancellableContext.cancel(e);
|
||||
assertTrue(cancellableContext.isCancelled());
|
||||
|
||||
Status status = statusFromCancelled(cancellableContext);
|
||||
assertNotNull(status);
|
||||
assertEquals(Status.Code.UNKNOWN, status.getCode());
|
||||
assertSame(e, status.getCause());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void statusFromCancelled_shouldThrowIfCtxIsNull() {
|
||||
try {
|
||||
statusFromCancelled(null);
|
||||
fail("NPE expected");
|
||||
} catch (NullPointerException npe) {
|
||||
assertEquals("context must not be null", npe.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -34,6 +34,7 @@ package io.grpc.internal;
|
|||
import static io.grpc.internal.GrpcUtil.ACCEPT_ENCODING_SPLITER;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertSame;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
|
@ -126,6 +127,9 @@ public class ClientCallImplTest {
|
|||
@Captor
|
||||
private ArgumentCaptor<ClientStreamListener> listenerArgumentCaptor;
|
||||
|
||||
@Captor
|
||||
private ArgumentCaptor<Status> statusArgumentCaptor;
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
MockitoAnnotations.initMocks(this);
|
||||
|
|
@ -362,7 +366,10 @@ public class ClientCallImplTest {
|
|||
|
||||
call.start(callListener, new Metadata());
|
||||
|
||||
cancellableContext.cancel(new Throwable());
|
||||
Throwable t = new Throwable();
|
||||
cancellableContext.cancel(t);
|
||||
|
||||
verify(stream, times(1)).cancel(statusArgumentCaptor.capture());
|
||||
|
||||
verify(stream, times(1)).cancel(statusCaptor.capture());
|
||||
assertEquals(Status.Code.CANCELLED, statusCaptor.getValue().getCode());
|
||||
|
|
@ -433,6 +440,107 @@ public class ClientCallImplTest {
|
|||
verifyZeroInteractions(provider);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void contextDeadlineShouldBePropagatedInMetadata() {
|
||||
long deadlineNanos = TimeUnit.SECONDS.toNanos(1);
|
||||
Context context = Context.current().withDeadlineAfter(deadlineNanos, TimeUnit.NANOSECONDS,
|
||||
deadlineCancellationExecutor);
|
||||
context.attach();
|
||||
|
||||
ClientCallImpl<Void, Void> call = new ClientCallImpl<Void, Void>(
|
||||
DESCRIPTOR,
|
||||
MoreExecutors.directExecutor(),
|
||||
CallOptions.DEFAULT,
|
||||
provider,
|
||||
deadlineCancellationExecutor);
|
||||
|
||||
Metadata headers = new Metadata();
|
||||
|
||||
call.start(callListener, headers);
|
||||
|
||||
assertTrue(headers.containsKey(GrpcUtil.TIMEOUT_KEY));
|
||||
Long timeout = headers.get(GrpcUtil.TIMEOUT_KEY);
|
||||
assertNotNull(timeout);
|
||||
|
||||
long deltaNanos = TimeUnit.MILLISECONDS.toNanos(400);
|
||||
assertTimeoutBetween(timeout, deadlineNanos - deltaNanos, deadlineNanos);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void contextDeadlineShouldOverrideLargerMetadataTimeout() {
|
||||
long deadlineNanos = TimeUnit.SECONDS.toNanos(1);
|
||||
Context context = Context.current().withDeadlineAfter(deadlineNanos, TimeUnit.NANOSECONDS,
|
||||
deadlineCancellationExecutor);
|
||||
context.attach();
|
||||
|
||||
CallOptions callOpts = CallOptions.DEFAULT.withDeadlineAfter(2, TimeUnit.SECONDS);
|
||||
ClientCallImpl<Void, Void> call = new ClientCallImpl<Void, Void>(
|
||||
DESCRIPTOR,
|
||||
MoreExecutors.directExecutor(),
|
||||
callOpts,
|
||||
provider,
|
||||
deadlineCancellationExecutor);
|
||||
|
||||
Metadata headers = new Metadata();
|
||||
|
||||
call.start(callListener, headers);
|
||||
|
||||
assertTrue(headers.containsKey(GrpcUtil.TIMEOUT_KEY));
|
||||
Long timeout = headers.get(GrpcUtil.TIMEOUT_KEY);
|
||||
assertNotNull(timeout);
|
||||
|
||||
long deltaNanos = TimeUnit.MILLISECONDS.toNanos(400);
|
||||
assertTimeoutBetween(timeout, deadlineNanos - deltaNanos, deadlineNanos);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void contextDeadlineShouldNotOverrideSmallerMetadataTimeout() {
|
||||
long deadlineNanos = TimeUnit.SECONDS.toNanos(2);
|
||||
Context context = Context.current().withDeadlineAfter(deadlineNanos, TimeUnit.NANOSECONDS,
|
||||
deadlineCancellationExecutor);
|
||||
context.attach();
|
||||
|
||||
CallOptions callOpts = CallOptions.DEFAULT.withDeadlineAfter(1, TimeUnit.SECONDS);
|
||||
ClientCallImpl<Void, Void> call = new ClientCallImpl<Void, Void>(
|
||||
DESCRIPTOR,
|
||||
MoreExecutors.directExecutor(),
|
||||
callOpts,
|
||||
provider,
|
||||
deadlineCancellationExecutor);
|
||||
|
||||
Metadata headers = new Metadata();
|
||||
|
||||
call.start(callListener, headers);
|
||||
|
||||
assertTrue(headers.containsKey(GrpcUtil.TIMEOUT_KEY));
|
||||
Long timeout = headers.get(GrpcUtil.TIMEOUT_KEY);
|
||||
assertNotNull(timeout);
|
||||
|
||||
long callOptsNanos = TimeUnit.SECONDS.toNanos(1);
|
||||
long deltaNanos = TimeUnit.MILLISECONDS.toNanos(400);
|
||||
assertTimeoutBetween(timeout, callOptsNanos - deltaNanos, callOptsNanos);
|
||||
}
|
||||
|
||||
/**
|
||||
* Without a context or call options deadline,
|
||||
* a timeout should not be set in metadata.
|
||||
*/
|
||||
@Test
|
||||
public void timeoutShouldNotBeSet() {
|
||||
ClientCallImpl<Void, Void> call = new ClientCallImpl<Void, Void>(
|
||||
DESCRIPTOR,
|
||||
MoreExecutors.directExecutor(),
|
||||
CallOptions.DEFAULT,
|
||||
provider,
|
||||
deadlineCancellationExecutor);
|
||||
|
||||
Metadata headers = new Metadata();
|
||||
|
||||
call.start(callListener, headers);
|
||||
|
||||
assertFalse(headers.containsKey(GrpcUtil.TIMEOUT_KEY));
|
||||
}
|
||||
|
||||
private static class TestMarshaller<T> implements Marshaller<T> {
|
||||
@Override
|
||||
public InputStream stream(T value) {
|
||||
|
|
@ -444,5 +552,11 @@ public class ClientCallImplTest {
|
|||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private static void assertTimeoutBetween(long timeout, long from, long to) {
|
||||
assertTrue("timeout: " + timeout + " ns", timeout <= to);
|
||||
assertTrue("timeout: " + timeout + " ns", timeout >= from);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -165,8 +165,10 @@ public class ManagedChannelImplTest {
|
|||
ClientCall<String, Integer> call =
|
||||
channel.newCall(method, CallOptions.DEFAULT.withDeadlineAfter(0, TimeUnit.NANOSECONDS));
|
||||
call.start(mockCallListener, new Metadata());
|
||||
verify(mockCallListener, timeout(1000)).onClose(
|
||||
same(Status.DEADLINE_EXCEEDED), any(Metadata.class));
|
||||
ArgumentCaptor<Status> statusCaptor = ArgumentCaptor.forClass(Status.class);
|
||||
verify(mockCallListener, timeout(1000)).onClose(statusCaptor.capture(), any(Metadata.class));
|
||||
Status status = statusCaptor.getValue();
|
||||
assertSame(Status.DEADLINE_EXCEEDED.getCode(), status.getCode());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
|||
|
|
@ -189,7 +189,7 @@ public class ServerCallImplTest {
|
|||
@Test
|
||||
public void streamListener_halfClosed() {
|
||||
ServerStreamListenerImpl<Long> streamListener =
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, timeout, context);
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, context);
|
||||
|
||||
streamListener.halfClosed();
|
||||
|
||||
|
|
@ -199,7 +199,7 @@ public class ServerCallImplTest {
|
|||
@Test
|
||||
public void streamListener_halfClosed_onlyOnce() {
|
||||
ServerStreamListenerImpl<Long> streamListener =
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, timeout, context);
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, context);
|
||||
streamListener.halfClosed();
|
||||
// canceling the call should short circuit future halfClosed() calls.
|
||||
streamListener.closed(Status.CANCELLED);
|
||||
|
|
@ -212,12 +212,11 @@ public class ServerCallImplTest {
|
|||
@Test
|
||||
public void streamListener_closedOk() {
|
||||
ServerStreamListenerImpl<Long> streamListener =
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, timeout, context);
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, context);
|
||||
|
||||
streamListener.closed(Status.OK);
|
||||
|
||||
verify(callListener).onComplete();
|
||||
assertTrue(timeout.isCancelled());
|
||||
assertTrue(context.isCancelled());
|
||||
assertNull(context.cancellationCause());
|
||||
}
|
||||
|
|
@ -225,12 +224,11 @@ public class ServerCallImplTest {
|
|||
@Test
|
||||
public void streamListener_closedCancelled() {
|
||||
ServerStreamListenerImpl<Long> streamListener =
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, timeout, context);
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, context);
|
||||
|
||||
streamListener.closed(Status.CANCELLED);
|
||||
|
||||
verify(callListener).onCancel();
|
||||
assertTrue(timeout.isCancelled());
|
||||
assertTrue(context.isCancelled());
|
||||
assertNull(context.cancellationCause());
|
||||
}
|
||||
|
|
@ -238,7 +236,7 @@ public class ServerCallImplTest {
|
|||
@Test
|
||||
public void streamListener_onReady() {
|
||||
ServerStreamListenerImpl<Long> streamListener =
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, timeout, context);
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, context);
|
||||
|
||||
streamListener.onReady();
|
||||
|
||||
|
|
@ -248,7 +246,7 @@ public class ServerCallImplTest {
|
|||
@Test
|
||||
public void streamListener_onReady_onlyOnce() {
|
||||
ServerStreamListenerImpl<Long> streamListener =
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, timeout, context);
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, context);
|
||||
streamListener.onReady();
|
||||
// canceling the call should short circuit future halfClosed() calls.
|
||||
streamListener.closed(Status.CANCELLED);
|
||||
|
|
@ -261,7 +259,7 @@ public class ServerCallImplTest {
|
|||
@Test
|
||||
public void streamListener_messageRead() {
|
||||
ServerStreamListenerImpl<Long> streamListener =
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, timeout, context);
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, context);
|
||||
streamListener.messageRead(method.streamRequest(1234L));
|
||||
|
||||
verify(callListener).onMessage(1234L);
|
||||
|
|
@ -270,7 +268,7 @@ public class ServerCallImplTest {
|
|||
@Test
|
||||
public void streamListener_messageRead_unaryFailsOnMultiple() {
|
||||
ServerStreamListenerImpl<Long> streamListener =
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, timeout, context);
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, context);
|
||||
streamListener.messageRead(method.streamRequest(1234L));
|
||||
streamListener.messageRead(method.streamRequest(1234L));
|
||||
|
||||
|
|
@ -284,7 +282,7 @@ public class ServerCallImplTest {
|
|||
@Test
|
||||
public void streamListener_messageRead_onlyOnce() {
|
||||
ServerStreamListenerImpl<Long> streamListener =
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, timeout, context);
|
||||
new ServerCallImpl.ServerStreamListenerImpl<Long>(call, callListener, context);
|
||||
streamListener.messageRead(method.streamRequest(1234L));
|
||||
// canceling the call should short circuit future halfClosed() calls.
|
||||
streamListener.closed(Status.CANCELLED);
|
||||
|
|
|
|||
|
|
@ -643,7 +643,7 @@ public abstract class AbstractInteropTest {
|
|||
.build()).next();
|
||||
fail("Expected deadline to be exceeded");
|
||||
} catch (Throwable t) {
|
||||
assertEquals(Status.DEADLINE_EXCEEDED, Status.fromThrowable(t));
|
||||
assertEquals(Status.DEADLINE_EXCEEDED.getCode(), Status.fromThrowable(t).getCode());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -666,7 +666,8 @@ public abstract class AbstractInteropTest {
|
|||
.withDeadlineAfter(30, TimeUnit.MILLISECONDS)
|
||||
.streamingOutputCall(request, recorder);
|
||||
recorder.awaitCompletion();
|
||||
assertEquals(Status.DEADLINE_EXCEEDED, Status.fromThrowable(recorder.getError()));
|
||||
assertEquals(Status.DEADLINE_EXCEEDED.getCode(),
|
||||
Status.fromThrowable(recorder.getError()).getCode());
|
||||
}
|
||||
|
||||
@Test(timeout = 10000)
|
||||
|
|
|
|||
|
|
@ -121,7 +121,7 @@ public class CascadingTest {
|
|||
} catch (StatusRuntimeException sre) {
|
||||
// Wait for the workers to finish
|
||||
Status status = Status.fromThrowable(sre);
|
||||
assertEquals(Status.Code.CANCELLED, status.getCode());
|
||||
assertEquals(Status.Code.DEADLINE_EXCEEDED, status.getCode());
|
||||
|
||||
// Should have 3 calls before timeout propagates
|
||||
assertEquals(3, nodeCount.get());
|
||||
|
|
@ -229,7 +229,8 @@ public class CascadingTest {
|
|||
blockingStub.unaryCall((Messages.SimpleRequest) message);
|
||||
} catch (Exception e) {
|
||||
Status status = Status.fromThrowable(e);
|
||||
if (status.getCode() == Status.Code.CANCELLED) {
|
||||
if (status.getCode() == Status.Code.CANCELLED
|
||||
|| status.getCode() == Status.Code.DEADLINE_EXCEEDED) {
|
||||
observedCancellations.countDown();
|
||||
} else if (status.getCode() == Status.Code.ABORTED) {
|
||||
// Propagate aborted back up
|
||||
|
|
|
|||
Loading…
Reference in New Issue