From a79f2c1f6fba8e41ef08a66199ed0c8d8a5b112c Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Fri, 16 Dec 2016 15:41:04 -0800 Subject: [PATCH 01/14] Fork ManagedChannelImpl and its tests. --- .../io/grpc/internal/ManagedChannelImpl2.java | 888 ++++++++++++++ .../ManagedChannelImpl2IdlenessTest.java | 440 +++++++ .../internal/ManagedChannelImpl2Test.java | 1082 +++++++++++++++++ 3 files changed, 2410 insertions(+) create mode 100644 core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java create mode 100644 core/src/test/java/io/grpc/internal/ManagedChannelImpl2IdlenessTest.java create mode 100644 core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java diff --git a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java new file mode 100644 index 00000000000..ee416482c5c --- /dev/null +++ b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java @@ -0,0 +1,888 @@ +/* + * Copyright 2014, 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.internal; + +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 com.google.census.CensusContextFactory; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Stopwatch; +import com.google.common.base.Supplier; + +import io.grpc.Attributes; +import io.grpc.CallOptions; +import io.grpc.Channel; +import io.grpc.ClientCall; +import io.grpc.ClientInterceptor; +import io.grpc.ClientInterceptors; +import io.grpc.CompressorRegistry; +import io.grpc.DecompressorRegistry; +import io.grpc.EquivalentAddressGroup; +import io.grpc.LoadBalancer; +import io.grpc.ManagedChannel; +import io.grpc.MethodDescriptor; +import io.grpc.NameResolver; +import io.grpc.ResolvedServerInfoGroup; +import io.grpc.Status; +import io.grpc.TransportManager; +import io.grpc.TransportManager.InterimTransport; +import io.grpc.TransportManager.OobTransportProvider; +import io.grpc.internal.ClientCallImpl.ClientTransportProvider; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +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 java.util.regex.Pattern; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; +import javax.annotation.concurrent.ThreadSafe; + +/** A communication channel for making outgoing RPCs. */ +@ThreadSafe +public final class ManagedChannelImpl extends ManagedChannel implements WithLogId { + private static final Logger log = Logger.getLogger(ManagedChannelImpl.class.getName()); + + // Matching this pattern means the target string is a URI target or at least intended to be one. + // A URI target must be an absolute hierarchical URI. + // From RFC 2396: scheme = alpha *( alpha | digit | "+" | "-" | "." ) + @VisibleForTesting + static final Pattern URI_PATTERN = Pattern.compile("[a-zA-Z][a-zA-Z0-9+.-]*:/.*"); + + static final long IDLE_TIMEOUT_MILLIS_DISABLE = -1; + + /** + * The time after idleTimeoutMillis expires before idleness takes effect. The time before + * idleTimeoutMillis expires is part of a fast path for acquiring the load balancer. After + * idleTimeoutMillis expires a slow path takes effect with extra synchronization. + * + *

Transports having open streams prevents entering idle mode. However, this creates an + * inherent race between acquiring a transport, which can't be done in idle mode, and the RPC + * actually being created on that transport, which inhibits idle mode. Thus we reset the idle + * timer when acquiring a transport, and impose a minimum idle time (IDLE_MODE_MIN_TIMEOUT_MILLIS) + * to make the chances of racing very small. If we do race, then the RPC will spuriously fail + * because the transport chosen was shut down. + * + *

For heavy users, resetting the idle timer each RPC becomes highly contended. We instead only + * need to reset the timer when it is close to expiring. We do the equivalent by having two + * periods: a reduced regular idle time period and the extra time as a grace period. We ignore the + * race during the regular idle time period, but any acquisition during the grace period must + * reset the timer. + */ + @VisibleForTesting + static final long IDLE_GRACE_PERIOD_MILLIS = TimeUnit.SECONDS.toMillis(1); + + private static final ClientTransport SHUTDOWN_TRANSPORT = + new FailingClientTransport(Status.UNAVAILABLE.withDescription("Channel is shutdown")); + + @VisibleForTesting + static final ClientTransport IDLE_MODE_TRANSPORT = + new FailingClientTransport(Status.INTERNAL.withDescription("Channel is in idle mode")); + + private final String target; + private final NameResolver.Factory nameResolverFactory; + private final Attributes nameResolverParams; + private final LoadBalancer.Factory loadBalancerFactory; + private final ClientTransportFactory transportFactory; + private final Executor executor; + private final boolean usingSharedExecutor; + private final Object lock = new Object(); + private final LogId logId = LogId.allocate(getClass().getName()); + + private final DecompressorRegistry decompressorRegistry; + private final CompressorRegistry compressorRegistry; + + private final SharedResourceHolder.Resource timerService; + private final Supplier stopwatchSupplier; + /** The timout before entering idle mode, less {@link #IDLE_GRACE_PERIOD_MILLIS}. */ + private final long idleTimeoutMillis; + private final CensusContextFactory censusFactory; + + /** + * Executor that runs deadline timers for requests. + */ + private ScheduledExecutorService scheduledExecutor; + + private final BackoffPolicy.Provider backoffPolicyProvider; + + /** + * We delegate to this channel, so that we can have interceptors as necessary. If there aren't + * any interceptors this will just be {@link RealChannel}. + */ + private final Channel interceptorChannel; + @Nullable private final String userAgent; + + // Never be null. Must be modified under lock. + private NameResolver nameResolver; + + /** {@code null} when idle or when in grace idle period. "lock" must be held when modifying. */ + @Nullable + private volatile LoadBalancer loadBalancer; + + /** non-{code null} iff channel is in grace idle period. */ + @GuardedBy("lock") + @Nullable + private LoadBalancer graceLoadBalancer; + + /** + * Maps EquivalentAddressGroups to transports for that server. "lock" must be held when mutating. + */ + // Even though we set a concurrency level of 1, this is better than Collections.synchronizedMap + // because it doesn't need to acquire a lock for reads. + private final ConcurrentMap transports = + new ConcurrentHashMap(16, .75f, 1); + + /** + * TransportSets that are shutdown (but not yet terminated) due to channel idleness or channel + * shut down. + */ + @GuardedBy("lock") + private final HashSet decommissionedTransports = new HashSet(); + + @GuardedBy("lock") + private final HashSet delayedTransports = + new HashSet(); + + @VisibleForTesting + final InUseStateAggregator inUseStateAggregator = + new InUseStateAggregator() { + @Override + Object getLock() { + return lock; + } + + @Override + @GuardedBy("lock") + void handleInUse() { + exitIdleMode(); + } + + @GuardedBy("lock") + @Override + void handleNotInUse() { + if (shutdown) { + return; + } + rescheduleIdleTimer(); + } + }; + + private class IdleModeTimer implements Runnable { + @GuardedBy("lock") + boolean cancelled; + + @Override + public void run() { + ArrayList transportsCopy = new ArrayList(); + LoadBalancer savedBalancer; + NameResolver oldResolver; + synchronized (lock) { + if (cancelled) { + // Race detected: this task started before cancelIdleTimer() could cancel it. + return; + } + if (loadBalancer != null) { + // Enter grace period. + graceLoadBalancer = loadBalancer; + loadBalancer = null; + assert idleModeTimer == this; + idleModeTimerFuture = scheduledExecutor.schedule(new LogExceptionRunnable(idleModeTimer), + IDLE_GRACE_PERIOD_MILLIS, TimeUnit.MILLISECONDS); + return; + } + // Enter idle mode + savedBalancer = graceLoadBalancer; + graceLoadBalancer = null; + oldResolver = nameResolver; + nameResolver = getNameResolver(target, nameResolverFactory, nameResolverParams); + transportsCopy.addAll(transports.values()); + transports.clear(); + decommissionedTransports.addAll(transportsCopy); + } + for (TransportSet ts : transportsCopy) { + ts.shutdown(); + } + savedBalancer.shutdown(); + oldResolver.shutdown(); + } + } + + @GuardedBy("lock") + @Nullable + private ScheduledFuture idleModeTimerFuture; + @GuardedBy("lock") + @Nullable + private IdleModeTimer idleModeTimer; + + /** + * Make the channel exit idle mode, if it's in it. Return a LoadBalancer that can be used for + * making new requests. Return null if the channel is shutdown. + * + *

May be called under the lock. + */ + @VisibleForTesting + LoadBalancer exitIdleMode() { + final LoadBalancer balancer; + final NameResolver resolver; + synchronized (lock) { + if (shutdown) { + return null; + } + if (inUseStateAggregator.isInUse()) { + cancelIdleTimer(); + } else { + // exitIdleMode() may be called outside of inUseStateAggregator, which may still in + // "not-in-use" state. If it's the case, we start the timer which will be soon cancelled if + // the aggregator receives actual uses. + rescheduleIdleTimer(); + } + if (graceLoadBalancer != null) { + // Exit grace period; timer already rescheduled above. + loadBalancer = graceLoadBalancer; + graceLoadBalancer = null; + } + if (loadBalancer != null) { + return loadBalancer; + } + balancer = loadBalancerFactory.newLoadBalancer(nameResolver.getServiceAuthority(), tm); + this.loadBalancer = balancer; + resolver = this.nameResolver; + } + class NameResolverStartTask implements Runnable { + @Override + public void run() { + NameResolverListenerImpl listener = new NameResolverListenerImpl(balancer); + // This may trigger quite a few non-trivial work in LoadBalancer and NameResolver, + // we don't want to do it in the lock. + try { + resolver.start(listener); + } catch (Throwable t) { + listener.onError(Status.fromThrowable(t)); + } + } + } + + scheduledExecutor.execute(new NameResolverStartTask()); + return balancer; + } + + @VisibleForTesting + boolean isInIdleGracePeriod() { + synchronized (lock) { + return graceLoadBalancer != null; + } + } + + // ErrorProne's GuardedByChecker can't figure out that the idleModeTimer is a nested instance of + // this particular instance. It is worried about something like: + // ManagedChannelImpl a = ...; + // ManagedChannelImpl b = ...; + // a.idleModeTimer = b.idleModeTimer; + // a.cancelIdleTimer(); // access of b.idleModeTimer is guarded by a.lock, not b.lock + // + // _We_ know that isn't happening, so we suppress the warning. + @SuppressWarnings("GuardedByChecker") + @GuardedBy("lock") + private void cancelIdleTimer() { + if (idleModeTimerFuture != null) { + idleModeTimerFuture.cancel(false); + idleModeTimer.cancelled = true; + idleModeTimerFuture = null; + idleModeTimer = null; + } + } + + @GuardedBy("lock") + private void rescheduleIdleTimer() { + if (idleTimeoutMillis == IDLE_TIMEOUT_MILLIS_DISABLE) { + return; + } + cancelIdleTimer(); + idleModeTimer = new IdleModeTimer(); + idleModeTimerFuture = scheduledExecutor.schedule(new LogExceptionRunnable(idleModeTimer), + idleTimeoutMillis, TimeUnit.MILLISECONDS); + } + + @GuardedBy("lock") + private final HashSet oobTransports = + new HashSet(); + + @GuardedBy("lock") + private boolean shutdown; + @GuardedBy("lock") + private boolean shutdownNowed; + @GuardedBy("lock") + private boolean terminated; + + private final ClientTransportProvider transportProvider = new ClientTransportProvider() { + @Override + public ClientTransport get(CallOptions callOptions) { + LoadBalancer balancer = loadBalancer; + if (balancer == null) { + // Current state is either idle or in grace period + balancer = exitIdleMode(); + } + if (balancer == null) { + return SHUTDOWN_TRANSPORT; + } + return balancer.pickTransport(callOptions.getAffinity()); + } + }; + + ManagedChannelImpl(String target, BackoffPolicy.Provider backoffPolicyProvider, + NameResolver.Factory nameResolverFactory, Attributes nameResolverParams, + LoadBalancer.Factory loadBalancerFactory, ClientTransportFactory transportFactory, + DecompressorRegistry decompressorRegistry, CompressorRegistry compressorRegistry, + SharedResourceHolder.Resource timerService, + Supplier stopwatchSupplier, long idleTimeoutMillis, + @Nullable Executor executor, @Nullable String userAgent, + List interceptors, CensusContextFactory censusFactory) { + this.target = checkNotNull(target, "target"); + this.nameResolverFactory = checkNotNull(nameResolverFactory, "nameResolverFactory"); + this.nameResolverParams = checkNotNull(nameResolverParams, "nameResolverParams"); + this.nameResolver = getNameResolver(target, nameResolverFactory, nameResolverParams); + this.loadBalancerFactory = checkNotNull(loadBalancerFactory, "loadBalancerFactory"); + if (executor == null) { + usingSharedExecutor = true; + this.executor = SharedResourceHolder.get(GrpcUtil.SHARED_CHANNEL_EXECUTOR); + } else { + usingSharedExecutor = false; + this.executor = executor; + } + this.backoffPolicyProvider = backoffPolicyProvider; + this.transportFactory = + new CallCredentialsApplyingTransportFactory(transportFactory, this.executor); + this.interceptorChannel = ClientInterceptors.intercept(new RealChannel(), interceptors); + this.timerService = timerService; + this.scheduledExecutor = SharedResourceHolder.get(timerService); + this.stopwatchSupplier = checkNotNull(stopwatchSupplier, "stopwatchSupplier"); + if (idleTimeoutMillis == IDLE_TIMEOUT_MILLIS_DISABLE) { + this.idleTimeoutMillis = idleTimeoutMillis; + } else { + assert IDLE_GRACE_PERIOD_MILLIS + <= AbstractManagedChannelImplBuilder.IDLE_MODE_MIN_TIMEOUT_MILLIS; + checkArgument(idleTimeoutMillis >= IDLE_GRACE_PERIOD_MILLIS, + "invalid idleTimeoutMillis %s", idleTimeoutMillis); + this.idleTimeoutMillis = idleTimeoutMillis - IDLE_GRACE_PERIOD_MILLIS; + } + this.decompressorRegistry = decompressorRegistry; + this.compressorRegistry = compressorRegistry; + this.userAgent = userAgent; + this.censusFactory = checkNotNull(censusFactory, "censusFactory"); + + if (log.isLoggable(Level.INFO)) { + log.log(Level.INFO, "[{0}] Created with target {1}", new Object[] {getLogId(), target}); + } + } + + @VisibleForTesting + static NameResolver getNameResolver(String target, NameResolver.Factory nameResolverFactory, + Attributes nameResolverParams) { + // Finding a NameResolver. Try using the target string as the URI. If that fails, try prepending + // "dns:///". + URI targetUri = null; + StringBuilder uriSyntaxErrors = new StringBuilder(); + try { + targetUri = new URI(target); + // For "localhost:8080" this would likely cause newNameResolver to return null, because + // "localhost" is parsed as the scheme. Will fall into the next branch and try + // "dns:///localhost:8080". + } catch (URISyntaxException e) { + // Can happen with ip addresses like "[::1]:1234" or 127.0.0.1:1234. + uriSyntaxErrors.append(e.getMessage()); + } + if (targetUri != null) { + NameResolver resolver = nameResolverFactory.newNameResolver(targetUri, nameResolverParams); + if (resolver != null) { + return resolver; + } + // "foo.googleapis.com:8080" cause resolver to be null, because "foo.googleapis.com" is an + // unmapped scheme. Just fall through and will try "dns:///foo.googleapis.com:8080" + } + + // If we reached here, the targetUri couldn't be used. + if (!URI_PATTERN.matcher(target).matches()) { + // It doesn't look like a URI target. Maybe it's an authority string. Try with the default + // scheme from the factory. + try { + targetUri = new URI(nameResolverFactory.getDefaultScheme(), "", "/" + target, null); + } catch (URISyntaxException e) { + // Should not be possible. + throw new IllegalArgumentException(e); + } + if (targetUri != null) { + NameResolver resolver = nameResolverFactory.newNameResolver(targetUri, nameResolverParams); + if (resolver != null) { + return resolver; + } + } + } + throw new IllegalArgumentException(String.format( + "cannot find a NameResolver for %s%s", + target, uriSyntaxErrors.length() > 0 ? " (" + uriSyntaxErrors + ")" : "")); + } + + /** + * Initiates an orderly shutdown in which preexisting calls continue but new calls are immediately + * cancelled. + */ + @Override + public ManagedChannelImpl shutdown() { + ArrayList transportsCopy = new ArrayList(); + ArrayList delayedTransportsCopy = + new ArrayList(); + ArrayList oobTransportsCopy = + new ArrayList(); + LoadBalancer balancer; + NameResolver resolver; + synchronized (lock) { + if (shutdown) { + return this; + } + shutdown = true; + // After shutdown there are no new calls, so no new cancellation tasks are needed + scheduledExecutor = SharedResourceHolder.release(timerService, scheduledExecutor); + maybeTerminateChannel(); + if (!terminated) { + transportsCopy.addAll(transports.values()); + transports.clear(); + decommissionedTransports.addAll(transportsCopy); + delayedTransportsCopy.addAll(delayedTransports); + oobTransportsCopy.addAll(oobTransports); + } + balancer = getCurrentLoadBalancer(); + resolver = nameResolver; + cancelIdleTimer(); + } + if (balancer != null) { + balancer.shutdown(); + } + resolver.shutdown(); + for (TransportSet ts : transportsCopy) { + ts.shutdown(); + } + for (DelayedClientTransport transport : delayedTransportsCopy) { + transport.shutdown(); + } + for (OobTransportProviderImpl provider : oobTransportsCopy) { + provider.close(); + } + if (log.isLoggable(Level.FINE)) { + log.log(Level.FINE, "[{0}] Shutting down", getLogId()); + } + return this; + } + + /** + * Initiates a forceful shutdown in which preexisting and new calls are cancelled. Although + * forceful, the shutdown process is still not instantaneous; {@link #isTerminated()} will likely + * return {@code false} immediately after this method returns. + */ + @Override + public ManagedChannelImpl shutdownNow() { + synchronized (lock) { + // Short-circuiting not strictly necessary, but prevents transports from needing to handle + // multiple shutdownNow invocations. + if (shutdownNowed) { + return this; + } + shutdownNowed = true; + } + shutdown(); + List transportsCopy; + List delayedTransportsCopy; + List oobTransportsCopy; + synchronized (lock) { + transportsCopy = new ArrayList(transports.values()); + transportsCopy.addAll(decommissionedTransports); + delayedTransportsCopy = new ArrayList(delayedTransports); + oobTransportsCopy = new ArrayList(oobTransports); + } + if (log.isLoggable(Level.FINE)) { + log.log(Level.FINE, "[{0}] Shutting down now", getLogId()); + } + Status nowStatus = Status.UNAVAILABLE.withDescription("Channel shutdownNow invoked"); + for (TransportSet ts : transportsCopy) { + ts.shutdownNow(nowStatus); + } + for (DelayedClientTransport transport : delayedTransportsCopy) { + transport.shutdownNow(nowStatus); + } + for (OobTransportProviderImpl provider : oobTransportsCopy) { + provider.shutdownNow(nowStatus); + } + return this; + } + + @Override + public boolean isShutdown() { + synchronized (lock) { + return shutdown; + } + } + + @Override + public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException { + synchronized (lock) { + long timeoutNanos = unit.toNanos(timeout); + long endTimeNanos = System.nanoTime() + timeoutNanos; + while (!terminated && (timeoutNanos = endTimeNanos - System.nanoTime()) > 0) { + TimeUnit.NANOSECONDS.timedWait(lock, timeoutNanos); + } + return terminated; + } + } + + @Override + public boolean isTerminated() { + synchronized (lock) { + return terminated; + } + } + + /* + * Creates a new outgoing call on the channel. + */ + @Override + public ClientCall newCall(MethodDescriptor method, + CallOptions callOptions) { + return interceptorChannel.newCall(method, callOptions); + } + + @Override + public String authority() { + return interceptorChannel.authority(); + } + + /** Returns {@code null} iff channel is in idle state. */ + @GuardedBy("lock") + private LoadBalancer getCurrentLoadBalancer() { + if (loadBalancer != null) { + return loadBalancer; + } + return graceLoadBalancer; + } + + private class RealChannel extends Channel { + @Override + public ClientCall newCall(MethodDescriptor method, + CallOptions callOptions) { + Executor executor = callOptions.getExecutor(); + if (executor == null) { + executor = ManagedChannelImpl.this.executor; + } + StatsTraceContext statsTraceCtx = StatsTraceContext.newClientContext( + method.getFullMethodName(), censusFactory, stopwatchSupplier); + return new ClientCallImpl( + method, + executor, + callOptions, + statsTraceCtx, + transportProvider, + scheduledExecutor) + .setDecompressorRegistry(decompressorRegistry) + .setCompressorRegistry(compressorRegistry); + } + + @Override + public String authority() { + String authority = nameResolver.getServiceAuthority(); + return checkNotNull(authority, "authority"); + } + } + + /** + * Terminate the channel if termination conditions are met. + */ + @GuardedBy("lock") + private void maybeTerminateChannel() { + if (terminated) { + return; + } + if (shutdown && transports.isEmpty() && decommissionedTransports.isEmpty() + && delayedTransports.isEmpty() && oobTransports.isEmpty()) { + if (log.isLoggable(Level.INFO)) { + log.log(Level.INFO, "[{0}] Terminated", getLogId()); + } + terminated = true; + lock.notifyAll(); + if (usingSharedExecutor) { + SharedResourceHolder.release(GrpcUtil.SHARED_CHANNEL_EXECUTOR, (ExecutorService) executor); + } + // Release the transport factory so that it can deallocate any resources. + transportFactory.close(); + } + } + + @VisibleForTesting + final TransportManager tm = new TransportManager() { + @Override + public void updateRetainedTransports(Collection addrs) { + // TODO(zhangkun83): warm-up new servers and discard removed servers. + } + + @Override + public ClientTransport getTransport(final EquivalentAddressGroup addressGroup) { + checkNotNull(addressGroup, "addressGroup"); + TransportSet ts = transports.get(addressGroup); + if (ts != null) { + return ts.obtainActiveTransport(); + } + synchronized (lock) { + if (shutdown) { + return SHUTDOWN_TRANSPORT; + } + if (getCurrentLoadBalancer() == null) { + return IDLE_MODE_TRANSPORT; + } + ts = transports.get(addressGroup); + if (ts == null) { + ts = new TransportSet(addressGroup, authority(), userAgent, getCurrentLoadBalancer(), + backoffPolicyProvider, transportFactory, scheduledExecutor, stopwatchSupplier, + executor, new TransportSet.Callback() { + @Override + public void onTerminated(TransportSet ts) { + synchronized (lock) { + transports.remove(addressGroup); + decommissionedTransports.remove(ts); + maybeTerminateChannel(); + } + } + + @Override + public void onAllAddressesFailed() { + nameResolver.refresh(); + } + + @Override + public void onConnectionClosedByServer(Status status) { + nameResolver.refresh(); + } + + @Override + public void onInUse(TransportSet ts) { + inUseStateAggregator.updateObjectInUse(ts, true); + } + + @Override + public void onNotInUse(TransportSet ts) { + inUseStateAggregator.updateObjectInUse(ts, false); + } + }); + if (log.isLoggable(Level.FINE)) { + log.log(Level.FINE, "[{0}] {1} created for {2}", + new Object[] {getLogId(), ts.getLogId(), addressGroup}); + } + transports.put(addressGroup, ts); + } + } + return ts.obtainActiveTransport(); + } + + @Override + public Channel makeChannel(ClientTransport transport) { + return new SingleTransportChannel( + censusFactory, transport, executor, scheduledExecutor, authority(), stopwatchSupplier); + } + + @Override + public ClientTransport createFailingTransport(Status error) { + return new FailingClientTransport(error); + } + + @Override + public InterimTransport createInterimTransport() { + return new InterimTransportImpl(); + } + + @Override + public OobTransportProvider createOobTransportProvider( + EquivalentAddressGroup addressGroup, String authority) { + return new OobTransportProviderImpl(addressGroup, authority); + } + }; + + @Override + public LogId getLogId() { + return logId; + } + + private static class NameResolverListenerImpl implements NameResolver.Listener { + final LoadBalancer balancer; + + NameResolverListenerImpl(LoadBalancer balancer) { + this.balancer = balancer; + } + + @Override + public void onUpdate(List servers, Attributes config) { + if (servers.isEmpty()) { + onError(Status.UNAVAILABLE.withDescription("NameResolver returned an empty list")); + return; + } + + try { + balancer.handleResolvedAddresses(servers, config); + } catch (Throwable e) { + // It must be a bug! Push the exception back to LoadBalancer in the hope that it may be + // propagated to the application. + balancer.handleNameResolutionError(Status.INTERNAL.withCause(e) + .withDescription("Thrown from handleResolvedAddresses(): " + e)); + } + } + + @Override + public void onError(Status error) { + checkArgument(!error.isOk(), "the error status must not be OK"); + balancer.handleNameResolutionError(error); + } + } + + private class InterimTransportImpl implements InterimTransport { + private final DelayedClientTransport delayedTransport; + private boolean closed; + + InterimTransportImpl() { + delayedTransport = new DelayedClientTransport(executor); + delayedTransport.start(new ManagedClientTransport.Listener() { + @Override public void transportShutdown(Status status) {} + + @Override public void transportTerminated() { + synchronized (lock) { + delayedTransports.remove(delayedTransport); + maybeTerminateChannel(); + } + inUseStateAggregator.updateObjectInUse(delayedTransport, false); + } + + @Override public void transportReady() {} + + @Override public void transportInUse(boolean inUse) { + inUseStateAggregator.updateObjectInUse(delayedTransport, inUse); + } + }); + boolean savedShutdown; + synchronized (lock) { + delayedTransports.add(delayedTransport); + savedShutdown = shutdown; + } + if (savedShutdown) { + delayedTransport.setTransport(SHUTDOWN_TRANSPORT); + delayedTransport.shutdown(); + } + } + + @Override + public ClientTransport transport() { + checkState(!closed, "already closed"); + return delayedTransport; + } + + @Override + public void closeWithRealTransports(Supplier realTransports) { + delayedTransport.setTransportSupplier(realTransports); + delayedTransport.shutdown(); + } + + @Override + public void closeWithError(Status error) { + delayedTransport.shutdownNow(error); + } + } + + private class OobTransportProviderImpl implements OobTransportProvider { + private final TransportSet transportSet; + private final ClientTransport transport; + + OobTransportProviderImpl(EquivalentAddressGroup addressGroup, String authority) { + synchronized (lock) { + if (shutdown) { + transportSet = null; + transport = SHUTDOWN_TRANSPORT; + } else if (getCurrentLoadBalancer() == null) { + transportSet = null; + transport = IDLE_MODE_TRANSPORT; + } else { + transport = null; + transportSet = new TransportSet(addressGroup, authority, userAgent, + getCurrentLoadBalancer(), backoffPolicyProvider, transportFactory, scheduledExecutor, + stopwatchSupplier, executor, new TransportSet.Callback() { + @Override + public void onTerminated(TransportSet ts) { + synchronized (lock) { + oobTransports.remove(OobTransportProviderImpl.this); + maybeTerminateChannel(); + } + } + }); + oobTransports.add(this); + } + } + } + + @Override + public ClientTransport get() { + if (transport != null) { + return transport; + } else { + return transportSet.obtainActiveTransport(); + } + } + + @Override + public void close() { + if (transportSet != null) { + transportSet.shutdown(); + } + } + + void shutdownNow(Status reason) { + if (transportSet != null) { + transportSet.shutdownNow(reason); + } + } + } +} diff --git a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2IdlenessTest.java b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2IdlenessTest.java new file mode 100644 index 00000000000..89dc573aea2 --- /dev/null +++ b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2IdlenessTest.java @@ -0,0 +1,440 @@ +/* + * 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.internal; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.same; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.collect.Lists; + +import io.grpc.Attributes; +import io.grpc.CallOptions; +import io.grpc.ClientCall; +import io.grpc.ClientInterceptor; +import io.grpc.CompressorRegistry; +import io.grpc.DecompressorRegistry; +import io.grpc.EquivalentAddressGroup; +import io.grpc.IntegerMarshaller; +import io.grpc.LoadBalancer; +import io.grpc.Metadata; +import io.grpc.MethodDescriptor; +import io.grpc.NameResolver; +import io.grpc.ResolvedServerInfo; +import io.grpc.ResolvedServerInfoGroup; +import io.grpc.Status; +import io.grpc.StringMarshaller; +import io.grpc.TransportManager.OobTransportProvider; +import io.grpc.TransportManager; +import io.grpc.internal.TestUtils.MockClientTransportInfo; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Matchers; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.net.SocketAddress; +import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +/** + * Unit tests for {@link ManagedChannelImpl}'s idle mode. + */ +@RunWith(JUnit4.class) +public class ManagedChannelImplIdlenessTest { + private final FakeClock timer = new FakeClock(); + private final FakeClock executor = new FakeClock(); + private static final String AUTHORITY = "fakeauthority"; + private static final String USER_AGENT = "fakeagent"; + private static final long IDLE_TIMEOUT_SECONDS = 30; + private ManagedChannelImpl channel; + + private final MethodDescriptor method = MethodDescriptor.create( + MethodDescriptor.MethodType.UNKNOWN, "/service/method", + new StringMarshaller(), new IntegerMarshaller()); + + private final List servers = Lists.newArrayList(); + private final List addressGroupList = + new ArrayList(); + + @Mock private SharedResourceHolder.Resource timerService; + @Mock private ClientTransportFactory mockTransportFactory; + @Mock private LoadBalancer mockLoadBalancer; + @Mock private LoadBalancer.Factory mockLoadBalancerFactory; + @Mock private NameResolver mockNameResolver; + @Mock private NameResolver.Factory mockNameResolverFactory; + @Mock private ClientCall.Listener mockCallListener; + @Captor private ArgumentCaptor nameResolverListenerCaptor; + private BlockingQueue newTransports; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + when(timerService.create()).thenReturn(timer.getScheduledExecutorService()); + when(mockLoadBalancerFactory + .newLoadBalancer(anyString(), Matchers.>any())) + .thenReturn(mockLoadBalancer); + when(mockNameResolver.getServiceAuthority()).thenReturn(AUTHORITY); + when(mockNameResolverFactory + .newNameResolver(any(URI.class), any(Attributes.class))) + .thenReturn(mockNameResolver); + + channel = new ManagedChannelImpl("fake://target", new FakeBackoffPolicyProvider(), + mockNameResolverFactory, Attributes.EMPTY, mockLoadBalancerFactory, + mockTransportFactory, DecompressorRegistry.getDefaultInstance(), + CompressorRegistry.getDefaultInstance(), timerService, timer.getStopwatchSupplier(), + TimeUnit.SECONDS.toMillis(IDLE_TIMEOUT_SECONDS), + executor.getScheduledExecutorService(), USER_AGENT, + Collections.emptyList(), + NoopCensusContextFactory.INSTANCE); + newTransports = TestUtils.captureTransports(mockTransportFactory); + + for (int i = 0; i < 2; i++) { + ResolvedServerInfoGroup.Builder resolvedServerInfoGroup = ResolvedServerInfoGroup.builder(); + for (int j = 0; j < 2; j++) { + resolvedServerInfoGroup.add( + new ResolvedServerInfo(new FakeSocketAddress("servergroup" + i + "server" + j))); + } + servers.add(resolvedServerInfoGroup.build()); + addressGroupList.add(resolvedServerInfoGroup.build().toEquivalentAddressGroup()); + } + verify(mockNameResolverFactory).newNameResolver(any(URI.class), any(Attributes.class)); + // Verify the initial idleness + verify(mockLoadBalancerFactory, never()).newLoadBalancer( + anyString(), Matchers.>any()); + verify(mockTransportFactory, never()).newClientTransport( + any(SocketAddress.class), anyString(), anyString()); + verify(mockNameResolver, never()).start(any(NameResolver.Listener.class)); + } + + @After + public void allPendingTasksAreRun() { + assertEquals(timer.getPendingTasks() + " should be empty", 0, timer.numPendingTasks()); + assertEquals(executor.getPendingTasks() + " should be empty", 0, executor.numPendingTasks()); + } + + @Test + public void newCallExitsIdleness() throws Exception { + final EquivalentAddressGroup addressGroup = addressGroupList.get(1); + doAnswer(new Answer() { + @Override + public ClientTransport answer(InvocationOnMock invocation) throws Throwable { + return channel.tm.getTransport(addressGroup); + } + }).when(mockLoadBalancer).pickTransport(any(Attributes.class)); + + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + call.start(mockCallListener, new Metadata()); + + verify(mockLoadBalancerFactory).newLoadBalancer(anyString(), same(channel.tm)); + // NameResolver is started in the scheduled executor + timer.runDueTasks(); + verify(mockNameResolver).start(nameResolverListenerCaptor.capture()); + + // LoadBalancer is used right after created. + verify(mockLoadBalancer).pickTransport(any(Attributes.class)); + verify(mockTransportFactory).newClientTransport( + addressGroup.getAddresses().get(0), AUTHORITY, USER_AGENT); + + // Simulate new address resolved + nameResolverListenerCaptor.getValue().onUpdate(servers, Attributes.EMPTY); + verify(mockLoadBalancer).handleResolvedAddresses(servers, Attributes.EMPTY); + } + + @Test + public void newCallResetsGracePeriod() throws Exception { + final EquivalentAddressGroup addressGroup = addressGroupList.get(1); + doAnswer(new Answer() { + @Override + public ClientTransport answer(InvocationOnMock invocation) throws Throwable { + return channel.tm.getTransport(addressGroup); + } + }).when(mockLoadBalancer).pickTransport(any(Attributes.class)); + + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + call.start(mockCallListener, new Metadata()); + call.cancel("cleanup", null); + executor.runDueTasks(); + + timer.runDueTasks(); + verify(mockLoadBalancerFactory).newLoadBalancer(anyString(), same(channel.tm)); + verify(mockLoadBalancer).pickTransport(any(Attributes.class)); + + // Enter grace period + timer.forwardTime(TimeUnit.SECONDS.toMillis(IDLE_TIMEOUT_SECONDS) + - ManagedChannelImpl.IDLE_GRACE_PERIOD_MILLIS, TimeUnit.MILLISECONDS); + assertTrue(channel.isInIdleGracePeriod()); + + call = channel.newCall(method, CallOptions.DEFAULT); + call.start(mockCallListener, new Metadata()); + assertFalse(channel.isInIdleGracePeriod()); + call.cancel("cleanup", null); + executor.runDueTasks(); + + // Load balancer was reused. + timer.runDueTasks(); + verify(mockLoadBalancerFactory).newLoadBalancer(anyString(), same(channel.tm)); + verify(mockLoadBalancer, times(2)).pickTransport(any(Attributes.class)); + + // Now just let time pass to allow the original idle time to be well past expired. + timer.forwardTime(IDLE_TIMEOUT_SECONDS - 1, TimeUnit.SECONDS); + + call = channel.newCall(method, CallOptions.DEFAULT); + call.start(mockCallListener, new Metadata()); + + // Load balancer was reused; the idle time period must have been reset. + timer.runDueTasks(); + verify(mockLoadBalancerFactory).newLoadBalancer(anyString(), same(channel.tm)); + verify(mockLoadBalancer, times(3)).pickTransport(any(Attributes.class)); + } + + @Test + public void shutdownDuringGracePeriodShutdownLb() throws Exception { + forceExitIdleMode(); + verify(mockLoadBalancerFactory).newLoadBalancer(anyString(), same(channel.tm)); + // Enter grace period + timer.forwardTime(TimeUnit.SECONDS.toMillis(IDLE_TIMEOUT_SECONDS) + - ManagedChannelImpl.IDLE_GRACE_PERIOD_MILLIS, TimeUnit.MILLISECONDS); + verify(mockLoadBalancer, never()).shutdown(); + channel.shutdown(); + verify(mockLoadBalancer).shutdown(); + } + + @Test + public void enterIdleModeAfterForceExit() throws Exception { + forceExitIdleMode(); + + // Trigger the creation of TransportSets + for (EquivalentAddressGroup addressGroup : addressGroupList) { + channel.tm.getTransport(addressGroup); + verify(mockTransportFactory).newClientTransport( + addressGroup.getAddresses().get(0), AUTHORITY, USER_AGENT); + } + ArrayList transports = new ArrayList(); + newTransports.drainTo(transports); + assertEquals(addressGroupList.size(), transports.size()); + + channel.tm.createInterimTransport(); + + // Without actually using these transports, will eventually enter idle mode + walkIntoIdleMode(transports); + } + + @Test + public void interimTransportHoldsOffIdleness() throws Exception { + doAnswer(new Answer() { + @Override + public ClientTransport answer(InvocationOnMock invocation) throws Throwable { + return channel.tm.createInterimTransport().transport(); + } + }).when(mockLoadBalancer).pickTransport(any(Attributes.class)); + + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + call.start(mockCallListener, new Metadata()); + assertTrue(channel.inUseStateAggregator.isInUse()); + // NameResolver is started in the scheduled executor + timer.runDueTasks(); + + // As long as the interim transport is in-use (by the pending RPC), the channel won't go idle. + timer.forwardTime(IDLE_TIMEOUT_SECONDS * 2, TimeUnit.SECONDS); + assertTrue(channel.inUseStateAggregator.isInUse()); + + // Cancelling the only RPC will reset the in-use state. + assertEquals(0, executor.numPendingTasks()); + call.cancel("In test", null); + assertEquals(1, executor.runDueTasks()); + assertFalse(channel.inUseStateAggregator.isInUse()); + // And allow the channel to go idle. + walkIntoIdleMode(Collections.emptyList()); + } + + @Test + public void realTransportsHoldsOffIdleness() throws Exception { + final EquivalentAddressGroup addressGroup = addressGroupList.get(1); + doAnswer(new Answer() { + @Override + public ClientTransport answer(InvocationOnMock invocation) throws Throwable { + return channel.tm.getTransport(addressGroup); + } + }).when(mockLoadBalancer).pickTransport(any(Attributes.class)); + + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + call.start(mockCallListener, new Metadata()); + + // A TransportSet is in-use, while the stream is pending in a delayed transport + assertTrue(channel.inUseStateAggregator.isInUse()); + // NameResolver is started in the scheduled executor + timer.runDueTasks(); + + // Making the real transport ready, will release the delayed transport. + // The TransportSet is *not* in-use before the real transport become in-use. + MockClientTransportInfo t0 = newTransports.poll(); + assertEquals(0, executor.numPendingTasks()); + t0.listener.transportReady(); + // Real streams are started in the executor + assertEquals(1, executor.runDueTasks()); + assertFalse(channel.inUseStateAggregator.isInUse()); + t0.listener.transportInUse(true); + assertTrue(channel.inUseStateAggregator.isInUse()); + + // As long as the transport is in-use, the channel won't go idle. + timer.forwardTime(IDLE_TIMEOUT_SECONDS * 2, TimeUnit.SECONDS); + + t0.listener.transportInUse(false); + assertFalse(channel.inUseStateAggregator.isInUse()); + // And allow the channel to go idle. + walkIntoIdleMode(Arrays.asList(t0)); + } + + @Test + public void idlenessDecommissionsTransports() throws Exception { + EquivalentAddressGroup addressGroup = addressGroupList.get(0); + forceExitIdleMode(); + + channel.tm.getTransport(addressGroup); + MockClientTransportInfo t0 = newTransports.poll(); + t0.listener.transportReady(); + assertSame(t0.transport, channelTmGetTransportUnwrapped(addressGroup)); + + walkIntoIdleMode(Arrays.asList(t0)); + verify(t0.transport).shutdown(); + + forceExitIdleMode(); + channel.tm.getTransport(addressGroup); + MockClientTransportInfo t1 = newTransports.poll(); + t1.listener.transportReady(); + + assertSame(t1.transport, channelTmGetTransportUnwrapped(addressGroup)); + assertNotSame(t0.transport, channelTmGetTransportUnwrapped(addressGroup)); + + channel.shutdown(); + verify(t1.transport).shutdown(); + channel.shutdownNow(); + verify(t0.transport).shutdownNow(any(Status.class)); + verify(t1.transport).shutdownNow(any(Status.class)); + + t1.listener.transportTerminated(); + assertFalse(channel.isTerminated()); + t0.listener.transportTerminated(); + assertTrue(channel.isTerminated()); + } + + @Test + public void loadBalancerShouldNotCreateConnectionsWhenIdle() throws Exception { + // Acts as a misbehaving LoadBalancer that tries to create connections when channel is in idle, + // which means the LoadBalancer is supposedly shutdown. + assertSame(ManagedChannelImpl.IDLE_MODE_TRANSPORT, + channel.tm.getTransport(addressGroupList.get(0))); + OobTransportProvider oobProvider = + channel.tm.createOobTransportProvider(addressGroupList.get(0), AUTHORITY); + assertSame(ManagedChannelImpl.IDLE_MODE_TRANSPORT, oobProvider.get()); + oobProvider.close(); + verify(mockTransportFactory, never()).newClientTransport( + any(SocketAddress.class), anyString(), anyString()); + // We don't care for delayed (interim) transports, because they don't create connections. + } + + private void walkIntoIdleMode(Collection currentTransports) { + timer.forwardTime(IDLE_TIMEOUT_SECONDS - 1, TimeUnit.SECONDS); + verify(mockLoadBalancer, never()).shutdown(); + verify(mockNameResolver, never()).shutdown(); + for (MockClientTransportInfo transport : currentTransports) { + verify(transport.transport, never()).shutdown(); + } + timer.forwardTime(1, TimeUnit.SECONDS); + verify(mockLoadBalancer).shutdown(); + verify(mockNameResolver).shutdown(); + for (MockClientTransportInfo transport : currentTransports) { + verify(transport.transport).shutdown(); + } + } + + private void forceExitIdleMode() { + channel.exitIdleMode(); + // NameResolver is started in the scheduled executor + timer.runDueTasks(); + } + + private ClientTransport channelTmGetTransportUnwrapped(EquivalentAddressGroup addressGroup) { + return ((ForwardingConnectionClientTransport) channel.tm.getTransport(addressGroup)).delegate(); + } + + private static class FakeBackoffPolicyProvider implements BackoffPolicy.Provider { + @Override + public BackoffPolicy get() { + return new BackoffPolicy() { + @Override + public long nextBackoffMillis() { + return 1; + } + }; + } + } + + private static class FakeSocketAddress extends SocketAddress { + final String name; + + FakeSocketAddress(String name) { + this.name = name; + } + + @Override + public String toString() { + return "FakeSocketAddress-" + name; + } + } +} diff --git a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java new file mode 100644 index 00000000000..a719c39bbbf --- /dev/null +++ b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java @@ -0,0 +1,1082 @@ +/* + * Copyright 2015, 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.internal; + +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; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.isA; +import static org.mockito.Matchers.same; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.atMost; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableList; + +import io.grpc.Attributes; +import io.grpc.CallCredentials.MetadataApplier; +import io.grpc.CallCredentials; +import io.grpc.CallOptions; +import io.grpc.Channel; +import io.grpc.ClientCall; +import io.grpc.ClientInterceptor; +import io.grpc.Compressor; +import io.grpc.CompressorRegistry; +import io.grpc.Context; +import io.grpc.DecompressorRegistry; +import io.grpc.IntegerMarshaller; +import io.grpc.LoadBalancer; +import io.grpc.Metadata; +import io.grpc.MethodDescriptor; +import io.grpc.NameResolver; +import io.grpc.PickFirstBalancerFactory; +import io.grpc.ResolvedServerInfo; +import io.grpc.ResolvedServerInfoGroup; +import io.grpc.SecurityLevel; +import io.grpc.Status; +import io.grpc.StringMarshaller; +import io.grpc.TransportManager; +import io.grpc.internal.testing.CensusTestUtils.FakeCensusContextFactory; + +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Matchers; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.net.SocketAddress; +import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +/** Unit tests for {@link ManagedChannelImpl}. */ +@RunWith(JUnit4.class) +public class ManagedChannelImplTest { + private static final List NO_INTERCEPTOR = + Collections.emptyList(); + private static final Attributes NAME_RESOLVER_PARAMS = + Attributes.newBuilder().set(NameResolver.Factory.PARAMS_DEFAULT_PORT, 447).build(); + private final MethodDescriptor method = MethodDescriptor.create( + MethodDescriptor.MethodType.UNKNOWN, "/service/method", + new StringMarshaller(), new IntegerMarshaller()); + private final String serviceName = "fake.example.com"; + private final String authority = serviceName; + private final String userAgent = "userAgent"; + private final String target = "fake://" + serviceName; + private URI expectedUri; + private final SocketAddress socketAddress = new SocketAddress() {}; + private final ResolvedServerInfo server = new ResolvedServerInfo(socketAddress, Attributes.EMPTY); + private final FakeClock timer = new FakeClock(); + private final FakeClock executor = new FakeClock(); + private final FakeCensusContextFactory censusCtxFactory = new FakeCensusContextFactory(); + private SpyingLoadBalancerFactory loadBalancerFactory = + new SpyingLoadBalancerFactory(PickFirstBalancerFactory.getInstance()); + + @Rule public final ExpectedException thrown = ExpectedException.none(); + + private ManagedChannelImpl channel; + @Captor + private ArgumentCaptor statusCaptor; + @Captor + private ArgumentCaptor statsTraceCtxCaptor; + @Mock + private ConnectionClientTransport mockTransport; + @Mock + private ClientTransportFactory mockTransportFactory; + @Mock + private ClientCall.Listener mockCallListener; + @Mock + private ClientCall.Listener mockCallListener2; + @Mock + private ClientCall.Listener mockCallListener3; + @Mock + private SharedResourceHolder.Resource timerService; + @Mock + private CallCredentials creds; + + private ArgumentCaptor transportListenerCaptor = + ArgumentCaptor.forClass(ManagedClientTransport.Listener.class); + private ArgumentCaptor streamListenerCaptor = + ArgumentCaptor.forClass(ClientStreamListener.class); + + private void createChannel( + NameResolver.Factory nameResolverFactory, List interceptors) { + channel = new ManagedChannelImpl(target, new FakeBackoffPolicyProvider(), + nameResolverFactory, NAME_RESOLVER_PARAMS, loadBalancerFactory, + mockTransportFactory, DecompressorRegistry.getDefaultInstance(), + CompressorRegistry.getDefaultInstance(), timerService, timer.getStopwatchSupplier(), + ManagedChannelImpl.IDLE_TIMEOUT_MILLIS_DISABLE, + executor.getScheduledExecutorService(), userAgent, interceptors, censusCtxFactory); + // Force-exit the initial idle-mode + channel.exitIdleMode(); + // Will start NameResolver in the scheduled executor + assertEquals(1, timer.runDueTasks()); + } + + @Before + public void setUp() throws Exception { + MockitoAnnotations.initMocks(this); + expectedUri = new URI(target); + when(mockTransportFactory.newClientTransport( + any(SocketAddress.class), any(String.class), any(String.class))) + .thenReturn(mockTransport); + when(timerService.create()).thenReturn(timer.getScheduledExecutorService()); + } + + @After + public void allPendingTasksAreRun() throws Exception { + // The "never" verifications in the tests only hold up if all due tasks are done. + // As for timer, although there may be scheduled tasks in a future time, since we don't test + // any time-related behavior in this test suite, we only care the tasks that are due. This + // would ignore any time-sensitive tasks, e.g., back-off and the idle timer. + assertTrue(timer.getDueTasks() + " should be empty", timer.getDueTasks().isEmpty()); + assertEquals(executor.getPendingTasks() + " should be empty", 0, executor.numPendingTasks()); + } + + /** + * The counterpart of {@link ManagedChannelImplIdlenessTest#enterIdleModeAfterForceExit}. + */ + @Test + @SuppressWarnings("unchecked") + public void idleModeDisabled() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + assertEquals(1, loadBalancerFactory.balancers.size()); + + // No task is scheduled to enter idle mode + assertEquals(0, timer.numPendingTasks()); + assertEquals(0, executor.numPendingTasks()); + } + + @Test + public void immediateDeadlineExceeded() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + ClientCall call = + channel.newCall(method, CallOptions.DEFAULT.withDeadlineAfter(0, TimeUnit.NANOSECONDS)); + call.start(mockCallListener, new Metadata()); + assertEquals(1, executor.runDueTasks()); + + verify(mockCallListener).onClose(statusCaptor.capture(), any(Metadata.class)); + Status status = statusCaptor.getValue(); + assertSame(Status.DEADLINE_EXCEEDED.getCode(), status.getCode()); + } + + @Test + public void shutdownWithNoTransportsEverCreated() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + verifyNoMoreInteractions(mockTransportFactory); + channel.shutdown(); + assertTrue(channel.isShutdown()); + assertTrue(channel.isTerminated()); + } + + @Test + public void twoCallsAndGracefulShutdown() { + FakeNameResolverFactory nameResolverFactory = new FakeNameResolverFactory(true); + createChannel(nameResolverFactory, NO_INTERCEPTOR); + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + verifyNoMoreInteractions(mockTransportFactory); + + // Create transport and call + ClientStream mockStream = mock(ClientStream.class); + Metadata headers = new Metadata(); + when(mockTransportFactory.newClientTransport( + any(SocketAddress.class), any(String.class), any(String.class))) + .thenReturn(mockTransport); + when(mockTransport.newStream(same(method), same(headers), same(CallOptions.DEFAULT), + any(StatsTraceContext.class))) + .thenReturn(mockStream); + call.start(mockCallListener, headers); + timer.runDueTasks(); + executor.runDueTasks(); + + verify(mockTransportFactory) + .newClientTransport(same(socketAddress), eq(authority), eq(userAgent)); + verify(mockTransport).start(transportListenerCaptor.capture()); + ManagedClientTransport.Listener transportListener = transportListenerCaptor.getValue(); + transportListener.transportReady(); + executor.runDueTasks(); + + verify(mockTransport).newStream(same(method), same(headers), same(CallOptions.DEFAULT), + statsTraceCtxCaptor.capture()); + assertEquals(censusCtxFactory.pollContextOrFail(), + statsTraceCtxCaptor.getValue().getCensusContext()); + verify(mockStream).start(streamListenerCaptor.capture()); + verify(mockStream).setCompressor(isA(Compressor.class)); + ClientStreamListener streamListener = streamListenerCaptor.getValue(); + + // Second call + ClientCall call2 = channel.newCall(method, CallOptions.DEFAULT); + ClientStream mockStream2 = mock(ClientStream.class); + Metadata headers2 = new Metadata(); + when(mockTransport.newStream(same(method), same(headers2), same(CallOptions.DEFAULT), + any(StatsTraceContext.class))) + .thenReturn(mockStream2); + call2.start(mockCallListener2, headers2); + verify(mockTransport).newStream(same(method), same(headers2), same(CallOptions.DEFAULT), + statsTraceCtxCaptor.capture()); + assertEquals(censusCtxFactory.pollContextOrFail(), + statsTraceCtxCaptor.getValue().getCensusContext()); + + verify(mockStream2).start(streamListenerCaptor.capture()); + ClientStreamListener streamListener2 = streamListenerCaptor.getValue(); + Metadata trailers = new Metadata(); + streamListener2.closed(Status.CANCELLED, trailers); + executor.runDueTasks(); + + verify(mockCallListener2).onClose(Status.CANCELLED, trailers); + + // Shutdown + channel.shutdown(); + assertTrue(channel.isShutdown()); + assertFalse(channel.isTerminated()); + verify(mockTransport).shutdown(); + assertEquals(1, nameResolverFactory.resolvers.size()); + assertTrue(nameResolverFactory.resolvers.get(0).shutdown); + assertEquals(1, loadBalancerFactory.balancers.size()); + verify(loadBalancerFactory.balancers.get(0)).shutdown(); + + // Further calls should fail without going to the transport + ClientCall call3 = channel.newCall(method, CallOptions.DEFAULT); + call3.start(mockCallListener3, new Metadata()); + timer.runDueTasks(); + executor.runDueTasks(); + + verify(mockCallListener3).onClose(statusCaptor.capture(), any(Metadata.class)); + assertSame(Status.Code.UNAVAILABLE, statusCaptor.getValue().getCode()); + + // Finish shutdown + transportListener.transportShutdown(Status.CANCELLED); + assertFalse(channel.isTerminated()); + streamListener.closed(Status.CANCELLED, trailers); + executor.runDueTasks(); + + verify(mockCallListener).onClose(Status.CANCELLED, trailers); + assertFalse(channel.isTerminated()); + + transportListener.transportTerminated(); + assertTrue(channel.isTerminated()); + + verify(mockTransportFactory).close(); + verifyNoMoreInteractions(mockTransportFactory); + verify(mockTransport, atLeast(0)).getLogId(); + verifyNoMoreInteractions(mockTransport); + verifyNoMoreInteractions(mockStream); + } + + @Test + public void callAndShutdownNow() { + FakeNameResolverFactory nameResolverFactory = new FakeNameResolverFactory(true); + createChannel(nameResolverFactory, NO_INTERCEPTOR); + verifyNoMoreInteractions(mockTransportFactory); + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + verifyNoMoreInteractions(mockTransportFactory); + + // Create transport and call + ClientStream mockStream = mock(ClientStream.class); + Metadata headers = new Metadata(); + when(mockTransportFactory.newClientTransport( + any(SocketAddress.class), any(String.class), any(String.class))) + .thenReturn(mockTransport); + when(mockTransport.newStream(same(method), same(headers), same(CallOptions.DEFAULT), + any(StatsTraceContext.class))) + .thenReturn(mockStream); + call.start(mockCallListener, headers); + timer.runDueTasks(); + executor.runDueTasks(); + + verify(mockTransportFactory) + .newClientTransport(same(socketAddress), eq(authority), any(String.class)); + verify(mockTransport).start(transportListenerCaptor.capture()); + ManagedClientTransport.Listener transportListener = transportListenerCaptor.getValue(); + transportListener.transportReady(); + executor.runDueTasks(); + + verify(mockTransport).newStream(same(method), same(headers), same(CallOptions.DEFAULT), + any(StatsTraceContext.class)); + + verify(mockStream).start(streamListenerCaptor.capture()); + verify(mockStream).setCompressor(isA(Compressor.class)); + ClientStreamListener streamListener = streamListenerCaptor.getValue(); + + // ShutdownNow + channel.shutdownNow(); + assertTrue(channel.isShutdown()); + assertFalse(channel.isTerminated()); + // ShutdownNow may or may not invoke shutdown. Ideally it wouldn't, but it doesn't matter much + // either way. + verify(mockTransport, atMost(1)).shutdown(); + verify(mockTransport).shutdownNow(any(Status.class)); + assertEquals(1, nameResolverFactory.resolvers.size()); + assertTrue(nameResolverFactory.resolvers.get(0).shutdown); + assertEquals(1, loadBalancerFactory.balancers.size()); + verify(loadBalancerFactory.balancers.get(0)).shutdown(); + + // Further calls should fail without going to the transport + ClientCall call3 = channel.newCall(method, CallOptions.DEFAULT); + call3.start(mockCallListener3, new Metadata()); + executor.runDueTasks(); + + verify(mockCallListener3).onClose(statusCaptor.capture(), any(Metadata.class)); + assertSame(Status.Code.UNAVAILABLE, statusCaptor.getValue().getCode()); + + // Finish shutdown + transportListener.transportShutdown(Status.CANCELLED); + assertFalse(channel.isTerminated()); + Metadata trailers = new Metadata(); + streamListener.closed(Status.CANCELLED, trailers); + executor.runDueTasks(); + + verify(mockCallListener).onClose(Status.CANCELLED, trailers); + assertFalse(channel.isTerminated()); + + transportListener.transportTerminated(); + assertTrue(channel.isTerminated()); + + verify(mockTransportFactory).close(); + verifyNoMoreInteractions(mockTransportFactory); + verify(mockTransport, atLeast(0)).getLogId(); + verifyNoMoreInteractions(mockTransport); + verifyNoMoreInteractions(mockStream); + } + + /** Make sure shutdownNow() after shutdown() has an effect. */ + @Test + public void callAndShutdownAndShutdownNow() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + + // Create transport and call + ClientStream mockStream = mock(ClientStream.class); + Metadata headers = new Metadata(); + when(mockTransport.newStream(same(method), same(headers), same(CallOptions.DEFAULT), + any(StatsTraceContext.class))) + .thenReturn(mockStream); + call.start(mockCallListener, headers); + timer.runDueTasks(); + executor.runDueTasks(); + + verify(mockTransport).start(transportListenerCaptor.capture()); + ManagedClientTransport.Listener transportListener = transportListenerCaptor.getValue(); + transportListener.transportReady(); + executor.runDueTasks(); + + verify(mockStream).start(streamListenerCaptor.capture()); + ClientStreamListener streamListener = streamListenerCaptor.getValue(); + + // ShutdownNow + channel.shutdown(); + channel.shutdownNow(); + // ShutdownNow may or may not invoke shutdown. Ideally it wouldn't, but it doesn't matter much + // either way. + verify(mockTransport, atMost(2)).shutdown(); + verify(mockTransport).shutdownNow(any(Status.class)); + + // Finish shutdown + transportListener.transportShutdown(Status.CANCELLED); + assertFalse(channel.isTerminated()); + Metadata trailers = new Metadata(); + streamListener.closed(Status.CANCELLED, trailers); + executor.runDueTasks(); + + verify(mockCallListener).onClose(Status.CANCELLED, trailers); + assertFalse(channel.isTerminated()); + + transportListener.transportTerminated(); + assertTrue(channel.isTerminated()); + } + + + @Test + public void interceptor() throws Exception { + final AtomicLong atomic = new AtomicLong(); + ClientInterceptor interceptor = new ClientInterceptor() { + @Override + public ClientCall interceptCall( + MethodDescriptor method, CallOptions callOptions, + Channel next) { + atomic.set(1); + return next.newCall(method, callOptions); + } + }; + createChannel(new FakeNameResolverFactory(true), Arrays.asList(interceptor)); + assertNotNull(channel.newCall(method, CallOptions.DEFAULT)); + assertEquals(1, atomic.get()); + } + + @Test + public void testNoDeadlockOnShutdown() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + // Force creation of transport + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + Metadata headers = new Metadata(); + ClientStream mockStream = mock(ClientStream.class); + when(mockTransport.newStream(same(method), same(headers))).thenReturn(mockStream); + call.start(mockCallListener, headers); + timer.runDueTasks(); + executor.runDueTasks(); + call.cancel("Cancel for test", null); + executor.runDueTasks(); + + verify(mockTransport).start(transportListenerCaptor.capture()); + final ManagedClientTransport.Listener transportListener = transportListenerCaptor.getValue(); + final Object lock = new Object(); + final CyclicBarrier barrier = new CyclicBarrier(2); + new Thread() { + @Override + public void run() { + synchronized (lock) { + try { + barrier.await(); + } catch (Exception ex) { + throw new AssertionError(ex); + } + // To deadlock, a lock would be needed for this call to proceed. + transportListener.transportShutdown(Status.CANCELLED); + } + } + }.start(); + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) { + // To deadlock, a lock would need to be held while this method is in progress. + try { + barrier.await(); + } catch (Exception ex) { + throw new AssertionError(ex); + } + // If deadlock is possible with this setup, this sychronization completes the loop because + // the transportShutdown needs a lock that Channel is holding while calling this method. + synchronized (lock) { + } + return null; + } + }).when(mockTransport).shutdown(); + channel.shutdown(); + + transportListener.transportTerminated(); + } + + @Test + public void callOptionsExecutor() { + Metadata headers = new Metadata(); + ClientStream mockStream = mock(ClientStream.class); + when(mockTransport.newStream(same(method), same(headers), any(CallOptions.class), + any(StatsTraceContext.class))) + .thenReturn(mockStream); + FakeClock callExecutor = new FakeClock(); + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + CallOptions options = + CallOptions.DEFAULT.withExecutor(callExecutor.getScheduledExecutorService()); + + ClientCall call = channel.newCall(method, options); + call.start(mockCallListener, headers); + timer.runDueTasks(); + executor.runDueTasks(); + + verify(mockTransport).start(transportListenerCaptor.capture()); + assertEquals(0, executor.numPendingTasks()); + transportListenerCaptor.getValue().transportReady(); + // Real streams are started in the channel's executor + assertEquals(1, executor.runDueTasks()); + + verify(mockTransport).newStream(same(method), same(headers), same(options), + any(StatsTraceContext.class)); + verify(mockStream).start(streamListenerCaptor.capture()); + ClientStreamListener streamListener = streamListenerCaptor.getValue(); + Metadata trailers = new Metadata(); + assertEquals(0, callExecutor.numPendingTasks()); + streamListener.closed(Status.CANCELLED, trailers); + verify(mockCallListener, never()).onClose(same(Status.CANCELLED), same(trailers)); + assertEquals(1, callExecutor.runDueTasks()); + + verify(mockCallListener).onClose(same(Status.CANCELLED), same(trailers)); + } + + @Test + public void nameResolutionFailed() { + Status error = Status.UNAVAILABLE.withCause(new Throwable("fake name resolution error")); + + // Name resolution is started as soon as channel is created. + createChannel(new FailingNameResolverFactory(error), NO_INTERCEPTOR); + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + call.start(mockCallListener, new Metadata()); + timer.runDueTasks(); + executor.runDueTasks(); + + // The call failed with the name resolution error + verify(mockCallListener).onClose(statusCaptor.capture(), any(Metadata.class)); + Status status = statusCaptor.getValue(); + assertSame(error.getCode(), status.getCode()); + assertSame(error.getCause(), status.getCause()); + // LoadBalancer received the same error + assertEquals(1, loadBalancerFactory.balancers.size()); + verify(loadBalancerFactory.balancers.get(0)).handleNameResolutionError(same(error)); + } + + @Test + public void nameResolverReturnsEmptySubLists() { + String errorDescription = "NameResolver returned an empty list"; + + // Name resolution is started as soon as channel is created + createChannel(new FakeNameResolverFactory(), NO_INTERCEPTOR); + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + call.start(mockCallListener, new Metadata()); + timer.runDueTasks(); + executor.runDueTasks(); + + // The call failed with the name resolution error + verify(mockCallListener).onClose(statusCaptor.capture(), any(Metadata.class)); + Status status = statusCaptor.getValue(); + assertSame(Status.Code.UNAVAILABLE, status.getCode()); + assertTrue(status.getDescription(), status.getDescription().contains(errorDescription)); + // LoadBalancer received the same error + assertEquals(1, loadBalancerFactory.balancers.size()); + verify(loadBalancerFactory.balancers.get(0)).handleNameResolutionError(statusCaptor.capture()); + status = statusCaptor.getValue(); + assertSame(Status.Code.UNAVAILABLE, status.getCode()); + assertEquals(errorDescription, status.getDescription()); + } + + @Test + public void loadBalancerThrowsInHandleResolvedAddresses() { + RuntimeException ex = new RuntimeException("simulated"); + // Delay the success of name resolution until allResolved() is called + FakeNameResolverFactory nameResolverFactory = new FakeNameResolverFactory(false); + createChannel(nameResolverFactory, NO_INTERCEPTOR); + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + call.start(mockCallListener, new Metadata()); + timer.runDueTasks(); + executor.runDueTasks(); + + assertEquals(1, loadBalancerFactory.balancers.size()); + LoadBalancer loadBalancer = loadBalancerFactory.balancers.get(0); + doThrow(ex).when(loadBalancer).handleResolvedAddresses( + Matchers.>anyObject(), any(Attributes.class)); + + // NameResolver returns addresses. + nameResolverFactory.allResolved(); + executor.runDueTasks(); + + // The call failed with the error thrown from handleResolvedAddresses() + verify(mockCallListener).onClose(statusCaptor.capture(), any(Metadata.class)); + Status status = statusCaptor.getValue(); + assertSame(Status.Code.INTERNAL, status.getCode()); + assertSame(ex, status.getCause()); + // The LoadBalancer received the same error + verify(loadBalancer).handleNameResolutionError(statusCaptor.capture()); + status = statusCaptor.getValue(); + assertSame(Status.Code.INTERNAL, status.getCode()); + assertSame(ex, status.getCause()); + } + + @Test + public void nameResolvedAfterChannelShutdown() { + // Delay the success of name resolution until allResolved() is called. + FakeNameResolverFactory nameResolverFactory = new FakeNameResolverFactory(false); + createChannel(nameResolverFactory, NO_INTERCEPTOR); + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + Metadata headers = new Metadata(); + + call.start(mockCallListener, headers); + timer.runDueTasks(); + executor.runDueTasks(); + channel.shutdown(); + + assertTrue(channel.isShutdown()); + // Name resolved after the channel is shut down, which is possible if the name resolution takes + // time and is not cancellable. The resolved address will still be passed to the LoadBalancer. + nameResolverFactory.allResolved(); + executor.runDueTasks(); + + verify(mockTransportFactory, never()) + .newClientTransport(any(SocketAddress.class), any(String.class), any(String.class)); + } + + /** + * Verify that if the first resolved address points to a server that cannot be connected, the call + * will end up with the second address which works. + */ + @Test + public void firstResolvedServerFailedToConnect() throws Exception { + final SocketAddress goodAddress = new SocketAddress() { + @Override public String toString() { + return "goodAddress"; + } + }; + final SocketAddress badAddress = new SocketAddress() { + @Override public String toString() { + return "badAddress"; + } + }; + final ResolvedServerInfo goodServer = new ResolvedServerInfo(goodAddress, Attributes.EMPTY); + final ResolvedServerInfo badServer = new ResolvedServerInfo(badAddress, Attributes.EMPTY); + final ConnectionClientTransport goodTransport = mock(ConnectionClientTransport.class); + final ConnectionClientTransport badTransport = mock(ConnectionClientTransport.class); + when(goodTransport.newStream( + any(MethodDescriptor.class), any(Metadata.class), any(CallOptions.class), + any(StatsTraceContext.class))) + .thenReturn(mock(ClientStream.class)); + when(mockTransportFactory.newClientTransport( + same(goodAddress), any(String.class), any(String.class))) + .thenReturn(goodTransport); + when(mockTransportFactory.newClientTransport( + same(badAddress), any(String.class), any(String.class))) + .thenReturn(badTransport); + + FakeNameResolverFactory nameResolverFactory = + new FakeNameResolverFactory(Arrays.asList(badServer, goodServer)); + createChannel(nameResolverFactory, NO_INTERCEPTOR); + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + Metadata headers = new Metadata(); + + // Start a call. The channel will starts with the first address (badAddress) + call.start(mockCallListener, headers); + timer.runDueTasks(); + executor.runDueTasks(); + + ArgumentCaptor badTransportListenerCaptor = + ArgumentCaptor.forClass(ManagedClientTransport.Listener.class); + verify(badTransport).start(badTransportListenerCaptor.capture()); + verify(mockTransportFactory) + .newClientTransport(same(badAddress), any(String.class), any(String.class)); + verify(mockTransportFactory, times(0)) + .newClientTransport(same(goodAddress), any(String.class), any(String.class)); + badTransportListenerCaptor.getValue().transportShutdown(Status.UNAVAILABLE); + + // The channel then try the second address (goodAddress) + ArgumentCaptor goodTransportListenerCaptor = + ArgumentCaptor.forClass(ManagedClientTransport.Listener.class); + verify(mockTransportFactory) + .newClientTransport(same(goodAddress), any(String.class), any(String.class)); + verify(goodTransport).start(goodTransportListenerCaptor.capture()); + goodTransportListenerCaptor.getValue().transportReady(); + executor.runDueTasks(); + + verify(goodTransport).newStream(same(method), same(headers), same(CallOptions.DEFAULT), + any(StatsTraceContext.class)); + // The bad transport was never used. + verify(badTransport, times(0)).newStream(any(MethodDescriptor.class), any(Metadata.class)); + } + + /** + * Verify that if all resolved addresses failed to connect, the call will fail. + */ + @Test + public void allServersFailedToConnect() throws Exception { + final SocketAddress addr1 = new SocketAddress() { + @Override public String toString() { + return "addr1"; + } + }; + final SocketAddress addr2 = new SocketAddress() { + @Override public String toString() { + return "addr2"; + } + }; + final ResolvedServerInfo server1 = new ResolvedServerInfo(addr1, Attributes.EMPTY); + final ResolvedServerInfo server2 = new ResolvedServerInfo(addr2, Attributes.EMPTY); + final ConnectionClientTransport transport1 = mock(ConnectionClientTransport.class); + final ConnectionClientTransport transport2 = mock(ConnectionClientTransport.class); + when(mockTransportFactory.newClientTransport(same(addr1), any(String.class), any(String.class))) + .thenReturn(transport1); + when(mockTransportFactory.newClientTransport(same(addr2), any(String.class), any(String.class))) + .thenReturn(transport2); + + FakeNameResolverFactory nameResolverFactory = + new FakeNameResolverFactory(Arrays.asList(server1, server2)); + createChannel(nameResolverFactory, NO_INTERCEPTOR); + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + Metadata headers = new Metadata(); + + // Start a call. The channel will starts with the first address, which will fail to connect. + call.start(mockCallListener, headers); + timer.runDueTasks(); + executor.runDueTasks(); + + verify(transport1).start(transportListenerCaptor.capture()); + verify(mockTransportFactory) + .newClientTransport(same(addr1), any(String.class), any(String.class)); + verify(mockTransportFactory, times(0)) + .newClientTransport(same(addr2), any(String.class), any(String.class)); + transportListenerCaptor.getValue().transportShutdown(Status.UNAVAILABLE); + + // The channel then try the second address, which will fail to connect too. + verify(transport2).start(transportListenerCaptor.capture()); + verify(mockTransportFactory) + .newClientTransport(same(addr2), any(String.class), any(String.class)); + verify(transport2).start(transportListenerCaptor.capture()); + transportListenerCaptor.getValue().transportShutdown(Status.UNAVAILABLE); + executor.runDueTasks(); + + // Call fails + verify(mockCallListener).onClose(statusCaptor.capture(), any(Metadata.class)); + assertEquals(Status.Code.UNAVAILABLE, statusCaptor.getValue().getCode()); + // No real stream was ever created + verify(transport1, times(0)).newStream(any(MethodDescriptor.class), any(Metadata.class)); + verify(transport2, times(0)).newStream(any(MethodDescriptor.class), any(Metadata.class)); + } + + /** + * Verify that if the first resolved address points to a server that is at first connected, but + * disconnected later, all calls will stick to the first address. + */ + @Test + public void firstResolvedServerConnectedThenDisconnected() throws Exception { + final SocketAddress addr1 = new SocketAddress() { + @Override public String toString() { + return "addr1"; + } + }; + final SocketAddress addr2 = new SocketAddress() { + @Override public String toString() { + return "addr2"; + } + }; + final ResolvedServerInfo server1 = new ResolvedServerInfo(addr1, Attributes.EMPTY); + final ResolvedServerInfo server2 = new ResolvedServerInfo(addr2, Attributes.EMPTY); + // Addr1 will have two transports throughout this test. + final ConnectionClientTransport transport1 = mock(ConnectionClientTransport.class); + final ConnectionClientTransport transport2 = mock(ConnectionClientTransport.class); + when(transport1.newStream( + any(MethodDescriptor.class), any(Metadata.class), any(CallOptions.class), + any(StatsTraceContext.class))) + .thenReturn(mock(ClientStream.class)); + when(transport2.newStream( + any(MethodDescriptor.class), any(Metadata.class), any(CallOptions.class), + any(StatsTraceContext.class))) + .thenReturn(mock(ClientStream.class)); + when(mockTransportFactory.newClientTransport(same(addr1), any(String.class), any(String.class))) + .thenReturn(transport1, transport2); + + FakeNameResolverFactory nameResolverFactory = + new FakeNameResolverFactory(Arrays.asList(server1, server2)); + createChannel(nameResolverFactory, NO_INTERCEPTOR); + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + Metadata headers = new Metadata(); + + // First call will use the first address + call.start(mockCallListener, headers); + timer.runDueTasks(); + executor.runDueTasks(); + + verify(mockTransportFactory) + .newClientTransport(same(addr1), any(String.class), any(String.class)); + verify(transport1).start(transportListenerCaptor.capture()); + transportListenerCaptor.getValue().transportReady(); + executor.runDueTasks(); + + verify(transport1).newStream(same(method), same(headers), same(CallOptions.DEFAULT), + any(StatsTraceContext.class)); + transportListenerCaptor.getValue().transportShutdown(Status.UNAVAILABLE); + + // Second call still use the first address, since it was successfully connected. + ClientCall call2 = channel.newCall(method, CallOptions.DEFAULT); + call2.start(mockCallListener, headers); + verify(transport2).start(transportListenerCaptor.capture()); + verify(mockTransportFactory, times(2)) + .newClientTransport(same(addr1), any(String.class), any(String.class)); + transportListenerCaptor.getValue().transportReady(); + executor.runDueTasks(); + + verify(transport2).newStream(same(method), same(headers), same(CallOptions.DEFAULT), + any(StatsTraceContext.class)); + } + + @Test + public void uriPattern() { + assertTrue(ManagedChannelImpl.URI_PATTERN.matcher("a:/").matches()); + assertTrue(ManagedChannelImpl.URI_PATTERN.matcher("Z019+-.:/!@ #~ ").matches()); + assertFalse(ManagedChannelImpl.URI_PATTERN.matcher("a/:").matches()); // "/:" not matched + assertFalse(ManagedChannelImpl.URI_PATTERN.matcher("0a:/").matches()); // '0' not matched + assertFalse(ManagedChannelImpl.URI_PATTERN.matcher("a,:/").matches()); // ',' not matched + assertFalse(ManagedChannelImpl.URI_PATTERN.matcher(" a:/").matches()); // space not matched + } + + /** + * Test that information such as the Call's context, MethodDescriptor, authority, executor are + * propagated to newStream() and applyRequestMetadata(). + */ + @Test + public void informationPropagatedToNewStreamAndCallCredentials() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + CallOptions callOptions = CallOptions.DEFAULT.withCallCredentials(creds); + final Context.Key testKey = Context.key("testing"); + Context ctx = Context.current().withValue(testKey, "testValue"); + final LinkedList credsApplyContexts = new LinkedList(); + final LinkedList newStreamContexts = new LinkedList(); + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock in) throws Throwable { + credsApplyContexts.add(Context.current()); + return null; + } + }).when(creds).applyRequestMetadata( + any(MethodDescriptor.class), any(Attributes.class), any(Executor.class), + any(MetadataApplier.class)); + + final ConnectionClientTransport transport = mock(ConnectionClientTransport.class); + when(transport.getAttrs()).thenReturn(Attributes.EMPTY); + when(mockTransportFactory.newClientTransport(any(SocketAddress.class), any(String.class), + any(String.class))).thenReturn(transport); + doAnswer(new Answer() { + @Override + public ClientStream answer(InvocationOnMock in) throws Throwable { + newStreamContexts.add(Context.current()); + return mock(ClientStream.class); + } + }).when(transport).newStream( + any(MethodDescriptor.class), any(Metadata.class), any(CallOptions.class), + any(StatsTraceContext.class)); + + // First call will be on delayed transport. Only newCall() is run within the expected context, + // so that we can verify that the context is explicitly attached before calling newStream() and + // applyRequestMetadata(), which happens after we detach the context from the thread. + Context origCtx = ctx.attach(); + assertEquals("testValue", testKey.get()); + ClientCall call = channel.newCall(method, callOptions); + ctx.detach(origCtx); + assertNull(testKey.get()); + call.start(mockCallListener, new Metadata()); + + ArgumentCaptor transportListenerCaptor = + ArgumentCaptor.forClass(ManagedClientTransport.Listener.class); + verify(mockTransportFactory).newClientTransport( + same(socketAddress), eq(authority), eq(userAgent)); + verify(transport).start(transportListenerCaptor.capture()); + verify(creds, never()).applyRequestMetadata( + any(MethodDescriptor.class), any(Attributes.class), any(Executor.class), + any(MetadataApplier.class)); + + // applyRequestMetadata() is called after the transport becomes ready. + transportListenerCaptor.getValue().transportReady(); + executor.runDueTasks(); + ArgumentCaptor attrsCaptor = ArgumentCaptor.forClass(Attributes.class); + ArgumentCaptor applierCaptor = ArgumentCaptor.forClass(MetadataApplier.class); + verify(creds).applyRequestMetadata(same(method), attrsCaptor.capture(), + same(executor.getScheduledExecutorService()), applierCaptor.capture()); + assertEquals("testValue", testKey.get(credsApplyContexts.poll())); + assertEquals(authority, attrsCaptor.getValue().get(CallCredentials.ATTR_AUTHORITY)); + assertEquals(SecurityLevel.NONE, + attrsCaptor.getValue().get(CallCredentials.ATTR_SECURITY_LEVEL)); + verify(transport, never()).newStream( + any(MethodDescriptor.class), any(Metadata.class), any(CallOptions.class), + any(StatsTraceContext.class)); + + // newStream() is called after apply() is called + applierCaptor.getValue().apply(new Metadata()); + verify(transport).newStream(same(method), any(Metadata.class), same(callOptions), + any(StatsTraceContext.class)); + assertEquals("testValue", testKey.get(newStreamContexts.poll())); + // The context should not live beyond the scope of newStream() and applyRequestMetadata() + assertNull(testKey.get()); + + + // Second call will not be on delayed transport + origCtx = ctx.attach(); + call = channel.newCall(method, callOptions); + ctx.detach(origCtx); + call.start(mockCallListener, new Metadata()); + + verify(creds, times(2)).applyRequestMetadata(same(method), attrsCaptor.capture(), + same(executor.getScheduledExecutorService()), applierCaptor.capture()); + assertEquals("testValue", testKey.get(credsApplyContexts.poll())); + assertEquals(authority, attrsCaptor.getValue().get(CallCredentials.ATTR_AUTHORITY)); + assertEquals(SecurityLevel.NONE, + attrsCaptor.getValue().get(CallCredentials.ATTR_SECURITY_LEVEL)); + // This is from the first call + verify(transport).newStream( + any(MethodDescriptor.class), any(Metadata.class), any(CallOptions.class), + any(StatsTraceContext.class)); + + // Still, newStream() is called after apply() is called + applierCaptor.getValue().apply(new Metadata()); + verify(transport, times(2)).newStream(same(method), any(Metadata.class), same(callOptions), + any(StatsTraceContext.class)); + assertEquals("testValue", testKey.get(newStreamContexts.poll())); + + assertNull(testKey.get()); + } + + private static class FakeBackoffPolicyProvider implements BackoffPolicy.Provider { + @Override + public BackoffPolicy get() { + return new BackoffPolicy() { + @Override + public long nextBackoffMillis() { + return 1; + } + }; + } + } + + private class FakeNameResolverFactory extends NameResolver.Factory { + final List servers; + final boolean resolvedAtStart; + final ArrayList resolvers = new ArrayList(); + + FakeNameResolverFactory(boolean resolvedAtStart) { + this.resolvedAtStart = resolvedAtStart; + servers = Collections.singletonList(ResolvedServerInfoGroup.builder().add(server).build()); + } + + FakeNameResolverFactory(List servers) { + resolvedAtStart = true; + this.servers = Collections.singletonList( + ResolvedServerInfoGroup.builder().addAll(servers).build()); + } + + public FakeNameResolverFactory() { + resolvedAtStart = true; + this.servers = ImmutableList.of(); + } + + @Override + public NameResolver newNameResolver(final URI targetUri, Attributes params) { + if (!expectedUri.equals(targetUri)) { + return null; + } + assertSame(NAME_RESOLVER_PARAMS, params); + FakeNameResolver resolver = new FakeNameResolver(); + resolvers.add(resolver); + return resolver; + } + + @Override + public String getDefaultScheme() { + return "fake"; + } + + void allResolved() { + for (FakeNameResolver resolver : resolvers) { + resolver.resolved(); + } + } + + private class FakeNameResolver extends NameResolver { + Listener listener; + boolean shutdown; + + @Override public String getServiceAuthority() { + return expectedUri.getAuthority(); + } + + @Override public void start(final Listener listener) { + this.listener = listener; + if (resolvedAtStart) { + resolved(); + } + } + + void resolved() { + listener.onUpdate(servers, Attributes.EMPTY); + } + + @Override public void shutdown() { + shutdown = true; + } + } + } + + private static class FailingNameResolverFactory extends NameResolver.Factory { + final Status error; + + FailingNameResolverFactory(Status error) { + this.error = error; + } + + @Override + public NameResolver newNameResolver(URI notUsedUri, Attributes params) { + return new NameResolver() { + @Override public String getServiceAuthority() { + return "irrelevant-authority"; + } + + @Override public void start(final Listener listener) { + listener.onError(error); + } + + @Override public void shutdown() {} + }; + } + + @Override + public String getDefaultScheme() { + return "fake"; + } + } + + private static class SpyingLoadBalancerFactory extends LoadBalancer.Factory { + private final LoadBalancer.Factory delegate; + private final List> balancers = new ArrayList>(); + + private SpyingLoadBalancerFactory(LoadBalancer.Factory delegate) { + this.delegate = delegate; + } + + @Override + public LoadBalancer newLoadBalancer(String serviceName, TransportManager tm) { + LoadBalancer lb = spy(delegate.newLoadBalancer(serviceName, tm)); + balancers.add(lb); + return lb; + } + } +} From 850aefccc58c63ee00b98ed94ec024254cc3e2a4 Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Fri, 16 Dec 2016 15:52:48 -0800 Subject: [PATCH 02/14] core: ManagedChannelImpl2. 1. Adapt to LoadBalancer2 interface. Channel holds on to a single DelayedClientTransport2. 2. Lock-free: every channel state mutation, including Subchannel mutations, calling into LoadBalancer, idleness and shutdown, is made from channelExecutor. 3. Idleness grace period is no longer needed. --- core/src/main/java/io/grpc/LoadBalancer2.java | 7 +- .../io/grpc/internal/ChannelExecutor.java | 4 +- .../java/io/grpc/internal/ClientCallImpl.java | 4 +- .../io/grpc/internal/ManagedChannelImpl.java | 3 +- .../io/grpc/internal/ManagedChannelImpl2.java | 900 +++++++++--------- .../java/io/grpc/internal/OobChannel.java | 244 +++++ .../grpc/internal/SingleTransportChannel.java | 3 +- .../java/io/grpc/internal/TransportSet.java | 3 +- .../io/grpc/internal/ClientCallImplTest.java | 2 +- .../ManagedChannelImpl2IdlenessTest.java | 302 +++--- .../internal/ManagedChannelImpl2Test.java | 892 +++++++++-------- 11 files changed, 1246 insertions(+), 1118 deletions(-) create mode 100644 core/src/main/java/io/grpc/internal/OobChannel.java diff --git a/core/src/main/java/io/grpc/LoadBalancer2.java b/core/src/main/java/io/grpc/LoadBalancer2.java index 2945efbb097..204a3df8cd5 100644 --- a/core/src/main/java/io/grpc/LoadBalancer2.java +++ b/core/src/main/java/io/grpc/LoadBalancer2.java @@ -34,6 +34,7 @@ import com.google.common.base.Preconditions; import java.util.List; +import java.util.concurrent.Executor; import javax.annotation.Nullable; import javax.annotation.concurrent.Immutable; import javax.annotation.concurrent.NotThreadSafe; @@ -337,9 +338,13 @@ public abstract static class Helper { * *

The LoadBalancer is responsible for closing unused OOB channels, and closing all OOB * channels within {@link #shutdown}. + * + * @param eag the address(es) of the channel + * @param authority the authority of the destination this channel connects to + * @param executor the default executor for running RPC callbacks */ public abstract ManagedChannel createOobChannel( - EquivalentAddressGroup eag, String authority); + EquivalentAddressGroup eag, String authority, Executor executor); /** * Set a new picker to the channel. diff --git a/core/src/main/java/io/grpc/internal/ChannelExecutor.java b/core/src/main/java/io/grpc/internal/ChannelExecutor.java index 5de3b56f8cc..1c92f564618 100644 --- a/core/src/main/java/io/grpc/internal/ChannelExecutor.java +++ b/core/src/main/java/io/grpc/internal/ChannelExecutor.java @@ -31,6 +31,8 @@ package io.grpc.internal; +import static com.google.common.base.Preconditions.checkNotNull; + import com.google.common.annotations.VisibleForTesting; import java.util.LinkedList; @@ -97,7 +99,7 @@ void drain() { */ ChannelExecutor executeLater(Runnable runnable) { synchronized (lock) { - queue.add(runnable); + queue.add(checkNotNull(runnable, "runnable is null")); } return this; } diff --git a/core/src/main/java/io/grpc/internal/ClientCallImpl.java b/core/src/main/java/io/grpc/internal/ClientCallImpl.java index 5a964614e33..ac80a2e589f 100644 --- a/core/src/main/java/io/grpc/internal/ClientCallImpl.java +++ b/core/src/main/java/io/grpc/internal/ClientCallImpl.java @@ -127,7 +127,7 @@ interface ClientTransportProvider { /** * Returns a transport for a new call. */ - ClientTransport get(CallOptions callOptions); + ClientTransport get(CallOptions callOptions, Metadata headers); } ClientCallImpl setDecompressorRegistry(DecompressorRegistry decompressorRegistry) { @@ -215,7 +215,7 @@ public void runInContext() { if (!deadlineExceeded) { updateTimeoutHeaders(effectiveDeadline, callOptions.getDeadline(), context.getDeadline(), headers); - ClientTransport transport = clientTransportProvider.get(callOptions); + ClientTransport transport = clientTransportProvider.get(callOptions, headers); Context origContext = context.attach(); try { stream = transport.newStream(method, headers, callOptions, statsTraceCtx); diff --git a/core/src/main/java/io/grpc/internal/ManagedChannelImpl.java b/core/src/main/java/io/grpc/internal/ManagedChannelImpl.java index ee416482c5c..b05277a463a 100644 --- a/core/src/main/java/io/grpc/internal/ManagedChannelImpl.java +++ b/core/src/main/java/io/grpc/internal/ManagedChannelImpl.java @@ -51,6 +51,7 @@ import io.grpc.EquivalentAddressGroup; import io.grpc.LoadBalancer; import io.grpc.ManagedChannel; +import io.grpc.Metadata; import io.grpc.MethodDescriptor; import io.grpc.NameResolver; import io.grpc.ResolvedServerInfoGroup; @@ -359,7 +360,7 @@ private void rescheduleIdleTimer() { private final ClientTransportProvider transportProvider = new ClientTransportProvider() { @Override - public ClientTransport get(CallOptions callOptions) { + public ClientTransport get(CallOptions callOptions, Metadata headers) { LoadBalancer balancer = loadBalancer; if (balancer == null) { // Current state is either idle or in grace period diff --git a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java index ee416482c5c..4ca3ce15449 100644 --- a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java +++ b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java @@ -1,5 +1,5 @@ /* - * Copyright 2014, Google Inc. All rights reserved. + * 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 @@ -34,6 +34,8 @@ 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 io.grpc.ConnectivityState.IDLE; +import static io.grpc.ConnectivityState.TRANSIENT_FAILURE; import com.google.census.CensusContextFactory; import com.google.common.annotations.VisibleForTesting; @@ -47,32 +49,32 @@ import io.grpc.ClientInterceptor; import io.grpc.ClientInterceptors; import io.grpc.CompressorRegistry; +import io.grpc.ConnectivityStateInfo; import io.grpc.DecompressorRegistry; import io.grpc.EquivalentAddressGroup; -import io.grpc.LoadBalancer; +import io.grpc.LoadBalancer2.PickResult; +import io.grpc.LoadBalancer2.SubchannelPicker; +import io.grpc.LoadBalancer2; import io.grpc.ManagedChannel; +import io.grpc.Metadata; import io.grpc.MethodDescriptor; import io.grpc.NameResolver; import io.grpc.ResolvedServerInfoGroup; import io.grpc.Status; -import io.grpc.TransportManager; -import io.grpc.TransportManager.InterimTransport; -import io.grpc.TransportManager.OobTransportProvider; import io.grpc.internal.ClientCallImpl.ClientTransportProvider; import java.net.URI; import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.Collection; import java.util.HashSet; import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; +import java.util.Set; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.logging.Level; import java.util.logging.Logger; import java.util.regex.Pattern; @@ -83,8 +85,8 @@ /** A communication channel for making outgoing RPCs. */ @ThreadSafe -public final class ManagedChannelImpl extends ManagedChannel implements WithLogId { - private static final Logger log = Logger.getLogger(ManagedChannelImpl.class.getName()); +public final class ManagedChannelImpl2 extends ManagedChannel implements WithLogId { + private static final Logger log = Logger.getLogger(ManagedChannelImpl2.class.getName()); // Matching this pattern means the target string is a URI target or at least intended to be one. // A URI target must be an absolute hierarchical URI. @@ -94,57 +96,41 @@ public final class ManagedChannelImpl extends ManagedChannel implements WithLogI static final long IDLE_TIMEOUT_MILLIS_DISABLE = -1; - /** - * The time after idleTimeoutMillis expires before idleness takes effect. The time before - * idleTimeoutMillis expires is part of a fast path for acquiring the load balancer. After - * idleTimeoutMillis expires a slow path takes effect with extra synchronization. - * - *

Transports having open streams prevents entering idle mode. However, this creates an - * inherent race between acquiring a transport, which can't be done in idle mode, and the RPC - * actually being created on that transport, which inhibits idle mode. Thus we reset the idle - * timer when acquiring a transport, and impose a minimum idle time (IDLE_MODE_MIN_TIMEOUT_MILLIS) - * to make the chances of racing very small. If we do race, then the RPC will spuriously fail - * because the transport chosen was shut down. - * - *

For heavy users, resetting the idle timer each RPC becomes highly contended. We instead only - * need to reset the timer when it is close to expiring. We do the equivalent by having two - * periods: a reduced regular idle time period and the extra time as a grace period. We ignore the - * race during the regular idle time period, but any acquisition during the grace period must - * reset the timer. - */ @VisibleForTesting - static final long IDLE_GRACE_PERIOD_MILLIS = TimeUnit.SECONDS.toMillis(1); + static final long SUBCHANNEL_SHUTDOWN_DELAY_SECONDS = 5; private static final ClientTransport SHUTDOWN_TRANSPORT = new FailingClientTransport(Status.UNAVAILABLE.withDescription("Channel is shutdown")); @VisibleForTesting - static final ClientTransport IDLE_MODE_TRANSPORT = - new FailingClientTransport(Status.INTERNAL.withDescription("Channel is in idle mode")); + static final Status SHUTDOWN_NOW_STATUS = + Status.UNAVAILABLE.withDescription("Channel shutdownNow invoked"); private final String target; private final NameResolver.Factory nameResolverFactory; private final Attributes nameResolverParams; - private final LoadBalancer.Factory loadBalancerFactory; + private final LoadBalancer2.Factory loadBalancerFactory; private final ClientTransportFactory transportFactory; private final Executor executor; private final boolean usingSharedExecutor; - private final Object lock = new Object(); private final LogId logId = LogId.allocate(getClass().getName()); + private final ChannelExecutor channelExecutor = new ChannelExecutor(); + private final DecompressorRegistry decompressorRegistry; private final CompressorRegistry compressorRegistry; private final SharedResourceHolder.Resource timerService; private final Supplier stopwatchSupplier; - /** The timout before entering idle mode, less {@link #IDLE_GRACE_PERIOD_MILLIS}. */ + /** The timout before entering idle mode. */ private final long idleTimeoutMillis; private final CensusContextFactory censusFactory; /** * Executor that runs deadline timers for requests. */ - private ScheduledExecutorService scheduledExecutor; + // Must be assigned from channelExecutor + private volatile ScheduledExecutorService scheduledExecutor; private final BackoffPolicy.Provider backoffPolicyProvider; @@ -155,177 +141,187 @@ public final class ManagedChannelImpl extends ManagedChannel implements WithLogI private final Channel interceptorChannel; @Nullable private final String userAgent; - // Never be null. Must be modified under lock. + // Only null after channel is terminated. Must be assigned from the channelExecutor. private NameResolver nameResolver; - /** {@code null} when idle or when in grace idle period. "lock" must be held when modifying. */ + // null when channel is in idle mode. Must be assigned from channelExecutor. @Nullable - private volatile LoadBalancer loadBalancer; + private LoadBalancer2 loadBalancer; - /** non-{code null} iff channel is in grace idle period. */ - @GuardedBy("lock") + // Must be assigned from channelExecutor. null if channel is in idle mode. @Nullable - private LoadBalancer graceLoadBalancer; + private volatile SubchannelPicker subchannelPicker; - /** - * Maps EquivalentAddressGroups to transports for that server. "lock" must be held when mutating. - */ - // Even though we set a concurrency level of 1, this is better than Collections.synchronizedMap - // because it doesn't need to acquire a lock for reads. - private final ConcurrentMap transports = - new ConcurrentHashMap(16, .75f, 1); + // Must be mutated from channelExecutor + // If any monitoring hook to be added later needs to get a snapshot of this Set, we could + // switch to a ConcurrentHashMap. + private final Set subchannels = new HashSet(16, .75f); - /** - * TransportSets that are shutdown (but not yet terminated) due to channel idleness or channel - * shut down. - */ - @GuardedBy("lock") - private final HashSet decommissionedTransports = new HashSet(); + // Must be mutated from channelExecutor + private final Set oobChannels = new HashSet(1, .75f); - @GuardedBy("lock") - private final HashSet delayedTransports = - new HashSet(); + // reprocess() must be run from channelExecutor + private final DelayedClientTransport2 delayedTransport; + + // Shutdown states. + // + // Channel's shutdown process: + // 1. shutdown(): stop accepting new calls from applications + // 1a shutdown <- true + // 1b subchannelPicker <- null + // 1c delayedTransport.shutdown() + // 2. delayedTransport terminated: stop stream-creation functionality + // 2a terminating <- true + // 2b loadBalancer <- null + // 2c nameResolver <- null + // 2d loadBalancer.shutdown() + // * LoadBalancer will shutdown subchannels and OOB channels + // 2e nameResolver.shutdown() + // 3. All subchannels and OOB channels terminated: Channel considered terminated + + private final AtomicBoolean shutdown = new AtomicBoolean(false); + // Must be mutated from channelExecutor + private boolean shutdownNowed; + // Must be mutated from channelExecutor + private volatile boolean terminating; + // Must be mutated from channelExecutor + private volatile boolean terminated; + private final CountDownLatch terminatedLatch = new CountDownLatch(1); + + // Called from channelExecutor + private final ManagedClientTransport.Listener delayedTransportListener = + new ManagedClientTransport.Listener() { + @Override + public void transportShutdown(Status s) { + checkState(shutdown.get(), "Channel must have been shut down"); + } + + @Override + public void transportReady() { + // Don't care + } - @VisibleForTesting - final InUseStateAggregator inUseStateAggregator = - new InUseStateAggregator() { @Override - Object getLock() { - return lock; + public void transportInUse(final boolean inUse) { + inUseStateAggregator.updateObjectInUse(delayedTransport, inUse); } @Override - @GuardedBy("lock") + public void transportTerminated() { + checkState(shutdown.get(), "Channel must have been shut down"); + terminating = true; + if (loadBalancer != null) { + loadBalancer.shutdown(); + loadBalancer = null; + } + if (nameResolver != null) { + nameResolver.shutdown(); + nameResolver = null; + } + + // Until LoadBalancer is shutdown, it may still create new subchannels. We catch them + // here. + maybeShutdownNowSubchannels(); + maybeTerminateChannel(); + } + }; + + // Must be called from channelExecutor + private void maybeShutdownNowSubchannels() { + Status nowStatus = Status.UNAVAILABLE.withDescription("Channel shutdownNow invoked"); + if (shutdownNowed) { + for (InternalSubchannel subchannel : subchannels) { + subchannel.shutdownNow(SHUTDOWN_NOW_STATUS); + } + subchannels.clear(); + for (InternalSubchannel oobChannel : oobChannels) { + oobChannel.shutdownNow(SHUTDOWN_NOW_STATUS); + } + oobChannels.clear(); + } + } + + // Must be accessed from channelExecutor + @VisibleForTesting + final InUseStateAggregator2 inUseStateAggregator = + new InUseStateAggregator2() { + @Override void handleInUse() { exitIdleMode(); } - @GuardedBy("lock") @Override void handleNotInUse() { - if (shutdown) { + if (shutdown.get()) { return; } rescheduleIdleTimer(); } }; + // Run from channelExecutor private class IdleModeTimer implements Runnable { - @GuardedBy("lock") + // Only mutated from channelExecutor boolean cancelled; @Override public void run() { - ArrayList transportsCopy = new ArrayList(); - LoadBalancer savedBalancer; - NameResolver oldResolver; - synchronized (lock) { - if (cancelled) { - // Race detected: this task started before cancelIdleTimer() could cancel it. - return; - } - if (loadBalancer != null) { - // Enter grace period. - graceLoadBalancer = loadBalancer; - loadBalancer = null; - assert idleModeTimer == this; - idleModeTimerFuture = scheduledExecutor.schedule(new LogExceptionRunnable(idleModeTimer), - IDLE_GRACE_PERIOD_MILLIS, TimeUnit.MILLISECONDS); - return; - } - // Enter idle mode - savedBalancer = graceLoadBalancer; - graceLoadBalancer = null; - oldResolver = nameResolver; - nameResolver = getNameResolver(target, nameResolverFactory, nameResolverParams); - transportsCopy.addAll(transports.values()); - transports.clear(); - decommissionedTransports.addAll(transportsCopy); - } - for (TransportSet ts : transportsCopy) { - ts.shutdown(); + if (cancelled) { + // Race detected: this task was scheduled on channelExecutor before cancelIdleTimer() + // could cancel the timer. + return; } - savedBalancer.shutdown(); - oldResolver.shutdown(); + log.log(Level.FINE, "[{0}] Entering idle mode", getLogId()); + nameResolver.shutdown(); + nameResolver = getNameResolver(target, nameResolverFactory, nameResolverParams); + loadBalancer.shutdown(); + loadBalancer = null; + subchannelPicker = null; } } - @GuardedBy("lock") + // Must be used from channelExecutor @Nullable private ScheduledFuture idleModeTimerFuture; - @GuardedBy("lock") + // Must be used from channelExecutor @Nullable private IdleModeTimer idleModeTimer; /** - * Make the channel exit idle mode, if it's in it. Return a LoadBalancer that can be used for - * making new requests. Return null if the channel is shutdown. + * Make the channel exit idle mode, if it's in it. * - *

May be called under the lock. + *

Must be called from channelExecutor */ @VisibleForTesting - LoadBalancer exitIdleMode() { - final LoadBalancer balancer; - final NameResolver resolver; - synchronized (lock) { - if (shutdown) { - return null; - } - if (inUseStateAggregator.isInUse()) { - cancelIdleTimer(); - } else { - // exitIdleMode() may be called outside of inUseStateAggregator, which may still in - // "not-in-use" state. If it's the case, we start the timer which will be soon cancelled if - // the aggregator receives actual uses. - rescheduleIdleTimer(); - } - if (graceLoadBalancer != null) { - // Exit grace period; timer already rescheduled above. - loadBalancer = graceLoadBalancer; - graceLoadBalancer = null; - } - if (loadBalancer != null) { - return loadBalancer; - } - balancer = loadBalancerFactory.newLoadBalancer(nameResolver.getServiceAuthority(), tm); - this.loadBalancer = balancer; - resolver = this.nameResolver; - } - class NameResolverStartTask implements Runnable { - @Override - public void run() { - NameResolverListenerImpl listener = new NameResolverListenerImpl(balancer); - // This may trigger quite a few non-trivial work in LoadBalancer and NameResolver, - // we don't want to do it in the lock. - try { - resolver.start(listener); - } catch (Throwable t) { - listener.onError(Status.fromThrowable(t)); - } - } + void exitIdleMode() { + if (shutdown.get()) { + return; } + // Cancel the timer now, so that a racing due timer will not put Channel on idleness + // when the caller of exitIdleMode() is about to use the returned loadBalancer. + cancelIdleTimer(); + // exitIdleMode() may be called outside of inUseStateAggregator.handleNotInUse() while + // isInUse() == false, in which case we still need to schedule the timer. + if (!inUseStateAggregator.isInUse()) { + rescheduleIdleTimer(); + } + if (loadBalancer != null) { + return; + } + log.log(Level.FINE, "[{0}] Exiting idle mode", getLogId()); + LbHelperImpl helper = new LbHelperImpl(nameResolver); + helper.lb = loadBalancerFactory.newLoadBalancer(helper); + this.loadBalancer = helper.lb; - scheduledExecutor.execute(new NameResolverStartTask()); - return balancer; - } - - @VisibleForTesting - boolean isInIdleGracePeriod() { - synchronized (lock) { - return graceLoadBalancer != null; + NameResolverListenerImpl listener = new NameResolverListenerImpl(helper.lb); + try { + nameResolver.start(listener); + } catch (Throwable t) { + listener.onError(Status.fromThrowable(t)); } } - // ErrorProne's GuardedByChecker can't figure out that the idleModeTimer is a nested instance of - // this particular instance. It is worried about something like: - // ManagedChannelImpl a = ...; - // ManagedChannelImpl b = ...; - // a.idleModeTimer = b.idleModeTimer; - // a.cancelIdleTimer(); // access of b.idleModeTimer is guarded by a.lock, not b.lock - // - // _We_ know that isn't happening, so we suppress the warning. - @SuppressWarnings("GuardedByChecker") - @GuardedBy("lock") + // Must be run from channelExecutor private void cancelIdleTimer() { if (idleModeTimerFuture != null) { idleModeTimerFuture.cancel(false); @@ -335,46 +331,64 @@ private void cancelIdleTimer() { } } - @GuardedBy("lock") + // Always run from channelExecutor private void rescheduleIdleTimer() { if (idleTimeoutMillis == IDLE_TIMEOUT_MILLIS_DISABLE) { return; } cancelIdleTimer(); idleModeTimer = new IdleModeTimer(); - idleModeTimerFuture = scheduledExecutor.schedule(new LogExceptionRunnable(idleModeTimer), + idleModeTimerFuture = scheduledExecutor.schedule( + new LogExceptionRunnable(new Runnable() { + @Override + public void run() { + channelExecutor.executeLater(idleModeTimer).drain(); + } + }), idleTimeoutMillis, TimeUnit.MILLISECONDS); } - @GuardedBy("lock") - private final HashSet oobTransports = - new HashSet(); - - @GuardedBy("lock") - private boolean shutdown; - @GuardedBy("lock") - private boolean shutdownNowed; - @GuardedBy("lock") - private boolean terminated; - private final ClientTransportProvider transportProvider = new ClientTransportProvider() { @Override - public ClientTransport get(CallOptions callOptions) { - LoadBalancer balancer = loadBalancer; - if (balancer == null) { - // Current state is either idle or in grace period - balancer = exitIdleMode(); + public ClientTransport get(CallOptions callOptions, Metadata headers) { + SubchannelPicker pickerCopy = subchannelPicker; + if (shutdown.get()) { + // If channel is shut down, delayedTransport is also shut down which will fail the stream + // properly. + return delayedTransport; + } + if (pickerCopy == null) { + channelExecutor.executeLater(new Runnable() { + @Override + public void run() { + exitIdleMode(); + } + }).drain(); + return delayedTransport; } - if (balancer == null) { - return SHUTDOWN_TRANSPORT; + // There is no need to reschedule the idle timer here. + // + // pickerCopy != null, which means idle timer has not expired when this method starts. + // Even if idle timer expires right after we grab pickerCopy, and it shuts down LoadBalancer + // which calls Subchannel.shutdown(), the InternalSubchannel will be actually shutdown after + // SUBCHANNEL_SHUTDOWN_DELAY_SECONDS, which gives the caller time to start RPC on it. + // + // In most cases the idle timer is scheduled to fire after the transport has created the + // stream, which would have reported in-use state to the channel that would have cancelled + // the idle timer. + PickResult pickResult = pickerCopy.pickSubchannel(callOptions.getAffinity(), headers); + ClientTransport transport = GrpcUtil.getTransportFromPickResult( + pickResult, callOptions.isWaitForReady()); + if (transport != null) { + return transport; } - return balancer.pickTransport(callOptions.getAffinity()); + return delayedTransport; } }; - ManagedChannelImpl(String target, BackoffPolicy.Provider backoffPolicyProvider, + ManagedChannelImpl2(String target, BackoffPolicy.Provider backoffPolicyProvider, NameResolver.Factory nameResolverFactory, Attributes nameResolverParams, - LoadBalancer.Factory loadBalancerFactory, ClientTransportFactory transportFactory, + LoadBalancer2.Factory loadBalancerFactory, ClientTransportFactory transportFactory, DecompressorRegistry decompressorRegistry, CompressorRegistry compressorRegistry, SharedResourceHolder.Resource timerService, Supplier stopwatchSupplier, long idleTimeoutMillis, @@ -392,6 +406,8 @@ public ClientTransport get(CallOptions callOptions) { usingSharedExecutor = false; this.executor = executor; } + this.delayedTransport = new DelayedClientTransport2(this.executor, this.channelExecutor); + this.delayedTransport.start(delayedTransportListener); this.backoffPolicyProvider = backoffPolicyProvider; this.transportFactory = new CallCredentialsApplyingTransportFactory(transportFactory, this.executor); @@ -402,20 +418,17 @@ public ClientTransport get(CallOptions callOptions) { if (idleTimeoutMillis == IDLE_TIMEOUT_MILLIS_DISABLE) { this.idleTimeoutMillis = idleTimeoutMillis; } else { - assert IDLE_GRACE_PERIOD_MILLIS - <= AbstractManagedChannelImplBuilder.IDLE_MODE_MIN_TIMEOUT_MILLIS; - checkArgument(idleTimeoutMillis >= IDLE_GRACE_PERIOD_MILLIS, + checkArgument( + idleTimeoutMillis >= AbstractManagedChannelImplBuilder.IDLE_MODE_MIN_TIMEOUT_MILLIS, "invalid idleTimeoutMillis %s", idleTimeoutMillis); - this.idleTimeoutMillis = idleTimeoutMillis - IDLE_GRACE_PERIOD_MILLIS; + this.idleTimeoutMillis = idleTimeoutMillis; } this.decompressorRegistry = decompressorRegistry; this.compressorRegistry = compressorRegistry; this.userAgent = userAgent; this.censusFactory = checkNotNull(censusFactory, "censusFactory"); - if (log.isLoggable(Level.INFO)) { - log.log(Level.INFO, "[{0}] Created with target {1}", new Object[] {getLogId(), target}); - } + log.log(Level.FINE, "[{0}] Created with target {1}", new Object[] {getLogId(), target}); } @VisibleForTesting @@ -470,49 +483,20 @@ static NameResolver getNameResolver(String target, NameResolver.Factory nameReso * cancelled. */ @Override - public ManagedChannelImpl shutdown() { - ArrayList transportsCopy = new ArrayList(); - ArrayList delayedTransportsCopy = - new ArrayList(); - ArrayList oobTransportsCopy = - new ArrayList(); - LoadBalancer balancer; - NameResolver resolver; - synchronized (lock) { - if (shutdown) { - return this; - } - shutdown = true; - // After shutdown there are no new calls, so no new cancellation tasks are needed - scheduledExecutor = SharedResourceHolder.release(timerService, scheduledExecutor); - maybeTerminateChannel(); - if (!terminated) { - transportsCopy.addAll(transports.values()); - transports.clear(); - decommissionedTransports.addAll(transportsCopy); - delayedTransportsCopy.addAll(delayedTransports); - oobTransportsCopy.addAll(oobTransports); - } - balancer = getCurrentLoadBalancer(); - resolver = nameResolver; - cancelIdleTimer(); - } - if (balancer != null) { - balancer.shutdown(); - } - resolver.shutdown(); - for (TransportSet ts : transportsCopy) { - ts.shutdown(); - } - for (DelayedClientTransport transport : delayedTransportsCopy) { - transport.shutdown(); - } - for (OobTransportProviderImpl provider : oobTransportsCopy) { - provider.close(); - } - if (log.isLoggable(Level.FINE)) { - log.log(Level.FINE, "[{0}] Shutting down", getLogId()); + public ManagedChannelImpl2 shutdown() { + log.log(Level.FINE, "[{0}] shutdown() called", getLogId()); + if (!shutdown.compareAndSet(false, true)) { + return this; } + delayedTransport.shutdown(); + channelExecutor.executeLater(new Runnable() { + @Override + public void run() { + maybeTerminateChannel(); + cancelIdleTimer(); + } + }).drain(); + log.log(Level.FINE, "[{0}] Shutting down", getLogId()); return this; } @@ -522,65 +506,36 @@ public ManagedChannelImpl shutdown() { * return {@code false} immediately after this method returns. */ @Override - public ManagedChannelImpl shutdownNow() { - synchronized (lock) { - // Short-circuiting not strictly necessary, but prevents transports from needing to handle - // multiple shutdownNow invocations. - if (shutdownNowed) { - return this; - } - shutdownNowed = true; - } + public ManagedChannelImpl2 shutdownNow() { + log.log(Level.FINE, "[{0}] shutdownNow() called", getLogId()); shutdown(); - List transportsCopy; - List delayedTransportsCopy; - List oobTransportsCopy; - synchronized (lock) { - transportsCopy = new ArrayList(transports.values()); - transportsCopy.addAll(decommissionedTransports); - delayedTransportsCopy = new ArrayList(delayedTransports); - oobTransportsCopy = new ArrayList(oobTransports); - } - if (log.isLoggable(Level.FINE)) { - log.log(Level.FINE, "[{0}] Shutting down now", getLogId()); - } - Status nowStatus = Status.UNAVAILABLE.withDescription("Channel shutdownNow invoked"); - for (TransportSet ts : transportsCopy) { - ts.shutdownNow(nowStatus); - } - for (DelayedClientTransport transport : delayedTransportsCopy) { - transport.shutdownNow(nowStatus); - } - for (OobTransportProviderImpl provider : oobTransportsCopy) { - provider.shutdownNow(nowStatus); - } + delayedTransport.shutdownNow(SHUTDOWN_NOW_STATUS); + channelExecutor.executeLater(new Runnable() { + @Override + public void run() { + if (shutdownNowed) { + return; + } + shutdownNowed = true; + maybeShutdownNowSubchannels(); + } + }).drain(); return this; } @Override public boolean isShutdown() { - synchronized (lock) { - return shutdown; - } + return shutdown.get(); } @Override public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException { - synchronized (lock) { - long timeoutNanos = unit.toNanos(timeout); - long endTimeNanos = System.nanoTime() + timeoutNanos; - while (!terminated && (timeoutNanos = endTimeNanos - System.nanoTime()) > 0) { - TimeUnit.NANOSECONDS.timedWait(lock, timeoutNanos); - } - return terminated; - } + return terminatedLatch.await(timeout, unit); } @Override public boolean isTerminated() { - synchronized (lock) { - return terminated; - } + return terminated; } /* @@ -597,22 +552,13 @@ public String authority() { return interceptorChannel.authority(); } - /** Returns {@code null} iff channel is in idle state. */ - @GuardedBy("lock") - private LoadBalancer getCurrentLoadBalancer() { - if (loadBalancer != null) { - return loadBalancer; - } - return graceLoadBalancer; - } - private class RealChannel extends Channel { @Override public ClientCall newCall(MethodDescriptor method, CallOptions callOptions) { Executor executor = callOptions.getExecutor(); if (executor == null) { - executor = ManagedChannelImpl.this.executor; + executor = ManagedChannelImpl2.this.executor; } StatsTraceContext statsTraceCtx = StatsTraceContext.newClientContext( method.getFullMethodName(), censusFactory, stopwatchSupplier); @@ -637,252 +583,292 @@ public String authority() { /** * Terminate the channel if termination conditions are met. */ - @GuardedBy("lock") + // Must be run from channelExecutor private void maybeTerminateChannel() { if (terminated) { return; } - if (shutdown && transports.isEmpty() && decommissionedTransports.isEmpty() - && delayedTransports.isEmpty() && oobTransports.isEmpty()) { - if (log.isLoggable(Level.INFO)) { - log.log(Level.INFO, "[{0}] Terminated", getLogId()); - } + if (shutdown.get() && subchannels.isEmpty() && oobChannels.isEmpty()) { + log.log(Level.FINE, "[{0}] Terminated", getLogId()); terminated = true; - lock.notifyAll(); + terminatedLatch.countDown(); if (usingSharedExecutor) { SharedResourceHolder.release(GrpcUtil.SHARED_CHANNEL_EXECUTOR, (ExecutorService) executor); } + scheduledExecutor = SharedResourceHolder.release(timerService, scheduledExecutor); // Release the transport factory so that it can deallocate any resources. transportFactory.close(); } } - @VisibleForTesting - final TransportManager tm = new TransportManager() { - @Override - public void updateRetainedTransports(Collection addrs) { - // TODO(zhangkun83): warm-up new servers and discard removed servers. + private class LbHelperImpl extends LoadBalancer2.Helper { + LoadBalancer2 lb; + final NameResolver nr; + + LbHelperImpl(NameResolver nr) { + this.nr = checkNotNull(nr, "NameResolver"); } @Override - public ClientTransport getTransport(final EquivalentAddressGroup addressGroup) { + public SubchannelImpl createSubchannel(EquivalentAddressGroup addressGroup, Attributes attrs) { checkNotNull(addressGroup, "addressGroup"); - TransportSet ts = transports.get(addressGroup); - if (ts != null) { - return ts.obtainActiveTransport(); - } - synchronized (lock) { - if (shutdown) { - return SHUTDOWN_TRANSPORT; - } - if (getCurrentLoadBalancer() == null) { - return IDLE_MODE_TRANSPORT; - } - ts = transports.get(addressGroup); - if (ts == null) { - ts = new TransportSet(addressGroup, authority(), userAgent, getCurrentLoadBalancer(), - backoffPolicyProvider, transportFactory, scheduledExecutor, stopwatchSupplier, - executor, new TransportSet.Callback() { - @Override - public void onTerminated(TransportSet ts) { - synchronized (lock) { - transports.remove(addressGroup); - decommissionedTransports.remove(ts); - maybeTerminateChannel(); - } - } - - @Override - public void onAllAddressesFailed() { - nameResolver.refresh(); - } - - @Override - public void onConnectionClosedByServer(Status status) { - nameResolver.refresh(); + checkNotNull(attrs, "attrs"); + ScheduledExecutorService scheduledExecutorCopy = scheduledExecutor; + checkState(scheduledExecutorCopy != null, + "scheduledExecutor is already cleared. Looks like you are calling this method after " + + "you've already shut down"); + final SubchannelImplImpl subchannel = new SubchannelImplImpl(attrs); + final InternalSubchannel internalSubchannel = new InternalSubchannel( + addressGroup, authority(), userAgent, backoffPolicyProvider, transportFactory, + scheduledExecutorCopy, stopwatchSupplier, channelExecutor, + new InternalSubchannel.Callback() { + // All callbacks are run in channelExecutor + @Override + void onTerminated(InternalSubchannel is) { + subchannels.remove(is); + maybeTerminateChannel(); + } + + @Override + void onStateChange(InternalSubchannel is, ConnectivityStateInfo newState) { + if ((newState.getState() == TRANSIENT_FAILURE || newState.getState() == IDLE)) { + nr.refresh(); } + lb.handleSubchannelState(subchannel, newState); + } + + @Override + public void onInUse(InternalSubchannel is) { + inUseStateAggregator.updateObjectInUse(is, true); + } + + @Override + public void onNotInUse(InternalSubchannel is) { + inUseStateAggregator.updateObjectInUse(is, false); + } + }); + subchannel.subchannel = internalSubchannel; + log.log(Level.FINE, "[{0}] {1} created for {2}", + new Object[] {getLogId(), internalSubchannel.getLogId(), addressGroup}); + channelExecutor.executeLater(new Runnable() { + @Override + public void run() { + if (terminating) { + internalSubchannel.shutdown(); + } + if (!terminated) { + // If channel has not terminated, it will track the subchannel and block termination + // for it. + subchannels.add(internalSubchannel); + } + } + }).drain(); + return subchannel; + } - @Override - public void onInUse(TransportSet ts) { - inUseStateAggregator.updateObjectInUse(ts, true); - } + @Override + public ManagedChannel createOobChannel( + EquivalentAddressGroup addressGroup, String authority, Executor executor) { + ScheduledExecutorService scheduledExecutorCopy = scheduledExecutor; + checkState(scheduledExecutorCopy != null, + "scheduledExecutor is already cleared. Looks like you are calling this method after " + + "you've already shut down"); + final OobChannel oobChannel = new OobChannel(censusFactory, authority, executor, + scheduledExecutorCopy, stopwatchSupplier, channelExecutor); + final InternalSubchannel internalSubchannel = new InternalSubchannel( + addressGroup, authority, userAgent, backoffPolicyProvider, transportFactory, + scheduledExecutorCopy, stopwatchSupplier, channelExecutor, + // All callback methods are run from channelExecutor + new InternalSubchannel.Callback() { + @Override + void onTerminated(InternalSubchannel is) { + oobChannels.remove(is); + oobChannel.handleSubchannelTerminated(); + maybeTerminateChannel(); + } - @Override - public void onNotInUse(TransportSet ts) { - inUseStateAggregator.updateObjectInUse(ts, false); - } - }); - if (log.isLoggable(Level.FINE)) { - log.log(Level.FINE, "[{0}] {1} created for {2}", - new Object[] {getLogId(), ts.getLogId(), addressGroup}); + @Override + void onStateChange(InternalSubchannel is, ConnectivityStateInfo newState) { + oobChannel.handleSubchannelStateChange(newState); + } + }); + oobChannel.setSubchannel(internalSubchannel); + channelExecutor.executeLater(new Runnable() { + @Override + public void run() { + if (terminating) { + oobChannel.shutdown(); + } + if (!terminated) { + // If channel has not terminated, it will track the subchannel and block termination + // for it. + oobChannels.add(internalSubchannel); + } } - transports.put(addressGroup, ts); - } - } - return ts.obtainActiveTransport(); + }).drain(); + return oobChannel; } @Override - public Channel makeChannel(ClientTransport transport) { - return new SingleTransportChannel( - censusFactory, transport, executor, scheduledExecutor, authority(), stopwatchSupplier); + public String getAuthority() { + return ManagedChannelImpl2.this.authority(); } @Override - public ClientTransport createFailingTransport(Status error) { - return new FailingClientTransport(error); + public NameResolver.Factory getNameResolverFactory() { + return nameResolverFactory; } @Override - public InterimTransport createInterimTransport() { - return new InterimTransportImpl(); + public void runSerialized(Runnable task) { + channelExecutor.executeLater(task).drain(); } @Override - public OobTransportProvider createOobTransportProvider( - EquivalentAddressGroup addressGroup, String authority) { - return new OobTransportProviderImpl(addressGroup, authority); + public void updatePicker(final SubchannelPicker picker) { + runSerialized(new Runnable() { + @Override + public void run() { + subchannelPicker = picker; + delayedTransport.reprocess(picker); + } + }); } - }; + } @Override public LogId getLogId() { return logId; } - private static class NameResolverListenerImpl implements NameResolver.Listener { - final LoadBalancer balancer; + private class NameResolverListenerImpl implements NameResolver.Listener { + final LoadBalancer2 balancer; - NameResolverListenerImpl(LoadBalancer balancer) { + NameResolverListenerImpl(LoadBalancer2 balancer) { this.balancer = balancer; } @Override - public void onUpdate(List servers, Attributes config) { + public void onUpdate(final List servers, final Attributes config) { if (servers.isEmpty()) { onError(Status.UNAVAILABLE.withDescription("NameResolver returned an empty list")); return; } - - try { - balancer.handleResolvedAddresses(servers, config); - } catch (Throwable e) { - // It must be a bug! Push the exception back to LoadBalancer in the hope that it may be - // propagated to the application. - balancer.handleNameResolutionError(Status.INTERNAL.withCause(e) - .withDescription("Thrown from handleResolvedAddresses(): " + e)); - } + log.log(Level.FINE, "[{0}] resolved address: {1}, config={2}", + new Object[] {getLogId(), servers, config}); + channelExecutor.executeLater(new Runnable() { + @Override + public void run() { + if (terminated) { + return; + } + try { + balancer.handleResolvedAddresses(servers, config); + } catch (Throwable e) { + log.log(Level.WARNING, "[" + getLogId() + "] Caught exception from LoadBalancer", e); + // It must be a bug! Push the exception back to LoadBalancer in the hope that it may + // be propagated to the application. + balancer.handleNameResolutionError(Status.INTERNAL.withCause(e) + .withDescription("Thrown from handleResolvedAddresses(): " + e)); + } + } + }).drain(); } @Override - public void onError(Status error) { + public void onError(final Status error) { checkArgument(!error.isOk(), "the error status must not be OK"); - balancer.handleNameResolutionError(error); - } - } - - private class InterimTransportImpl implements InterimTransport { - private final DelayedClientTransport delayedTransport; - private boolean closed; - - InterimTransportImpl() { - delayedTransport = new DelayedClientTransport(executor); - delayedTransport.start(new ManagedClientTransport.Listener() { - @Override public void transportShutdown(Status status) {} - - @Override public void transportTerminated() { - synchronized (lock) { - delayedTransports.remove(delayedTransport); - maybeTerminateChannel(); + log.log(Level.WARNING, "[{0}] Failed to resolve name. status={1}", + new Object[] {getLogId(), error}); + channelExecutor.executeLater(new Runnable() { + @Override + public void run() { + if (terminated) { + return; } - inUseStateAggregator.updateObjectInUse(delayedTransport, false); + balancer.handleNameResolutionError(error); } + }).drain(); + } + } - @Override public void transportReady() {} + private final class SubchannelImplImpl extends SubchannelImpl { + // Set right after SubchannelImplImpl is created. + InternalSubchannel subchannel; + final Object shutdownLock = new Object(); + final Attributes attrs; - @Override public void transportInUse(boolean inUse) { - inUseStateAggregator.updateObjectInUse(delayedTransport, inUse); - } - }); - boolean savedShutdown; - synchronized (lock) { - delayedTransports.add(delayedTransport); - savedShutdown = shutdown; - } - if (savedShutdown) { - delayedTransport.setTransport(SHUTDOWN_TRANSPORT); - delayedTransport.shutdown(); - } - } + @GuardedBy("shutdownLock") + boolean shutdownRequested; + @GuardedBy("shutdownLock") + ScheduledFuture delayedShutdownTask; - @Override - public ClientTransport transport() { - checkState(!closed, "already closed"); - return delayedTransport; + SubchannelImplImpl(Attributes attrs) { + this.attrs = checkNotNull(attrs, "attrs"); } @Override - public void closeWithRealTransports(Supplier realTransports) { - delayedTransport.setTransportSupplier(realTransports); - delayedTransport.shutdown(); + ClientTransport obtainActiveTransport() { + return subchannel.obtainActiveTransport(); } @Override - public void closeWithError(Status error) { - delayedTransport.shutdownNow(error); - } - } - - private class OobTransportProviderImpl implements OobTransportProvider { - private final TransportSet transportSet; - private final ClientTransport transport; - - OobTransportProviderImpl(EquivalentAddressGroup addressGroup, String authority) { - synchronized (lock) { - if (shutdown) { - transportSet = null; - transport = SHUTDOWN_TRANSPORT; - } else if (getCurrentLoadBalancer() == null) { - transportSet = null; - transport = IDLE_MODE_TRANSPORT; + public void shutdown() { + synchronized (shutdownLock) { + if (shutdownRequested) { + if (terminating && delayedShutdownTask != null) { + // shutdown() was previously called when terminating == false, thus a delayed shutdown() + // was scheduled. Now since terminating == true, We should expedite the shutdown. + delayedShutdownTask.cancel(false); + delayedShutdownTask = null; + // Will fall through to the subchannel.shutdown() at the end. + } else { + return; + } } else { - transport = null; - transportSet = new TransportSet(addressGroup, authority, userAgent, - getCurrentLoadBalancer(), backoffPolicyProvider, transportFactory, scheduledExecutor, - stopwatchSupplier, executor, new TransportSet.Callback() { - @Override - public void onTerminated(TransportSet ts) { - synchronized (lock) { - oobTransports.remove(OobTransportProviderImpl.this); - maybeTerminateChannel(); - } - } - }); - oobTransports.add(this); + shutdownRequested = true; } } + ScheduledExecutorService scheduledExecutorCopy = scheduledExecutor; + // Add a delay to shutdown to deal with the race between 1) a transport being picked and + // newStream() being called on it, and 2) its Subchannel is shut down by LoadBalancer (e.g., + // because of address change, or because LoadBalancer is shutdown by Channel entering idle + // mode). If (2) wins, the app will see a spurious error. We work around this by delaying + // shutdown of Subchannel for a few seconds here. + if (!terminating && scheduledExecutorCopy != null) { + delayedShutdownTask = scheduledExecutorCopy.schedule( + new LogExceptionRunnable( + new Runnable() { + @Override + public void run() { + subchannel.shutdown(); + } + }), SUBCHANNEL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); + } else { + // Two possible ways to get here: + // + // 1. terminating == true: no more real streams will be created, it's safe and also + // desirable to shutdown timely. + // + // 2. scheduledExecutor == null: possible only when Channel has already been terminated. + // Though may not be necessary, we'll do it anyway. + subchannel.shutdown(); + } } @Override - public ClientTransport get() { - if (transport != null) { - return transport; - } else { - return transportSet.obtainActiveTransport(); - } + public void requestConnection() { + subchannel.obtainActiveTransport(); } @Override - public void close() { - if (transportSet != null) { - transportSet.shutdown(); - } + public EquivalentAddressGroup getAddresses() { + return subchannel.getAddressGroup(); } - void shutdownNow(Status reason) { - if (transportSet != null) { - transportSet.shutdownNow(reason); - } + @Override + public Attributes getAttributes() { + return attrs; } } } diff --git a/core/src/main/java/io/grpc/internal/OobChannel.java b/core/src/main/java/io/grpc/internal/OobChannel.java new file mode 100644 index 00000000000..f74861396f5 --- /dev/null +++ b/core/src/main/java/io/grpc/internal/OobChannel.java @@ -0,0 +1,244 @@ +/* + * 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.internal; + +import static com.google.common.base.Preconditions.checkNotNull; +import static io.grpc.ConnectivityState.READY; +import static io.grpc.ConnectivityState.TRANSIENT_FAILURE; + +import com.google.census.CensusContextFactory; +import com.google.common.base.Stopwatch; +import com.google.common.base.Supplier; + +import io.grpc.Attributes; +import io.grpc.CallOptions; +import io.grpc.ClientCall; +import io.grpc.ConnectivityStateInfo; +import io.grpc.EquivalentAddressGroup; +import io.grpc.LoadBalancer2.PickResult; +import io.grpc.LoadBalancer2.SubchannelPicker; +import io.grpc.ManagedChannel; +import io.grpc.Metadata; +import io.grpc.MethodDescriptor; +import io.grpc.Status; +import io.grpc.internal.ClientCallImpl.ClientTransportProvider; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.logging.Level; +import java.util.logging.Logger; +import javax.annotation.concurrent.ThreadSafe; + +/** + * A ManagedChannel backed by a single {@link InternalSubchannel} and used for {@link LoadBalancer2} + * to its own RPC needs. + */ +@ThreadSafe +final class OobChannel extends ManagedChannel implements WithLogId { + private static final Logger log = Logger.getLogger(OobChannel.class.getName()); + + private SubchannelImpl subchannelImpl; + private SubchannelPicker subchannelPicker; + + private final LogId logId = LogId.allocate(getClass().getName()); + private final CensusContextFactory censusFactory; + private final String authority; + private final DelayedClientTransport2 delayedTransport; + private final Executor executor; + private final ScheduledExecutorService deadlineCancellationExecutor; + private final Supplier stopwatchSupplier; + private final CountDownLatch terminatedLatch = new CountDownLatch(1); + private volatile boolean shutdown; + + private final ClientTransportProvider transportProvider = new ClientTransportProvider() { + @Override + public ClientTransport get(CallOptions callOptions, Metadata headers) { + // delayed transport's newStream() always acquires a lock, but concurrent performance doesn't + // matter here because OOB communication should be sparse, and it's not on application RPC's + // critical path. + return delayedTransport; + } + }; + + OobChannel(CensusContextFactory censusFactory, String authority, Executor executor, + ScheduledExecutorService deadlineCancellationExecutor, Supplier stopwatchSupplier, + ChannelExecutor channelExecutor) { + this.censusFactory = checkNotNull(censusFactory, "censusFactory"); + this.authority = checkNotNull(authority, "authority"); + this.executor = checkNotNull(executor, "executor"); + this.deadlineCancellationExecutor = checkNotNull( + deadlineCancellationExecutor, "deadlineCancellationExecutor"); + this.stopwatchSupplier = checkNotNull(stopwatchSupplier, "stopwatchSupplier"); + this.delayedTransport = new DelayedClientTransport2(executor, channelExecutor); + this.delayedTransport.start(new ManagedClientTransport.Listener() { + @Override + public void transportShutdown(Status s) { + // Don't care + } + + @Override + public void transportTerminated() { + subchannelImpl.shutdown(); + } + + @Override + public void transportReady() { + // Don't care + } + + @Override + public void transportInUse(boolean inUse) { + // Don't care + } + }); + } + + // Must be called only once, right after the OobChannel is created. + void setSubchannel(final InternalSubchannel subchannel) { + log.log(Level.FINE, "[{0}] Created with [{1}]", new Object[] {this, subchannel}); + subchannelImpl = new SubchannelImpl() { + @Override + public void shutdown() { + subchannel.shutdown(); + } + + @Override + ClientTransport obtainActiveTransport() { + return subchannel.obtainActiveTransport(); + } + + @Override + public void requestConnection() { + subchannel.obtainActiveTransport(); + } + + @Override + public EquivalentAddressGroup getAddresses() { + return subchannel.getAddressGroup(); + } + + @Override + public Attributes getAttributes() { + return Attributes.EMPTY; + } + }; + + subchannelPicker = new SubchannelPicker() { + final PickResult result = PickResult.withSubchannel(subchannelImpl); + + @Override + public PickResult pickSubchannel(Attributes affinity, Metadata headers) { + return result; + } + }; + delayedTransport.reprocess(subchannelPicker); + } + + @Override + public ClientCall newCall( + MethodDescriptor methodDescriptor, CallOptions callOptions) { + StatsTraceContext statsTraceCtx = StatsTraceContext.newClientContext( + methodDescriptor.getFullMethodName(), censusFactory, stopwatchSupplier); + return new ClientCallImpl(methodDescriptor, + callOptions.getExecutor() == null ? executor : callOptions.getExecutor(), + callOptions, statsTraceCtx, transportProvider, + deadlineCancellationExecutor); + } + + @Override + public String authority() { + return authority; + } + + @Override + public LogId getLogId() { + return logId; + } + + @Override + public boolean isTerminated() { + return terminatedLatch.getCount() == 0; + } + + @Override + public boolean awaitTermination(long time, TimeUnit unit) throws InterruptedException { + return terminatedLatch.await(time, unit); + } + + @Override + public ManagedChannel shutdown() { + shutdown = true; + delayedTransport.shutdown(); + return this; + } + + @Override + public boolean isShutdown() { + return shutdown; + } + + @Override + public ManagedChannel shutdownNow() { + delayedTransport.shutdownNow( + Status.UNAVAILABLE.withDescription("OobChannel.shutdownNow() called")); + return this; + } + + void handleSubchannelStateChange(final ConnectivityStateInfo newState) { + switch (newState.getState()) { + case READY: + case IDLE: + delayedTransport.reprocess(subchannelPicker); + break; + case TRANSIENT_FAILURE: + delayedTransport.reprocess(new SubchannelPicker() { + final PickResult errorResult = PickResult.withError(newState.getStatus()); + + @Override + public PickResult pickSubchannel(Attributes affinity, Metadata headers) { + return errorResult; + } + }); + break; + default: + // Do nothing + } + } + + void handleSubchannelTerminated() { + // When delayedTransport is terminated, it shuts down subchannel. Therefore, at this point + // both delayedTransport and subchannel have terminated. + terminatedLatch.countDown(); + } +} diff --git a/core/src/main/java/io/grpc/internal/SingleTransportChannel.java b/core/src/main/java/io/grpc/internal/SingleTransportChannel.java index 19ba5904304..130e9580b5d 100644 --- a/core/src/main/java/io/grpc/internal/SingleTransportChannel.java +++ b/core/src/main/java/io/grpc/internal/SingleTransportChannel.java @@ -39,6 +39,7 @@ import io.grpc.CallOptions; import io.grpc.Channel; import io.grpc.ClientCall; +import io.grpc.Metadata; import io.grpc.MethodDescriptor; import io.grpc.internal.ClientCallImpl.ClientTransportProvider; @@ -59,7 +60,7 @@ final class SingleTransportChannel extends Channel { private final ClientTransportProvider transportProvider = new ClientTransportProvider() { @Override - public ClientTransport get(CallOptions callOptions) { + public ClientTransport get(CallOptions callOptions, Metadata headers) { return transport; } }; diff --git a/core/src/main/java/io/grpc/internal/TransportSet.java b/core/src/main/java/io/grpc/internal/TransportSet.java index 9ab5b636631..501670e7451 100644 --- a/core/src/main/java/io/grpc/internal/TransportSet.java +++ b/core/src/main/java/io/grpc/internal/TransportSet.java @@ -41,6 +41,7 @@ import io.grpc.EquivalentAddressGroup; import io.grpc.LoadBalancer; import io.grpc.ManagedChannel; +import io.grpc.Metadata; import io.grpc.MethodDescriptor; import io.grpc.Status; import io.grpc.internal.ClientCallImpl.ClientTransportProvider; @@ -364,7 +365,7 @@ public final ClientCall newCall( new SerializingExecutor(appExecutor), callOptions, StatsTraceContext.NOOP, new ClientTransportProvider() { @Override - public ClientTransport get(CallOptions callOptions) { + public ClientTransport get(CallOptions callOptions, Metadata headers) { return obtainActiveTransport(); } }, diff --git a/core/src/test/java/io/grpc/internal/ClientCallImplTest.java b/core/src/test/java/io/grpc/internal/ClientCallImplTest.java index 4ef6346347c..98dfc8dd92b 100644 --- a/core/src/test/java/io/grpc/internal/ClientCallImplTest.java +++ b/core/src/test/java/io/grpc/internal/ClientCallImplTest.java @@ -155,7 +155,7 @@ public class ClientCallImplTest { public void setUp() { MockitoAnnotations.initMocks(this); assertNotNull(censusCtx); - when(provider.get(any(CallOptions.class))).thenReturn(transport); + when(provider.get(any(CallOptions.class), any(Metadata.class))).thenReturn(transport); when(transport.newStream(any(MethodDescriptor.class), any(Metadata.class), any(CallOptions.class), any(StatsTraceContext.class))).thenReturn(stream); } diff --git a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2IdlenessTest.java b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2IdlenessTest.java index 89dc573aea2..a73eea00a73 100644 --- a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2IdlenessTest.java +++ b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2IdlenessTest.java @@ -33,13 +33,11 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.same; -import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -55,7 +53,12 @@ import io.grpc.DecompressorRegistry; import io.grpc.EquivalentAddressGroup; import io.grpc.IntegerMarshaller; -import io.grpc.LoadBalancer; +import io.grpc.LoadBalancer2.Helper; +import io.grpc.LoadBalancer2.PickResult; +import io.grpc.LoadBalancer2.Subchannel; +import io.grpc.LoadBalancer2.SubchannelPicker; +import io.grpc.LoadBalancer2; +import io.grpc.ManagedChannel; import io.grpc.Metadata; import io.grpc.MethodDescriptor; import io.grpc.NameResolver; @@ -63,8 +66,6 @@ import io.grpc.ResolvedServerInfoGroup; import io.grpc.Status; import io.grpc.StringMarshaller; -import io.grpc.TransportManager.OobTransportProvider; -import io.grpc.TransportManager; import io.grpc.internal.TestUtils.MockClientTransportInfo; import org.junit.After; @@ -74,17 +75,12 @@ import org.junit.runners.JUnit4; import org.mockito.ArgumentCaptor; import org.mockito.Captor; -import org.mockito.Matchers; import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import java.net.SocketAddress; import java.net.URI; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.concurrent.BlockingQueue; @@ -95,13 +91,13 @@ * Unit tests for {@link ManagedChannelImpl}'s idle mode. */ @RunWith(JUnit4.class) -public class ManagedChannelImplIdlenessTest { +public class ManagedChannelImpl2IdlenessTest { private final FakeClock timer = new FakeClock(); private final FakeClock executor = new FakeClock(); private static final String AUTHORITY = "fakeauthority"; private static final String USER_AGENT = "fakeagent"; private static final long IDLE_TIMEOUT_SECONDS = 30; - private ManagedChannelImpl channel; + private ManagedChannelImpl2 channel; private final MethodDescriptor method = MethodDescriptor.create( MethodDescriptor.MethodType.UNKNOWN, "/service/method", @@ -113,11 +109,12 @@ public class ManagedChannelImplIdlenessTest { @Mock private SharedResourceHolder.Resource timerService; @Mock private ClientTransportFactory mockTransportFactory; - @Mock private LoadBalancer mockLoadBalancer; - @Mock private LoadBalancer.Factory mockLoadBalancerFactory; + @Mock private LoadBalancer2 mockLoadBalancer; + @Mock private LoadBalancer2.Factory mockLoadBalancerFactory; @Mock private NameResolver mockNameResolver; @Mock private NameResolver.Factory mockNameResolverFactory; @Mock private ClientCall.Listener mockCallListener; + @Mock private ClientCall.Listener mockCallListener2; @Captor private ArgumentCaptor nameResolverListenerCaptor; private BlockingQueue newTransports; @@ -125,15 +122,13 @@ public class ManagedChannelImplIdlenessTest { public void setUp() { MockitoAnnotations.initMocks(this); when(timerService.create()).thenReturn(timer.getScheduledExecutorService()); - when(mockLoadBalancerFactory - .newLoadBalancer(anyString(), Matchers.>any())) - .thenReturn(mockLoadBalancer); + when(mockLoadBalancerFactory.newLoadBalancer(any(Helper.class))).thenReturn(mockLoadBalancer); when(mockNameResolver.getServiceAuthority()).thenReturn(AUTHORITY); when(mockNameResolverFactory .newNameResolver(any(URI.class), any(Attributes.class))) .thenReturn(mockNameResolver); - channel = new ManagedChannelImpl("fake://target", new FakeBackoffPolicyProvider(), + channel = new ManagedChannelImpl2("fake://target", new FakeBackoffPolicyProvider(), mockNameResolverFactory, Attributes.EMPTY, mockLoadBalancerFactory, mockTransportFactory, DecompressorRegistry.getDefaultInstance(), CompressorRegistry.getDefaultInstance(), timerService, timer.getStopwatchSupplier(), @@ -154,8 +149,7 @@ public void setUp() { } verify(mockNameResolverFactory).newNameResolver(any(URI.class), any(Attributes.class)); // Verify the initial idleness - verify(mockLoadBalancerFactory, never()).newLoadBalancer( - anyString(), Matchers.>any()); + verify(mockLoadBalancerFactory, never()).newLoadBalancer(any(Helper.class)); verify(mockTransportFactory, never()).newClientTransport( any(SocketAddress.class), anyString(), anyString()); verify(mockNameResolver, never()).start(any(NameResolver.Listener.class)); @@ -170,126 +164,70 @@ public void allPendingTasksAreRun() { @Test public void newCallExitsIdleness() throws Exception { final EquivalentAddressGroup addressGroup = addressGroupList.get(1); - doAnswer(new Answer() { - @Override - public ClientTransport answer(InvocationOnMock invocation) throws Throwable { - return channel.tm.getTransport(addressGroup); - } - }).when(mockLoadBalancer).pickTransport(any(Attributes.class)); ClientCall call = channel.newCall(method, CallOptions.DEFAULT); call.start(mockCallListener, new Metadata()); - verify(mockLoadBalancerFactory).newLoadBalancer(anyString(), same(channel.tm)); - // NameResolver is started in the scheduled executor - timer.runDueTasks(); - verify(mockNameResolver).start(nameResolverListenerCaptor.capture()); + ArgumentCaptor helperCaptor = ArgumentCaptor.forClass(null); + verify(mockLoadBalancerFactory).newLoadBalancer(helperCaptor.capture()); + Helper helper = helperCaptor.getValue(); - // LoadBalancer is used right after created. - verify(mockLoadBalancer).pickTransport(any(Attributes.class)); - verify(mockTransportFactory).newClientTransport( - addressGroup.getAddresses().get(0), AUTHORITY, USER_AGENT); - - // Simulate new address resolved + verify(mockNameResolver).start(nameResolverListenerCaptor.capture()); + // Simulate new address resolved to make sure the LoadBalancer is correctly linked to + // the NameResolver. nameResolverListenerCaptor.getValue().onUpdate(servers, Attributes.EMPTY); verify(mockLoadBalancer).handleResolvedAddresses(servers, Attributes.EMPTY); } @Test - public void newCallResetsGracePeriod() throws Exception { + public void newCallRefreshesIdlenessTimer() throws Exception { final EquivalentAddressGroup addressGroup = addressGroupList.get(1); - doAnswer(new Answer() { - @Override - public ClientTransport answer(InvocationOnMock invocation) throws Throwable { - return channel.tm.getTransport(addressGroup); - } - }).when(mockLoadBalancer).pickTransport(any(Attributes.class)); + // First call to exit the initial idleness, then immediately cancel the call. ClientCall call = channel.newCall(method, CallOptions.DEFAULT); call.start(mockCallListener, new Metadata()); - call.cancel("cleanup", null); - executor.runDueTasks(); - - timer.runDueTasks(); - verify(mockLoadBalancerFactory).newLoadBalancer(anyString(), same(channel.tm)); - verify(mockLoadBalancer).pickTransport(any(Attributes.class)); - - // Enter grace period - timer.forwardTime(TimeUnit.SECONDS.toMillis(IDLE_TIMEOUT_SECONDS) - - ManagedChannelImpl.IDLE_GRACE_PERIOD_MILLIS, TimeUnit.MILLISECONDS); - assertTrue(channel.isInIdleGracePeriod()); + call.cancel("For testing", null); - call = channel.newCall(method, CallOptions.DEFAULT); - call.start(mockCallListener, new Metadata()); - assertFalse(channel.isInIdleGracePeriod()); - call.cancel("cleanup", null); - executor.runDueTasks(); - - // Load balancer was reused. - timer.runDueTasks(); - verify(mockLoadBalancerFactory).newLoadBalancer(anyString(), same(channel.tm)); - verify(mockLoadBalancer, times(2)).pickTransport(any(Attributes.class)); + // Verify that we have exited the idle mode + ArgumentCaptor helperCaptor = ArgumentCaptor.forClass(null); + verify(mockLoadBalancerFactory).newLoadBalancer(helperCaptor.capture()); + Helper helper = helperCaptor.getValue(); + assertFalse(channel.inUseStateAggregator.isInUse()); - // Now just let time pass to allow the original idle time to be well past expired. + // Move closer to idleness, but not yet. timer.forwardTime(IDLE_TIMEOUT_SECONDS - 1, TimeUnit.SECONDS); + verify(mockLoadBalancer, never()).shutdown(); + assertFalse(channel.inUseStateAggregator.isInUse()); + // A new call would refresh the timer call = channel.newCall(method, CallOptions.DEFAULT); call.start(mockCallListener, new Metadata()); + call.cancel("For testing", null); + assertFalse(channel.inUseStateAggregator.isInUse()); - // Load balancer was reused; the idle time period must have been reset. - timer.runDueTasks(); - verify(mockLoadBalancerFactory).newLoadBalancer(anyString(), same(channel.tm)); - verify(mockLoadBalancer, times(3)).pickTransport(any(Attributes.class)); - } - - @Test - public void shutdownDuringGracePeriodShutdownLb() throws Exception { - forceExitIdleMode(); - verify(mockLoadBalancerFactory).newLoadBalancer(anyString(), same(channel.tm)); - // Enter grace period - timer.forwardTime(TimeUnit.SECONDS.toMillis(IDLE_TIMEOUT_SECONDS) - - ManagedChannelImpl.IDLE_GRACE_PERIOD_MILLIS, TimeUnit.MILLISECONDS); + // ... so that passing the same length of time will not trigger idle mode + timer.forwardTime(IDLE_TIMEOUT_SECONDS - 1, TimeUnit.SECONDS); verify(mockLoadBalancer, never()).shutdown(); - channel.shutdown(); - verify(mockLoadBalancer).shutdown(); - } - - @Test - public void enterIdleModeAfterForceExit() throws Exception { - forceExitIdleMode(); - - // Trigger the creation of TransportSets - for (EquivalentAddressGroup addressGroup : addressGroupList) { - channel.tm.getTransport(addressGroup); - verify(mockTransportFactory).newClientTransport( - addressGroup.getAddresses().get(0), AUTHORITY, USER_AGENT); - } - ArrayList transports = new ArrayList(); - newTransports.drainTo(transports); - assertEquals(addressGroupList.size(), transports.size()); + assertFalse(channel.inUseStateAggregator.isInUse()); - channel.tm.createInterimTransport(); + // ... until the time since last call has reached the timeout + timer.forwardTime(1, TimeUnit.SECONDS); + verify(mockLoadBalancer).shutdown(); + assertFalse(channel.inUseStateAggregator.isInUse()); - // Without actually using these transports, will eventually enter idle mode - walkIntoIdleMode(transports); + // Drain the app executor, which runs the call listeners + verify(mockCallListener, never()).onClose(any(Status.class), any(Metadata.class)); + assertEquals(2, executor.runDueTasks()); + verify(mockCallListener, times(2)).onClose(any(Status.class), any(Metadata.class)); } - + @Test - public void interimTransportHoldsOffIdleness() throws Exception { - doAnswer(new Answer() { - @Override - public ClientTransport answer(InvocationOnMock invocation) throws Throwable { - return channel.tm.createInterimTransport().transport(); - } - }).when(mockLoadBalancer).pickTransport(any(Attributes.class)); - + public void delayedTransportHoldsOffIdleness() throws Exception { ClientCall call = channel.newCall(method, CallOptions.DEFAULT); call.start(mockCallListener, new Metadata()); assertTrue(channel.inUseStateAggregator.isInUse()); - // NameResolver is started in the scheduled executor - timer.runDueTasks(); - // As long as the interim transport is in-use (by the pending RPC), the channel won't go idle. + // As long as the delayed transport is in-use (by the pending RPC), the channel won't go idle. timer.forwardTime(IDLE_TIMEOUT_SECONDS * 2, TimeUnit.SECONDS); assertTrue(channel.inUseStateAggregator.isInUse()); @@ -299,118 +237,100 @@ public ClientTransport answer(InvocationOnMock invocation) throws Throwable { assertEquals(1, executor.runDueTasks()); assertFalse(channel.inUseStateAggregator.isInUse()); // And allow the channel to go idle. - walkIntoIdleMode(Collections.emptyList()); + timer.forwardTime(IDLE_TIMEOUT_SECONDS - 1, TimeUnit.SECONDS); + verify(mockLoadBalancer, never()).shutdown(); + timer.forwardTime(1, TimeUnit.SECONDS); + verify(mockLoadBalancer).shutdown(); } @Test public void realTransportsHoldsOffIdleness() throws Exception { final EquivalentAddressGroup addressGroup = addressGroupList.get(1); - doAnswer(new Answer() { - @Override - public ClientTransport answer(InvocationOnMock invocation) throws Throwable { - return channel.tm.getTransport(addressGroup); - } - }).when(mockLoadBalancer).pickTransport(any(Attributes.class)); + // Start a call, which goes to delayed transport ClientCall call = channel.newCall(method, CallOptions.DEFAULT); call.start(mockCallListener, new Metadata()); - // A TransportSet is in-use, while the stream is pending in a delayed transport + // Verify that we have exited the idle mode + ArgumentCaptor helperCaptor = ArgumentCaptor.forClass(null); + verify(mockLoadBalancerFactory).newLoadBalancer(helperCaptor.capture()); + Helper helper = helperCaptor.getValue(); assertTrue(channel.inUseStateAggregator.isInUse()); - // NameResolver is started in the scheduled executor - timer.runDueTasks(); - // Making the real transport ready, will release the delayed transport. - // The TransportSet is *not* in-use before the real transport become in-use. + // Assume LoadBalancer has received an address, then create a subchannel. + Subchannel subchannel = helper.createSubchannel(addressGroup, Attributes.EMPTY); + subchannel.requestConnection(); MockClientTransportInfo t0 = newTransports.poll(); - assertEquals(0, executor.numPendingTasks()); t0.listener.transportReady(); - // Real streams are started in the executor - assertEquals(1, executor.runDueTasks()); + + SubchannelPicker mockPicker = mock(SubchannelPicker.class); + when(mockPicker.pickSubchannel(any(Attributes.class), any(Metadata.class))) + .thenReturn(PickResult.withSubchannel(subchannel)); + helper.updatePicker(mockPicker); + // Delayed transport creates real streams in the app executor + executor.runDueTasks(); + + // Delayed transport exits in-use, while real transport has not entered in-use yet. assertFalse(channel.inUseStateAggregator.isInUse()); + + // Now it's in-use t0.listener.transportInUse(true); assertTrue(channel.inUseStateAggregator.isInUse()); // As long as the transport is in-use, the channel won't go idle. timer.forwardTime(IDLE_TIMEOUT_SECONDS * 2, TimeUnit.SECONDS); + assertTrue(channel.inUseStateAggregator.isInUse()); t0.listener.transportInUse(false); assertFalse(channel.inUseStateAggregator.isInUse()); // And allow the channel to go idle. - walkIntoIdleMode(Arrays.asList(t0)); + timer.forwardTime(IDLE_TIMEOUT_SECONDS - 1, TimeUnit.SECONDS); + verify(mockLoadBalancer, never()).shutdown(); + timer.forwardTime(1, TimeUnit.SECONDS); + verify(mockLoadBalancer).shutdown(); } @Test - public void idlenessDecommissionsTransports() throws Exception { - EquivalentAddressGroup addressGroup = addressGroupList.get(0); - forceExitIdleMode(); - - channel.tm.getTransport(addressGroup); - MockClientTransportInfo t0 = newTransports.poll(); - t0.listener.transportReady(); - assertSame(t0.transport, channelTmGetTransportUnwrapped(addressGroup)); - - walkIntoIdleMode(Arrays.asList(t0)); - verify(t0.transport).shutdown(); - - forceExitIdleMode(); - channel.tm.getTransport(addressGroup); - MockClientTransportInfo t1 = newTransports.poll(); - t1.listener.transportReady(); - - assertSame(t1.transport, channelTmGetTransportUnwrapped(addressGroup)); - assertNotSame(t0.transport, channelTmGetTransportUnwrapped(addressGroup)); + public void oobTransportDoesNotAffectIdleness() { + FakeClock oobExecutor = new FakeClock(); + // Start a call, which goes to delayed transport + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + call.start(mockCallListener, new Metadata()); - channel.shutdown(); - verify(t1.transport).shutdown(); - channel.shutdownNow(); - verify(t0.transport).shutdownNow(any(Status.class)); - verify(t1.transport).shutdownNow(any(Status.class)); + // Verify that we have exited the idle mode + ArgumentCaptor helperCaptor = ArgumentCaptor.forClass(null); + verify(mockLoadBalancerFactory).newLoadBalancer(helperCaptor.capture()); + Helper helper = helperCaptor.getValue(); - t1.listener.transportTerminated(); - assertFalse(channel.isTerminated()); - t0.listener.transportTerminated(); - assertTrue(channel.isTerminated()); - } + // Fail the RPC + SubchannelPicker failingPicker = mock(SubchannelPicker.class); + when(failingPicker.pickSubchannel(any(Attributes.class), any(Metadata.class))) + .thenReturn(PickResult.withError(Status.UNAVAILABLE)); + helper.updatePicker(failingPicker); + executor.runDueTasks(); + verify(mockCallListener).onClose(same(Status.UNAVAILABLE), any(Metadata.class)); - @Test - public void loadBalancerShouldNotCreateConnectionsWhenIdle() throws Exception { - // Acts as a misbehaving LoadBalancer that tries to create connections when channel is in idle, - // which means the LoadBalancer is supposedly shutdown. - assertSame(ManagedChannelImpl.IDLE_MODE_TRANSPORT, - channel.tm.getTransport(addressGroupList.get(0))); - OobTransportProvider oobProvider = - channel.tm.createOobTransportProvider(addressGroupList.get(0), AUTHORITY); - assertSame(ManagedChannelImpl.IDLE_MODE_TRANSPORT, oobProvider.get()); - oobProvider.close(); - verify(mockTransportFactory, never()).newClientTransport( - any(SocketAddress.class), anyString(), anyString()); - // We don't care for delayed (interim) transports, because they don't create connections. - } + // ... so that the channel resets its in-use state + assertFalse(channel.inUseStateAggregator.isInUse()); - private void walkIntoIdleMode(Collection currentTransports) { - timer.forwardTime(IDLE_TIMEOUT_SECONDS - 1, TimeUnit.SECONDS); + // Now make an RPC on an OOB channel + ManagedChannel oob = helper.createOobChannel(addressGroupList.get(0), "oobauthority", + oobExecutor.getScheduledExecutorService()); + verify(mockTransportFactory, never()) + .newClientTransport(any(SocketAddress.class), same("oobauthority"), same(USER_AGENT)); + ClientCall oobCall = oob.newCall(method, CallOptions.DEFAULT); + oobCall.start(mockCallListener2, new Metadata()); + verify(mockTransportFactory) + .newClientTransport(any(SocketAddress.class), same("oobauthority"), same(USER_AGENT)); + MockClientTransportInfo oobTransportInfo = newTransports.poll(); + assertEquals(0, newTransports.size()); + // The OOB transport reports in-use state + oobTransportInfo.listener.transportInUse(true); + + // But it won't stop the channel from going idle verify(mockLoadBalancer, never()).shutdown(); - verify(mockNameResolver, never()).shutdown(); - for (MockClientTransportInfo transport : currentTransports) { - verify(transport.transport, never()).shutdown(); - } - timer.forwardTime(1, TimeUnit.SECONDS); + timer.forwardTime(IDLE_TIMEOUT_SECONDS, TimeUnit.SECONDS); verify(mockLoadBalancer).shutdown(); - verify(mockNameResolver).shutdown(); - for (MockClientTransportInfo transport : currentTransports) { - verify(transport.transport).shutdown(); - } - } - - private void forceExitIdleMode() { - channel.exitIdleMode(); - // NameResolver is started in the scheduled executor - timer.runDueTasks(); - } - - private ClientTransport channelTmGetTransportUnwrapped(EquivalentAddressGroup addressGroup) { - return ((ForwardingConnectionClientTransport) channel.tm.getTransport(addressGroup)).delegate(); } private static class FakeBackoffPolicyProvider implements BackoffPolicy.Provider { diff --git a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java index a719c39bbbf..40b89f59b1d 100644 --- a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java +++ b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java @@ -31,6 +31,10 @@ package io.grpc.internal; +import static io.grpc.ConnectivityState.CONNECTING; +import static io.grpc.ConnectivityState.READY; +import static io.grpc.ConnectivityState.TRANSIENT_FAILURE; +import static junit.framework.TestCase.assertNotSame; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -39,15 +43,13 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; -import static org.mockito.Matchers.isA; import static org.mockito.Matchers.same; import static org.mockito.Mockito.atLeast; -import static org.mockito.Mockito.atMost; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.inOrder; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; -import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoMoreInteractions; @@ -62,22 +64,27 @@ import io.grpc.Channel; import io.grpc.ClientCall; import io.grpc.ClientInterceptor; -import io.grpc.Compressor; import io.grpc.CompressorRegistry; +import io.grpc.ConnectivityStateInfo; import io.grpc.Context; import io.grpc.DecompressorRegistry; +import io.grpc.EquivalentAddressGroup; import io.grpc.IntegerMarshaller; -import io.grpc.LoadBalancer; +import io.grpc.LoadBalancer2; +import io.grpc.LoadBalancer2.Helper; +import io.grpc.LoadBalancer2.PickResult; +import io.grpc.LoadBalancer2.Subchannel; +import io.grpc.LoadBalancer2.SubchannelPicker; +import io.grpc.ManagedChannel; import io.grpc.Metadata; import io.grpc.MethodDescriptor; import io.grpc.NameResolver; -import io.grpc.PickFirstBalancerFactory; import io.grpc.ResolvedServerInfo; import io.grpc.ResolvedServerInfoGroup; import io.grpc.SecurityLevel; import io.grpc.Status; import io.grpc.StringMarshaller; -import io.grpc.TransportManager; +import io.grpc.internal.TestUtils.MockClientTransportInfo; import io.grpc.internal.testing.CensusTestUtils.FakeCensusContextFactory; import org.junit.After; @@ -89,6 +96,7 @@ import org.junit.runners.JUnit4; import org.mockito.ArgumentCaptor; import org.mockito.Captor; +import org.mockito.InOrder; import org.mockito.Matchers; import org.mockito.Mock; import org.mockito.MockitoAnnotations; @@ -102,44 +110,52 @@ import java.util.Collections; import java.util.LinkedList; import java.util.List; -import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -/** Unit tests for {@link ManagedChannelImpl}. */ +/** Unit tests for {@link ManagedChannelImpl2}. */ @RunWith(JUnit4.class) -public class ManagedChannelImplTest { +public class ManagedChannelImpl2Test { private static final List NO_INTERCEPTOR = Collections.emptyList(); private static final Attributes NAME_RESOLVER_PARAMS = Attributes.newBuilder().set(NameResolver.Factory.PARAMS_DEFAULT_PORT, 447).build(); - private final MethodDescriptor method = MethodDescriptor.create( + private static final MethodDescriptor method = MethodDescriptor.create( MethodDescriptor.MethodType.UNKNOWN, "/service/method", new StringMarshaller(), new IntegerMarshaller()); + private static final Attributes.Key SUBCHANNEL_ATTR_KEY = + Attributes.Key.of("subchannel-attr-key"); private final String serviceName = "fake.example.com"; private final String authority = serviceName; private final String userAgent = "userAgent"; private final String target = "fake://" + serviceName; private URI expectedUri; private final SocketAddress socketAddress = new SocketAddress() {}; + private final EquivalentAddressGroup addressGroup = new EquivalentAddressGroup(socketAddress); private final ResolvedServerInfo server = new ResolvedServerInfo(socketAddress, Attributes.EMPTY); private final FakeClock timer = new FakeClock(); private final FakeClock executor = new FakeClock(); private final FakeCensusContextFactory censusCtxFactory = new FakeCensusContextFactory(); - private SpyingLoadBalancerFactory loadBalancerFactory = - new SpyingLoadBalancerFactory(PickFirstBalancerFactory.getInstance()); @Rule public final ExpectedException thrown = ExpectedException.none(); - private ManagedChannelImpl channel; + private ManagedChannelImpl2 channel; + private Helper helper; @Captor private ArgumentCaptor statusCaptor; @Captor private ArgumentCaptor statsTraceCtxCaptor; @Mock - private ConnectionClientTransport mockTransport; + private LoadBalancer2.Factory mockLoadBalancerFactory; + @Mock + private LoadBalancer2 mockLoadBalancer; + @Captor + private ArgumentCaptor stateInfoCaptor; + @Mock + private SubchannelPicker mockPicker; @Mock private ClientTransportFactory mockTransportFactory; @Mock @@ -149,36 +165,39 @@ public class ManagedChannelImplTest { @Mock private ClientCall.Listener mockCallListener3; @Mock + private ClientCall.Listener mockCallListener4; + @Mock private SharedResourceHolder.Resource timerService; @Mock private CallCredentials creds; + private BlockingQueue transports; - private ArgumentCaptor transportListenerCaptor = - ArgumentCaptor.forClass(ManagedClientTransport.Listener.class); private ArgumentCaptor streamListenerCaptor = ArgumentCaptor.forClass(ClientStreamListener.class); private void createChannel( NameResolver.Factory nameResolverFactory, List interceptors) { - channel = new ManagedChannelImpl(target, new FakeBackoffPolicyProvider(), - nameResolverFactory, NAME_RESOLVER_PARAMS, loadBalancerFactory, + channel = new ManagedChannelImpl2(target, new FakeBackoffPolicyProvider(), + nameResolverFactory, NAME_RESOLVER_PARAMS, mockLoadBalancerFactory, mockTransportFactory, DecompressorRegistry.getDefaultInstance(), CompressorRegistry.getDefaultInstance(), timerService, timer.getStopwatchSupplier(), - ManagedChannelImpl.IDLE_TIMEOUT_MILLIS_DISABLE, + ManagedChannelImpl2.IDLE_TIMEOUT_MILLIS_DISABLE, executor.getScheduledExecutorService(), userAgent, interceptors, censusCtxFactory); // Force-exit the initial idle-mode channel.exitIdleMode(); - // Will start NameResolver in the scheduled executor - assertEquals(1, timer.runDueTasks()); + assertEquals(0, timer.numPendingTasks()); + + ArgumentCaptor helperCaptor = ArgumentCaptor.forClass(null); + verify(mockLoadBalancerFactory).newLoadBalancer(helperCaptor.capture()); + helper = helperCaptor.getValue(); } @Before public void setUp() throws Exception { MockitoAnnotations.initMocks(this); expectedUri = new URI(target); - when(mockTransportFactory.newClientTransport( - any(SocketAddress.class), any(String.class), any(String.class))) - .thenReturn(mockTransport); + when(mockLoadBalancerFactory.newLoadBalancer(any(Helper.class))).thenReturn(mockLoadBalancer); + transports = TestUtils.captureTransports(mockTransportFactory); when(timerService.create()).thenReturn(timer.getScheduledExecutorService()); } @@ -193,14 +212,14 @@ public void allPendingTasksAreRun() throws Exception { } /** - * The counterpart of {@link ManagedChannelImplIdlenessTest#enterIdleModeAfterForceExit}. + * The counterpart of {@link ManagedChannelImpl2IdlenessTest#enterIdleModeAfterForceExit}. */ @Test @SuppressWarnings("unchecked") public void idleModeDisabled() { createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); - assertEquals(1, loadBalancerFactory.balancers.size()); + // In this test suite, the channel is always created with idle mode disabled. // No task is scheduled to enter idle mode assertEquals(0, timer.numPendingTasks()); assertEquals(0, executor.numPendingTasks()); @@ -229,165 +248,137 @@ public void shutdownWithNoTransportsEverCreated() { } @Test - public void twoCallsAndGracefulShutdown() { + public void callsAndShutdown() { + subtestCallsAndShutdown(false, false); + } + + @Test + public void callsAndShutdownNow() { + subtestCallsAndShutdown(true, false); + } + + /** Make sure shutdownNow() after shutdown() has an effect. */ + @Test + public void callsAndShutdownAndShutdownNow() { + subtestCallsAndShutdown(false, true); + } + + private void subtestCallsAndShutdown(boolean shutdownNow, boolean shutdownNowAfterShutdown) { FakeNameResolverFactory nameResolverFactory = new FakeNameResolverFactory(true); createChannel(nameResolverFactory, NO_INTERCEPTOR); - ClientCall call = channel.newCall(method, CallOptions.DEFAULT); - verifyNoMoreInteractions(mockTransportFactory); - - // Create transport and call ClientStream mockStream = mock(ClientStream.class); + ClientStream mockStream2 = mock(ClientStream.class); Metadata headers = new Metadata(); - when(mockTransportFactory.newClientTransport( - any(SocketAddress.class), any(String.class), any(String.class))) - .thenReturn(mockTransport); + Metadata headers2 = new Metadata(); + + // Configure the picker so that first RPC goes to delayed transport, and second RPC goes to + // real transport. + Subchannel subchannel = helper.createSubchannel(addressGroup, Attributes.EMPTY); + subchannel.requestConnection(); + verify(mockTransportFactory).newClientTransport( + any(SocketAddress.class), any(String.class), any(String.class)); + MockClientTransportInfo transportInfo = transports.poll(); + ConnectionClientTransport mockTransport = transportInfo.transport; + verify(mockTransport).start(any(ManagedClientTransport.Listener.class)); + ManagedClientTransport.Listener transportListener = transportInfo.listener; when(mockTransport.newStream(same(method), same(headers), same(CallOptions.DEFAULT), any(StatsTraceContext.class))) .thenReturn(mockStream); - call.start(mockCallListener, headers); - timer.runDueTasks(); - executor.runDueTasks(); - - verify(mockTransportFactory) - .newClientTransport(same(socketAddress), eq(authority), eq(userAgent)); - verify(mockTransport).start(transportListenerCaptor.capture()); - ManagedClientTransport.Listener transportListener = transportListenerCaptor.getValue(); + when(mockTransport.newStream(same(method), same(headers2), same(CallOptions.DEFAULT), + any(StatsTraceContext.class))) + .thenReturn(mockStream2); transportListener.transportReady(); - executor.runDueTasks(); + when(mockPicker.pickSubchannel(any(Attributes.class), same(headers))).thenReturn( + PickResult.withNoResult()); + when(mockPicker.pickSubchannel(any(Attributes.class), same(headers2))).thenReturn( + PickResult.withSubchannel(subchannel)); + helper.updatePicker(mockPicker); - verify(mockTransport).newStream(same(method), same(headers), same(CallOptions.DEFAULT), - statsTraceCtxCaptor.capture()); - assertEquals(censusCtxFactory.pollContextOrFail(), - statsTraceCtxCaptor.getValue().getCensusContext()); - verify(mockStream).start(streamListenerCaptor.capture()); - verify(mockStream).setCompressor(isA(Compressor.class)); - ClientStreamListener streamListener = streamListenerCaptor.getValue(); + // First RPC, will be pending + ClientCall call = channel.newCall(method, CallOptions.DEFAULT); + verifyNoMoreInteractions(mockTransportFactory); + call.start(mockCallListener, headers); - // Second call + verify(mockTransport, never()).newStream(same(method), same(headers), same(CallOptions.DEFAULT), + any(StatsTraceContext.class)); + censusCtxFactory.pollContextOrFail(); + + // Second RPC, will be assigned to the real transport ClientCall call2 = channel.newCall(method, CallOptions.DEFAULT); - ClientStream mockStream2 = mock(ClientStream.class); - Metadata headers2 = new Metadata(); - when(mockTransport.newStream(same(method), same(headers2), same(CallOptions.DEFAULT), - any(StatsTraceContext.class))) - .thenReturn(mockStream2); call2.start(mockCallListener2, headers2); verify(mockTransport).newStream(same(method), same(headers2), same(CallOptions.DEFAULT), statsTraceCtxCaptor.capture()); assertEquals(censusCtxFactory.pollContextOrFail(), statsTraceCtxCaptor.getValue().getCensusContext()); - - verify(mockStream2).start(streamListenerCaptor.capture()); - ClientStreamListener streamListener2 = streamListenerCaptor.getValue(); - Metadata trailers = new Metadata(); - streamListener2.closed(Status.CANCELLED, trailers); - executor.runDueTasks(); - - verify(mockCallListener2).onClose(Status.CANCELLED, trailers); + verify(mockTransport).newStream(same(method), same(headers2), same(CallOptions.DEFAULT), + statsTraceCtxCaptor.capture()); + verify(mockStream2).start(any(ClientStreamListener.class)); // Shutdown - channel.shutdown(); + if (shutdownNow) { + channel.shutdownNow(); + } else { + channel.shutdown(); + if (shutdownNowAfterShutdown) { + channel.shutdownNow(); + shutdownNow = true; + } + } assertTrue(channel.isShutdown()); assertFalse(channel.isTerminated()); - verify(mockTransport).shutdown(); assertEquals(1, nameResolverFactory.resolvers.size()); - assertTrue(nameResolverFactory.resolvers.get(0).shutdown); - assertEquals(1, loadBalancerFactory.balancers.size()); - verify(loadBalancerFactory.balancers.get(0)).shutdown(); + verify(mockLoadBalancerFactory).newLoadBalancer(any(Helper.class)); // Further calls should fail without going to the transport ClientCall call3 = channel.newCall(method, CallOptions.DEFAULT); - call3.start(mockCallListener3, new Metadata()); + call3.start(mockCallListener3, headers2); timer.runDueTasks(); executor.runDueTasks(); verify(mockCallListener3).onClose(statusCaptor.capture(), any(Metadata.class)); assertSame(Status.Code.UNAVAILABLE, statusCaptor.getValue().getCode()); - // Finish shutdown - transportListener.transportShutdown(Status.CANCELLED); - assertFalse(channel.isTerminated()); - streamListener.closed(Status.CANCELLED, trailers); - executor.runDueTasks(); - - verify(mockCallListener).onClose(Status.CANCELLED, trailers); - assertFalse(channel.isTerminated()); - - transportListener.transportTerminated(); - assertTrue(channel.isTerminated()); - - verify(mockTransportFactory).close(); - verifyNoMoreInteractions(mockTransportFactory); - verify(mockTransport, atLeast(0)).getLogId(); - verifyNoMoreInteractions(mockTransport); - verifyNoMoreInteractions(mockStream); - } - - @Test - public void callAndShutdownNow() { - FakeNameResolverFactory nameResolverFactory = new FakeNameResolverFactory(true); - createChannel(nameResolverFactory, NO_INTERCEPTOR); - verifyNoMoreInteractions(mockTransportFactory); - ClientCall call = channel.newCall(method, CallOptions.DEFAULT); - verifyNoMoreInteractions(mockTransportFactory); - - // Create transport and call - ClientStream mockStream = mock(ClientStream.class); - Metadata headers = new Metadata(); - when(mockTransportFactory.newClientTransport( - any(SocketAddress.class), any(String.class), any(String.class))) - .thenReturn(mockTransport); - when(mockTransport.newStream(same(method), same(headers), same(CallOptions.DEFAULT), - any(StatsTraceContext.class))) - .thenReturn(mockStream); - call.start(mockCallListener, headers); - timer.runDueTasks(); - executor.runDueTasks(); - - verify(mockTransportFactory) - .newClientTransport(same(socketAddress), eq(authority), any(String.class)); - verify(mockTransport).start(transportListenerCaptor.capture()); - ManagedClientTransport.Listener transportListener = transportListenerCaptor.getValue(); - transportListener.transportReady(); - executor.runDueTasks(); - - verify(mockTransport).newStream(same(method), same(headers), same(CallOptions.DEFAULT), - any(StatsTraceContext.class)); - - verify(mockStream).start(streamListenerCaptor.capture()); - verify(mockStream).setCompressor(isA(Compressor.class)); - ClientStreamListener streamListener = streamListenerCaptor.getValue(); + if (shutdownNow) { + // LoadBalancer and NameResolver are shut down as soon as delayed transport is terminated. + verify(mockLoadBalancer).shutdown(); + assertTrue(nameResolverFactory.resolvers.get(0).shutdown); + // call should have been aborted by delayed transport + executor.runDueTasks(); + verify(mockCallListener).onClose(same(ManagedChannelImpl2.SHUTDOWN_NOW_STATUS), + any(Metadata.class)); + } else { + // LoadBalancer and NameResolver are still running. + verify(mockLoadBalancer, never()).shutdown(); + assertFalse(nameResolverFactory.resolvers.get(0).shutdown); + // call and call2 are still alive, and can still be assigned to a real transport + SubchannelPicker picker2 = mock(SubchannelPicker.class); + when(picker2.pickSubchannel(any(Attributes.class), same(headers))).thenReturn( + PickResult.withSubchannel(subchannel)); + helper.updatePicker(picker2); + executor.runDueTasks(); + verify(mockTransport).newStream(same(method), same(headers), same(CallOptions.DEFAULT), + any(StatsTraceContext.class)); + verify(mockStream).start(any(ClientStreamListener.class)); + } - // ShutdownNow - channel.shutdownNow(); - assertTrue(channel.isShutdown()); - assertFalse(channel.isTerminated()); - // ShutdownNow may or may not invoke shutdown. Ideally it wouldn't, but it doesn't matter much - // either way. - verify(mockTransport, atMost(1)).shutdown(); - verify(mockTransport).shutdownNow(any(Status.class)); - assertEquals(1, nameResolverFactory.resolvers.size()); + // After call is moved out of delayed transport, LoadBalancer, NameResolver and the transports + // will be shutdown. + verify(mockLoadBalancer).shutdown(); assertTrue(nameResolverFactory.resolvers.get(0).shutdown); - assertEquals(1, loadBalancerFactory.balancers.size()); - verify(loadBalancerFactory.balancers.get(0)).shutdown(); - - // Further calls should fail without going to the transport - ClientCall call3 = channel.newCall(method, CallOptions.DEFAULT); - call3.start(mockCallListener3, new Metadata()); - executor.runDueTasks(); - - verify(mockCallListener3).onClose(statusCaptor.capture(), any(Metadata.class)); - assertSame(Status.Code.UNAVAILABLE, statusCaptor.getValue().getCode()); - // Finish shutdown - transportListener.transportShutdown(Status.CANCELLED); - assertFalse(channel.isTerminated()); - Metadata trailers = new Metadata(); - streamListener.closed(Status.CANCELLED, trailers); - executor.runDueTasks(); + if (shutdownNow) { + // Channel shutdownNow() all subchannels after shutting down LoadBalancer + verify(mockTransport).shutdownNow(ManagedChannelImpl2.SHUTDOWN_NOW_STATUS); + } else { + verify(mockTransport, never()).shutdownNow(any(Status.class)); + } + // LoadBalancer should shutdown the subchannel + subchannel.shutdown(); + verify(mockTransport).shutdown(); - verify(mockCallListener).onClose(Status.CANCELLED, trailers); + // Killing the remaining real transport will terminate the channel + transportListener.transportShutdown(Status.UNAVAILABLE); assertFalse(channel.isTerminated()); - transportListener.transportTerminated(); assertTrue(channel.isTerminated()); @@ -395,56 +386,8 @@ public void callAndShutdownNow() { verifyNoMoreInteractions(mockTransportFactory); verify(mockTransport, atLeast(0)).getLogId(); verifyNoMoreInteractions(mockTransport); - verifyNoMoreInteractions(mockStream); - } - - /** Make sure shutdownNow() after shutdown() has an effect. */ - @Test - public void callAndShutdownAndShutdownNow() { - createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); - ClientCall call = channel.newCall(method, CallOptions.DEFAULT); - - // Create transport and call - ClientStream mockStream = mock(ClientStream.class); - Metadata headers = new Metadata(); - when(mockTransport.newStream(same(method), same(headers), same(CallOptions.DEFAULT), - any(StatsTraceContext.class))) - .thenReturn(mockStream); - call.start(mockCallListener, headers); - timer.runDueTasks(); - executor.runDueTasks(); - - verify(mockTransport).start(transportListenerCaptor.capture()); - ManagedClientTransport.Listener transportListener = transportListenerCaptor.getValue(); - transportListener.transportReady(); - executor.runDueTasks(); - - verify(mockStream).start(streamListenerCaptor.capture()); - ClientStreamListener streamListener = streamListenerCaptor.getValue(); - - // ShutdownNow - channel.shutdown(); - channel.shutdownNow(); - // ShutdownNow may or may not invoke shutdown. Ideally it wouldn't, but it doesn't matter much - // either way. - verify(mockTransport, atMost(2)).shutdown(); - verify(mockTransport).shutdownNow(any(Status.class)); - - // Finish shutdown - transportListener.transportShutdown(Status.CANCELLED); - assertFalse(channel.isTerminated()); - Metadata trailers = new Metadata(); - streamListener.closed(Status.CANCELLED, trailers); - executor.runDueTasks(); - - verify(mockCallListener).onClose(Status.CANCELLED, trailers); - assertFalse(channel.isTerminated()); - - transportListener.transportTerminated(); - assertTrue(channel.isTerminated()); } - @Test public void interceptor() throws Exception { final AtomicLong atomic = new AtomicLong(); @@ -462,92 +405,51 @@ public ClientCall interceptCall( assertEquals(1, atomic.get()); } - @Test - public void testNoDeadlockOnShutdown() { - createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); - // Force creation of transport - ClientCall call = channel.newCall(method, CallOptions.DEFAULT); - Metadata headers = new Metadata(); - ClientStream mockStream = mock(ClientStream.class); - when(mockTransport.newStream(same(method), same(headers))).thenReturn(mockStream); - call.start(mockCallListener, headers); - timer.runDueTasks(); - executor.runDueTasks(); - call.cancel("Cancel for test", null); - executor.runDueTasks(); - - verify(mockTransport).start(transportListenerCaptor.capture()); - final ManagedClientTransport.Listener transportListener = transportListenerCaptor.getValue(); - final Object lock = new Object(); - final CyclicBarrier barrier = new CyclicBarrier(2); - new Thread() { - @Override - public void run() { - synchronized (lock) { - try { - barrier.await(); - } catch (Exception ex) { - throw new AssertionError(ex); - } - // To deadlock, a lock would be needed for this call to proceed. - transportListener.transportShutdown(Status.CANCELLED); - } - } - }.start(); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) { - // To deadlock, a lock would need to be held while this method is in progress. - try { - barrier.await(); - } catch (Exception ex) { - throw new AssertionError(ex); - } - // If deadlock is possible with this setup, this sychronization completes the loop because - // the transportShutdown needs a lock that Channel is holding while calling this method. - synchronized (lock) { - } - return null; - } - }).when(mockTransport).shutdown(); - channel.shutdown(); - - transportListener.transportTerminated(); - } - @Test public void callOptionsExecutor() { Metadata headers = new Metadata(); ClientStream mockStream = mock(ClientStream.class); - when(mockTransport.newStream(same(method), same(headers), any(CallOptions.class), - any(StatsTraceContext.class))) - .thenReturn(mockStream); FakeClock callExecutor = new FakeClock(); createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + + // Start a call with a call executor CallOptions options = CallOptions.DEFAULT.withExecutor(callExecutor.getScheduledExecutorService()); - ClientCall call = channel.newCall(method, options); call.start(mockCallListener, headers); - timer.runDueTasks(); - executor.runDueTasks(); - verify(mockTransport).start(transportListenerCaptor.capture()); - assertEquals(0, executor.numPendingTasks()); - transportListenerCaptor.getValue().transportReady(); - // Real streams are started in the channel's executor - assertEquals(1, executor.runDueTasks()); + // Make the transport available + Subchannel subchannel = helper.createSubchannel(addressGroup, Attributes.EMPTY); + verify(mockTransportFactory, never()).newClientTransport( + any(SocketAddress.class), any(String.class), any(String.class)); + subchannel.requestConnection(); + verify(mockTransportFactory).newClientTransport( + any(SocketAddress.class), any(String.class), any(String.class)); + MockClientTransportInfo transportInfo = transports.poll(); + ConnectionClientTransport mockTransport = transportInfo.transport; + ManagedClientTransport.Listener transportListener = transportInfo.listener; + when(mockTransport.newStream(same(method), same(headers), any(CallOptions.class), + any(StatsTraceContext.class))) + .thenReturn(mockStream); + transportListener.transportReady(); + when(mockPicker.pickSubchannel(any(Attributes.class), any(Metadata.class))) + .thenReturn(PickResult.withSubchannel(subchannel)); + assertEquals(0, callExecutor.numPendingTasks()); + helper.updatePicker(mockPicker); + // Real streams are started in the call executor if they were previously buffered. + assertEquals(1, callExecutor.runDueTasks()); verify(mockTransport).newStream(same(method), same(headers), same(options), any(StatsTraceContext.class)); verify(mockStream).start(streamListenerCaptor.capture()); + + // Call listener callbacks are also run in the call executor ClientStreamListener streamListener = streamListenerCaptor.getValue(); Metadata trailers = new Metadata(); assertEquals(0, callExecutor.numPendingTasks()); streamListener.closed(Status.CANCELLED, trailers); verify(mockCallListener, never()).onClose(same(Status.CANCELLED), same(trailers)); assertEquals(1, callExecutor.runDueTasks()); - verify(mockCallListener).onClose(same(Status.CANCELLED), same(trailers)); } @@ -557,42 +459,21 @@ public void nameResolutionFailed() { // Name resolution is started as soon as channel is created. createChannel(new FailingNameResolverFactory(error), NO_INTERCEPTOR); - ClientCall call = channel.newCall(method, CallOptions.DEFAULT); - call.start(mockCallListener, new Metadata()); - timer.runDueTasks(); - executor.runDueTasks(); - - // The call failed with the name resolution error - verify(mockCallListener).onClose(statusCaptor.capture(), any(Metadata.class)); - Status status = statusCaptor.getValue(); - assertSame(error.getCode(), status.getCode()); - assertSame(error.getCause(), status.getCause()); - // LoadBalancer received the same error - assertEquals(1, loadBalancerFactory.balancers.size()); - verify(loadBalancerFactory.balancers.get(0)).handleNameResolutionError(same(error)); + verify(mockLoadBalancer).handleNameResolutionError(same(error)); } @Test public void nameResolverReturnsEmptySubLists() { String errorDescription = "NameResolver returned an empty list"; - // Name resolution is started as soon as channel is created + // Pass a FakeNameResolverFactory with an empty list createChannel(new FakeNameResolverFactory(), NO_INTERCEPTOR); - ClientCall call = channel.newCall(method, CallOptions.DEFAULT); - call.start(mockCallListener, new Metadata()); - timer.runDueTasks(); - executor.runDueTasks(); - // The call failed with the name resolution error - verify(mockCallListener).onClose(statusCaptor.capture(), any(Metadata.class)); + // LoadBalancer received the error + verify(mockLoadBalancerFactory).newLoadBalancer(any(Helper.class)); + verify(mockLoadBalancer).handleNameResolutionError(statusCaptor.capture()); Status status = statusCaptor.getValue(); assertSame(Status.Code.UNAVAILABLE, status.getCode()); - assertTrue(status.getDescription(), status.getDescription().contains(errorDescription)); - // LoadBalancer received the same error - assertEquals(1, loadBalancerFactory.balancers.size()); - verify(loadBalancerFactory.balancers.get(0)).handleNameResolutionError(statusCaptor.capture()); - status = statusCaptor.getValue(); - assertSame(Status.Code.UNAVAILABLE, status.getCode()); assertEquals(errorDescription, status.getDescription()); } @@ -602,30 +483,19 @@ public void loadBalancerThrowsInHandleResolvedAddresses() { // Delay the success of name resolution until allResolved() is called FakeNameResolverFactory nameResolverFactory = new FakeNameResolverFactory(false); createChannel(nameResolverFactory, NO_INTERCEPTOR); - ClientCall call = channel.newCall(method, CallOptions.DEFAULT); - call.start(mockCallListener, new Metadata()); - timer.runDueTasks(); - executor.runDueTasks(); - assertEquals(1, loadBalancerFactory.balancers.size()); - LoadBalancer loadBalancer = loadBalancerFactory.balancers.get(0); - doThrow(ex).when(loadBalancer).handleResolvedAddresses( + verify(mockLoadBalancerFactory).newLoadBalancer(any(Helper.class)); + doThrow(ex).when(mockLoadBalancer).handleResolvedAddresses( Matchers.>anyObject(), any(Attributes.class)); // NameResolver returns addresses. nameResolverFactory.allResolved(); - executor.runDueTasks(); - // The call failed with the error thrown from handleResolvedAddresses() - verify(mockCallListener).onClose(statusCaptor.capture(), any(Metadata.class)); + // The LoadBalancer will receive the error that it has thrown. + verify(mockLoadBalancer).handleNameResolutionError(statusCaptor.capture()); Status status = statusCaptor.getValue(); assertSame(Status.Code.INTERNAL, status.getCode()); assertSame(ex, status.getCause()); - // The LoadBalancer received the same error - verify(loadBalancer).handleNameResolutionError(statusCaptor.capture()); - status = statusCaptor.getValue(); - assertSame(Status.Code.INTERNAL, status.getCode()); - assertSame(ex, status.getCause()); } @Test @@ -633,22 +503,16 @@ public void nameResolvedAfterChannelShutdown() { // Delay the success of name resolution until allResolved() is called. FakeNameResolverFactory nameResolverFactory = new FakeNameResolverFactory(false); createChannel(nameResolverFactory, NO_INTERCEPTOR); - ClientCall call = channel.newCall(method, CallOptions.DEFAULT); - Metadata headers = new Metadata(); - call.start(mockCallListener, headers); - timer.runDueTasks(); - executor.runDueTasks(); channel.shutdown(); assertTrue(channel.isShutdown()); + assertTrue(channel.isTerminated()); + verify(mockLoadBalancer).shutdown(); // Name resolved after the channel is shut down, which is possible if the name resolution takes - // time and is not cancellable. The resolved address will still be passed to the LoadBalancer. + // time and is not cancellable. The resolved address will be dropped. nameResolverFactory.allResolved(); - executor.runDueTasks(); - - verify(mockTransportFactory, never()) - .newClientTransport(any(SocketAddress.class), any(String.class), any(String.class)); + verifyNoMoreInteractions(mockLoadBalancer); } /** @@ -669,56 +533,74 @@ public void firstResolvedServerFailedToConnect() throws Exception { }; final ResolvedServerInfo goodServer = new ResolvedServerInfo(goodAddress, Attributes.EMPTY); final ResolvedServerInfo badServer = new ResolvedServerInfo(badAddress, Attributes.EMPTY); - final ConnectionClientTransport goodTransport = mock(ConnectionClientTransport.class); - final ConnectionClientTransport badTransport = mock(ConnectionClientTransport.class); - when(goodTransport.newStream( - any(MethodDescriptor.class), any(Metadata.class), any(CallOptions.class), - any(StatsTraceContext.class))) - .thenReturn(mock(ClientStream.class)); - when(mockTransportFactory.newClientTransport( - same(goodAddress), any(String.class), any(String.class))) - .thenReturn(goodTransport); - when(mockTransportFactory.newClientTransport( - same(badAddress), any(String.class), any(String.class))) - .thenReturn(badTransport); + InOrder inOrder = inOrder(mockLoadBalancer); + ResolvedServerInfoGroup serverInfoGroup = ResolvedServerInfoGroup.builder() + .add(badServer) + .add(goodServer) + .build(); FakeNameResolverFactory nameResolverFactory = - new FakeNameResolverFactory(Arrays.asList(badServer, goodServer)); + new FakeNameResolverFactory(serverInfoGroup.getResolvedServerInfoList()); createChannel(nameResolverFactory, NO_INTERCEPTOR); + + // Start the call ClientCall call = channel.newCall(method, CallOptions.DEFAULT); Metadata headers = new Metadata(); - - // Start a call. The channel will starts with the first address (badAddress) call.start(mockCallListener, headers); - timer.runDueTasks(); executor.runDueTasks(); - ArgumentCaptor badTransportListenerCaptor = - ArgumentCaptor.forClass(ManagedClientTransport.Listener.class); - verify(badTransport).start(badTransportListenerCaptor.capture()); + // Simulate name resolution results + inOrder.verify(mockLoadBalancer).handleResolvedAddresses( + eq(Arrays.asList(serverInfoGroup)), eq(Attributes.EMPTY)); + Subchannel subchannel = helper.createSubchannel( + serverInfoGroup.toEquivalentAddressGroup(), Attributes.EMPTY); + when(mockPicker.pickSubchannel(any(Attributes.class), any(Metadata.class))) + .thenReturn(PickResult.withSubchannel(subchannel)); + subchannel.requestConnection(); + inOrder.verify(mockLoadBalancer).handleSubchannelState( + same(subchannel), stateInfoCaptor.capture()); + assertEquals(CONNECTING, stateInfoCaptor.getValue().getState()); + + // The channel will starts with the first address (badAddress) verify(mockTransportFactory) .newClientTransport(same(badAddress), any(String.class), any(String.class)); verify(mockTransportFactory, times(0)) .newClientTransport(same(goodAddress), any(String.class), any(String.class)); - badTransportListenerCaptor.getValue().transportShutdown(Status.UNAVAILABLE); + + MockClientTransportInfo badTransportInfo = transports.poll(); + // Which failed to connect + badTransportInfo.listener.transportShutdown(Status.UNAVAILABLE); + inOrder.verifyNoMoreInteractions(); // The channel then try the second address (goodAddress) - ArgumentCaptor goodTransportListenerCaptor = - ArgumentCaptor.forClass(ManagedClientTransport.Listener.class); verify(mockTransportFactory) .newClientTransport(same(goodAddress), any(String.class), any(String.class)); - verify(goodTransport).start(goodTransportListenerCaptor.capture()); - goodTransportListenerCaptor.getValue().transportReady(); + MockClientTransportInfo goodTransportInfo = transports.poll(); + when(goodTransportInfo.transport.newStream( + any(MethodDescriptor.class), any(Metadata.class), any(CallOptions.class), + any(StatsTraceContext.class))) + .thenReturn(mock(ClientStream.class)); + + goodTransportInfo.listener.transportReady(); + inOrder.verify(mockLoadBalancer).handleSubchannelState( + same(subchannel), stateInfoCaptor.capture()); + assertEquals(READY, stateInfoCaptor.getValue().getState()); + + // A typical LoadBalancer will call this once the subchannel becomes READY + helper.updatePicker(mockPicker); + // Delayed transport uses the app executor to create real streams. executor.runDueTasks(); - verify(goodTransport).newStream(same(method), same(headers), same(CallOptions.DEFAULT), - any(StatsTraceContext.class)); + verify(goodTransportInfo.transport).newStream(same(method), same(headers), + same(CallOptions.DEFAULT), any(StatsTraceContext.class)); // The bad transport was never used. - verify(badTransport, times(0)).newStream(any(MethodDescriptor.class), any(Metadata.class)); + verify(badTransportInfo.transport, times(0)).newStream(any(MethodDescriptor.class), + any(Metadata.class), any(CallOptions.class), any(StatsTraceContext.class)); } /** - * Verify that if all resolved addresses failed to connect, the call will fail. + * Verify that if all resolved addresses failed to connect, a fail-fast call will fail, while a + * wait-for-ready call will still be buffered. */ @Test public void allServersFailedToConnect() throws Exception { @@ -734,121 +616,218 @@ public void allServersFailedToConnect() throws Exception { }; final ResolvedServerInfo server1 = new ResolvedServerInfo(addr1, Attributes.EMPTY); final ResolvedServerInfo server2 = new ResolvedServerInfo(addr2, Attributes.EMPTY); - final ConnectionClientTransport transport1 = mock(ConnectionClientTransport.class); - final ConnectionClientTransport transport2 = mock(ConnectionClientTransport.class); - when(mockTransportFactory.newClientTransport(same(addr1), any(String.class), any(String.class))) - .thenReturn(transport1); - when(mockTransportFactory.newClientTransport(same(addr2), any(String.class), any(String.class))) - .thenReturn(transport2); + InOrder inOrder = inOrder(mockLoadBalancer); + + ResolvedServerInfoGroup serverInfoGroup = ResolvedServerInfoGroup.builder() + .add(server1) + .add(server2) + .build(); FakeNameResolverFactory nameResolverFactory = - new FakeNameResolverFactory(Arrays.asList(server1, server2)); + new FakeNameResolverFactory(serverInfoGroup.getResolvedServerInfoList()); createChannel(nameResolverFactory, NO_INTERCEPTOR); - ClientCall call = channel.newCall(method, CallOptions.DEFAULT); - Metadata headers = new Metadata(); - // Start a call. The channel will starts with the first address, which will fail to connect. + // Start a wait-for-ready call + ClientCall call = + channel.newCall(method, CallOptions.DEFAULT.withWaitForReady()); + Metadata headers = new Metadata(); call.start(mockCallListener, headers); - timer.runDueTasks(); + // ... and a fail-fast call + ClientCall call2 = + channel.newCall(method, CallOptions.DEFAULT.withoutWaitForReady()); + call2.start(mockCallListener2, headers); executor.runDueTasks(); - verify(transport1).start(transportListenerCaptor.capture()); + // Simulate name resolution results + inOrder.verify(mockLoadBalancer).handleResolvedAddresses( + eq(Arrays.asList(serverInfoGroup)), eq(Attributes.EMPTY)); + Subchannel subchannel = helper.createSubchannel( + serverInfoGroup.toEquivalentAddressGroup(), Attributes.EMPTY); + when(mockPicker.pickSubchannel(any(Attributes.class), any(Metadata.class))) + .thenReturn(PickResult.withSubchannel(subchannel)); + subchannel.requestConnection(); + inOrder.verify(mockLoadBalancer).handleSubchannelState( + same(subchannel), stateInfoCaptor.capture()); + assertEquals(CONNECTING, stateInfoCaptor.getValue().getState()); + + // Connecting to server1, which will fail verify(mockTransportFactory) .newClientTransport(same(addr1), any(String.class), any(String.class)); verify(mockTransportFactory, times(0)) .newClientTransport(same(addr2), any(String.class), any(String.class)); - transportListenerCaptor.getValue().transportShutdown(Status.UNAVAILABLE); + MockClientTransportInfo transportInfo1 = transports.poll(); + transportInfo1.listener.transportShutdown(Status.UNAVAILABLE); - // The channel then try the second address, which will fail to connect too. - verify(transport2).start(transportListenerCaptor.capture()); + // Connecting to server2, which will fail too verify(mockTransportFactory) .newClientTransport(same(addr2), any(String.class), any(String.class)); - verify(transport2).start(transportListenerCaptor.capture()); - transportListenerCaptor.getValue().transportShutdown(Status.UNAVAILABLE); + MockClientTransportInfo transportInfo2 = transports.poll(); + Status server2Error = Status.UNAVAILABLE.withDescription("Server2 failed to connect"); + transportInfo2.listener.transportShutdown(server2Error); + + // ... which makes the subchannel enter TRANSIENT_FAILURE. The last error Status is propagated + // to LoadBalancer. + inOrder.verify(mockLoadBalancer).handleSubchannelState( + same(subchannel), stateInfoCaptor.capture()); + assertEquals(TRANSIENT_FAILURE, stateInfoCaptor.getValue().getState()); + assertSame(server2Error, stateInfoCaptor.getValue().getStatus()); + + // A typical LoadBalancer would create a picker with error + SubchannelPicker picker2 = mock(SubchannelPicker.class); + when(picker2.pickSubchannel(any(Attributes.class), any(Metadata.class))) + .thenReturn(PickResult.withError(server2Error)); + helper.updatePicker(picker2); executor.runDueTasks(); - // Call fails - verify(mockCallListener).onClose(statusCaptor.capture(), any(Metadata.class)); - assertEquals(Status.Code.UNAVAILABLE, statusCaptor.getValue().getCode()); + // ... which fails the fail-fast call + verify(mockCallListener2).onClose(same(server2Error), any(Metadata.class)); + // ... while the wait-for-ready call stays + verifyNoMoreInteractions(mockCallListener); // No real stream was ever created - verify(transport1, times(0)).newStream(any(MethodDescriptor.class), any(Metadata.class)); - verify(transport2, times(0)).newStream(any(MethodDescriptor.class), any(Metadata.class)); + verify(transportInfo1.transport, times(0)) + .newStream(any(MethodDescriptor.class), any(Metadata.class)); + verify(transportInfo2.transport, times(0)) + .newStream(any(MethodDescriptor.class), any(Metadata.class)); } - /** - * Verify that if the first resolved address points to a server that is at first connected, but - * disconnected later, all calls will stick to the first address. - */ @Test - public void firstResolvedServerConnectedThenDisconnected() throws Exception { - final SocketAddress addr1 = new SocketAddress() { - @Override public String toString() { - return "addr1"; - } - }; - final SocketAddress addr2 = new SocketAddress() { - @Override public String toString() { - return "addr2"; - } - }; - final ResolvedServerInfo server1 = new ResolvedServerInfo(addr1, Attributes.EMPTY); - final ResolvedServerInfo server2 = new ResolvedServerInfo(addr2, Attributes.EMPTY); - // Addr1 will have two transports throughout this test. - final ConnectionClientTransport transport1 = mock(ConnectionClientTransport.class); - final ConnectionClientTransport transport2 = mock(ConnectionClientTransport.class); - when(transport1.newStream( - any(MethodDescriptor.class), any(Metadata.class), any(CallOptions.class), - any(StatsTraceContext.class))) - .thenReturn(mock(ClientStream.class)); - when(transport2.newStream( - any(MethodDescriptor.class), any(Metadata.class), any(CallOptions.class), - any(StatsTraceContext.class))) - .thenReturn(mock(ClientStream.class)); - when(mockTransportFactory.newClientTransport(same(addr1), any(String.class), any(String.class))) - .thenReturn(transport1, transport2); + public void subchannels() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); - FakeNameResolverFactory nameResolverFactory = - new FakeNameResolverFactory(Arrays.asList(server1, server2)); - createChannel(nameResolverFactory, NO_INTERCEPTOR); - ClientCall call = channel.newCall(method, CallOptions.DEFAULT); - Metadata headers = new Metadata(); + // createSubchannel() always return a new Subchannel + Attributes attrs1 = Attributes.newBuilder().set(SUBCHANNEL_ATTR_KEY, "attr1").build(); + Attributes attrs2 = Attributes.newBuilder().set(SUBCHANNEL_ATTR_KEY, "attr2").build(); + Subchannel sub1 = helper.createSubchannel(addressGroup, attrs1); + Subchannel sub2 = helper.createSubchannel(addressGroup, attrs2); + assertNotSame(sub1, sub2); + assertNotSame(attrs1, attrs2); + assertSame(attrs1, sub1.getAttributes()); + assertSame(attrs2, sub2.getAttributes()); + assertSame(addressGroup, sub1.getAddresses()); + assertSame(addressGroup, sub2.getAddresses()); + + // requestConnection() + verify(mockTransportFactory, never()).newClientTransport( + any(SocketAddress.class), any(String.class), any(String.class)); + sub1.requestConnection(); + verify(mockTransportFactory).newClientTransport(socketAddress, authority, userAgent); + MockClientTransportInfo transportInfo1 = transports.poll(); + assertNotNull(transportInfo1); + + sub2.requestConnection(); + verify(mockTransportFactory, times(2)).newClientTransport(socketAddress, authority, userAgent); + MockClientTransportInfo transportInfo2 = transports.poll(); + assertNotNull(transportInfo2); + + sub1.requestConnection(); + sub2.requestConnection(); + verify(mockTransportFactory, times(2)).newClientTransport(socketAddress, authority, userAgent); + + // shutdown() has a delay + sub1.shutdown(); + timer.forwardTime(ManagedChannelImpl2.SUBCHANNEL_SHUTDOWN_DELAY_SECONDS - 1, TimeUnit.SECONDS); + sub1.shutdown(); + verify(transportInfo1.transport, never()).shutdown(); + timer.forwardTime(1, TimeUnit.SECONDS); + verify(transportInfo1.transport).shutdown(); + + // ... but not after Channel is terminating + verify(mockLoadBalancer, never()).shutdown(); + channel.shutdown(); + verify(mockLoadBalancer).shutdown(); + verify(transportInfo2.transport, never()).shutdown(); - // First call will use the first address - call.start(mockCallListener, headers); - timer.runDueTasks(); - executor.runDueTasks(); + sub2.shutdown(); + verify(transportInfo2.transport).shutdown(); + } - verify(mockTransportFactory) - .newClientTransport(same(addr1), any(String.class), any(String.class)); - verify(transport1).start(transportListenerCaptor.capture()); - transportListenerCaptor.getValue().transportReady(); - executor.runDueTasks(); + @Test + public void oobchannels() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + FakeClock oobExecutor = new FakeClock(); - verify(transport1).newStream(same(method), same(headers), same(CallOptions.DEFAULT), - any(StatsTraceContext.class)); - transportListenerCaptor.getValue().transportShutdown(Status.UNAVAILABLE); + ManagedChannel oob = helper.createOobChannel(addressGroup, "oobauthority", + oobExecutor.getScheduledExecutorService()); + assertEquals("oobauthority", oob.authority()); - // Second call still use the first address, since it was successfully connected. - ClientCall call2 = channel.newCall(method, CallOptions.DEFAULT); - call2.start(mockCallListener, headers); - verify(transport2).start(transportListenerCaptor.capture()); - verify(mockTransportFactory, times(2)) - .newClientTransport(same(addr1), any(String.class), any(String.class)); - transportListenerCaptor.getValue().transportReady(); - executor.runDueTasks(); + // OOB channels create connections lazily. A new call will initiate the connection. + Metadata headers = new Metadata(); + ClientCall call = oob.newCall(method, CallOptions.DEFAULT); + call.start(mockCallListener, headers); + verify(mockTransportFactory).newClientTransport(socketAddress, "oobauthority", userAgent); + MockClientTransportInfo transportInfo = transports.poll(); + assertNotNull(transportInfo); + + assertEquals(0, oobExecutor.numPendingTasks()); + transportInfo.listener.transportReady(); + assertEquals(1, oobExecutor.runDueTasks()); + verify(transportInfo.transport).newStream(same(method), same(headers), + same(CallOptions.DEFAULT), any(StatsTraceContext.class)); + + // The transport goes away + transportInfo.listener.transportShutdown(Status.UNAVAILABLE); + transportInfo.listener.transportTerminated(); + + // A new call will trigger a new transport + ClientCall call2 = oob.newCall(method, CallOptions.DEFAULT); + call2.start(mockCallListener2, headers); + ClientCall call3 = oob.newCall(method, CallOptions.DEFAULT.withWaitForReady()); + call3.start(mockCallListener3, headers); + verify(mockTransportFactory, times(2)).newClientTransport( + socketAddress, "oobauthority", userAgent); + transportInfo = transports.poll(); + assertNotNull(transportInfo); + + // This transport fails + Status transportError = Status.UNAVAILABLE.withDescription("Connection refused"); + assertEquals(0, oobExecutor.numPendingTasks()); + transportInfo.listener.transportShutdown(transportError); + assertTrue(oobExecutor.runDueTasks() > 0); + + // Fail-fast RPC will fail, while wait-for-ready RPC will still be pending + verify(mockCallListener2).onClose(same(transportError), any(Metadata.class)); + verify(mockCallListener3, never()).onClose(any(Status.class), any(Metadata.class)); - verify(transport2).newStream(same(method), same(headers), same(CallOptions.DEFAULT), - any(StatsTraceContext.class)); + // Shutdown + assertFalse(oob.isShutdown()); + oob.shutdown(); + assertTrue(oob.isShutdown()); + + // New RPCs will be rejected. + assertEquals(0, oobExecutor.numPendingTasks()); + ClientCall call4 = oob.newCall(method, CallOptions.DEFAULT); + call4.start(mockCallListener4, headers); + assertTrue(oobExecutor.runDueTasks() > 0); + verify(mockCallListener4).onClose(statusCaptor.capture(), any(Metadata.class)); + + // The pending RPC will still be pending + verify(mockCallListener3, never()).onClose(any(Status.class), any(Metadata.class)); + + // This will shutdownNow() the delayed transport, terminating the pending RPC + assertEquals(0, oobExecutor.numPendingTasks()); + oob.shutdownNow(); + assertTrue(oobExecutor.runDueTasks() > 0); + verify(mockCallListener3).onClose(any(Status.class), any(Metadata.class)); + + // Shut down the channel, and it will not terminated because OOB channel has not. + channel.shutdown(); + assertFalse(channel.isTerminated()); + // Delayed transport has already terminated. Terminating the transport terminates the + // subchannel, which in turn terimates the OOB channel, which terminates the channel. + assertFalse(oob.isTerminated()); + transportInfo.listener.transportTerminated(); + assertTrue(oob.isTerminated()); + assertTrue(channel.isTerminated()); } @Test public void uriPattern() { - assertTrue(ManagedChannelImpl.URI_PATTERN.matcher("a:/").matches()); - assertTrue(ManagedChannelImpl.URI_PATTERN.matcher("Z019+-.:/!@ #~ ").matches()); - assertFalse(ManagedChannelImpl.URI_PATTERN.matcher("a/:").matches()); // "/:" not matched - assertFalse(ManagedChannelImpl.URI_PATTERN.matcher("0a:/").matches()); // '0' not matched - assertFalse(ManagedChannelImpl.URI_PATTERN.matcher("a,:/").matches()); // ',' not matched - assertFalse(ManagedChannelImpl.URI_PATTERN.matcher(" a:/").matches()); // space not matched + assertTrue(ManagedChannelImpl2.URI_PATTERN.matcher("a:/").matches()); + assertTrue(ManagedChannelImpl2.URI_PATTERN.matcher("Z019+-.:/!@ #~ ").matches()); + assertFalse(ManagedChannelImpl2.URI_PATTERN.matcher("a/:").matches()); // "/:" not matched + assertFalse(ManagedChannelImpl2.URI_PATTERN.matcher("0a:/").matches()); // '0' not matched + assertFalse(ManagedChannelImpl2.URI_PATTERN.matcher("a,:/").matches()); // ',' not matched + assertFalse(ManagedChannelImpl2.URI_PATTERN.matcher(" a:/").matches()); // space not matched } /** @@ -857,6 +836,8 @@ public void uriPattern() { */ @Test public void informationPropagatedToNewStreamAndCallCredentials() { + ResolvedServerInfoGroup serverInfoGroup = ResolvedServerInfoGroup.builder() + .add(server).build(); createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); CallOptions callOptions = CallOptions.DEFAULT.withCallCredentials(creds); final Context.Key testKey = Context.key("testing"); @@ -873,20 +854,6 @@ public Void answer(InvocationOnMock in) throws Throwable { any(MethodDescriptor.class), any(Attributes.class), any(Executor.class), any(MetadataApplier.class)); - final ConnectionClientTransport transport = mock(ConnectionClientTransport.class); - when(transport.getAttrs()).thenReturn(Attributes.EMPTY); - when(mockTransportFactory.newClientTransport(any(SocketAddress.class), any(String.class), - any(String.class))).thenReturn(transport); - doAnswer(new Answer() { - @Override - public ClientStream answer(InvocationOnMock in) throws Throwable { - newStreamContexts.add(Context.current()); - return mock(ClientStream.class); - } - }).when(transport).newStream( - any(MethodDescriptor.class), any(Metadata.class), any(CallOptions.class), - any(StatsTraceContext.class)); - // First call will be on delayed transport. Only newCall() is run within the expected context, // so that we can verify that the context is explicitly attached before calling newStream() and // applyRequestMetadata(), which happens after we detach the context from the thread. @@ -897,17 +864,34 @@ public ClientStream answer(InvocationOnMock in) throws Throwable { assertNull(testKey.get()); call.start(mockCallListener, new Metadata()); - ArgumentCaptor transportListenerCaptor = - ArgumentCaptor.forClass(ManagedClientTransport.Listener.class); + // Simulate name resolution results + Subchannel subchannel = helper.createSubchannel( + serverInfoGroup.toEquivalentAddressGroup(), Attributes.EMPTY); + subchannel.requestConnection(); verify(mockTransportFactory).newClientTransport( same(socketAddress), eq(authority), eq(userAgent)); - verify(transport).start(transportListenerCaptor.capture()); + MockClientTransportInfo transportInfo = transports.poll(); + final ConnectionClientTransport transport = transportInfo.transport; + when(transport.getAttrs()).thenReturn(Attributes.EMPTY); + doAnswer(new Answer() { + @Override + public ClientStream answer(InvocationOnMock in) throws Throwable { + newStreamContexts.add(Context.current()); + return mock(ClientStream.class); + } + }).when(transport).newStream( + any(MethodDescriptor.class), any(Metadata.class), any(CallOptions.class), + any(StatsTraceContext.class)); + verify(creds, never()).applyRequestMetadata( any(MethodDescriptor.class), any(Attributes.class), any(Executor.class), any(MetadataApplier.class)); // applyRequestMetadata() is called after the transport becomes ready. - transportListenerCaptor.getValue().transportReady(); + transportInfo.listener.transportReady(); + when(mockPicker.pickSubchannel(any(Attributes.class), any(Metadata.class))) + .thenReturn(PickResult.withSubchannel(subchannel)); + helper.updatePicker(mockPicker); executor.runDueTasks(); ArgumentCaptor attrsCaptor = ArgumentCaptor.forClass(Attributes.class); ArgumentCaptor applierCaptor = ArgumentCaptor.forClass(MetadataApplier.class); @@ -1063,20 +1047,4 @@ public String getDefaultScheme() { return "fake"; } } - - private static class SpyingLoadBalancerFactory extends LoadBalancer.Factory { - private final LoadBalancer.Factory delegate; - private final List> balancers = new ArrayList>(); - - private SpyingLoadBalancerFactory(LoadBalancer.Factory delegate) { - this.delegate = delegate; - } - - @Override - public LoadBalancer newLoadBalancer(String serviceName, TransportManager tm) { - LoadBalancer lb = spy(delegate.newLoadBalancer(serviceName, tm)); - balancers.add(lb); - return lb; - } - } } From c0e1b2daa7f8021db2ebb06740e3aa1b5313f397 Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Mon, 19 Dec 2016 16:48:43 -0800 Subject: [PATCH 03/14] Cosmetic and documentation fixes. --- .../io/grpc/internal/ManagedChannelImpl2.java | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java index 4ca3ce15449..d2fb75e42d4 100644 --- a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java +++ b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java @@ -172,15 +172,15 @@ public final class ManagedChannelImpl2 extends ManagedChannel implements WithLog // 1c delayedTransport.shutdown() // 2. delayedTransport terminated: stop stream-creation functionality // 2a terminating <- true - // 2b loadBalancer <- null - // 2c nameResolver <- null - // 2d loadBalancer.shutdown() + // 2b loadBalancer.shutdown() // * LoadBalancer will shutdown subchannels and OOB channels - // 2e nameResolver.shutdown() + // 2c loadBalancer <- null + // 2d nameResolver.shutdown() + // 2e nameResolver <- null // 3. All subchannels and OOB channels terminated: Channel considered terminated private final AtomicBoolean shutdown = new AtomicBoolean(false); - // Must be mutated from channelExecutor + // Must only be mutated and read from channelExecutor private boolean shutdownNowed; // Must be mutated from channelExecutor private volatile boolean terminating; @@ -228,7 +228,6 @@ public void transportTerminated() { // Must be called from channelExecutor private void maybeShutdownNowSubchannels() { - Status nowStatus = Status.UNAVAILABLE.withDescription("Channel shutdownNow invoked"); if (shutdownNowed) { for (InternalSubchannel subchannel : subchannels) { subchannel.shutdownNow(SHUTDOWN_NOW_STATUS); @@ -272,6 +271,9 @@ public void run() { return; } log.log(Level.FINE, "[{0}] Entering idle mode", getLogId()); + // nameResolver and loadBalancer are guaranteed to be non-null. If any of them were null, + // either the idleModeTimer ran twice without exiting the idle mode, or the task in shutdown() + // did not cancel idleModeTimer, both of which are bugs. nameResolver.shutdown(); nameResolver = getNameResolver(target, nameResolverFactory, nameResolverParams); loadBalancer.shutdown(); @@ -650,7 +652,7 @@ public void onNotInUse(InternalSubchannel is) { subchannel.subchannel = internalSubchannel; log.log(Level.FINE, "[{0}] {1} created for {2}", new Object[] {getLogId(), internalSubchannel.getLogId(), addressGroup}); - channelExecutor.executeLater(new Runnable() { + runSerialized(new Runnable() { @Override public void run() { if (terminating) { @@ -662,7 +664,7 @@ public void run() { subchannels.add(internalSubchannel); } } - }).drain(); + }); return subchannel; } @@ -693,7 +695,7 @@ void onStateChange(InternalSubchannel is, ConnectivityStateInfo newState) { } }); oobChannel.setSubchannel(internalSubchannel); - channelExecutor.executeLater(new Runnable() { + runSerialized(new Runnable() { @Override public void run() { if (terminating) { @@ -705,7 +707,7 @@ public void run() { oobChannels.add(internalSubchannel); } } - }).drain(); + }); return oobChannel; } From 5225693bd2998a3b6939a831f8c11e594771a86e Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Mon, 19 Dec 2016 17:06:03 -0800 Subject: [PATCH 04/14] Bug fix: OobChannel.shutdownNow() should set shutdown to true. --- .../java/io/grpc/internal/OobChannel.java | 1 + .../internal/ManagedChannelImpl2Test.java | 45 +++++++++++++------ 2 files changed, 32 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/io/grpc/internal/OobChannel.java b/core/src/main/java/io/grpc/internal/OobChannel.java index f74861396f5..d357e887122 100644 --- a/core/src/main/java/io/grpc/internal/OobChannel.java +++ b/core/src/main/java/io/grpc/internal/OobChannel.java @@ -210,6 +210,7 @@ public boolean isShutdown() { @Override public ManagedChannel shutdownNow() { + shutdown = true; delayedTransport.shutdownNow( Status.UNAVAILABLE.withDescription("OobChannel.shutdownNow() called")); return this; diff --git a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java index 40b89f59b1d..0c73fbe5daa 100644 --- a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java +++ b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java @@ -167,6 +167,8 @@ public class ManagedChannelImpl2Test { @Mock private ClientCall.Listener mockCallListener4; @Mock + private ClientCall.Listener mockCallListener5; + @Mock private SharedResourceHolder.Resource timerService; @Mock private CallCredentials creds; @@ -746,15 +748,19 @@ public void oobchannels() { createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); FakeClock oobExecutor = new FakeClock(); - ManagedChannel oob = helper.createOobChannel(addressGroup, "oobauthority", + ManagedChannel oob1 = helper.createOobChannel(addressGroup, "oob1authority", + oobExecutor.getScheduledExecutorService()); + ManagedChannel oob2 = helper.createOobChannel(addressGroup, "oob2authority", oobExecutor.getScheduledExecutorService()); - assertEquals("oobauthority", oob.authority()); + + assertEquals("oob1authority", oob1.authority()); + assertEquals("oob2authority", oob2.authority()); // OOB channels create connections lazily. A new call will initiate the connection. Metadata headers = new Metadata(); - ClientCall call = oob.newCall(method, CallOptions.DEFAULT); + ClientCall call = oob1.newCall(method, CallOptions.DEFAULT); call.start(mockCallListener, headers); - verify(mockTransportFactory).newClientTransport(socketAddress, "oobauthority", userAgent); + verify(mockTransportFactory).newClientTransport(socketAddress, "oob1authority", userAgent); MockClientTransportInfo transportInfo = transports.poll(); assertNotNull(transportInfo); @@ -769,12 +775,13 @@ public void oobchannels() { transportInfo.listener.transportTerminated(); // A new call will trigger a new transport - ClientCall call2 = oob.newCall(method, CallOptions.DEFAULT); + ClientCall call2 = oob1.newCall(method, CallOptions.DEFAULT); call2.start(mockCallListener2, headers); - ClientCall call3 = oob.newCall(method, CallOptions.DEFAULT.withWaitForReady()); + ClientCall call3 = + oob1.newCall(method, CallOptions.DEFAULT.withWaitForReady()); call3.start(mockCallListener3, headers); verify(mockTransportFactory, times(2)).newClientTransport( - socketAddress, "oobauthority", userAgent); + socketAddress, "oob1authority", userAgent); transportInfo = transports.poll(); assertNotNull(transportInfo); @@ -789,23 +796,33 @@ public void oobchannels() { verify(mockCallListener3, never()).onClose(any(Status.class), any(Metadata.class)); // Shutdown - assertFalse(oob.isShutdown()); - oob.shutdown(); - assertTrue(oob.isShutdown()); + assertFalse(oob1.isShutdown()); + assertFalse(oob2.isShutdown()); + oob1.shutdown(); + oob2.shutdownNow(); + assertTrue(oob1.isShutdown()); + assertTrue(oob2.isShutdown()); // New RPCs will be rejected. assertEquals(0, oobExecutor.numPendingTasks()); - ClientCall call4 = oob.newCall(method, CallOptions.DEFAULT); + ClientCall call4 = oob1.newCall(method, CallOptions.DEFAULT); + ClientCall call5 = oob2.newCall(method, CallOptions.DEFAULT); call4.start(mockCallListener4, headers); + call5.start(mockCallListener5, headers); assertTrue(oobExecutor.runDueTasks() > 0); verify(mockCallListener4).onClose(statusCaptor.capture(), any(Metadata.class)); + Status status4 = statusCaptor.getValue(); + assertEquals(Status.Code.UNAVAILABLE, status4.getCode()); + verify(mockCallListener5).onClose(statusCaptor.capture(), any(Metadata.class)); + Status status5 = statusCaptor.getValue(); + assertEquals(Status.Code.UNAVAILABLE, status5.getCode()); // The pending RPC will still be pending verify(mockCallListener3, never()).onClose(any(Status.class), any(Metadata.class)); // This will shutdownNow() the delayed transport, terminating the pending RPC assertEquals(0, oobExecutor.numPendingTasks()); - oob.shutdownNow(); + oob1.shutdownNow(); assertTrue(oobExecutor.runDueTasks() > 0); verify(mockCallListener3).onClose(any(Status.class), any(Metadata.class)); @@ -814,9 +831,9 @@ public void oobchannels() { assertFalse(channel.isTerminated()); // Delayed transport has already terminated. Terminating the transport terminates the // subchannel, which in turn terimates the OOB channel, which terminates the channel. - assertFalse(oob.isTerminated()); + assertFalse(oob1.isTerminated()); transportInfo.listener.transportTerminated(); - assertTrue(oob.isTerminated()); + assertTrue(oob1.isTerminated()); assertTrue(channel.isTerminated()); } From 0ae084e6b864d60faa438fddfe15e528cf3abc5b Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Mon, 19 Dec 2016 17:13:23 -0800 Subject: [PATCH 05/14] Bug fix: SubchannelImplImpl should schedule the timer within the lock. --- .../io/grpc/internal/ManagedChannelImpl2.java | 50 +++++++++---------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java index d2fb75e42d4..a1bc532a22b 100644 --- a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java +++ b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java @@ -830,32 +830,32 @@ public void shutdown() { } else { shutdownRequested = true; } + ScheduledExecutorService scheduledExecutorCopy = scheduledExecutor; + // Add a delay to shutdown to deal with the race between 1) a transport being picked and + // newStream() being called on it, and 2) its Subchannel is shut down by LoadBalancer (e.g., + // because of address change, or because LoadBalancer is shutdown by Channel entering idle + // mode). If (2) wins, the app will see a spurious error. We work around this by delaying + // shutdown of Subchannel for a few seconds here. + if (!terminating && scheduledExecutorCopy != null) { + delayedShutdownTask = scheduledExecutorCopy.schedule( + new LogExceptionRunnable( + new Runnable() { + @Override + public void run() { + subchannel.shutdown(); + } + }), SUBCHANNEL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); + return; + } } - ScheduledExecutorService scheduledExecutorCopy = scheduledExecutor; - // Add a delay to shutdown to deal with the race between 1) a transport being picked and - // newStream() being called on it, and 2) its Subchannel is shut down by LoadBalancer (e.g., - // because of address change, or because LoadBalancer is shutdown by Channel entering idle - // mode). If (2) wins, the app will see a spurious error. We work around this by delaying - // shutdown of Subchannel for a few seconds here. - if (!terminating && scheduledExecutorCopy != null) { - delayedShutdownTask = scheduledExecutorCopy.schedule( - new LogExceptionRunnable( - new Runnable() { - @Override - public void run() { - subchannel.shutdown(); - } - }), SUBCHANNEL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); - } else { - // Two possible ways to get here: - // - // 1. terminating == true: no more real streams will be created, it's safe and also - // desirable to shutdown timely. - // - // 2. scheduledExecutor == null: possible only when Channel has already been terminated. - // Though may not be necessary, we'll do it anyway. - subchannel.shutdown(); - } + // Two possible ways to get here: + // + // 1. terminating == true: no more real streams will be created, it's safe and also desirable + // to shutdown timely. + // + // 2. scheduledExecutor == null: possible only when Channel has already been terminated. + // Though may not be necessary, we'll do it anyway. + subchannel.shutdown(); } @Override From 97c463b2b12199cc23178aefda8402f4f2e4241a Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Mon, 19 Dec 2016 17:15:10 -0800 Subject: [PATCH 06/14] Remove unused field from test. --- .../test/java/io/grpc/internal/DelayedClientTransport2Test.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/java/io/grpc/internal/DelayedClientTransport2Test.java b/core/src/test/java/io/grpc/internal/DelayedClientTransport2Test.java index 5efdcf4bbac..186e251abe2 100644 --- a/core/src/test/java/io/grpc/internal/DelayedClientTransport2Test.java +++ b/core/src/test/java/io/grpc/internal/DelayedClientTransport2Test.java @@ -105,7 +105,6 @@ public class DelayedClientTransport2Test { private final Metadata headers = new Metadata(); private final Metadata headers2 = new Metadata(); - private final Metadata headers3 = new Metadata(); private final CallOptions callOptions = CallOptions.DEFAULT.withAuthority("dummy_value"); private final CallOptions callOptions2 = CallOptions.DEFAULT.withAuthority("dummy_value2"); From 3bbe8c30869174731d8fae3c6e3bbb111905ac97 Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Wed, 28 Dec 2016 11:42:16 -0800 Subject: [PATCH 07/14] Switch to ObjectPool. ObjectPool is simpler to use and more test-friendly than the raw SharedResourceHolder. Also assign ManagedChannel's executor to directExecutor after returning the real executor to pool. --- core/src/main/java/io/grpc/LoadBalancer2.java | 7 +-- .../io/grpc/internal/ManagedChannelImpl2.java | 43 ++++++++--------- .../java/io/grpc/internal/ObjectPool.java | 46 +++++++++++++++++++ .../java/io/grpc/internal/OobChannel.java | 13 ++++-- .../ManagedChannelImpl2IdlenessTest.java | 18 +++++--- .../internal/ManagedChannelImpl2Test.java | 24 ++++++---- 6 files changed, 102 insertions(+), 49 deletions(-) create mode 100644 core/src/main/java/io/grpc/internal/ObjectPool.java diff --git a/core/src/main/java/io/grpc/LoadBalancer2.java b/core/src/main/java/io/grpc/LoadBalancer2.java index 204a3df8cd5..2945efbb097 100644 --- a/core/src/main/java/io/grpc/LoadBalancer2.java +++ b/core/src/main/java/io/grpc/LoadBalancer2.java @@ -34,7 +34,6 @@ import com.google.common.base.Preconditions; import java.util.List; -import java.util.concurrent.Executor; import javax.annotation.Nullable; import javax.annotation.concurrent.Immutable; import javax.annotation.concurrent.NotThreadSafe; @@ -338,13 +337,9 @@ public abstract static class Helper { * *

The LoadBalancer is responsible for closing unused OOB channels, and closing all OOB * channels within {@link #shutdown}. - * - * @param eag the address(es) of the channel - * @param authority the authority of the destination this channel connects to - * @param executor the default executor for running RPC callbacks */ public abstract ManagedChannel createOobChannel( - EquivalentAddressGroup eag, String authority, Executor executor); + EquivalentAddressGroup eag, String authority); /** * Set a new picker to the channel. diff --git a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java index a1bc532a22b..66f4259fb2a 100644 --- a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java +++ b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java @@ -41,6 +41,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; import com.google.common.base.Supplier; +import com.google.common.util.concurrent.MoreExecutors; import io.grpc.Attributes; import io.grpc.CallOptions; @@ -70,7 +71,6 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; @@ -111,8 +111,9 @@ public final class ManagedChannelImpl2 extends ManagedChannel implements WithLog private final Attributes nameResolverParams; private final LoadBalancer2.Factory loadBalancerFactory; private final ClientTransportFactory transportFactory; - private final Executor executor; - private final boolean usingSharedExecutor; + private volatile Executor executor; + private final ObjectPool executorPool; + private final ObjectPool oobExecutorPool; private final LogId logId = LogId.allocate(getClass().getName()); private final ChannelExecutor channelExecutor = new ChannelExecutor(); @@ -120,7 +121,7 @@ public final class ManagedChannelImpl2 extends ManagedChannel implements WithLog private final DecompressorRegistry decompressorRegistry; private final CompressorRegistry compressorRegistry; - private final SharedResourceHolder.Resource timerService; + private final ObjectPool timerServicePool; private final Supplier stopwatchSupplier; /** The timout before entering idle mode. */ private final long idleTimeoutMillis; @@ -392,30 +393,27 @@ public void run() { NameResolver.Factory nameResolverFactory, Attributes nameResolverParams, LoadBalancer2.Factory loadBalancerFactory, ClientTransportFactory transportFactory, DecompressorRegistry decompressorRegistry, CompressorRegistry compressorRegistry, - SharedResourceHolder.Resource timerService, + ObjectPool timerServicePool, + ObjectPool executorPool, ObjectPool oobExecutorPool, Supplier stopwatchSupplier, long idleTimeoutMillis, - @Nullable Executor executor, @Nullable String userAgent, + @Nullable String userAgent, List interceptors, CensusContextFactory censusFactory) { this.target = checkNotNull(target, "target"); this.nameResolverFactory = checkNotNull(nameResolverFactory, "nameResolverFactory"); this.nameResolverParams = checkNotNull(nameResolverParams, "nameResolverParams"); this.nameResolver = getNameResolver(target, nameResolverFactory, nameResolverParams); this.loadBalancerFactory = checkNotNull(loadBalancerFactory, "loadBalancerFactory"); - if (executor == null) { - usingSharedExecutor = true; - this.executor = SharedResourceHolder.get(GrpcUtil.SHARED_CHANNEL_EXECUTOR); - } else { - usingSharedExecutor = false; - this.executor = executor; - } + this.executorPool = checkNotNull(executorPool, "executorPool"); + this.oobExecutorPool = checkNotNull(oobExecutorPool, "oobExecutorPool"); + this.executor = checkNotNull(executorPool.getObject(), "executor"); this.delayedTransport = new DelayedClientTransport2(this.executor, this.channelExecutor); this.delayedTransport.start(delayedTransportListener); this.backoffPolicyProvider = backoffPolicyProvider; this.transportFactory = new CallCredentialsApplyingTransportFactory(transportFactory, this.executor); this.interceptorChannel = ClientInterceptors.intercept(new RealChannel(), interceptors); - this.timerService = timerService; - this.scheduledExecutor = SharedResourceHolder.get(timerService); + this.timerServicePool = timerServicePool; + this.scheduledExecutor = timerServicePool.getObject(); this.stopwatchSupplier = checkNotNull(stopwatchSupplier, "stopwatchSupplier"); if (idleTimeoutMillis == IDLE_TIMEOUT_MILLIS_DISABLE) { this.idleTimeoutMillis = idleTimeoutMillis; @@ -594,10 +592,10 @@ private void maybeTerminateChannel() { log.log(Level.FINE, "[{0}] Terminated", getLogId()); terminated = true; terminatedLatch.countDown(); - if (usingSharedExecutor) { - SharedResourceHolder.release(GrpcUtil.SHARED_CHANNEL_EXECUTOR, (ExecutorService) executor); - } - scheduledExecutor = SharedResourceHolder.release(timerService, scheduledExecutor); + executorPool.returnObject(executor); + // Needed for delivering rejections to new calls after OobChannel is terminated. + executor = MoreExecutors.directExecutor(); + scheduledExecutor = timerServicePool.returnObject(scheduledExecutor); // Release the transport factory so that it can deallocate any resources. transportFactory.close(); } @@ -669,14 +667,13 @@ public void run() { } @Override - public ManagedChannel createOobChannel( - EquivalentAddressGroup addressGroup, String authority, Executor executor) { + public ManagedChannel createOobChannel(EquivalentAddressGroup addressGroup, String authority) { ScheduledExecutorService scheduledExecutorCopy = scheduledExecutor; checkState(scheduledExecutorCopy != null, "scheduledExecutor is already cleared. Looks like you are calling this method after " + "you've already shut down"); - final OobChannel oobChannel = new OobChannel(censusFactory, authority, executor, - scheduledExecutorCopy, stopwatchSupplier, channelExecutor); + final OobChannel oobChannel = new OobChannel(censusFactory, authority, + oobExecutorPool, scheduledExecutorCopy, stopwatchSupplier, channelExecutor); final InternalSubchannel internalSubchannel = new InternalSubchannel( addressGroup, authority, userAgent, backoffPolicyProvider, transportFactory, scheduledExecutorCopy, stopwatchSupplier, channelExecutor, diff --git a/core/src/main/java/io/grpc/internal/ObjectPool.java b/core/src/main/java/io/grpc/internal/ObjectPool.java new file mode 100644 index 00000000000..20ae7191e4e --- /dev/null +++ b/core/src/main/java/io/grpc/internal/ObjectPool.java @@ -0,0 +1,46 @@ +/* + * 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.internal; + +public interface ObjectPool { + /** + * Get an object from the pool. + */ + T getObject(); + + /** + * Return the object to the pool. The caller should not use the object beyond this point. + * + * @return always {@code null} + */ + T returnObject(Object object); +} diff --git a/core/src/main/java/io/grpc/internal/OobChannel.java b/core/src/main/java/io/grpc/internal/OobChannel.java index d357e887122..9cb10557295 100644 --- a/core/src/main/java/io/grpc/internal/OobChannel.java +++ b/core/src/main/java/io/grpc/internal/OobChannel.java @@ -38,6 +38,7 @@ import com.google.census.CensusContextFactory; import com.google.common.base.Stopwatch; import com.google.common.base.Supplier; +import com.google.common.util.concurrent.MoreExecutors; import io.grpc.Attributes; import io.grpc.CallOptions; @@ -75,7 +76,8 @@ final class OobChannel extends ManagedChannel implements WithLogId { private final CensusContextFactory censusFactory; private final String authority; private final DelayedClientTransport2 delayedTransport; - private final Executor executor; + private final ObjectPool executorPool; + private volatile Executor executor; private final ScheduledExecutorService deadlineCancellationExecutor; private final Supplier stopwatchSupplier; private final CountDownLatch terminatedLatch = new CountDownLatch(1); @@ -91,12 +93,14 @@ public ClientTransport get(CallOptions callOptions, Metadata headers) { } }; - OobChannel(CensusContextFactory censusFactory, String authority, Executor executor, + OobChannel(CensusContextFactory censusFactory, String authority, + ObjectPool executorPool, ScheduledExecutorService deadlineCancellationExecutor, Supplier stopwatchSupplier, ChannelExecutor channelExecutor) { this.censusFactory = checkNotNull(censusFactory, "censusFactory"); this.authority = checkNotNull(authority, "authority"); - this.executor = checkNotNull(executor, "executor"); + this.executorPool = checkNotNull(executorPool, "executorPool"); + this.executor = checkNotNull(executorPool.getObject(), "executor"); this.deadlineCancellationExecutor = checkNotNull( deadlineCancellationExecutor, "deadlineCancellationExecutor"); this.stopwatchSupplier = checkNotNull(stopwatchSupplier, "stopwatchSupplier"); @@ -240,6 +244,9 @@ public PickResult pickSubchannel(Attributes affinity, Metadata headers) { void handleSubchannelTerminated() { // When delayedTransport is terminated, it shuts down subchannel. Therefore, at this point // both delayedTransport and subchannel have terminated. + executorPool.returnObject(executor); + // Needed for delivering rejections to new calls after OobChannel is terminated. + executor = MoreExecutors.directExecutor(); terminatedLatch.countDown(); } } diff --git a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2IdlenessTest.java b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2IdlenessTest.java index a73eea00a73..5d685f71769 100644 --- a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2IdlenessTest.java +++ b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2IdlenessTest.java @@ -84,6 +84,7 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -94,6 +95,7 @@ public class ManagedChannelImpl2IdlenessTest { private final FakeClock timer = new FakeClock(); private final FakeClock executor = new FakeClock(); + private final FakeClock oobExecutor = new FakeClock(); private static final String AUTHORITY = "fakeauthority"; private static final String USER_AGENT = "fakeagent"; private static final long IDLE_TIMEOUT_SECONDS = 30; @@ -107,7 +109,9 @@ public class ManagedChannelImpl2IdlenessTest { private final List addressGroupList = new ArrayList(); - @Mock private SharedResourceHolder.Resource timerService; + @Mock private ObjectPool timerServicePool; + @Mock private ObjectPool executorPool; + @Mock private ObjectPool oobExecutorPool; @Mock private ClientTransportFactory mockTransportFactory; @Mock private LoadBalancer2 mockLoadBalancer; @Mock private LoadBalancer2.Factory mockLoadBalancerFactory; @@ -121,7 +125,9 @@ public class ManagedChannelImpl2IdlenessTest { @Before public void setUp() { MockitoAnnotations.initMocks(this); - when(timerService.create()).thenReturn(timer.getScheduledExecutorService()); + when(timerServicePool.getObject()).thenReturn(timer.getScheduledExecutorService()); + when(executorPool.getObject()).thenReturn(executor.getScheduledExecutorService()); + when(oobExecutorPool.getObject()).thenReturn(oobExecutor.getScheduledExecutorService()); when(mockLoadBalancerFactory.newLoadBalancer(any(Helper.class))).thenReturn(mockLoadBalancer); when(mockNameResolver.getServiceAuthority()).thenReturn(AUTHORITY); when(mockNameResolverFactory @@ -131,9 +137,8 @@ public void setUp() { channel = new ManagedChannelImpl2("fake://target", new FakeBackoffPolicyProvider(), mockNameResolverFactory, Attributes.EMPTY, mockLoadBalancerFactory, mockTransportFactory, DecompressorRegistry.getDefaultInstance(), - CompressorRegistry.getDefaultInstance(), timerService, timer.getStopwatchSupplier(), - TimeUnit.SECONDS.toMillis(IDLE_TIMEOUT_SECONDS), - executor.getScheduledExecutorService(), USER_AGENT, + CompressorRegistry.getDefaultInstance(), timerServicePool, executorPool, oobExecutorPool, + timer.getStopwatchSupplier(), TimeUnit.SECONDS.toMillis(IDLE_TIMEOUT_SECONDS), USER_AGENT, Collections.emptyList(), NoopCensusContextFactory.INSTANCE); newTransports = TestUtils.captureTransports(mockTransportFactory); @@ -314,8 +319,7 @@ public void oobTransportDoesNotAffectIdleness() { assertFalse(channel.inUseStateAggregator.isInUse()); // Now make an RPC on an OOB channel - ManagedChannel oob = helper.createOobChannel(addressGroupList.get(0), "oobauthority", - oobExecutor.getScheduledExecutorService()); + ManagedChannel oob = helper.createOobChannel(addressGroupList.get(0), "oobauthority"); verify(mockTransportFactory, never()) .newClientTransport(any(SocketAddress.class), same("oobauthority"), same(USER_AGENT)); ClientCall oobCall = oob.newCall(method, CallOptions.DEFAULT); diff --git a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java index 0c73fbe5daa..56c9efb6d00 100644 --- a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java +++ b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java @@ -138,6 +138,7 @@ public class ManagedChannelImpl2Test { private final ResolvedServerInfo server = new ResolvedServerInfo(socketAddress, Attributes.EMPTY); private final FakeClock timer = new FakeClock(); private final FakeClock executor = new FakeClock(); + private final FakeClock oobExecutor = new FakeClock(); private final FakeCensusContextFactory censusCtxFactory = new FakeCensusContextFactory(); @Rule public final ExpectedException thrown = ExpectedException.none(); @@ -169,7 +170,11 @@ public class ManagedChannelImpl2Test { @Mock private ClientCall.Listener mockCallListener5; @Mock - private SharedResourceHolder.Resource timerService; + private ObjectPool timerServicePool; + @Mock + private ObjectPool executorPool; + @Mock + private ObjectPool oobExecutorPool; @Mock private CallCredentials creds; private BlockingQueue transports; @@ -182,9 +187,9 @@ private void createChannel( channel = new ManagedChannelImpl2(target, new FakeBackoffPolicyProvider(), nameResolverFactory, NAME_RESOLVER_PARAMS, mockLoadBalancerFactory, mockTransportFactory, DecompressorRegistry.getDefaultInstance(), - CompressorRegistry.getDefaultInstance(), timerService, timer.getStopwatchSupplier(), - ManagedChannelImpl2.IDLE_TIMEOUT_MILLIS_DISABLE, - executor.getScheduledExecutorService(), userAgent, interceptors, censusCtxFactory); + CompressorRegistry.getDefaultInstance(), timerServicePool, executorPool, oobExecutorPool, + timer.getStopwatchSupplier(), ManagedChannelImpl2.IDLE_TIMEOUT_MILLIS_DISABLE, userAgent, + interceptors, censusCtxFactory); // Force-exit the initial idle-mode channel.exitIdleMode(); assertEquals(0, timer.numPendingTasks()); @@ -200,7 +205,9 @@ public void setUp() throws Exception { expectedUri = new URI(target); when(mockLoadBalancerFactory.newLoadBalancer(any(Helper.class))).thenReturn(mockLoadBalancer); transports = TestUtils.captureTransports(mockTransportFactory); - when(timerService.create()).thenReturn(timer.getScheduledExecutorService()); + when(timerServicePool.getObject()).thenReturn(timer.getScheduledExecutorService()); + when(executorPool.getObject()).thenReturn(executor.getScheduledExecutorService()); + when(oobExecutorPool.getObject()).thenReturn(oobExecutor.getScheduledExecutorService()); } @After @@ -746,12 +753,9 @@ public void subchannels() { @Test public void oobchannels() { createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); - FakeClock oobExecutor = new FakeClock(); - ManagedChannel oob1 = helper.createOobChannel(addressGroup, "oob1authority", - oobExecutor.getScheduledExecutorService()); - ManagedChannel oob2 = helper.createOobChannel(addressGroup, "oob2authority", - oobExecutor.getScheduledExecutorService()); + ManagedChannel oob1 = helper.createOobChannel(addressGroup, "oob1authority"); + ManagedChannel oob2 = helper.createOobChannel(addressGroup, "oob2authority"); assertEquals("oob1authority", oob1.authority()); assertEquals("oob2authority", oob2.authority()); From 014ffc2154586217461b40d2198a26c74b929746 Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Wed, 28 Dec 2016 17:24:34 -0800 Subject: [PATCH 08/14] Add tests for returning the shared executors. --- .../internal/ManagedChannelImpl2Test.java | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java index 56c9efb6d00..6d05aece083 100644 --- a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java +++ b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java @@ -42,6 +42,7 @@ import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyObject; import static org.mockito.Matchers.eq; import static org.mockito.Matchers.same; import static org.mockito.Mockito.atLeast; @@ -250,10 +251,16 @@ public void immediateDeadlineExceeded() { @Test public void shutdownWithNoTransportsEverCreated() { createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + verify(executorPool).getObject(); + verify(timerServicePool).getObject(); + verify(executorPool, never()).returnObject(anyObject()); + verify(timerServicePool, never()).returnObject(anyObject()); verifyNoMoreInteractions(mockTransportFactory); channel.shutdown(); assertTrue(channel.isShutdown()); assertTrue(channel.isTerminated()); + verify(executorPool).returnObject(executor.getScheduledExecutorService()); + verify(timerServicePool).returnObject(timer.getScheduledExecutorService()); } @Test @@ -275,6 +282,8 @@ public void callsAndShutdownAndShutdownNow() { private void subtestCallsAndShutdown(boolean shutdownNow, boolean shutdownNowAfterShutdown) { FakeNameResolverFactory nameResolverFactory = new FakeNameResolverFactory(true); createChannel(nameResolverFactory, NO_INTERCEPTOR); + verify(executorPool).getObject(); + verify(timerServicePool).getObject(); ClientStream mockStream = mock(ClientStream.class); ClientStream mockStream2 = mock(ClientStream.class); Metadata headers = new Metadata(); @@ -388,8 +397,13 @@ private void subtestCallsAndShutdown(boolean shutdownNow, boolean shutdownNowAft // Killing the remaining real transport will terminate the channel transportListener.transportShutdown(Status.UNAVAILABLE); assertFalse(channel.isTerminated()); + verify(executorPool, never()).returnObject(anyObject()); + verify(timerServicePool, never()).returnObject(anyObject()); transportListener.transportTerminated(); assertTrue(channel.isTerminated()); + verify(executorPool).returnObject(executor.getScheduledExecutorService()); + verify(timerServicePool).returnObject(timer.getScheduledExecutorService()); + verifyNoMoreInteractions(oobExecutorPool); verify(mockTransportFactory).close(); verifyNoMoreInteractions(mockTransportFactory); @@ -756,6 +770,7 @@ public void oobchannels() { ManagedChannel oob1 = helper.createOobChannel(addressGroup, "oob1authority"); ManagedChannel oob2 = helper.createOobChannel(addressGroup, "oob2authority"); + verify(oobExecutorPool, times(2)).getObject(); assertEquals("oob1authority", oob1.authority()); assertEquals("oob2authority", oob2.authority()); @@ -803,9 +818,12 @@ public void oobchannels() { assertFalse(oob1.isShutdown()); assertFalse(oob2.isShutdown()); oob1.shutdown(); + verify(oobExecutorPool, never()).returnObject(anyObject()); oob2.shutdownNow(); assertTrue(oob1.isShutdown()); assertTrue(oob2.isShutdown()); + assertTrue(oob2.isTerminated()); + verify(oobExecutorPool).returnObject(oobExecutor.getScheduledExecutorService()); // New RPCs will be rejected. assertEquals(0, oobExecutor.numPendingTasks()); @@ -836,9 +854,11 @@ public void oobchannels() { // Delayed transport has already terminated. Terminating the transport terminates the // subchannel, which in turn terimates the OOB channel, which terminates the channel. assertFalse(oob1.isTerminated()); + verify(oobExecutorPool).returnObject(oobExecutor.getScheduledExecutorService()); transportInfo.listener.transportTerminated(); assertTrue(oob1.isTerminated()); assertTrue(channel.isTerminated()); + verify(oobExecutorPool, times(2)).returnObject(oobExecutor.getScheduledExecutorService()); } @Test From 108345e99eeb25e36a6d4ac13ed77178d457c39c Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Thu, 29 Dec 2016 11:12:38 -0800 Subject: [PATCH 09/14] Stop clearing app executor when channel is terminated. --- core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java | 5 +---- core/src/main/java/io/grpc/internal/OobChannel.java | 5 +---- 2 files changed, 2 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java index 66f4259fb2a..a10ac779a8c 100644 --- a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java +++ b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java @@ -41,7 +41,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; import com.google.common.base.Supplier; -import com.google.common.util.concurrent.MoreExecutors; import io.grpc.Attributes; import io.grpc.CallOptions; @@ -111,7 +110,7 @@ public final class ManagedChannelImpl2 extends ManagedChannel implements WithLog private final Attributes nameResolverParams; private final LoadBalancer2.Factory loadBalancerFactory; private final ClientTransportFactory transportFactory; - private volatile Executor executor; + private final Executor executor; private final ObjectPool executorPool; private final ObjectPool oobExecutorPool; private final LogId logId = LogId.allocate(getClass().getName()); @@ -593,8 +592,6 @@ private void maybeTerminateChannel() { terminated = true; terminatedLatch.countDown(); executorPool.returnObject(executor); - // Needed for delivering rejections to new calls after OobChannel is terminated. - executor = MoreExecutors.directExecutor(); scheduledExecutor = timerServicePool.returnObject(scheduledExecutor); // Release the transport factory so that it can deallocate any resources. transportFactory.close(); diff --git a/core/src/main/java/io/grpc/internal/OobChannel.java b/core/src/main/java/io/grpc/internal/OobChannel.java index 9cb10557295..969c00ceeff 100644 --- a/core/src/main/java/io/grpc/internal/OobChannel.java +++ b/core/src/main/java/io/grpc/internal/OobChannel.java @@ -38,7 +38,6 @@ import com.google.census.CensusContextFactory; import com.google.common.base.Stopwatch; import com.google.common.base.Supplier; -import com.google.common.util.concurrent.MoreExecutors; import io.grpc.Attributes; import io.grpc.CallOptions; @@ -77,7 +76,7 @@ final class OobChannel extends ManagedChannel implements WithLogId { private final String authority; private final DelayedClientTransport2 delayedTransport; private final ObjectPool executorPool; - private volatile Executor executor; + private final Executor executor; private final ScheduledExecutorService deadlineCancellationExecutor; private final Supplier stopwatchSupplier; private final CountDownLatch terminatedLatch = new CountDownLatch(1); @@ -245,8 +244,6 @@ void handleSubchannelTerminated() { // When delayedTransport is terminated, it shuts down subchannel. Therefore, at this point // both delayedTransport and subchannel have terminated. executorPool.returnObject(executor); - // Needed for delivering rejections to new calls after OobChannel is terminated. - executor = MoreExecutors.directExecutor(); terminatedLatch.countDown(); } } From 55c05f74f71190c357ecf179464294e8763e810f Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Thu, 29 Dec 2016 15:03:10 -0800 Subject: [PATCH 10/14] shutdown() doesn't need to call maybeTerminateChannel(); Because it's handled by delayedTransport termination. --- core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java index a10ac779a8c..f8858014c80 100644 --- a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java +++ b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java @@ -491,7 +491,6 @@ public ManagedChannelImpl2 shutdown() { channelExecutor.executeLater(new Runnable() { @Override public void run() { - maybeTerminateChannel(); cancelIdleTimer(); } }).drain(); From 072b6c37734f61d0230ab631a717029e0fafc961 Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Thu, 29 Dec 2016 17:45:37 -0800 Subject: [PATCH 11/14] Add a comment. --- core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java index f8858014c80..88018de31e1 100644 --- a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java +++ b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java @@ -650,6 +650,11 @@ public void onNotInUse(InternalSubchannel is) { @Override public void run() { if (terminating) { + // Because runSerialized() doesn't guarantee the runnable has been executed upon when + // returning, the subchannel may still be returned to the balancer without being + // shutdown even if "terminating" is already true. The subchannel will not be used in + // this case, because delayed transport has terminated when "terminating" becomes + // true, and no more requests will be sent to balancer beyond this point. internalSubchannel.shutdown(); } if (!terminated) { From c52ee3e5ec801d66eea514d685bfbe6640009b40 Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Thu, 29 Dec 2016 18:06:41 -0800 Subject: [PATCH 12/14] Prefer runSerialized() to channelExecutor. --- .../java/io/grpc/internal/ManagedChannelImpl2.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java index 88018de31e1..58385e77235 100644 --- a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java +++ b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java @@ -315,7 +315,7 @@ void exitIdleMode() { helper.lb = loadBalancerFactory.newLoadBalancer(helper); this.loadBalancer = helper.lb; - NameResolverListenerImpl listener = new NameResolverListenerImpl(helper.lb); + NameResolverListenerImpl listener = new NameResolverListenerImpl(helper); try { nameResolver.start(listener); } catch (Throwable t) { @@ -743,9 +743,11 @@ public LogId getLogId() { private class NameResolverListenerImpl implements NameResolver.Listener { final LoadBalancer2 balancer; + final LoadBalancer2.Helper helper; - NameResolverListenerImpl(LoadBalancer2 balancer) { - this.balancer = balancer; + NameResolverListenerImpl(LbHelperImpl helperImpl) { + this.balancer = helperImpl.lb; + this.helper = helperImpl; } @Override @@ -756,7 +758,7 @@ public void onUpdate(final List servers, final Attribut } log.log(Level.FINE, "[{0}] resolved address: {1}, config={2}", new Object[] {getLogId(), servers, config}); - channelExecutor.executeLater(new Runnable() { + helper.runSerialized(new Runnable() { @Override public void run() { if (terminated) { @@ -772,7 +774,7 @@ public void run() { .withDescription("Thrown from handleResolvedAddresses(): " + e)); } } - }).drain(); + }); } @Override From 60bc5a23b9ba92c7686cb4fb6281d9d1ce5f841b Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Tue, 3 Jan 2017 10:39:09 -0800 Subject: [PATCH 13/14] Do not clear subchannels and oobchannels at shutdownNow. --- .../io/grpc/internal/ManagedChannelImpl2.java | 2 - .../internal/ManagedChannelImpl2Test.java | 126 ++++++++++++++++++ 2 files changed, 126 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java index 58385e77235..3064ee58af0 100644 --- a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java +++ b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java @@ -232,11 +232,9 @@ private void maybeShutdownNowSubchannels() { for (InternalSubchannel subchannel : subchannels) { subchannel.shutdownNow(SHUTDOWN_NOW_STATUS); } - subchannels.clear(); for (InternalSubchannel oobChannel : oobChannels) { oobChannel.shutdownNow(SHUTDOWN_NOW_STATUS); } - oobChannels.clear(); } } diff --git a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java index 6d05aece083..845f558a1f5 100644 --- a/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java +++ b/core/src/test/java/io/grpc/internal/ManagedChannelImpl2Test.java @@ -43,6 +43,7 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyObject; +import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; import static org.mockito.Matchers.same; import static org.mockito.Mockito.atLeast; @@ -411,6 +412,10 @@ private void subtestCallsAndShutdown(boolean shutdownNow, boolean shutdownNowAft verifyNoMoreInteractions(mockTransport); } + @Test + public void shutdownNowWithMultipleOobChannels() { + } + @Test public void interceptor() throws Exception { final AtomicLong atomic = new AtomicLong(); @@ -764,6 +769,65 @@ public void subchannels() { verify(transportInfo2.transport).shutdown(); } + @Test + public void subchannelsWhenChannelShutdownNow() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + Subchannel sub1 = helper.createSubchannel(addressGroup, Attributes.EMPTY); + Subchannel sub2 = helper.createSubchannel(addressGroup, Attributes.EMPTY); + sub1.requestConnection(); + sub2.requestConnection(); + + assertEquals(2, transports.size()); + MockClientTransportInfo ti1 = transports.poll(); + MockClientTransportInfo ti2 = transports.poll(); + + ti1.listener.transportReady(); + ti2.listener.transportReady(); + + channel.shutdownNow(); + verify(ti1.transport).shutdownNow(any(Status.class)); + verify(ti2.transport).shutdownNow(any(Status.class)); + + ti1.listener.transportShutdown(Status.UNAVAILABLE.withDescription("shutdown now")); + ti2.listener.transportShutdown(Status.UNAVAILABLE.withDescription("shutdown now")); + ti1.listener.transportTerminated(); + + assertFalse(channel.isTerminated()); + ti2.listener.transportTerminated(); + assertTrue(channel.isTerminated()); + } + + @Test + public void subchannelsNoConnectionShutdown() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + Subchannel sub1 = helper.createSubchannel(addressGroup, Attributes.EMPTY); + Subchannel sub2 = helper.createSubchannel(addressGroup, Attributes.EMPTY); + + channel.shutdown(); + verify(mockLoadBalancer).shutdown(); + sub1.shutdown(); + assertFalse(channel.isTerminated()); + sub2.shutdown(); + assertTrue(channel.isTerminated()); + verify(mockTransportFactory, never()).newClientTransport(any(SocketAddress.class), anyString(), + anyString()); + } + + @Test + public void subchannelsNoConnectionShutdownNow() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + Subchannel sub1 = helper.createSubchannel(addressGroup, Attributes.EMPTY); + Subchannel sub2 = helper.createSubchannel(addressGroup, Attributes.EMPTY); + channel.shutdownNow(); + + verify(mockLoadBalancer).shutdown(); + // Channel's shutdownNow() will call shutdownNow() on all subchannels and oobchannels. + // Therefore, channel is terminated without relying on LoadBalancer to shutdown subchannels. + assertTrue(channel.isTerminated()); + verify(mockTransportFactory, never()).newClientTransport(any(SocketAddress.class), anyString(), + anyString()); + } + @Test public void oobchannels() { createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); @@ -861,6 +925,68 @@ public void oobchannels() { verify(oobExecutorPool, times(2)).returnObject(oobExecutor.getScheduledExecutorService()); } + @Test + public void oobChannelsWhenChannelShutdownNow() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + ManagedChannel oob1 = helper.createOobChannel(addressGroup, "oob1Authority"); + ManagedChannel oob2 = helper.createOobChannel(addressGroup, "oob2Authority"); + + oob1.newCall(method, CallOptions.DEFAULT).start(mockCallListener, new Metadata()); + oob2.newCall(method, CallOptions.DEFAULT).start(mockCallListener2, new Metadata()); + + assertEquals(2, transports.size()); + MockClientTransportInfo ti1 = transports.poll(); + MockClientTransportInfo ti2 = transports.poll(); + + ti1.listener.transportReady(); + ti2.listener.transportReady(); + + channel.shutdownNow(); + verify(ti1.transport).shutdownNow(any(Status.class)); + verify(ti2.transport).shutdownNow(any(Status.class)); + + ti1.listener.transportShutdown(Status.UNAVAILABLE.withDescription("shutdown now")); + ti2.listener.transportShutdown(Status.UNAVAILABLE.withDescription("shutdown now")); + ti1.listener.transportTerminated(); + + assertFalse(channel.isTerminated()); + ti2.listener.transportTerminated(); + assertTrue(channel.isTerminated()); + } + + @Test + public void oobChannelsNoConnectionShutdown() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + ManagedChannel oob1 = helper.createOobChannel(addressGroup, "oob1Authority"); + ManagedChannel oob2 = helper.createOobChannel(addressGroup, "oob2Authority"); + channel.shutdown(); + + verify(mockLoadBalancer).shutdown(); + oob1.shutdown(); + assertTrue(oob1.isTerminated()); + assertFalse(channel.isTerminated()); + oob2.shutdown(); + assertTrue(oob2.isTerminated()); + assertTrue(channel.isTerminated()); + verify(mockTransportFactory, never()).newClientTransport(any(SocketAddress.class), anyString(), + anyString()); + } + + @Test + public void oobChannelsNoConnectionShutdownNow() { + createChannel(new FakeNameResolverFactory(true), NO_INTERCEPTOR); + ManagedChannel oob1 = helper.createOobChannel(addressGroup, "oob1Authority"); + ManagedChannel oob2 = helper.createOobChannel(addressGroup, "oob2Authority"); + channel.shutdownNow(); + + verify(mockLoadBalancer).shutdown(); + assertTrue(channel.isTerminated()); + // Channel's shutdownNow() will call shutdownNow() on all subchannels and oobchannels. + // Therefore, channel is terminated without relying on LoadBalancer to shutdown oobchannels. + verify(mockTransportFactory, never()).newClientTransport(any(SocketAddress.class), anyString(), + anyString()); + } + @Test public void uriPattern() { assertTrue(ManagedChannelImpl2.URI_PATTERN.matcher("a:/").matches()); From 68578f5f16aa097cf29dc0f6bed9b42ffad389e3 Mon Sep 17 00:00:00 2001 From: Kun Zhang Date: Tue, 3 Jan 2017 10:49:54 -0800 Subject: [PATCH 14/14] A few no-op changes. --- .../io/grpc/internal/ManagedChannelImpl2.java | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java index 3064ee58af0..63e28c03f77 100644 --- a/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java +++ b/core/src/main/java/io/grpc/internal/ManagedChannelImpl2.java @@ -297,12 +297,13 @@ void exitIdleMode() { if (shutdown.get()) { return; } - // Cancel the timer now, so that a racing due timer will not put Channel on idleness - // when the caller of exitIdleMode() is about to use the returned loadBalancer. - cancelIdleTimer(); - // exitIdleMode() may be called outside of inUseStateAggregator.handleNotInUse() while - // isInUse() == false, in which case we still need to schedule the timer. - if (!inUseStateAggregator.isInUse()) { + if (inUseStateAggregator.isInUse()) { + // Cancel the timer now, so that a racing due timer will not put Channel on idleness + // when the caller of exitIdleMode() is about to use the returned loadBalancer. + cancelIdleTimer(); + } else { + // exitIdleMode() may be called outside of inUseStateAggregator.handleNotInUse() while + // isInUse() == false, in which case we still need to schedule the timer. rescheduleIdleTimer(); } if (loadBalancer != null) { @@ -409,8 +410,8 @@ public void run() { this.transportFactory = new CallCredentialsApplyingTransportFactory(transportFactory, this.executor); this.interceptorChannel = ClientInterceptors.intercept(new RealChannel(), interceptors); - this.timerServicePool = timerServicePool; - this.scheduledExecutor = timerServicePool.getObject(); + this.timerServicePool = checkNotNull(timerServicePool, "timerServicePool"); + this.scheduledExecutor = checkNotNull(timerServicePool.getObject(), "timerService"); this.stopwatchSupplier = checkNotNull(stopwatchSupplier, "stopwatchSupplier"); if (idleTimeoutMillis == IDLE_TIMEOUT_MILLIS_DISABLE) { this.idleTimeoutMillis = idleTimeoutMillis; @@ -420,8 +421,8 @@ public void run() { "invalid idleTimeoutMillis %s", idleTimeoutMillis); this.idleTimeoutMillis = idleTimeoutMillis; } - this.decompressorRegistry = decompressorRegistry; - this.compressorRegistry = compressorRegistry; + this.decompressorRegistry = checkNotNull(decompressorRegistry, "decompressorRegistry"); + this.compressorRegistry = checkNotNull(compressorRegistry, "compressorRegistry"); this.userAgent = userAgent; this.censusFactory = checkNotNull(censusFactory, "censusFactory"); @@ -463,11 +464,9 @@ static NameResolver getNameResolver(String target, NameResolver.Factory nameReso // Should not be possible. throw new IllegalArgumentException(e); } - if (targetUri != null) { - NameResolver resolver = nameResolverFactory.newNameResolver(targetUri, nameResolverParams); - if (resolver != null) { - return resolver; - } + NameResolver resolver = nameResolverFactory.newNameResolver(targetUri, nameResolverParams); + if (resolver != null) { + return resolver; } } throw new IllegalArgumentException(String.format( @@ -834,6 +833,9 @@ public void shutdown() { // because of address change, or because LoadBalancer is shutdown by Channel entering idle // mode). If (2) wins, the app will see a spurious error. We work around this by delaying // shutdown of Subchannel for a few seconds here. + // + // TODO(zhangkun83): consider a better approach + // (https://github.com/grpc/grpc-java/issues/2562). if (!terminating && scheduledExecutorCopy != null) { delayedShutdownTask = scheduledExecutorCopy.schedule( new LogExceptionRunnable(