xds: reimplement stats management (#7848)

This change reimplements stats recording for the client side:

1. Implemented the new stats objects: ClusterDropStats and ClusterLocalityStats, which match C-core's implementation. The XdsClient APIs for accessing stats objects are

- addClusterDropStats(String clusterName, String edsServiceName)
- addClusterLocalityStats(String clusterName, String edsServiceName, Locality locality)

2. Eliminated the LRS LB policy and incorporate locality load recording in ClusterImplLoadBalancer. The endpoint addresses resolved in ClusterResolverLoadBalancer will attach the locality in each address attributes. In ClusterImplLoadBalancer, its helper's createSubchannel() will populate the address locality and then call XdsClient.addClusterLocalityStats(...) to obtain the per-locality stats object for recording RPCs. This stats object is attached to the created subchannel's attribute. Therefore, ClusterImplLoadBalancer receives Picker update from its child LB policy, the Picker's subchannel will always have the per-locality stats object attached. Helper.pickSubchannel(...) will populate the per-locality stats object and wrap it into the stream tracer for counting RPCs. Note the subchannel's shutdown() is wrapped to call the stats object's Release().
This commit is contained in:
Chengyuan Zhang 2021-02-03 10:45:59 -08:00 committed by GitHub
parent 563288f0ad
commit 0dc37ab77d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
24 changed files with 1173 additions and 2417 deletions

View File

@ -1,448 +0,0 @@
/*
* Copyright 2019 The gRPC Authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.grpc.xds;
import static com.google.common.base.Preconditions.checkNotNull;
import com.github.udpa.udpa.data.orca.v1.OrcaLoadReport;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.MoreObjects;
import io.grpc.ClientStreamTracer;
import io.grpc.ClientStreamTracer.StreamInfo;
import io.grpc.LoadBalancer.PickResult;
import io.grpc.LoadBalancer.PickSubchannelArgs;
import io.grpc.LoadBalancer.SubchannelPicker;
import io.grpc.Metadata;
import io.grpc.Status;
import io.grpc.util.ForwardingClientStreamTracer;
import io.grpc.xds.OrcaOobUtil.OrcaOobReportListener;
import io.grpc.xds.OrcaPerRequestUtil.OrcaPerRequestReportListener;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import javax.annotation.concurrent.Immutable;
import javax.annotation.concurrent.ThreadSafe;
/**
* Client side load stats recorder that provides RPC counting and metrics recording as name-value
* pairs.
*/
@ThreadSafe
final class ClientLoadCounter {
private static final int THREAD_BALANCING_FACTOR = 64;
private final AtomicLong callsInProgress = new AtomicLong();
private final AtomicLong callsSucceeded = new AtomicLong();
private final AtomicLong callsFailed = new AtomicLong();
private final AtomicLong callsIssued = new AtomicLong();
private final MetricRecorder[] metricRecorders = new MetricRecorder[THREAD_BALANCING_FACTOR];
ClientLoadCounter() {
for (int i = 0; i < THREAD_BALANCING_FACTOR; i++) {
metricRecorders[i] = new MetricRecorder();
}
}
void recordCallStarted() {
callsIssued.getAndIncrement();
callsInProgress.getAndIncrement();
}
void recordCallFinished(Status status) {
callsInProgress.getAndDecrement();
if (status.isOk()) {
callsSucceeded.getAndIncrement();
} else {
callsFailed.getAndIncrement();
}
}
void recordMetric(String name, double value) {
MetricRecorder recorder =
metricRecorders[(int) (Thread.currentThread().getId() % THREAD_BALANCING_FACTOR)];
recorder.addValue(name, value);
}
/**
* Generates a snapshot for load stats recorded in this counter for the interval between calls
* of this method.
*/
synchronized ClientLoadSnapshot snapshot() {
Map<String, MetricValue> aggregatedValues = new HashMap<>();
for (MetricRecorder recorder : metricRecorders) {
Map<String, MetricValue> map = recorder.takeAll();
for (Map.Entry<String, MetricValue> entry : map.entrySet()) {
MetricValue curr = aggregatedValues.get(entry.getKey());
if (curr == null) {
curr = new MetricValue();
aggregatedValues.put(entry.getKey(), curr);
}
MetricValue diff = entry.getValue();
curr.numReports += diff.numReports;
curr.totalValue += diff.totalValue;
}
}
return new ClientLoadSnapshot(callsSucceeded.getAndSet(0),
callsInProgress.get(),
callsFailed.getAndSet(0),
callsIssued.getAndSet(0),
aggregatedValues);
}
@VisibleForTesting
void setCallsIssued(long callsIssued) {
this.callsIssued.set(callsIssued);
}
@VisibleForTesting
void setCallsInProgress(long callsInProgress) {
this.callsInProgress.set(callsInProgress);
}
@VisibleForTesting
void setCallsSucceeded(long callsSucceeded) {
this.callsSucceeded.set(callsSucceeded);
}
@VisibleForTesting
void setCallsFailed(long callsFailed) {
this.callsFailed.set(callsFailed);
}
/**
* A {@link ClientLoadSnapshot} represents a snapshot of {@link ClientLoadCounter}, which is a
* read-only copy of load stats recorded for some period of time.
*/
@Immutable
static final class ClientLoadSnapshot {
private final long callsSucceeded;
private final long callsInProgress;
private final long callsFailed;
private final long callsIssued;
private final Map<String, MetricValue> metricValues;
/**
* External usage must only be for testing.
*/
@VisibleForTesting
ClientLoadSnapshot(long callsSucceeded,
long callsInProgress,
long callsFailed,
long callsIssued,
Map<String, MetricValue> metricValues) {
this.callsSucceeded = callsSucceeded;
this.callsInProgress = callsInProgress;
this.callsFailed = callsFailed;
this.callsIssued = callsIssued;
this.metricValues = checkNotNull(metricValues, "metricValues");
}
long getCallsSucceeded() {
return callsSucceeded;
}
long getCallsInProgress() {
return callsInProgress;
}
long getCallsFailed() {
return callsFailed;
}
long getCallsIssued() {
return callsIssued;
}
Map<String, MetricValue> getMetricValues() {
return Collections.unmodifiableMap(metricValues);
}
@Override
public String toString() {
return MoreObjects.toStringHelper(this)
.add("callsSucceeded", callsSucceeded)
.add("callsInProgress", callsInProgress)
.add("callsFailed", callsFailed)
.add("callsIssued", callsIssued)
.add("metricValues", metricValues)
.toString();
}
}
/**
* Atomic unit of recording for metric data.
*/
@Immutable
static final class MetricValue {
private int numReports;
private double totalValue;
private MetricValue() {
this(0, 0);
}
/**
* Must only be used for testing.
*/
@VisibleForTesting
MetricValue(int numReports, double totalValue) {
this.numReports = numReports;
this.totalValue = totalValue;
}
long getNumReports() {
return numReports;
}
double getTotalValue() {
return totalValue;
}
@Override
public String toString() {
return MoreObjects.toStringHelper(this)
.add("numReports", numReports)
.add("totalValue", totalValue)
.toString();
}
}
/**
* Single contention-balanced bucket for recording metric data.
*/
@ThreadSafe
private static class MetricRecorder {
private Map<String, MetricValue> metricValues = new HashMap<>();
synchronized void addValue(String metricName, double value) {
MetricValue currValue = metricValues.get(metricName);
if (currValue == null) {
currValue = new MetricValue();
}
currValue.numReports++;
currValue.totalValue += value;
metricValues.put(metricName, currValue);
}
synchronized Map<String, MetricValue> takeAll() {
Map<String, MetricValue> ret = metricValues;
metricValues = new HashMap<>();
return ret;
}
}
/**
* An {@link LoadRecordingStreamTracerFactory} instance for creating client stream tracers that
* records and aggregates client-side load data into an {@link ClientLoadCounter} object.
*/
@ThreadSafe
@VisibleForTesting
static final class LoadRecordingStreamTracerFactory extends ClientStreamTracer.Factory {
private final ClientStreamTracer.Factory delegate;
private final ClientLoadCounter counter;
LoadRecordingStreamTracerFactory(ClientLoadCounter counter,
ClientStreamTracer.Factory delegate) {
this.counter = checkNotNull(counter, "counter");
this.delegate = checkNotNull(delegate, "delegate");
}
@Override
public ClientStreamTracer newClientStreamTracer(StreamInfo info, Metadata headers) {
counter.recordCallStarted();
final ClientStreamTracer delegateTracer = delegate.newClientStreamTracer(info, headers);
return new ForwardingClientStreamTracer() {
@Override
protected ClientStreamTracer delegate() {
return delegateTracer;
}
@Override
public void streamClosed(Status status) {
counter.recordCallFinished(status);
delegate().streamClosed(status);
}
};
}
@VisibleForTesting
ClientLoadCounter getCounter() {
return counter;
}
@VisibleForTesting
ClientStreamTracer.Factory delegate() {
return delegate;
}
}
/**
* Listener implementation to receive backend metrics and record metric values in the provided
* {@link ClientLoadCounter}.
*/
@ThreadSafe
static final class MetricsRecordingListener
implements OrcaPerRequestReportListener, OrcaOobReportListener {
private final ClientLoadCounter counter;
MetricsRecordingListener(ClientLoadCounter counter) {
this.counter = checkNotNull(counter, "counter");
}
@Override
public void onLoadReport(OrcaLoadReport report) {
counter.recordMetric("cpu_utilization", report.getCpuUtilization());
counter.recordMetric("mem_utilization", report.getMemUtilization());
for (Map.Entry<String, Double> entry : report.getRequestCostMap().entrySet()) {
counter.recordMetric(entry.getKey(), entry.getValue());
}
for (Map.Entry<String, Double> entry : report.getUtilizationMap().entrySet()) {
counter.recordMetric(entry.getKey(), entry.getValue());
}
}
@VisibleForTesting
ClientLoadCounter getCounter() {
return counter;
}
}
/**
* Base class for {@link SubchannelPicker} wrapper classes that intercept "RPC-capable"
* {@link PickResult}s with applying a custom {@link ClientStreamTracer.Factory} for stream
* instrumenting purposes.
*/
@VisibleForTesting
abstract static class TracerWrappingSubchannelPicker extends SubchannelPicker {
private static final ClientStreamTracer NOOP_CLIENT_STREAM_TRACER =
new ClientStreamTracer() {
};
private static final ClientStreamTracer.Factory NOOP_CLIENT_STREAM_TRACER_FACTORY =
new ClientStreamTracer.Factory() {
@Override
public ClientStreamTracer newClientStreamTracer(StreamInfo info, Metadata headers) {
return NOOP_CLIENT_STREAM_TRACER;
}
};
protected abstract SubchannelPicker delegate();
protected abstract ClientStreamTracer.Factory wrapTracerFactory(
ClientStreamTracer.Factory originFactory);
@Override
public PickResult pickSubchannel(PickSubchannelArgs args) {
PickResult result = delegate().pickSubchannel(args);
if (!result.getStatus().isOk()) {
return result;
}
if (result.getSubchannel() == null) {
return result;
}
ClientStreamTracer.Factory originFactory = result.getStreamTracerFactory();
if (originFactory == null) {
originFactory = NOOP_CLIENT_STREAM_TRACER_FACTORY;
}
return PickResult.withSubchannel(result.getSubchannel(), wrapTracerFactory(originFactory));
}
@Override
public String toString() {
return MoreObjects.toStringHelper(this).add("delegate", delegate()).toString();
}
}
/**
* A wrapper class that wraps a {@link SubchannelPicker} instance and associate it with a {@link
* ClientLoadCounter}. All "RPC-capable" {@link PickResult}s picked will be intercepted with
* client side load recording logic such that RPC activities occurring in the {@link PickResult}'s
* {@link io.grpc.LoadBalancer.Subchannel} will be recorded in the associated {@link
* ClientLoadCounter}.
*/
@ThreadSafe
static final class LoadRecordingSubchannelPicker extends TracerWrappingSubchannelPicker {
private final ClientLoadCounter counter;
private final SubchannelPicker delegate;
LoadRecordingSubchannelPicker(ClientLoadCounter counter, SubchannelPicker delegate) {
this.counter = checkNotNull(counter, "counter");
this.delegate = checkNotNull(delegate, "delegate");
}
@Override
protected SubchannelPicker delegate() {
return delegate;
}
@Override
protected ClientStreamTracer.Factory wrapTracerFactory(
ClientStreamTracer.Factory originFactory) {
return new LoadRecordingStreamTracerFactory(counter, originFactory);
}
@Override
public String toString() {
return MoreObjects.toStringHelper(LoadRecordingSubchannelPicker.class)
.add("delegate", delegate)
.toString();
}
}
/**
* A wrapper class that wraps {@link SubchannelPicker} instance and associate it with an {@link
* OrcaPerRequestReportListener}. All "RPC-capable" {@link PickResult}s picked will be intercepted
* with the logic of registering the listener for observing backend metrics.
*/
@ThreadSafe
static final class MetricsObservingSubchannelPicker extends TracerWrappingSubchannelPicker {
private final OrcaPerRequestReportListener listener;
private final SubchannelPicker delegate;
private final OrcaPerRequestUtil orcaPerRequestUtil;
MetricsObservingSubchannelPicker(OrcaPerRequestReportListener listener,
SubchannelPicker delegate,
OrcaPerRequestUtil orcaPerRequestUtil) {
this.listener = checkNotNull(listener, "listener");
this.delegate = checkNotNull(delegate, "delegate");
this.orcaPerRequestUtil = checkNotNull(orcaPerRequestUtil, "orcaPerRequestUtil");
}
@Override
protected SubchannelPicker delegate() {
return delegate;
}
@Override
protected ClientStreamTracer.Factory wrapTracerFactory(
ClientStreamTracer.Factory originFactory) {
return orcaPerRequestUtil.newOrcaClientStreamTracerFactory(originFactory, listener);
}
@Override
public String toString() {
return MoreObjects.toStringHelper(MetricsObservingSubchannelPicker.class)
.add("delegate", delegate)
.toString();
}
}
}

View File

@ -52,7 +52,8 @@ import io.grpc.xds.EnvoyProtoData.LocalityLbEndpoints;
import io.grpc.xds.EnvoyProtoData.Node;
import io.grpc.xds.EnvoyProtoData.StructOrError;
import io.grpc.xds.EnvoyServerProtoData.UpstreamTlsContext;
import io.grpc.xds.LoadStatsManager.LoadStatsStore;
import io.grpc.xds.LoadStatsManager2.ClusterDropStats;
import io.grpc.xds.LoadStatsManager2.ClusterLocalityStats;
import io.grpc.xds.XdsClient.CdsUpdate.AggregateClusterConfig;
import io.grpc.xds.XdsClient.CdsUpdate.ClusterType;
import io.grpc.xds.XdsClient.CdsUpdate.EdsClusterConfig;
@ -100,7 +101,7 @@ final class ClientXdsClient extends AbstractXdsClient {
private final Map<String, ResourceSubscriber> rdsResourceSubscribers = new HashMap<>();
private final Map<String, ResourceSubscriber> cdsResourceSubscribers = new HashMap<>();
private final Map<String, ResourceSubscriber> edsResourceSubscribers = new HashMap<>();
private final LoadStatsManager loadStatsManager = new LoadStatsManager();
private final LoadStatsManager2 loadStatsManager;
private final LoadReportClient lrsClient;
private boolean reportingLoad;
@ -108,6 +109,7 @@ final class ClientXdsClient extends AbstractXdsClient {
ScheduledExecutorService timeService, BackoffPolicy.Provider backoffPolicyProvider,
Supplier<Stopwatch> stopwatchSupplier) {
super(channel, useProtocolV3, node, timeService, backoffPolicyProvider, stopwatchSupplier);
loadStatsManager = new LoadStatsManager2(stopwatchSupplier);
lrsClient = new LoadReportClient(loadStatsManager, channel, useProtocolV3, node,
getSyncContext(), timeService, backoffPolicyProvider, stopwatchSupplier);
}
@ -755,11 +757,9 @@ final class ClientXdsClient extends AbstractXdsClient {
}
@Override
LoadStatsStore addClientStats(String clusterName, @Nullable String clusterServiceName) {
LoadStatsStore loadStatsStore;
synchronized (this) {
loadStatsStore = loadStatsManager.addLoadStats(clusterName, clusterServiceName);
}
ClusterDropStats addClusterDropStats(String clusterName, @Nullable String edsServiceName) {
ClusterDropStats dropCounter =
loadStatsManager.getClusterDropStats(clusterName, edsServiceName);
getSyncContext().execute(new Runnable() {
@Override
public void run() {
@ -769,14 +769,24 @@ final class ClientXdsClient extends AbstractXdsClient {
}
}
});
return loadStatsStore;
return dropCounter;
}
@Override
void removeClientStats(String clusterName, @Nullable String clusterServiceName) {
synchronized (this) {
loadStatsManager.removeLoadStats(clusterName, clusterServiceName);
}
ClusterLocalityStats addClusterLocalityStats(String clusterName,
@Nullable String edsServiceName, Locality locality) {
ClusterLocalityStats loadCounter =
loadStatsManager.getClusterLocalityStats(clusterName, edsServiceName, locality);
getSyncContext().execute(new Runnable() {
@Override
public void run() {
if (!reportingLoad) {
lrsClient.startLoadReporting();
reportingLoad = true;
}
}
});
return loadCounter;
}
private void cleanUpResourceTimers() {

View File

@ -32,10 +32,13 @@ import io.grpc.Status;
import io.grpc.internal.ObjectPool;
import io.grpc.util.ForwardingClientStreamTracer;
import io.grpc.util.ForwardingLoadBalancerHelper;
import io.grpc.util.ForwardingSubchannel;
import io.grpc.xds.ClusterImplLoadBalancerProvider.ClusterImplConfig;
import io.grpc.xds.EnvoyProtoData.DropOverload;
import io.grpc.xds.EnvoyProtoData.Locality;
import io.grpc.xds.EnvoyServerProtoData.UpstreamTlsContext;
import io.grpc.xds.LoadStatsManager.LoadStatsStore;
import io.grpc.xds.LoadStatsManager2.ClusterDropStats;
import io.grpc.xds.LoadStatsManager2.ClusterLocalityStats;
import io.grpc.xds.ThreadSafeRandom.ThreadSafeRandomImpl;
import io.grpc.xds.XdsLogger.XdsLogLevel;
import io.grpc.xds.XdsNameResolverProvider.CallCounterProvider;
@ -67,6 +70,8 @@ final class ClusterImplLoadBalancer extends LoadBalancer {
@VisibleForTesting
static boolean enableSecurity =
Boolean.parseBoolean(System.getenv("GRPC_XDS_EXPERIMENTAL_SECURITY_SUPPORT"));
private static final Attributes.Key<ClusterLocalityStats> ATTR_CLUSTER_LOCALITY_STATS =
Attributes.Key.create("io.grpc.xds.ClusterImplLoadBalancer.clusterLocalityStats");
private final XdsLogger logger;
private final Helper helper;
@ -79,7 +84,7 @@ final class ClusterImplLoadBalancer extends LoadBalancer {
private ObjectPool<XdsClient> xdsClientPool;
private XdsClient xdsClient;
private CallCounterProvider callCounterProvider;
private LoadStatsStore loadStatsStore;
private ClusterDropStats dropStats;
private ClusterImplLbHelper childLbHelper;
private LoadBalancer childLb;
@ -119,7 +124,7 @@ final class ClusterImplLoadBalancer extends LoadBalancer {
// Assume load report server does not change throughout cluster lifetime.
if (config.lrsServerName != null) {
if (config.lrsServerName.isEmpty()) {
loadStatsStore = xdsClient.addClientStats(cluster, edsServiceName);
dropStats = xdsClient.addClusterDropStats(cluster, edsServiceName);
} else {
logger.log(XdsLogLevel.WARNING, "Can only report load to the same management server");
}
@ -128,10 +133,6 @@ final class ClusterImplLoadBalancer extends LoadBalancer {
childLbHelper.updateDropPolicies(config.dropCategories);
childLbHelper.updateMaxConcurrentRequests(config.maxConcurrentRequests);
childLbHelper.updateSslContextProviderSupplier(config.tlsContext);
if (loadStatsStore != null) {
attributes = attributes.toBuilder()
.set(InternalXdsAttributes.ATTR_CLUSTER_SERVICE_LOAD_STATS_STORE, loadStatsStore).build();
}
childLb.handleResolvedAddresses(
resolvedAddresses.toBuilder()
.setAttributes(attributes)
@ -150,8 +151,8 @@ final class ClusterImplLoadBalancer extends LoadBalancer {
@Override
public void shutdown() {
if (loadStatsStore != null) {
xdsClient.removeClientStats(cluster, edsServiceName);
if (dropStats != null) {
dropStats.release();
}
if (childLb != null) {
childLb.shutdown();
@ -172,7 +173,7 @@ final class ClusterImplLoadBalancer extends LoadBalancer {
* or requests to endpoints in the cluster.
*/
private final class ClusterImplLbHelper extends ForwardingLoadBalancerHelper {
private final AtomicLong requestCount;
private final AtomicLong inFlights;
private ConnectivityState currentState = ConnectivityState.IDLE;
private SubchannelPicker currentPicker = BUFFER_PICKER;
private List<DropOverload> dropPolicies = Collections.emptyList();
@ -180,8 +181,8 @@ final class ClusterImplLoadBalancer extends LoadBalancer {
@Nullable
private SslContextProviderSupplier sslContextProviderSupplier;
private ClusterImplLbHelper(AtomicLong requestCount) {
this.requestCount = requestCount;
private ClusterImplLbHelper(AtomicLong inFlights) {
this.inFlights = checkNotNull(inFlights, "inFlights");
}
@Override
@ -206,8 +207,33 @@ final class ClusterImplLoadBalancer extends LoadBalancer {
}
addresses.add(new EquivalentAddressGroup(eag.getAddresses(), attrBuilder.build()));
}
args = args.toBuilder().setAddresses(addresses).build();
return delegate().createSubchannel(args);
Locality locality = args.getAddresses().get(0).getAttributes().get(
InternalXdsAttributes.ATTR_LOCALITY); // all addresses should be in the same locality
// Endpoint addresses resolved by ClusterResolverLoadBalancer should always contain
// attributes with its locality, including endpoints in LOGICAL_DNS clusters.
// In case of not (which really shouldn't), loads are aggregated under an empty locality.
if (locality == null) {
locality = new Locality("", "", "");
}
final ClusterLocalityStats localityStats = xdsClient.addClusterLocalityStats(
cluster, edsServiceName, locality);
Attributes attrs = args.getAttributes().toBuilder().set(
ATTR_CLUSTER_LOCALITY_STATS, localityStats).build();
args = args.toBuilder().setAddresses(addresses).setAttributes(attrs).build();
final Subchannel subchannel = delegate().createSubchannel(args);
return new ForwardingSubchannel() {
@Override
public void shutdown() {
localityStats.release();
delegate().shutdown();
}
@Override
protected Subchannel delegate() {
return subchannel;
}
};
}
@Override
@ -269,57 +295,59 @@ final class ClusterImplLoadBalancer extends LoadBalancer {
if (rand < dropOverload.getDropsPerMillion()) {
logger.log(XdsLogLevel.INFO, "Drop request with category: {0}",
dropOverload.getCategory());
if (loadStatsStore != null) {
loadStatsStore.recordDroppedRequest(dropOverload.getCategory());
if (dropStats != null) {
dropStats.recordDroppedRequest(dropOverload.getCategory());
}
return PickResult.withDrop(
Status.UNAVAILABLE.withDescription("Dropped: " + dropOverload.getCategory()));
}
}
PickResult result = delegate.pickSubchannel(args);
if (enableCircuitBreaking) {
if (result.getStatus().isOk() && result.getSubchannel() != null) {
if (requestCount.get() >= maxConcurrentRequests) {
if (loadStatsStore != null) {
loadStatsStore.recordDroppedRequest();
final PickResult result = delegate.pickSubchannel(args);
if (result.getStatus().isOk() && result.getSubchannel() != null) {
if (enableCircuitBreaking) {
if (inFlights.get() >= maxConcurrentRequests) {
if (dropStats != null) {
dropStats.recordDroppedRequest();
}
return PickResult.withDrop(Status.UNAVAILABLE.withDescription(
"Cluster max concurrent requests limit exceeded"));
} else {
ClientStreamTracer.Factory tracerFactory = new RequestCountingStreamTracerFactory(
result.getStreamTracerFactory(), requestCount);
return PickResult.withSubchannel(result.getSubchannel(), tracerFactory);
}
}
final ClusterLocalityStats stats =
result.getSubchannel().getAttributes().get(ATTR_CLUSTER_LOCALITY_STATS);
ClientStreamTracer.Factory tracerFactory = new CountingStreamTracerFactory(
stats, inFlights, result.getStreamTracerFactory());
return PickResult.withSubchannel(result.getSubchannel(), tracerFactory);
}
return result;
}
}
}
/**
* Counts the number of outstanding requests.
*/
private static final class RequestCountingStreamTracerFactory
extends ClientStreamTracer.Factory {
private static final class CountingStreamTracerFactory extends ClientStreamTracer.Factory {
private ClusterLocalityStats stats;
private final AtomicLong inFlights;
@Nullable
private final ClientStreamTracer.Factory delegate;
private final AtomicLong counter;
private RequestCountingStreamTracerFactory(@Nullable ClientStreamTracer.Factory delegate,
AtomicLong counter) {
private CountingStreamTracerFactory(
ClusterLocalityStats stats, AtomicLong inFlights,
@Nullable ClientStreamTracer.Factory delegate) {
this.stats = checkNotNull(stats, "stats");
this.inFlights = checkNotNull(inFlights, "inFlights");
this.delegate = delegate;
this.counter = counter;
}
@Override
public ClientStreamTracer newClientStreamTracer(StreamInfo info, Metadata headers) {
counter.incrementAndGet();
stats.recordCallStarted();
inFlights.incrementAndGet();
if (delegate == null) {
return new ClientStreamTracer() {
@Override
public void streamClosed(Status status) {
counter.decrementAndGet();
stats.recordCallFinished(status);
inFlights.decrementAndGet();
}
};
}
@ -332,7 +360,8 @@ final class ClusterImplLoadBalancer extends LoadBalancer {
@Override
public void streamClosed(Status status) {
counter.decrementAndGet();
stats.recordCallFinished(status);
inFlights.decrementAndGet();
delegate().streamClosed(status);
}
};

View File

@ -19,11 +19,11 @@ package io.grpc.xds;
import static com.google.common.base.Preconditions.checkNotNull;
import static io.grpc.ConnectivityState.CONNECTING;
import static io.grpc.ConnectivityState.TRANSIENT_FAILURE;
import static io.grpc.xds.XdsLbPolicies.LRS_POLICY_NAME;
import static io.grpc.xds.XdsLbPolicies.PRIORITY_POLICY_NAME;
import static io.grpc.xds.XdsSubchannelPickers.BUFFER_PICKER;
import com.google.common.annotations.VisibleForTesting;
import io.grpc.Attributes;
import io.grpc.EquivalentAddressGroup;
import io.grpc.InternalLogId;
import io.grpc.LoadBalancer;
@ -48,7 +48,6 @@ import io.grpc.xds.EnvoyProtoData.LbEndpoint;
import io.grpc.xds.EnvoyProtoData.Locality;
import io.grpc.xds.EnvoyProtoData.LocalityLbEndpoints;
import io.grpc.xds.EnvoyServerProtoData.UpstreamTlsContext;
import io.grpc.xds.LrsLoadBalancerProvider.LrsConfig;
import io.grpc.xds.PriorityLoadBalancerProvider.PriorityLbConfig;
import io.grpc.xds.PriorityLoadBalancerProvider.PriorityLbConfig.PriorityChildConfig;
import io.grpc.xds.WeightedTargetLoadBalancerProvider.WeightedPolicySelection;
@ -392,10 +391,12 @@ final class ClusterResolverLoadBalancer extends LoadBalancer {
for (LbEndpoint endpoint : localityLbInfo.getEndpoints()) {
if (endpoint.isHealthy()) {
discard = false;
Attributes attr = endpoint.getAddress().getAttributes().toBuilder()
.set(InternalXdsAttributes.ATTR_LOCALITY, locality).build();
EquivalentAddressGroup eag =
AddressFilter.setPathFilter(
endpoint.getAddress(),
Arrays.asList(priorityName, localityName(locality)));
new EquivalentAddressGroup(endpoint.getAddress().getAddresses(), attr);
eag = AddressFilter.setPathFilter(
eag, Arrays.asList(priorityName, localityName(locality)));
addresses.add(eag);
}
}
@ -533,19 +534,18 @@ final class ClusterResolverLoadBalancer extends LoadBalancer {
String priorityName = priorityName(name, 0); // value doesn't matter
List<EquivalentAddressGroup> addresses = new ArrayList<>();
for (EquivalentAddressGroup eag : resolutionResult.getAddresses()) {
EquivalentAddressGroup annotatedAddr =
AddressFilter.setPathFilter(
eag, Arrays.asList(
priorityName, LOGICAL_DNS_CLUSTER_LOCALITY.toString()));
addresses.add(annotatedAddr);
Attributes attr = eag.getAttributes().toBuilder().set(
InternalXdsAttributes.ATTR_LOCALITY, LOGICAL_DNS_CLUSTER_LOCALITY).build();
eag = new EquivalentAddressGroup(eag.getAddresses(), attr);
eag = AddressFilter.setPathFilter(
eag, Arrays.asList(priorityName, LOGICAL_DNS_CLUSTER_LOCALITY.toString()));
addresses.add(eag);
}
LoadBalancerProvider endpointPickingLbProvider =
lbRegistry.getProvider("pick_first");
PolicySelection endpointPickingPolicy =
new PolicySelection(endpointPickingLbProvider, null);
new PolicySelection(lbRegistry.getProvider("pick_first"), null);
PriorityChildConfig priorityChildConfig = generatePriorityChildConfig(
name, edsServiceName, lrsServerName, maxConcurrentRequests, tlsContext,
endpointPickingPolicy, false, lbRegistry, LOGICAL_DNS_CLUSTER_LOCALITY,
endpointPickingPolicy, false, lbRegistry,
Collections.<DropOverload>emptyList());
status = Status.OK;
resolved = true;
@ -615,19 +615,16 @@ final class ClusterResolverLoadBalancer extends LoadBalancer {
/**
* Generates the config to be used in the priority LB policy for a single priority.
*
* <p>priority LB -> cluster_impl LB -> (lrs LB) -> pick_first
* <p>priority LB -> cluster_impl LB -> pick_first
*/
private static PriorityChildConfig generatePriorityChildConfig(
String cluster, @Nullable String edsServiceName, @Nullable String lrsServerName,
@Nullable Long maxConcurrentRequests, @Nullable UpstreamTlsContext tlsContext,
PolicySelection endpointPickingPolicy, boolean ignoreReresolution,
LoadBalancerRegistry lbRegistry, Locality locality, List<DropOverload> dropOverloads) {
PolicySelection localityLbPolicy =
generateLocalityLbConfig(locality, cluster, edsServiceName, lrsServerName,
endpointPickingPolicy, lbRegistry);
LoadBalancerRegistry lbRegistry, List<DropOverload> dropOverloads) {
ClusterImplConfig clusterImplConfig =
new ClusterImplConfig(cluster, edsServiceName, lrsServerName, maxConcurrentRequests,
dropOverloads, localityLbPolicy, tlsContext);
dropOverloads, endpointPickingPolicy, tlsContext);
LoadBalancerProvider clusterImplLbProvider =
lbRegistry.getProvider(XdsLbPolicies.CLUSTER_IMPL_POLICY_NAME);
PolicySelection clusterImplPolicy =
@ -639,7 +636,7 @@ final class ClusterResolverLoadBalancer extends LoadBalancer {
* Generates configs to be used in the priority LB policy for priorities in the cluster.
*
* <p>priority LB -> cluster_impl LB (one per priority) -> weighted_target LB
* -> (lrs LB (one per locality)) -> round_robin
* -> round_robin (one per locality))
*/
private static Map<String, PriorityChildConfig> generatePriorityChildConfigs(
String cluster, @Nullable String edsServiceName, @Nullable String lrsServerName,
@ -650,11 +647,16 @@ final class ClusterResolverLoadBalancer extends LoadBalancer {
List<DropOverload> dropOverloads) {
Map<String, PriorityChildConfig> configs = new HashMap<>();
for (String priority : prioritizedLocalityWeights.keySet()) {
WeightedTargetConfig localityPickingLbConfig =
generateLocalityPickingLbConfig(cluster, edsServiceName, lrsServerName,
endpointPickingPolicy, lbRegistry, prioritizedLocalityWeights.get(priority));
PolicySelection localityPicking =
new PolicySelection(localityPickingPolicy.getProvider(), localityPickingLbConfig);
Map<Locality, Integer> localityWeights = prioritizedLocalityWeights.get(priority);
Map<String, WeightedPolicySelection> targets = new HashMap<>();
for (Locality locality : localityWeights.keySet()) {
int weight = localityWeights.get(locality);
targets.put(localityName(locality),
new WeightedPolicySelection(weight, endpointPickingPolicy));
}
PolicySelection localityPicking = new PolicySelection(
localityPickingPolicy.getProvider(),
new WeightedTargetConfig(Collections.unmodifiableMap(targets)));
ClusterImplConfig clusterImplConfig =
new ClusterImplConfig(cluster, edsServiceName, lrsServerName, maxConcurrentRequests,
dropOverloads, localityPicking, tlsContext);
@ -667,40 +669,6 @@ final class ClusterResolverLoadBalancer extends LoadBalancer {
return configs;
}
private static WeightedTargetConfig generateLocalityPickingLbConfig(
String cluster, @Nullable String edsServiceName, @Nullable String lrsServerName,
PolicySelection endpointPickingPolicy, LoadBalancerRegistry lbRegistry,
Map<Locality, Integer> localityWeights) {
Map<String, WeightedPolicySelection> targets = new HashMap<>();
for (Locality locality : localityWeights.keySet()) {
int weight = localityWeights.get(locality);
PolicySelection childPolicy =
generateLocalityLbConfig(locality, cluster, edsServiceName, lrsServerName,
endpointPickingPolicy, lbRegistry);
targets.put(localityName(locality), new WeightedPolicySelection(weight, childPolicy));
}
return new WeightedTargetConfig(Collections.unmodifiableMap(targets));
}
/**
* Generates intra-locality LB policy (with config) for the given locality.
*/
private static PolicySelection generateLocalityLbConfig(
Locality locality, String cluster, @Nullable String edsServiceName,
@Nullable String lrsServerName, PolicySelection endpointPickingPolicy,
LoadBalancerRegistry lbRegistry) {
PolicySelection policy;
if (lrsServerName != null) {
LrsConfig childConfig =
new LrsConfig(cluster, edsServiceName, lrsServerName, locality, endpointPickingPolicy);
LoadBalancerProvider childPolicyProvider = lbRegistry.getProvider(LRS_POLICY_NAME);
policy = new PolicySelection(childPolicyProvider, childConfig);
} else {
policy = endpointPickingPolicy;
}
return policy;
}
/**
* Generates a string that represents the priority in the LB policy config. The string is unique
* across priorities in all clusters and priorityName(c, p1) < priorityName(c, p2) iff p1 < p2.

View File

@ -1804,6 +1804,10 @@ final class EnvoyProtoData {
return this;
}
long getLoadReportIntervalNanos() {
return loadReportIntervalNanos;
}
Builder addUpstreamLocalityStats(UpstreamLocalityStats upstreamLocalityStats) {
upstreamLocalityStatsList.add(checkNotNull(upstreamLocalityStats, "upstreamLocalityStats"));
return this;

View File

@ -22,7 +22,7 @@ import io.grpc.Grpc;
import io.grpc.Internal;
import io.grpc.NameResolver;
import io.grpc.internal.ObjectPool;
import io.grpc.xds.LoadStatsManager.LoadStatsStore;
import io.grpc.xds.EnvoyProtoData.Locality;
import io.grpc.xds.XdsNameResolverProvider.CallCounterProvider;
import io.grpc.xds.internal.sds.SslContextProviderSupplier;
@ -62,10 +62,12 @@ public final class InternalXdsAttributes {
public static final Attributes.Key<String> ATTR_CLUSTER_NAME =
Attributes.Key.create("io.grpc.xds.InternalXdsAttributes.clusterName");
// TODO (chengyuanzhang): temporary solution for migrating to LRS policy. Should access
// stats object via XdsClient interface.
static final Attributes.Key<LoadStatsStore> ATTR_CLUSTER_SERVICE_LOAD_STATS_STORE =
Attributes.Key.create("io.grpc.xds.InternalXdsAttributes.loadStatsStore");
/**
* The locality that this EquivalentAddressGroup is in.
*/
@EquivalentAddressGroup.Attr
static final Attributes.Key<Locality> ATTR_LOCALITY =
Attributes.Key.create("io.grpc.xds.InternalXdsAttributes.locality");
private InternalXdsAttributes() {}
}

View File

@ -59,7 +59,7 @@ final class LoadReportClient {
private final ScheduledExecutorService timerService;
private final Stopwatch retryStopwatch;
private final BackoffPolicy.Provider backoffPolicyProvider;
private final LoadStatsManager loadStatsManager;
private final LoadStatsManager2 loadStatsManager;
private boolean started;
@Nullable
@ -70,7 +70,7 @@ final class LoadReportClient {
private LrsStream lrsStream;
LoadReportClient(
LoadStatsManager loadStatsManager,
LoadStatsManager2 loadStatsManager,
ManagedChannel channel,
boolean useProtocolV3,
Node node,
@ -213,11 +213,11 @@ final class LoadReportClient {
}
List<ClusterStats> clusterStatsList;
if (reportAllClusters) {
clusterStatsList = loadStatsManager.getAllLoadReports();
clusterStatsList = loadStatsManager.getAllClusterStatsReports();
} else {
clusterStatsList = new ArrayList<>();
for (String name : clusterNames) {
clusterStatsList.addAll(loadStatsManager.getClusterLoadReports(name));
clusterStatsList.addAll(loadStatsManager.getClusterStatsReports(name));
}
}
sendLoadStatsRequest(clusterStatsList);

View File

@ -1,169 +0,0 @@
/*
* Copyright 2019 The gRPC Authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.grpc.xds;
import static com.google.common.base.Preconditions.checkState;
import com.google.common.annotations.VisibleForTesting;
import io.grpc.xds.EnvoyProtoData.ClusterStats;
import io.grpc.xds.EnvoyProtoData.Locality;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import javax.annotation.Nullable;
/**
* Manages all stats for client side load.
*/
final class LoadStatsManager {
private final LoadStatsStoreFactory loadStatsStoreFactory;
private final Map<String, Map<String, ReferenceCounted<LoadStatsStore>>> loadStatsStores
= new HashMap<>();
LoadStatsManager() {
this(LoadStatsStoreImpl.getDefaultFactory());
}
@VisibleForTesting
LoadStatsManager(LoadStatsStoreFactory factory) {
this.loadStatsStoreFactory = factory;
}
/**
* Adds and retrieves the stats object for tracking loads for the given cluster:cluster_service.
* The returned {@link LoadStatsStore} is reference-counted, caller should use
* {@link #removeLoadStats} to release the reference when it is no longer used.
*/
LoadStatsStore addLoadStats(String cluster, @Nullable String clusterService) {
if (!loadStatsStores.containsKey(cluster)) {
loadStatsStores.put(cluster, new HashMap<String, ReferenceCounted<LoadStatsStore>>());
}
Map<String, ReferenceCounted<LoadStatsStore>> clusterLoadStatsStores
= loadStatsStores.get(cluster);
if (!clusterLoadStatsStores.containsKey(clusterService)) {
clusterLoadStatsStores.put(
clusterService,
ReferenceCounted.wrap(loadStatsStoreFactory.newLoadStatsStore(cluster, clusterService)));
}
ReferenceCounted<LoadStatsStore> ref = clusterLoadStatsStores.get(clusterService);
ref.retain();
return ref.get();
}
/**
* Discards stats object used for tracking loads for the given cluster:cluster_service.
*/
void removeLoadStats(String cluster, @Nullable String clusterService) {
checkState(
loadStatsStores.containsKey(cluster)
&& loadStatsStores.get(cluster).containsKey(clusterService),
"stats for cluster %s, cluster service %s not exits");
Map<String, ReferenceCounted<LoadStatsStore>> clusterLoadStatsStores =
loadStatsStores.get(cluster);
ReferenceCounted<LoadStatsStore> ref = clusterLoadStatsStores.get(clusterService);
ref.release();
if (ref.getReferenceCount() == 0) {
clusterLoadStatsStores.remove(clusterService);
}
if (clusterLoadStatsStores.isEmpty()) {
loadStatsStores.remove(cluster);
}
}
/**
* Generates reports summarizing the stats recorded for loads sent to the given cluster for
* the interval between calls of this method or {@link #getAllLoadReports}. A cluster may send
* loads to more than one cluster_service, they are included in separate stats reports.
*/
List<ClusterStats> getClusterLoadReports(String cluster) {
List<ClusterStats> res = new ArrayList<>();
Map<String, ReferenceCounted<LoadStatsStore>> clusterLoadStatsStores =
loadStatsStores.get(cluster);
if (clusterLoadStatsStores == null) {
return res;
}
for (ReferenceCounted<LoadStatsStore> ref : clusterLoadStatsStores.values()) {
res.add(ref.get().generateLoadReport());
}
return res;
}
/**
* Generates reports summarized the stats recorded for loads sent to all clusters for the
* interval between calls of this method or {@link #getClusterLoadReports}. Each report
* includes stats for one cluster:cluster_service.
*/
List<ClusterStats> getAllLoadReports() {
List<ClusterStats> res = new ArrayList<>();
for (Map<String, ReferenceCounted<LoadStatsStore>> clusterLoadStatsStores
: loadStatsStores.values()) {
for (ReferenceCounted<LoadStatsStore> ref : clusterLoadStatsStores.values()) {
res.add(ref.get().generateLoadReport());
}
}
return res;
}
// Introduced for testing.
@VisibleForTesting
interface LoadStatsStoreFactory {
LoadStatsStore newLoadStatsStore(String cluster, String clusterService);
}
/**
* Interface for client side load stats store. A {@link LoadStatsStore} instance holds the load
* stats for a cluster from an gRPC client's perspective by maintaining a set of locality
* counters for each locality it is tracking loads for.
*/
interface LoadStatsStore {
/**
* Generates a report based on recorded load stats (including RPC counts, backend metrics and
* dropped calls) for the interval since the previous call of this method.
*/
ClusterStats generateLoadReport();
/**
* Adds tracking for load stats sent to the given {@code locality}. Returns the counter
* object responsible for tracking the client load stats to the given {@code locality}.
* Only load stats for tracked localities will be included in generated load reports.
*/
ClientLoadCounter addLocality(Locality locality);
/**
* Drops tracking for load stats sent to the given {@code locality}. Load stats for removed
* localities will no longer be included in future generated load reports after
* their currently recording stats have been fully reported.
*/
void removeLocality(Locality locality);
/**
* Records a drop decision with the given category.
*
* <p>This method must be thread-safe.
*/
void recordDroppedRequest(String category);
/**
* Records a uncategorized drop decision.
*
* <p>This method must be thread-safe.
*/
void recordDroppedRequest();
}
}

View File

@ -0,0 +1,396 @@
/*
* Copyright 2021 The gRPC Authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.grpc.xds;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import com.google.common.base.Stopwatch;
import com.google.common.base.Supplier;
import com.google.common.collect.Sets;
import io.grpc.Status;
import io.grpc.xds.EnvoyProtoData.ClusterStats;
import io.grpc.xds.EnvoyProtoData.ClusterStats.DroppedRequests;
import io.grpc.xds.EnvoyProtoData.Locality;
import io.grpc.xds.EnvoyProtoData.UpstreamLocalityStats;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import javax.annotation.Nullable;
import javax.annotation.concurrent.ThreadSafe;
/**
* Manages client side traffic stats. Drop stats are maintained in cluster (with edsServiceName)
* granularity and load stats (request counts) are maintained in locality granularity.
*/
@ThreadSafe
final class LoadStatsManager2 {
// Recorders for drops of each cluster:edsServiceName.
private final Map<String, Map<String, ReferenceCounted<ClusterDropStats>>> allDropStats =
new HashMap<>();
// Recorders for loads of each cluster:edsServiceName:locality.
private final Map<String, Map<String,
Map<Locality, ReferenceCounted<ClusterLocalityStats>>>> allLoadStats = new HashMap<>();
private final Supplier<Stopwatch> stopwatchSupplier;
LoadStatsManager2(Supplier<Stopwatch> stopwatchSupplier) {
this.stopwatchSupplier = checkNotNull(stopwatchSupplier, "stopwatchSupplier");
}
/**
* Gets or creates the stats object for recording drops for the specified cluster with
* edsServiceName. The returned object is reference counted and the caller should use {@link
* ClusterDropStats#release()} to release its <i>hard</i> reference when it is safe to discard
* future stats for the cluster.
*/
synchronized ClusterDropStats getClusterDropStats(
String cluster, @Nullable String edsServiceName) {
if (!allDropStats.containsKey(cluster)) {
allDropStats.put(cluster, new HashMap<String, ReferenceCounted<ClusterDropStats>>());
}
Map<String, ReferenceCounted<ClusterDropStats>> perClusterCounters = allDropStats.get(cluster);
if (!perClusterCounters.containsKey(edsServiceName)) {
perClusterCounters.put(
edsServiceName,
ReferenceCounted.wrap(new ClusterDropStats(
cluster, edsServiceName, stopwatchSupplier.get())));
}
ReferenceCounted<ClusterDropStats> ref = perClusterCounters.get(edsServiceName);
ref.retain();
return ref.get();
}
private synchronized void releaseClusterDropCounter(
String cluster, @Nullable String edsServiceName) {
checkState(allDropStats.containsKey(cluster)
&& allDropStats.get(cluster).containsKey(edsServiceName),
"stats for cluster %s, edsServiceName %s not exits", cluster, edsServiceName);
ReferenceCounted<ClusterDropStats> ref = allDropStats.get(cluster).get(edsServiceName);
ref.release();
}
/**
* Gets or creates the stats object for recording loads for the specified locality (in the
* specified cluster with edsServiceName). The returned object is reference counted and the
* caller should use {@link ClusterLocalityStats#release} to release its <i>hard</i> reference
* when it is safe to discard the future stats for the locality.
*/
synchronized ClusterLocalityStats getClusterLocalityStats(
String cluster, @Nullable String edsServiceName, Locality locality) {
if (!allLoadStats.containsKey(cluster)) {
allLoadStats.put(
cluster,
new HashMap<String, Map<Locality, ReferenceCounted<ClusterLocalityStats>>>());
}
Map<String, Map<Locality, ReferenceCounted<ClusterLocalityStats>>> perClusterCounters =
allLoadStats.get(cluster);
if (!perClusterCounters.containsKey(edsServiceName)) {
perClusterCounters.put(
edsServiceName, new HashMap<Locality, ReferenceCounted<ClusterLocalityStats>>());
}
Map<Locality, ReferenceCounted<ClusterLocalityStats>> localityStats =
perClusterCounters.get(edsServiceName);
if (!localityStats.containsKey(locality)) {
localityStats.put(
locality,
ReferenceCounted.wrap(new ClusterLocalityStats(
cluster, edsServiceName, locality, stopwatchSupplier.get())));
}
ReferenceCounted<ClusterLocalityStats> ref = localityStats.get(locality);
ref.retain();
return ref.get();
}
private synchronized void releaseClusterLocalityLoadCounter(
String cluster, @Nullable String edsServiceName, Locality locality) {
checkState(allLoadStats.containsKey(cluster)
&& allLoadStats.get(cluster).containsKey(edsServiceName)
&& allLoadStats.get(cluster).get(edsServiceName).containsKey(locality),
"stats for cluster %s, edsServiceName %s, locality %s not exits",
cluster, edsServiceName, locality);
ReferenceCounted<ClusterLocalityStats> ref =
allLoadStats.get(cluster).get(edsServiceName).get(locality);
ref.release();
}
/**
* Gets the traffic stats (drops and loads) as a list of {@link ClusterStats} recorded for the
* specified cluster since the previous call of this method or {@link
* #getAllClusterStatsReports}. A {@link ClusterStats} includes stats for a specific cluster with
* edsServiceName.
*/
synchronized List<ClusterStats> getClusterStatsReports(String cluster) {
if (!allDropStats.containsKey(cluster) && !allLoadStats.containsKey(cluster)) {
return Collections.emptyList();
}
Map<String, ReferenceCounted<ClusterDropStats>> clusterDropStats = allDropStats.get(cluster);
Map<String, Map<Locality, ReferenceCounted<ClusterLocalityStats>>> clusterLoadStats =
allLoadStats.get(cluster);
Map<String, ClusterStats.Builder> statsReportBuilders = new HashMap<>();
// Populate drop stats.
if (clusterDropStats != null) {
Set<String> toDiscard = new HashSet<>();
for (String edsServiceName : clusterDropStats.keySet()) {
ClusterStats.Builder builder = ClusterStats.newBuilder().setClusterName(cluster);
if (edsServiceName != null) {
builder.setClusterServiceName(edsServiceName);
}
ReferenceCounted<ClusterDropStats> ref = clusterDropStats.get(edsServiceName);
if (ref.getReferenceCount() == 0) { // stats object no longer needed after snapshot
toDiscard.add(edsServiceName);
}
ClusterDropStatsSnapshot dropStatsSnapshot = ref.get().snapshot();
long totalCategorizedDrops = 0L;
for (Map.Entry<String, Long> entry : dropStatsSnapshot.categorizedDrops.entrySet()) {
builder.addDroppedRequests(new DroppedRequests(entry.getKey(), entry.getValue()));
totalCategorizedDrops += entry.getValue();
}
builder.setTotalDroppedRequests(
totalCategorizedDrops + dropStatsSnapshot.uncategorizedDrops);
builder.setLoadReportIntervalNanos(dropStatsSnapshot.durationNano);
statsReportBuilders.put(edsServiceName, builder);
}
clusterDropStats.keySet().removeAll(toDiscard);
}
// Populate load stats for all localities in the cluster.
if (clusterLoadStats != null) {
Set<String> toDiscard = new HashSet<>();
for (String edsServiceName : clusterLoadStats.keySet()) {
ClusterStats.Builder builder = statsReportBuilders.get(edsServiceName);
if (builder == null) {
builder = ClusterStats.newBuilder().setClusterName(cluster);
if (edsServiceName != null) {
builder.setClusterServiceName(edsServiceName);
}
statsReportBuilders.put(edsServiceName, builder);
}
Map<Locality, ReferenceCounted<ClusterLocalityStats>> localityStats =
clusterLoadStats.get(edsServiceName);
Set<Locality> localitiesToDiscard = new HashSet<>();
for (Locality locality : localityStats.keySet()) {
ReferenceCounted<ClusterLocalityStats> ref = localityStats.get(locality);
ClusterLocalityStatsSnapshot snapshot = ref.get().snapshot();
// Only discard stats object after all in-flight calls under recording had finished.
if (ref.getReferenceCount() == 0 && snapshot.callsInProgress == 0) {
localitiesToDiscard.add(locality);
}
UpstreamLocalityStats.Builder localityStatsBuilder = UpstreamLocalityStats.newBuilder();
localityStatsBuilder.setLocality(locality);
localityStatsBuilder.setTotalIssuedRequests(snapshot.callsIssued);
localityStatsBuilder.setTotalSuccessfulRequests(snapshot.callsSucceeded);
localityStatsBuilder.setTotalErrorRequests(snapshot.callsFailed);
localityStatsBuilder.setTotalRequestsInProgress(snapshot.callsInProgress);
builder.addUpstreamLocalityStats(localityStatsBuilder.build());
// Use the max (drops/loads) recording interval as the overall interval for the
// cluster's stats. In general, they should be mostly identical.
builder.setLoadReportIntervalNanos(
Math.max(builder.getLoadReportIntervalNanos(), snapshot.durationNano));
}
localityStats.keySet().removeAll(localitiesToDiscard);
if (localityStats.isEmpty()) {
toDiscard.add(edsServiceName);
}
}
clusterLoadStats.keySet().removeAll(toDiscard);
}
List<ClusterStats> res = new ArrayList<>();
for (ClusterStats.Builder builder : statsReportBuilders.values()) {
res.add(builder.build());
}
return Collections.unmodifiableList(res);
}
/**
* Gets the traffic stats (drops and loads) as a list of {@link ClusterStats} recorded for all
* clusters since the previous call of this method or {@link #getClusterStatsReports} for each
* specific cluster. A {@link ClusterStats} includes stats for a specific cluster with
* edsServiceName.
*/
synchronized List<ClusterStats> getAllClusterStatsReports() {
Set<String> allClusters = Sets.union(allDropStats.keySet(), allLoadStats.keySet());
List<ClusterStats> res = new ArrayList<>();
for (String cluster : allClusters) {
res.addAll(getClusterStatsReports(cluster));
}
return Collections.unmodifiableList(res);
}
/**
* Recorder for dropped requests. One instance per cluster with edsServiceName.
*/
@ThreadSafe
final class ClusterDropStats {
private final String clusterName;
@Nullable
private final String edsServiceName;
private final AtomicLong uncategorizedDrops = new AtomicLong();
private final ConcurrentMap<String, AtomicLong> categorizedDrops = new ConcurrentHashMap<>();
private final Stopwatch stopwatch;
private ClusterDropStats(
String clusterName, @Nullable String edsServiceName, Stopwatch stopwatch) {
this.clusterName = checkNotNull(clusterName, "clusterName");
this.edsServiceName = edsServiceName;
this.stopwatch = checkNotNull(stopwatch, "stopwatch");
stopwatch.reset().start();
}
/**
* Records a dropped request with the specified category.
*/
void recordDroppedRequest(String category) {
// There is a race between this method and snapshot(), causing one drop recorded but may not
// be included in any snapshot. This is acceptable and the race window is extremely small.
AtomicLong counter = categorizedDrops.putIfAbsent(category, new AtomicLong(1L));
if (counter != null) {
counter.getAndIncrement();
}
}
/**
* Records a dropped request without category.
*/
void recordDroppedRequest() {
uncategorizedDrops.getAndIncrement();
}
/**
* Release the <i>hard</i> reference for this stats object (previously obtained via {@link
* LoadStatsManager2#getClusterDropStats}). The object may still be recording
* drops after this method, but there is no guarantee drops recorded after this point will
* be included in load reports.
*/
void release() {
LoadStatsManager2.this.releaseClusterDropCounter(clusterName, edsServiceName);
}
private ClusterDropStatsSnapshot snapshot() {
Map<String, Long> drops = new HashMap<>();
for (Map.Entry<String, AtomicLong> entry : categorizedDrops.entrySet()) {
drops.put(entry.getKey(), entry.getValue().get());
}
categorizedDrops.clear();
long duration = stopwatch.elapsed(TimeUnit.NANOSECONDS);
stopwatch.reset().start();
return new ClusterDropStatsSnapshot(drops, uncategorizedDrops.getAndSet(0), duration);
}
}
private static final class ClusterDropStatsSnapshot {
private final Map<String, Long> categorizedDrops;
private final long uncategorizedDrops;
private final long durationNano;
private ClusterDropStatsSnapshot(
Map<String, Long> categorizedDrops, long uncategorizedDrops, long durationNano) {
this.categorizedDrops = Collections.unmodifiableMap(
checkNotNull(categorizedDrops, "categorizedDrops"));
this.uncategorizedDrops = uncategorizedDrops;
this.durationNano = durationNano;
}
}
/**
* Recorder for client loads. One instance per locality (in cluster with edsService).
*/
@ThreadSafe
final class ClusterLocalityStats {
private final String clusterName;
@Nullable
private final String edsServiceName;
private final Locality locality;
private final Stopwatch stopwatch;
private final AtomicLong callsInProgress = new AtomicLong();
private final AtomicLong callsSucceeded = new AtomicLong();
private final AtomicLong callsFailed = new AtomicLong();
private final AtomicLong callsIssued = new AtomicLong();
private ClusterLocalityStats(
String clusterName, @Nullable String edsServiceName, Locality locality,
Stopwatch stopwatch) {
this.clusterName = checkNotNull(clusterName, "clusterName");
this.edsServiceName = edsServiceName;
this.locality = checkNotNull(locality, "locality");
this.stopwatch = checkNotNull(stopwatch, "stopwatch");
stopwatch.reset().start();
}
/**
* Records a request being issued.
*/
void recordCallStarted() {
callsIssued.getAndIncrement();
callsInProgress.getAndIncrement();
}
/**
* Records a request finished with the given status.
*/
void recordCallFinished(Status status) {
callsInProgress.getAndDecrement();
if (status.isOk()) {
callsSucceeded.getAndIncrement();
} else {
callsFailed.getAndIncrement();
}
}
/**
* Release the <i>hard</i> reference for this stats object (previously obtained via {@link
* LoadStatsManager2#getClusterLocalityStats}). The object may still be
* recording loads after this method, but there is no guarantee loads recorded after this
* point will be included in load reports.
*/
void release() {
LoadStatsManager2.this.releaseClusterLocalityLoadCounter(
clusterName, edsServiceName, locality);
}
private ClusterLocalityStatsSnapshot snapshot() {
long duration = stopwatch.elapsed(TimeUnit.NANOSECONDS);
stopwatch.reset().start();
return new ClusterLocalityStatsSnapshot(callsSucceeded.getAndSet(0), callsInProgress.get(),
callsFailed.getAndSet(0), callsIssued.getAndSet(0), duration);
}
}
private static final class ClusterLocalityStatsSnapshot {
private final long callsSucceeded;
private final long callsInProgress;
private final long callsFailed;
private final long callsIssued;
private final long durationNano;
private ClusterLocalityStatsSnapshot(
long callsSucceeded, long callsInProgress, long callsFailed, long callsIssued,
long durationNano) {
this.callsSucceeded = callsSucceeded;
this.callsInProgress = callsInProgress;
this.callsFailed = callsFailed;
this.callsIssued = callsIssued;
this.durationNano = durationNano;
}
}
}

View File

@ -1,162 +0,0 @@
/*
* Copyright 2019 The gRPC Authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.grpc.xds;
import static com.google.common.base.Preconditions.checkNotNull;
import static java.util.concurrent.TimeUnit.NANOSECONDS;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Stopwatch;
import io.grpc.internal.GrpcUtil;
import io.grpc.xds.ClientLoadCounter.ClientLoadSnapshot;
import io.grpc.xds.ClientLoadCounter.MetricValue;
import io.grpc.xds.EnvoyProtoData.ClusterStats;
import io.grpc.xds.EnvoyProtoData.ClusterStats.DroppedRequests;
import io.grpc.xds.EnvoyProtoData.EndpointLoadMetricStats;
import io.grpc.xds.EnvoyProtoData.Locality;
import io.grpc.xds.EnvoyProtoData.UpstreamLocalityStats;
import io.grpc.xds.LoadStatsManager.LoadStatsStore;
import io.grpc.xds.LoadStatsManager.LoadStatsStoreFactory;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicLong;
import javax.annotation.Nullable;
import javax.annotation.concurrent.GuardedBy;
import javax.annotation.concurrent.ThreadSafe;
/**
* A {@link LoadStatsStoreImpl} maintains load stats per cluster:cluster_service. Load stats for
* endpoints are aggregated in locality granularity while the numbers of dropped calls are
* aggregated in cluster:cluster_service granularity.
*/
@ThreadSafe
final class LoadStatsStoreImpl implements LoadStatsStore {
private final String clusterName;
@Nullable
private final String clusterServiceName;
@GuardedBy("this")
private final Map<Locality, ReferenceCounted<ClientLoadCounter>> localityLoadCounters
= new HashMap<>();
private final AtomicLong uncategorizedDrops = new AtomicLong();
// Cluster level dropped request counts for each category decision.
private final ConcurrentMap<String, AtomicLong> dropCounters = new ConcurrentHashMap<>();
private final Stopwatch stopwatch;
LoadStatsStoreImpl(String clusterName, @Nullable String clusterServiceName) {
this(clusterName, clusterServiceName, GrpcUtil.STOPWATCH_SUPPLIER.get());
}
@VisibleForTesting
LoadStatsStoreImpl(String clusterName, @Nullable String clusterServiceName,
Stopwatch stopwatch) {
this.clusterName = checkNotNull(clusterName, "clusterName");
this.clusterServiceName = clusterServiceName;
this.stopwatch = checkNotNull(stopwatch, "stopwatch");
stopwatch.reset().start();
}
@Override
public synchronized ClusterStats generateLoadReport() {
ClusterStats.Builder statsBuilder = ClusterStats.newBuilder();
statsBuilder.setClusterName(clusterName);
if (clusterServiceName != null) {
statsBuilder.setClusterServiceName(clusterServiceName);
}
Set<Locality> untrackedLocalities = new HashSet<>();
for (Map.Entry<Locality, ReferenceCounted<ClientLoadCounter>> entry
: localityLoadCounters.entrySet()) {
ClientLoadSnapshot snapshot = entry.getValue().get().snapshot();
UpstreamLocalityStats.Builder localityStatsBuilder =
UpstreamLocalityStats.newBuilder().setLocality(entry.getKey());
localityStatsBuilder
.setTotalSuccessfulRequests(snapshot.getCallsSucceeded())
.setTotalErrorRequests(snapshot.getCallsFailed())
.setTotalRequestsInProgress(snapshot.getCallsInProgress())
.setTotalIssuedRequests(snapshot.getCallsIssued());
for (Map.Entry<String, MetricValue> metric : snapshot.getMetricValues().entrySet()) {
localityStatsBuilder.addLoadMetricStats(
EndpointLoadMetricStats.newBuilder()
.setMetricName(metric.getKey())
.setNumRequestsFinishedWithMetric(metric.getValue().getNumReports())
.setTotalMetricValue(metric.getValue().getTotalValue())
.build());
}
statsBuilder.addUpstreamLocalityStats(localityStatsBuilder.build());
if (entry.getValue().getReferenceCount() == 0 && snapshot.getCallsInProgress() == 0) {
untrackedLocalities.add(entry.getKey());
}
}
localityLoadCounters.keySet().removeAll(untrackedLocalities);
long totalDrops = uncategorizedDrops.getAndSet(0);
for (Map.Entry<String, AtomicLong> entry : dropCounters.entrySet()) {
long drops = entry.getValue().getAndSet(0);
totalDrops += drops;
statsBuilder.addDroppedRequests(new DroppedRequests(entry.getKey(), drops));
}
statsBuilder.setTotalDroppedRequests(totalDrops);
statsBuilder.setLoadReportIntervalNanos(stopwatch.elapsed(NANOSECONDS));
stopwatch.reset().start();
return statsBuilder.build();
}
@Override
public synchronized ClientLoadCounter addLocality(final Locality locality) {
ReferenceCounted<ClientLoadCounter> counter = localityLoadCounters.get(locality);
if (counter == null) {
counter = ReferenceCounted.wrap(new ClientLoadCounter());
localityLoadCounters.put(locality, counter);
}
counter.retain();
return counter.get();
}
@Override
public synchronized void removeLocality(final Locality locality) {
ReferenceCounted<ClientLoadCounter> counter = localityLoadCounters.get(locality);
counter.release();
}
@Override
public void recordDroppedRequest(String category) {
AtomicLong counter = dropCounters.get(category);
if (counter == null) {
counter = dropCounters.putIfAbsent(category, new AtomicLong());
if (counter == null) {
counter = dropCounters.get(category);
}
}
counter.getAndIncrement();
}
@Override
public void recordDroppedRequest() {
uncategorizedDrops.getAndIncrement();
}
static LoadStatsStoreFactory getDefaultFactory() {
return new LoadStatsStoreFactory() {
@Override
public LoadStatsStore newLoadStatsStore(String cluster, String clusterService) {
return new LoadStatsStoreImpl(cluster, clusterService);
}
};
}
}

View File

@ -1,123 +0,0 @@
/*
* Copyright 2020 The gRPC Authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.grpc.xds;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import io.grpc.ConnectivityState;
import io.grpc.LoadBalancer;
import io.grpc.Status;
import io.grpc.util.ForwardingLoadBalancerHelper;
import io.grpc.util.GracefulSwitchLoadBalancer;
import io.grpc.xds.ClientLoadCounter.LoadRecordingSubchannelPicker;
import io.grpc.xds.EnvoyProtoData.Locality;
import io.grpc.xds.LoadStatsManager.LoadStatsStore;
import io.grpc.xds.LrsLoadBalancerProvider.LrsConfig;
import io.grpc.xds.XdsSubchannelPickers.ErrorPicker;
import java.util.Objects;
import javax.annotation.CheckForNull;
/**
* Load balancer for lrs policy.
*/
final class LrsLoadBalancer extends LoadBalancer {
private final LoadBalancer.Helper helper;
@CheckForNull
private GracefulSwitchLoadBalancer switchingLoadBalancer;
private LoadStatsStore loadStatsStore;
private String clusterName;
private String edsServiceName;
private Locality locality;
private String childPolicyName;
LrsLoadBalancer(LoadBalancer.Helper helper) {
this.helper = checkNotNull(helper, "helper");
}
@Override
public void handleResolvedAddresses(ResolvedAddresses resolvedAddresses) {
LrsConfig config = (LrsConfig) resolvedAddresses.getLoadBalancingPolicyConfig();
LoadStatsStore store = resolvedAddresses.getAttributes().get(
InternalXdsAttributes.ATTR_CLUSTER_SERVICE_LOAD_STATS_STORE);
checkNotNull(config, "missing LRS lb config");
checkNotNull(store, "missing cluster service stats object");
checkAndSetUp(config, store);
if (switchingLoadBalancer == null) {
final ClientLoadCounter counter = loadStatsStore.addLocality(config.locality);
LoadBalancer.Helper loadRecordingHelper = new ForwardingLoadBalancerHelper() {
@Override
protected Helper delegate() {
return helper;
}
@Override
public void updateBalancingState(ConnectivityState newState, SubchannelPicker newPicker) {
SubchannelPicker loadRecordingPicker =
new LoadRecordingSubchannelPicker(counter, newPicker);
super.updateBalancingState(newState, loadRecordingPicker);
}
};
switchingLoadBalancer = new GracefulSwitchLoadBalancer(loadRecordingHelper);
}
String updatedChildPolicyName = config.childPolicy.getProvider().getPolicyName();
if (!Objects.equals(childPolicyName, updatedChildPolicyName)) {
switchingLoadBalancer.switchTo(config.childPolicy.getProvider());
childPolicyName = updatedChildPolicyName;
}
ResolvedAddresses downStreamResult =
resolvedAddresses.toBuilder()
.setLoadBalancingPolicyConfig(config.childPolicy.getConfig())
.build();
switchingLoadBalancer.handleResolvedAddresses(downStreamResult);
}
@Override
public void handleNameResolutionError(Status error) {
if (switchingLoadBalancer != null) {
switchingLoadBalancer.handleNameResolutionError(error);
} else {
helper.updateBalancingState(ConnectivityState.TRANSIENT_FAILURE, new ErrorPicker(error));
}
}
@Override
public void shutdown() {
if (switchingLoadBalancer != null) {
loadStatsStore.removeLocality(locality);
switchingLoadBalancer.shutdown();
}
}
private void checkAndSetUp(LrsConfig config, LoadStatsStore store) {
checkState(
clusterName == null || clusterName.equals(config.clusterName),
"cluster name should not change");
checkState(
edsServiceName == null || edsServiceName.equals(config.edsServiceName),
"edsServiceName should not change");
checkState(locality == null || locality.equals(config.locality), "locality should not change");
checkState(
loadStatsStore == null || loadStatsStore.equals(store),
"loadStatsStore should not change");
clusterName = config.clusterName;
edsServiceName = config.edsServiceName;
locality = config.locality;
loadStatsStore = store;
}
}

View File

@ -1,96 +0,0 @@
/*
* Copyright 2020 The gRPC Authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.grpc.xds;
import static com.google.common.base.Preconditions.checkNotNull;
import com.google.common.base.MoreObjects;
import io.grpc.Internal;
import io.grpc.LoadBalancer;
import io.grpc.LoadBalancerProvider;
import io.grpc.NameResolver.ConfigOrError;
import io.grpc.internal.ServiceConfigUtil.PolicySelection;
import io.grpc.xds.EnvoyProtoData.Locality;
import java.util.Map;
import javax.annotation.Nullable;
/**
* Provider for lrs load balancing policy.
*/
@Internal
public final class LrsLoadBalancerProvider extends LoadBalancerProvider {
private static final String LRS_POLICY_NAME = "lrs_experimental";
@Override
public LoadBalancer newLoadBalancer(LoadBalancer.Helper helper) {
return new LrsLoadBalancer(helper);
}
@Override
public boolean isAvailable() {
return true;
}
@Override
public int getPriority() {
return 5;
}
@Override
public String getPolicyName() {
return LRS_POLICY_NAME;
}
@Override
public ConfigOrError parseLoadBalancingPolicyConfig(Map<String, ?> rawConfig) {
throw new UnsupportedOperationException();
}
static final class LrsConfig {
final String clusterName;
@Nullable
final String edsServiceName;
final String lrsServerName;
final Locality locality;
final PolicySelection childPolicy;
LrsConfig(
String clusterName,
@Nullable String edsServiceName,
String lrsServerName,
Locality locality,
PolicySelection childPolicy) {
this.clusterName = checkNotNull(clusterName, "clusterName");
this.edsServiceName = edsServiceName;
this.lrsServerName = checkNotNull(lrsServerName, "lrsServerName");
this.locality = checkNotNull(locality, "locality");
this.childPolicy = checkNotNull(childPolicy, "childPolicy");
}
@Override
public String toString() {
return MoreObjects.toStringHelper(this)
.add("clusterName", clusterName)
.add("edsServiceName", edsServiceName)
.add("lrsServerName", lrsServerName)
.add("locality", locality)
.add("childPolicy", childPolicy)
.toString();
}
}
}

View File

@ -28,6 +28,7 @@ import static com.google.common.base.Preconditions.checkState;
* wrapper itself should never be returned to the consumers of the elements to avoid reference
* counts being leaked.
*/
// TODO(chengyuanzhang): move this class into LoadStatsManager2.
final class ReferenceCounted<T> {
private final T instance;
private int refs;

View File

@ -29,7 +29,8 @@ import io.grpc.xds.EnvoyProtoData.LocalityLbEndpoints;
import io.grpc.xds.EnvoyProtoData.VirtualHost;
import io.grpc.xds.EnvoyServerProtoData.Listener;
import io.grpc.xds.EnvoyServerProtoData.UpstreamTlsContext;
import io.grpc.xds.LoadStatsManager.LoadStatsStore;
import io.grpc.xds.LoadStatsManager2.ClusterDropStats;
import io.grpc.xds.LoadStatsManager2.ClusterLocalityStats;
import java.util.ArrayList;
import java.util.Collections;
import java.util.LinkedHashMap;
@ -568,20 +569,26 @@ abstract class XdsClient {
}
/**
* Starts recording client load stats for the given cluster:cluster_service. Caller should use
* the returned {@link LoadStatsStore} to record and aggregate stats for load sent to the given
* cluster:cluster_service. The first call of this method starts load reporting via LRS.
* Adds drop stats for the specified cluster with edsServiceName by using the returned object
* to record dropped requests. Drop stats recorded with the returned object will be reported
* to the load reporting server. The returned object is reference counted and the caller should
* use {@link ClusterDropStats#release} to release its <i>hard</i> reference when it is safe to
* stop reporting dropped RPCs for the specified cluster in the future.
*/
LoadStatsStore addClientStats(String clusterName, @Nullable String clusterServiceName) {
ClusterDropStats addClusterDropStats(String clusterName, @Nullable String edsServiceName) {
throw new UnsupportedOperationException();
}
/**
* Stops recording client load stats for the given cluster:cluster_service. The load reporting
* server will no longer receive stats for the given cluster:cluster_service after this call.
* Load reporting may be terminated if there is no stats to be reported.
* Adds load stats for the specified locality (in the specified cluster with edsServiceName) by
* using the returned object to record RPCs. Load stats recorded with the returned object will
* be reported to the load reporting server. The returned object is reference counted and the
* caller should use {@link ClusterLocalityStats#release} to release its <i>hard</i>
* reference when it is safe to stop reporting RPC loads for the specified locality in the
* future.
*/
void removeClientStats(String clusterName, @Nullable String clusterServiceName) {
ClusterLocalityStats addClusterLocalityStats(
String clusterName, @Nullable String edsServiceName, Locality locality) {
throw new UnsupportedOperationException();
}
}

View File

@ -19,12 +19,10 @@ package io.grpc.xds;
final class XdsLbPolicies {
static final String CLUSTER_MANAGER_POLICY_NAME = "cluster_manager_experimental";
static final String CDS_POLICY_NAME = "cds_experimental";
static final String EDS_POLICY_NAME = "eds_experimental";
static final String CLUSTER_RESOLVER_POLICY_NAME = "cluster_resolver_experimental";
static final String PRIORITY_POLICY_NAME = "priority_experimental";
static final String CLUSTER_IMPL_POLICY_NAME = "cluster_impl_experimental";
static final String WEIGHTED_TARGET_POLICY_NAME = "weighted_target_experimental";
static final String LRS_POLICY_NAME = "lrs_experimental";
private XdsLbPolicies() {}
}

View File

@ -1,7 +1,6 @@
io.grpc.xds.CdsLoadBalancerProvider
io.grpc.xds.PriorityLoadBalancerProvider
io.grpc.xds.WeightedTargetLoadBalancerProvider
io.grpc.xds.LrsLoadBalancerProvider
io.grpc.xds.ClusterManagerLoadBalancerProvider
io.grpc.xds.ClusterResolverLoadBalancerProvider
io.grpc.xds.ClusterImplLoadBalancerProvider

View File

@ -1,353 +0,0 @@
/*
* Copyright 2019 The gRPC Authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.grpc.xds;
import static com.google.common.truth.Truth.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.same;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import com.github.udpa.udpa.data.orca.v1.OrcaLoadReport;
import io.grpc.ClientStreamTracer;
import io.grpc.ClientStreamTracer.StreamInfo;
import io.grpc.LoadBalancer.PickResult;
import io.grpc.LoadBalancer.PickSubchannelArgs;
import io.grpc.LoadBalancer.Subchannel;
import io.grpc.LoadBalancer.SubchannelPicker;
import io.grpc.Metadata;
import io.grpc.Status;
import io.grpc.xds.ClientLoadCounter.ClientLoadSnapshot;
import io.grpc.xds.ClientLoadCounter.LoadRecordingStreamTracerFactory;
import io.grpc.xds.ClientLoadCounter.LoadRecordingSubchannelPicker;
import io.grpc.xds.ClientLoadCounter.MetricValue;
import io.grpc.xds.ClientLoadCounter.MetricsObservingSubchannelPicker;
import io.grpc.xds.ClientLoadCounter.MetricsRecordingListener;
import io.grpc.xds.ClientLoadCounter.TracerWrappingSubchannelPicker;
import io.grpc.xds.OrcaPerRequestUtil.OrcaPerRequestReportListener;
import java.util.concurrent.ThreadLocalRandom;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
/** Unit tests for {@link ClientLoadCounter}. */
@RunWith(JUnit4.class)
public class ClientLoadCounterTest {
private static final ClientStreamTracer.StreamInfo STREAM_INFO =
ClientStreamTracer.StreamInfo.newBuilder().build();
private static final ClientStreamTracer.Factory NOOP_CLIENT_STREAM_TRACER_FACTORY =
new ClientStreamTracer.Factory() {
@Override
public ClientStreamTracer newClientStreamTracer(StreamInfo info, Metadata headers) {
return new ClientStreamTracer() {
};
}
};
private ClientLoadCounter counter;
@Before
public void setUp() {
counter = new ClientLoadCounter();
ClientLoadSnapshot emptySnapshot = counter.snapshot();
assertQueryCounts(emptySnapshot, 0, 0, 0, 0);
assertThat(emptySnapshot.getMetricValues()).isEmpty();
}
@Test
public void snapshotContainsDataInCounter() {
long numSucceededCalls = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
long numInProgressCalls = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
long numFailedCalls = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
long numIssuedCalls = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
counter = new ClientLoadCounter();
counter.setCallsSucceeded(numSucceededCalls);
counter.setCallsInProgress(numInProgressCalls);
counter.setCallsFailed(numFailedCalls);
counter.setCallsIssued(numIssuedCalls);
ClientLoadSnapshot snapshot = counter.snapshot();
assertQueryCounts(snapshot, numSucceededCalls, numInProgressCalls, numFailedCalls,
numIssuedCalls);
String snapshotStr = snapshot.toString();
assertThat(snapshotStr).contains("callsSucceeded=" + numSucceededCalls);
assertThat(snapshotStr).contains("callsInProgress=" + numInProgressCalls);
assertThat(snapshotStr).contains("callsFailed=" + numFailedCalls);
assertThat(snapshotStr).contains("callsIssued=" + numIssuedCalls);
assertThat(snapshotStr).contains("metricValues={}");
// Snapshot only accounts for stats happening after previous snapshot.
snapshot = counter.snapshot();
assertQueryCounts(snapshot, 0, numInProgressCalls, 0, 0);
snapshotStr = snapshot.toString();
assertThat(snapshotStr).contains("callsSucceeded=0");
assertThat(snapshotStr).contains("callsInProgress=" + numInProgressCalls);
assertThat(snapshotStr).contains("callsFailed=0");
assertThat(snapshotStr).contains("callsIssued=0");
assertThat(snapshotStr).contains("metricValues={}");
}
@Test
public void normalRecordingOperations() {
counter.recordCallStarted();
ClientLoadSnapshot snapshot = counter.snapshot();
assertQueryCounts(snapshot, 0, 1, 0, 1);
counter.recordCallFinished(Status.OK);
snapshot = counter.snapshot();
assertQueryCounts(snapshot, 1, 0, 0, 0);
counter.recordCallStarted();
counter.recordCallFinished(Status.CANCELLED);
snapshot = counter.snapshot();
assertQueryCounts(snapshot, 0, 0, 1, 1);
counter.recordMetric("test-metric-1", 0.75);
counter.recordMetric("test-metric-2", 0.342);
counter.recordMetric("test-metric-3", 0.512);
counter.recordMetric("test-metric-1", 0.543);
counter.recordMetric("test-metric-1", 4.412);
counter.recordMetric("test-metric-1", 100.353);
snapshot = counter.snapshot();
assertThat(snapshot.getMetricValues().get("test-metric-1").getNumReports()).isEqualTo(4);
assertThat(snapshot.getMetricValues().get("test-metric-1").getTotalValue())
.isEqualTo(0.75 + 0.543 + 4.412 + 100.353);
assertThat(snapshot.getMetricValues().get("test-metric-2").getNumReports()).isEqualTo(1);
assertThat(snapshot.getMetricValues().get("test-metric-2").getTotalValue()).isEqualTo(0.342);
assertThat(snapshot.getMetricValues().get("test-metric-3").getNumReports()).isEqualTo(1);
assertThat(snapshot.getMetricValues().get("test-metric-3").getTotalValue()).isEqualTo(0.512);
}
@Test
public void loadRecordingStreamTracerFactory_clientSideQueryCountsAggregation() {
LoadRecordingStreamTracerFactory factory1 =
new LoadRecordingStreamTracerFactory(counter, NOOP_CLIENT_STREAM_TRACER_FACTORY);
ClientStreamTracer tracer = factory1.newClientStreamTracer(STREAM_INFO, new Metadata());
ClientLoadSnapshot snapshot = counter.snapshot();
assertQueryCounts(snapshot, 0, 1, 0, 1);
tracer.streamClosed(Status.OK);
snapshot = counter.snapshot();
assertQueryCounts(snapshot, 1, 0, 0, 0);
// Create a second LoadRecordingStreamTracerFactory with the same counter, stats are aggregated
// together.
LoadRecordingStreamTracerFactory factory2 =
new LoadRecordingStreamTracerFactory(counter, NOOP_CLIENT_STREAM_TRACER_FACTORY);
factory1.newClientStreamTracer(STREAM_INFO, new Metadata()).streamClosed(Status.ABORTED);
factory2.newClientStreamTracer(STREAM_INFO, new Metadata()).streamClosed(Status.CANCELLED);
snapshot = counter.snapshot();
assertQueryCounts(snapshot, 0, 0, 2, 2);
}
@Test
public void metricsRecordingListener_backendMetricsAggregation() {
MetricsRecordingListener listener1 = new MetricsRecordingListener(counter);
OrcaLoadReport report =
OrcaLoadReport.newBuilder()
.setCpuUtilization(0.5345)
.setMemUtilization(0.647)
.putRequestCost("named-cost-1", 3453.3525)
.putRequestCost("named-cost-2", 532543.14234)
.build();
listener1.onLoadReport(report);
// Simulate an empty load report.
listener1.onLoadReport(OrcaLoadReport.getDefaultInstance());
ClientLoadSnapshot snapshot = counter.snapshot();
MetricValue cpuMetric = snapshot.getMetricValues().get("cpu_utilization");
assertThat(cpuMetric.getNumReports()).isEqualTo(2);
assertThat(cpuMetric.getTotalValue()).isEqualTo(0.5345);
MetricValue memMetric = snapshot.getMetricValues().get("mem_utilization");
assertThat(memMetric.getNumReports()).isEqualTo(2);
assertThat(memMetric.getTotalValue()).isEqualTo(0.647);
MetricValue namedMetric1 = snapshot.getMetricValues().get("named-cost-1");
assertThat(namedMetric1.getNumReports()).isEqualTo(1);
assertThat(namedMetric1.getTotalValue()).isEqualTo(3453.3525);
MetricValue namedMetric2 = snapshot.getMetricValues().get("named-cost-2");
assertThat(namedMetric2.getNumReports()).isEqualTo(1);
assertThat(namedMetric2.getTotalValue()).isEqualTo(532543.14234);
snapshot = counter.snapshot();
assertThat(snapshot.getMetricValues()).isEmpty();
MetricsRecordingListener listener2 = new MetricsRecordingListener(counter);
report =
OrcaLoadReport.newBuilder()
.setCpuUtilization(0.3423)
.setMemUtilization(0.654)
.putUtilization("named-utilization", 0.7563)
.build();
// Two listeners with the same counter aggregate metrics together.
listener1.onLoadReport(report);
listener2.onLoadReport(report);
snapshot = counter.snapshot();
cpuMetric = snapshot.getMetricValues().get("cpu_utilization");
assertThat(cpuMetric.getNumReports()).isEqualTo(2);
assertThat(cpuMetric.getTotalValue()).isEqualTo(0.3423 + 0.3423);
memMetric = snapshot.getMetricValues().get("mem_utilization");
assertThat(memMetric.getNumReports()).isEqualTo(2);
assertThat(memMetric.getTotalValue()).isEqualTo(0.654 + 0.654);
MetricValue namedMetric = snapshot.getMetricValues().get("named-utilization");
assertThat(namedMetric.getNumReports()).isEqualTo(2);
assertThat(namedMetric.getTotalValue()).isEqualTo(0.7563 + 0.7563);
}
@Test
public void tracerWrappingSubchannelPicker_interceptPickResult_invalidPickResultNotIntercepted() {
final SubchannelPicker picker = mock(SubchannelPicker.class);
SubchannelPicker streamInstrSubchannelPicker = new TracerWrappingSubchannelPicker() {
@Override
protected SubchannelPicker delegate() {
return picker;
}
@Override
protected ClientStreamTracer.Factory wrapTracerFactory(
ClientStreamTracer.Factory originFactory) {
// NO-OP
return originFactory;
}
};
PickResult errorResult = PickResult.withError(Status.UNAVAILABLE.withDescription("Error"));
PickResult droppedResult = PickResult.withDrop(Status.UNAVAILABLE.withDescription("Dropped"));
PickResult emptyResult = PickResult.withNoResult();
when(picker.pickSubchannel(any(PickSubchannelArgs.class)))
.thenReturn(errorResult, droppedResult, emptyResult);
PickSubchannelArgs args = mock(PickSubchannelArgs.class);
PickResult interceptedErrorResult = streamInstrSubchannelPicker.pickSubchannel(args);
PickResult interceptedDroppedResult = streamInstrSubchannelPicker.pickSubchannel(args);
PickResult interceptedEmptyResult = streamInstrSubchannelPicker.pickSubchannel(args);
assertThat(interceptedErrorResult).isSameInstanceAs(errorResult);
assertThat(interceptedDroppedResult).isSameInstanceAs(droppedResult);
assertThat(interceptedEmptyResult).isSameInstanceAs(emptyResult);
}
@Test
public void loadRecordingSubchannelPicker_interceptPickResult_applyLoadRecorderToPickResult() {
ClientStreamTracer.Factory mockFactory = mock(ClientStreamTracer.Factory.class);
ClientStreamTracer mockTracer = mock(ClientStreamTracer.class);
when(mockFactory
.newClientStreamTracer(any(ClientStreamTracer.StreamInfo.class), any(Metadata.class)))
.thenReturn(mockTracer);
ClientLoadCounter localityCounter1 = new ClientLoadCounter();
ClientLoadCounter localityCounter2 = new ClientLoadCounter();
final PickResult pickResult1 = PickResult.withSubchannel(mock(Subchannel.class), mockFactory);
final PickResult pickResult2 = PickResult.withSubchannel(mock(Subchannel.class));
SubchannelPicker picker1 = new SubchannelPicker() {
@Override
public PickResult pickSubchannel(PickSubchannelArgs args) {
return pickResult1;
}
};
SubchannelPicker picker2 = new SubchannelPicker() {
@Override
public PickResult pickSubchannel(PickSubchannelArgs args) {
return pickResult2;
}
};
SubchannelPicker loadRecordingPicker1 =
new LoadRecordingSubchannelPicker(localityCounter1, picker1);
SubchannelPicker loadRecordingPicker2 =
new LoadRecordingSubchannelPicker(localityCounter2, picker2);
PickSubchannelArgs args = mock(PickSubchannelArgs.class);
PickResult interceptedPickResult1 = loadRecordingPicker1.pickSubchannel(args);
PickResult interceptedPickResult2 = loadRecordingPicker2.pickSubchannel(args);
LoadRecordingStreamTracerFactory recorder1 =
(LoadRecordingStreamTracerFactory) interceptedPickResult1.getStreamTracerFactory();
LoadRecordingStreamTracerFactory recorder2 =
(LoadRecordingStreamTracerFactory) interceptedPickResult2.getStreamTracerFactory();
assertThat(recorder1.getCounter()).isSameInstanceAs(localityCounter1);
assertThat(recorder2.getCounter()).isSameInstanceAs(localityCounter2);
// Stream tracing is propagated to downstream tracers, which preserves PickResult's original
// tracing functionality.
Metadata metadata = new Metadata();
interceptedPickResult1.getStreamTracerFactory().newClientStreamTracer(STREAM_INFO, metadata)
.streamClosed(Status.OK);
verify(mockFactory).newClientStreamTracer(same(STREAM_INFO), same(metadata));
verify(mockTracer).streamClosed(Status.OK);
}
@Test
public void metricsObservingSubchannelPicker_interceptPickResult_applyOrcaListenerToPickResult() {
ClientStreamTracer.Factory mockFactory = mock(ClientStreamTracer.Factory.class);
ClientStreamTracer mockTracer = mock(ClientStreamTracer.class);
when(mockFactory
.newClientStreamTracer(any(ClientStreamTracer.StreamInfo.class), any(Metadata.class)))
.thenReturn(mockTracer);
final PickResult pickResult1 = PickResult.withSubchannel(mock(Subchannel.class), mockFactory);
final PickResult pickResult2 = PickResult.withSubchannel(mock(Subchannel.class));
SubchannelPicker picker1 = new SubchannelPicker() {
@Override
public PickResult pickSubchannel(PickSubchannelArgs args) {
return pickResult1;
}
};
SubchannelPicker picker2 = new SubchannelPicker() {
@Override
public PickResult pickSubchannel(PickSubchannelArgs args) {
return pickResult2;
}
};
OrcaPerRequestUtil orcaPerRequestUtil = mock(OrcaPerRequestUtil.class);
ClientStreamTracer.Factory metricsRecorder1 = mock(ClientStreamTracer.Factory.class);
ClientStreamTracer.Factory metricsRecorder2 = mock(ClientStreamTracer.Factory.class);
when(orcaPerRequestUtil.newOrcaClientStreamTracerFactory(any(ClientStreamTracer.Factory.class),
any(OrcaPerRequestReportListener.class))).thenReturn(metricsRecorder1, metricsRecorder2);
OrcaPerRequestReportListener listener1 = mock(OrcaPerRequestReportListener.class);
OrcaPerRequestReportListener listener2 = mock(OrcaPerRequestReportListener.class);
PickSubchannelArgs args = mock(PickSubchannelArgs.class);
SubchannelPicker metricsObservingPicker1 =
new MetricsObservingSubchannelPicker(listener1, picker1, orcaPerRequestUtil);
SubchannelPicker metricsObservingPicker2 =
new MetricsObservingSubchannelPicker(listener2, picker2, orcaPerRequestUtil);
PickResult interceptedPickResult1 = metricsObservingPicker1.pickSubchannel(args);
PickResult interceptedPickResult2 = metricsObservingPicker2.pickSubchannel(args);
verify(orcaPerRequestUtil)
.newOrcaClientStreamTracerFactory(any(ClientStreamTracer.Factory.class), same(listener1));
verify(orcaPerRequestUtil)
.newOrcaClientStreamTracerFactory(any(ClientStreamTracer.Factory.class), same(listener2));
assertThat(interceptedPickResult1.getStreamTracerFactory()).isSameInstanceAs(metricsRecorder1);
assertThat(interceptedPickResult2.getStreamTracerFactory()).isSameInstanceAs(metricsRecorder2);
}
private void assertQueryCounts(ClientLoadSnapshot snapshot,
long callsSucceeded,
long callsInProgress,
long callsFailed,
long callsIssued) {
assertThat(snapshot.getCallsSucceeded()).isEqualTo(callsSucceeded);
assertThat(snapshot.getCallsInProgress()).isEqualTo(callsInProgress);
assertThat(snapshot.getCallsFailed()).isEqualTo(callsFailed);
assertThat(snapshot.getCallsIssued()).isEqualTo(callsIssued);
}
}

View File

@ -50,6 +50,7 @@ import io.grpc.xds.EnvoyProtoData.Locality;
import io.grpc.xds.EnvoyProtoData.LocalityLbEndpoints;
import io.grpc.xds.EnvoyProtoData.Node;
import io.grpc.xds.EnvoyServerProtoData.UpstreamTlsContext;
import io.grpc.xds.LoadStatsManager2.ClusterDropStats;
import io.grpc.xds.XdsClient.CdsResourceWatcher;
import io.grpc.xds.XdsClient.CdsUpdate;
import io.grpc.xds.XdsClient.CdsUpdate.AggregateClusterConfig;
@ -1423,13 +1424,10 @@ public abstract class ClientXdsClientTestBase {
assertThat(fakeClock.getPendingTasks(EDS_RESOURCE_FETCH_TIMEOUT_TASK_FILTER)).hasSize(1);
}
/**
* Tests sending a streaming LRS RPC for each cluster to report loads for.
*/
@Test
public void reportLoadStatsToServer() {
String clusterName = "cluster-foo.googleapis.com";
xdsClient.addClientStats(clusterName, null);
ClusterDropStats dropStats = xdsClient.addClusterDropStats(clusterName, null);
LrsRpcCall lrsCall = loadReportCalls.poll();
lrsCall.verifyNextReportClusters(Collections.<String[]>emptyList()); // initial LRS request
@ -1437,8 +1435,15 @@ public abstract class ClientXdsClientTestBase {
fakeClock.forwardNanos(1000L);
lrsCall.verifyNextReportClusters(Collections.singletonList(new String[] {clusterName, null}));
xdsClient.removeClientStats(clusterName, null);
dropStats.release();
fakeClock.forwardNanos(1000L);
// In case of having unreported cluster stats, one last report will be sent after corresponding
// stats object released.
lrsCall.verifyNextReportClusters(Collections.singletonList(new String[] {clusterName, null}));
fakeClock.forwardNanos(1000L);
// Currently load reporting continues (with empty stats) even if all stats objects have been
// released.
lrsCall.verifyNextReportClusters(Collections.<String[]>emptyList()); // no more stats reported
// See more test on LoadReportClientTest.java

View File

@ -41,16 +41,18 @@ import io.grpc.Metadata;
import io.grpc.Status;
import io.grpc.Status.Code;
import io.grpc.SynchronizationContext;
import io.grpc.internal.FakeClock;
import io.grpc.internal.ObjectPool;
import io.grpc.internal.ServiceConfigUtil.PolicySelection;
import io.grpc.xds.ClusterImplLoadBalancerProvider.ClusterImplConfig;
import io.grpc.xds.EnvoyProtoData.ClusterStats;
import io.grpc.xds.EnvoyProtoData.DropOverload;
import io.grpc.xds.EnvoyProtoData.Locality;
import io.grpc.xds.EnvoyProtoData.UpstreamLocalityStats;
import io.grpc.xds.EnvoyServerProtoData.DownstreamTlsContext;
import io.grpc.xds.EnvoyServerProtoData.UpstreamTlsContext;
import io.grpc.xds.LoadStatsManager.LoadStatsStore;
import io.grpc.xds.LrsLoadBalancerProvider.LrsConfig;
import io.grpc.xds.LoadStatsManager2.ClusterDropStats;
import io.grpc.xds.LoadStatsManager2.ClusterLocalityStats;
import io.grpc.xds.WeightedTargetLoadBalancerProvider.WeightedPolicySelection;
import io.grpc.xds.WeightedTargetLoadBalancerProvider.WeightedTargetConfig;
import io.grpc.xds.XdsNameResolverProvider.CallCounterProvider;
@ -93,12 +95,15 @@ public class ClusterImplLoadBalancerTest {
throw new AssertionError(e);
}
});
private final FakeClock fakeClock = new FakeClock();
private final Locality locality =
new Locality("test-region", "test-zone", "test-subzone");
private final PolicySelection roundRobin =
new PolicySelection(new FakeLoadBalancerProvider("round_robin"), null);
private final List<FakeLoadBalancer> downstreamBalancers = new ArrayList<>();
private final FakeTlsContextManager tlsContextManager = new FakeTlsContextManager();
private final LoadStatsManager2 loadStatsManager =
new LoadStatsManager2(fakeClock.getStopwatchSupplier());
private final FakeXdsClient xdsClient = new FakeXdsClient();
private final ObjectPool<XdsClient> xdsClientPool = new ObjectPool<XdsClient>() {
@Override
@ -136,7 +141,6 @@ public class ClusterImplLoadBalancerTest {
@After
public void tearDown() {
loadBalancer.shutdown();
assertThat(xdsClient.clusterStats).isNull();
assertThat(xdsClientRefs).isEqualTo(0);
assertThat(downstreamBalancers).isEmpty();
}
@ -156,8 +160,6 @@ public class ClusterImplLoadBalancerTest {
assertThat(childBalancer.config).isSameInstanceAs(weightedTargetConfig);
assertThat(childBalancer.attributes.get(InternalXdsAttributes.XDS_CLIENT_POOL))
.isSameInstanceAs(xdsClientPool);
assertThat(childBalancer.attributes.get(
InternalXdsAttributes.ATTR_CLUSTER_SERVICE_LOAD_STATS_STORE)).isNotNull();
}
@Test
@ -189,6 +191,57 @@ public class ClusterImplLoadBalancerTest {
.isEqualTo("cannot reach server");
}
@Test
public void recordLoadStats() {
LoadBalancerProvider weightedTargetProvider = new WeightedTargetLoadBalancerProvider();
WeightedTargetConfig weightedTargetConfig =
buildWeightedTargetConfig(ImmutableMap.of(locality, 10));
ClusterImplConfig config = new ClusterImplConfig(CLUSTER, EDS_SERVICE_NAME, LRS_SERVER_NAME,
null, Collections.<DropOverload>emptyList(),
new PolicySelection(weightedTargetProvider, weightedTargetConfig), null);
EquivalentAddressGroup endpoint = makeAddress("endpoint-addr", locality);
deliverAddressesAndConfig(Collections.singletonList(endpoint), config);
FakeLoadBalancer leafBalancer = Iterables.getOnlyElement(downstreamBalancers);
Subchannel subchannel = leafBalancer.helper.createSubchannel(
CreateSubchannelArgs.newBuilder().setAddresses(leafBalancer.addresses).build());
leafBalancer.deliverSubchannelState(subchannel, ConnectivityState.READY);
assertThat(currentState).isEqualTo(ConnectivityState.READY);
PickResult result = currentPicker.pickSubchannel(mock(PickSubchannelArgs.class));
assertThat(result.getStatus().isOk()).isTrue();
ClientStreamTracer streamTracer1 = result.getStreamTracerFactory().newClientStreamTracer(
ClientStreamTracer.StreamInfo.newBuilder().build(), new Metadata()); // first RPC call
ClientStreamTracer streamTracer2 = result.getStreamTracerFactory().newClientStreamTracer(
ClientStreamTracer.StreamInfo.newBuilder().build(), new Metadata()); // second RPC call
ClientStreamTracer streamTracer3 = result.getStreamTracerFactory().newClientStreamTracer(
ClientStreamTracer.StreamInfo.newBuilder().build(), new Metadata()); // third RPC call
streamTracer1.streamClosed(Status.OK);
streamTracer2.streamClosed(Status.UNAVAILABLE);
ClusterStats clusterStats =
Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER));
UpstreamLocalityStats localityStats =
Iterables.getOnlyElement(clusterStats.getUpstreamLocalityStatsList());
assertThat(localityStats.getLocality()).isEqualTo(locality);
assertThat(localityStats.getTotalIssuedRequests()).isEqualTo(3L);
assertThat(localityStats.getTotalSuccessfulRequests()).isEqualTo(1L);
assertThat(localityStats.getTotalErrorRequests()).isEqualTo(1L);
assertThat(localityStats.getTotalRequestsInProgress()).isEqualTo(1L);
streamTracer3.streamClosed(Status.OK);
subchannel.shutdown(); // stats recorder released
clusterStats = Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER));
// Locality load is reported for one last time in case of loads occurred since the previous
// load report.
localityStats = Iterables.getOnlyElement(clusterStats.getUpstreamLocalityStatsList());
assertThat(localityStats.getLocality()).isEqualTo(locality);
assertThat(localityStats.getTotalIssuedRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalSuccessfulRequests()).isEqualTo(1L);
assertThat(localityStats.getTotalErrorRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalRequestsInProgress()).isEqualTo(0L);
clusterStats = Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER));
assertThat(clusterStats.getUpstreamLocalityStatsList()).isEmpty(); // no longer reported
}
@Test
public void dropRpcsWithRespectToLbConfigDropCategories() {
LoadBalancerProvider weightedTargetProvider = new WeightedTargetLoadBalancerProvider();
@ -213,8 +266,14 @@ public class ClusterImplLoadBalancerTest {
assertThat(result.getStatus().isOk()).isFalse();
assertThat(result.getStatus().getCode()).isEqualTo(Code.UNAVAILABLE);
assertThat(result.getStatus().getDescription()).isEqualTo("Dropped: throttle");
assertThat(xdsClient.clusterStats.categorizedDrops.get("throttle"))
.isEqualTo(1);
ClusterStats clusterStats =
Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER));
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME);
assertThat(Iterables.getOnlyElement(clusterStats.getDroppedRequestsList()).getCategory())
.isEqualTo("throttle");
assertThat(Iterables.getOnlyElement(clusterStats.getDroppedRequestsList()).getDroppedCount())
.isEqualTo(1L);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(1L);
// Config update updates drop policies.
config = new ClusterImplConfig(CLUSTER, EDS_SERVICE_NAME, LRS_SERVER_NAME, null,
@ -233,12 +292,17 @@ public class ClusterImplLoadBalancerTest {
assertThat(result.getStatus().isOk()).isFalse();
assertThat(result.getStatus().getCode()).isEqualTo(Code.UNAVAILABLE);
assertThat(result.getStatus().getDescription()).isEqualTo("Dropped: lb");
assertThat(xdsClient.clusterStats.categorizedDrops.get("lb"))
.isEqualTo(1);
clusterStats =
Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER));
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME);
assertThat(Iterables.getOnlyElement(clusterStats.getDroppedRequestsList()).getCategory())
.isEqualTo("lb");
assertThat(Iterables.getOnlyElement(clusterStats.getDroppedRequestsList()).getDroppedCount())
.isEqualTo(1L);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(1L);
result = currentPicker.pickSubchannel(mock(PickSubchannelArgs.class));
assertThat(result.getStatus().isOk()).isTrue();
assertThat(result.getSubchannel()).isSameInstanceAs(subchannel);
}
@Test
@ -276,25 +340,27 @@ public class ClusterImplLoadBalancerTest {
for (int i = 0; i < maxConcurrentRequests; i++) {
PickResult result = currentPicker.pickSubchannel(mock(PickSubchannelArgs.class));
assertThat(result.getStatus().isOk()).isTrue();
assertThat(result.getSubchannel()).isSameInstanceAs(subchannel);
assertThat(result.getStreamTracerFactory()).isNotNull();
ClientStreamTracer.Factory streamTracerFactory = result.getStreamTracerFactory();
streamTracerFactory.newClientStreamTracer(ClientStreamTracer.StreamInfo.newBuilder().build(),
new Metadata());
}
assertThat(xdsClient.clusterStats.totalDrops).isEqualTo(0L);
ClusterStats clusterStats =
Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER));
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(0L);
PickResult result = currentPicker.pickSubchannel(mock(PickSubchannelArgs.class));
clusterStats = Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER));
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME);
if (enableCircuitBreaking) {
assertThat(result.getStatus().isOk()).isFalse();
assertThat(result.getStatus().getCode()).isEqualTo(Code.UNAVAILABLE);
assertThat(result.getStatus().getDescription())
.isEqualTo("Cluster max concurrent requests limit exceeded");
assertThat(xdsClient.clusterStats.totalDrops).isEqualTo(1L);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(1L);
} else {
assertThat(result.getStatus().isOk()).isTrue();
assertThat(result.getSubchannel()).isSameInstanceAs(subchannel);
assertThat(xdsClient.clusterStats.totalDrops).isEqualTo(0L);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(0L);
}
// Config update increments circuit breakers max_concurrent_requests threshold.
@ -306,11 +372,24 @@ public class ClusterImplLoadBalancerTest {
result = currentPicker.pickSubchannel(mock(PickSubchannelArgs.class));
assertThat(result.getStatus().isOk()).isTrue();
assertThat(result.getSubchannel()).isSameInstanceAs(subchannel);
result.getStreamTracerFactory().newClientStreamTracer(
ClientStreamTracer.StreamInfo.newBuilder().build(), new Metadata()); // 101th request
clusterStats = Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER));
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(0L);
result = currentPicker.pickSubchannel(mock(PickSubchannelArgs.class)); // 102th request
clusterStats = Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER));
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME);
if (enableCircuitBreaking) {
assertThat(xdsClient.clusterStats.totalDrops).isEqualTo(1L);
assertThat(result.getStatus().isOk()).isFalse();
assertThat(result.getStatus().getCode()).isEqualTo(Code.UNAVAILABLE);
assertThat(result.getStatus().getDescription())
.isEqualTo("Cluster max concurrent requests limit exceeded");
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(1L);
} else {
assertThat(xdsClient.clusterStats.totalDrops).isEqualTo(0L);
assertThat(result.getStatus().isOk()).isTrue();
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(0L);
}
}
@ -349,25 +428,27 @@ public class ClusterImplLoadBalancerTest {
for (int i = 0; i < ClusterImplLoadBalancer.DEFAULT_PER_CLUSTER_MAX_CONCURRENT_REQUESTS; i++) {
PickResult result = currentPicker.pickSubchannel(mock(PickSubchannelArgs.class));
assertThat(result.getStatus().isOk()).isTrue();
assertThat(result.getSubchannel()).isSameInstanceAs(subchannel);
assertThat(result.getStreamTracerFactory()).isNotNull();
ClientStreamTracer.Factory streamTracerFactory = result.getStreamTracerFactory();
streamTracerFactory.newClientStreamTracer(ClientStreamTracer.StreamInfo.newBuilder().build(),
new Metadata());
}
assertThat(xdsClient.clusterStats.totalDrops).isEqualTo(0L);
ClusterStats clusterStats =
Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER));
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(0L);
PickResult result = currentPicker.pickSubchannel(mock(PickSubchannelArgs.class));
clusterStats = Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER));
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME);
if (enableCircuitBreaking) {
assertThat(result.getStatus().isOk()).isFalse();
assertThat(result.getStatus().getCode()).isEqualTo(Code.UNAVAILABLE);
assertThat(result.getStatus().getDescription())
.isEqualTo("Cluster max concurrent requests limit exceeded");
assertThat(xdsClient.clusterStats.totalDrops).isEqualTo(1L);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(1L);
} else {
assertThat(result.getStatus().isOk()).isTrue();
assertThat(result.getSubchannel()).isSameInstanceAs(subchannel);
assertThat(xdsClient.clusterStats.totalDrops).isEqualTo(0L);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(0L);
}
}
@ -496,14 +577,11 @@ public class ClusterImplLoadBalancerTest {
}
private WeightedTargetConfig buildWeightedTargetConfig(Map<Locality, Integer> localityWeights) {
LoadBalancerProvider lrsBalancerProvider = new LrsLoadBalancerProvider();
Map<String, WeightedPolicySelection> targets = new HashMap<>();
for (Locality locality : localityWeights.keySet()) {
int weight = localityWeights.get(locality);
LrsConfig lrsConfig =
new LrsConfig(CLUSTER, EDS_SERVICE_NAME, LRS_SERVER_NAME, locality, roundRobin);
WeightedPolicySelection weightedLocalityLbPolicy =
new WeightedPolicySelection(weight, new PolicySelection(lrsBalancerProvider, lrsConfig));
new WeightedPolicySelection(weight, roundRobin);
targets.put(locality.toString(), weightedLocalityLbPolicy);
}
return new WeightedTargetConfig(Collections.unmodifiableMap(targets));
@ -543,8 +621,9 @@ public class ClusterImplLoadBalancerTest {
}
}
return AddressFilter.setPathFilter(new EquivalentAddressGroup(new FakeSocketAddress(name)),
Collections.singletonList(locality.toString()));
EquivalentAddressGroup eag = new EquivalentAddressGroup(new FakeSocketAddress(name),
Attributes.newBuilder().set(InternalXdsAttributes.ATTR_LOCALITY, locality).build());
return AddressFilter.setPathFilter(eag, Collections.singletonList(locality.toString()));
}
private final class FakeLoadBalancerProvider extends LoadBalancerProvider {
@ -634,7 +713,7 @@ public class ClusterImplLoadBalancerTest {
@Override
public Subchannel createSubchannel(CreateSubchannelArgs args) {
return new FakeSubchannel(args.getAddresses());
return new FakeSubchannel(args.getAddresses(), args.getAttributes());
}
@Override
@ -650,9 +729,11 @@ public class ClusterImplLoadBalancerTest {
private static final class FakeSubchannel extends Subchannel {
private final List<EquivalentAddressGroup> eags;
private final Attributes attrs;
private FakeSubchannel(List<EquivalentAddressGroup> eags) {
private FakeSubchannel(List<EquivalentAddressGroup> eags, Attributes attrs) {
this.eags = eags;
this.attrs = attrs;
}
@Override
@ -670,59 +751,20 @@ public class ClusterImplLoadBalancerTest {
@Override
public Attributes getAttributes() {
return Attributes.EMPTY;
return attrs;
}
}
private static final class FakeXdsClient extends XdsClient {
private FakeLoadStatsStore clusterStats;
private final class FakeXdsClient extends XdsClient {
@Override
LoadStatsStore addClientStats(String clusterName, @Nullable String clusterServiceName) {
assertThat(clusterStats).isNull();
clusterStats = new FakeLoadStatsStore();
return clusterStats;
ClusterDropStats addClusterDropStats(String clusterName, @Nullable String edsServiceName) {
return loadStatsManager.getClusterDropStats(clusterName, edsServiceName);
}
@Override
void removeClientStats(String clusterName, @Nullable String clusterServiceName) {
assertThat(clusterStats).isNotNull();
clusterStats = null;
}
}
private static final class FakeLoadStatsStore implements LoadStatsStore {
private final Map<String, Long> categorizedDrops = new HashMap<>();
private int totalDrops;
@Override
public ClusterStats generateLoadReport() {
throw new UnsupportedOperationException("should not be called");
}
@Override
public ClientLoadCounter addLocality(Locality locality) {
return new ClientLoadCounter();
}
@Override
public void removeLocality(Locality locality) {
// no-op
}
@Override
public void recordDroppedRequest(String category) {
if (!categorizedDrops.containsKey(category)) {
categorizedDrops.put(category, 1L);
} else {
categorizedDrops.put(category, categorizedDrops.get(category) + 1L);
}
totalDrops++;
}
@Override
public void recordDroppedRequest() {
totalDrops++;
ClusterLocalityStats addClusterLocalityStats(String clusterName,
@Nullable String edsServiceName, Locality locality) {
return loadStatsManager.getClusterLocalityStats(clusterName, edsServiceName, locality);
}
}

View File

@ -18,7 +18,6 @@ package io.grpc.xds;
import static com.google.common.truth.Truth.assertThat;
import static io.grpc.xds.XdsLbPolicies.CLUSTER_IMPL_POLICY_NAME;
import static io.grpc.xds.XdsLbPolicies.LRS_POLICY_NAME;
import static io.grpc.xds.XdsLbPolicies.PRIORITY_POLICY_NAME;
import static io.grpc.xds.XdsLbPolicies.WEIGHTED_TARGET_POLICY_NAME;
import static org.mockito.ArgumentMatchers.any;
@ -65,7 +64,6 @@ import io.grpc.xds.EnvoyProtoData.LbEndpoint;
import io.grpc.xds.EnvoyProtoData.Locality;
import io.grpc.xds.EnvoyProtoData.LocalityLbEndpoints;
import io.grpc.xds.EnvoyServerProtoData.UpstreamTlsContext;
import io.grpc.xds.LrsLoadBalancerProvider.LrsConfig;
import io.grpc.xds.PriorityLoadBalancerProvider.PriorityLbConfig;
import io.grpc.xds.PriorityLoadBalancerProvider.PriorityLbConfig.PriorityChildConfig;
import io.grpc.xds.WeightedTargetLoadBalancerProvider.WeightedPolicySelection;
@ -168,7 +166,6 @@ public class ClusterResolverLoadBalancerTest {
lbRegistry.register(new FakeLoadBalancerProvider(PRIORITY_POLICY_NAME));
lbRegistry.register(new FakeLoadBalancerProvider(CLUSTER_IMPL_POLICY_NAME));
lbRegistry.register(new FakeLoadBalancerProvider(LRS_POLICY_NAME));
lbRegistry.register(
new FakeLoadBalancerProvider("pick_first")); // needed by logical_dns
URI targetUri = new URI(AUTHORITY);
@ -244,9 +241,7 @@ public class ClusterResolverLoadBalancerTest {
assertThat(weightedTargetConfig.targets.keySet()).containsExactly(locality1.toString());
WeightedPolicySelection target = weightedTargetConfig.targets.get(locality1.toString());
assertThat(target.weight).isEqualTo(70);
assertThat(target.policySelection.getProvider().getPolicyName()).isEqualTo(LRS_POLICY_NAME);
assertLrsConfig((LrsConfig) target.policySelection.getConfig(), CLUSTER2, EDS_SERVICE_NAME2,
LRS_SERVER_NAME, locality1, "round_robin");
assertThat(target.policySelection.getProvider().getPolicyName()).isEqualTo("round_robin");
priorityChildConfig = priorityLbConfig.childConfigs.get(priority2);
assertThat(priorityChildConfig.ignoreReresolution).isTrue();
@ -259,9 +254,7 @@ public class ClusterResolverLoadBalancerTest {
assertThat(weightedTargetConfig.targets.keySet()).containsExactly(locality3.toString());
target = weightedTargetConfig.targets.get(locality3.toString());
assertThat(target.weight).isEqualTo(20);
assertThat(target.policySelection.getProvider().getPolicyName()).isEqualTo(LRS_POLICY_NAME);
assertLrsConfig((LrsConfig) target.policySelection.getConfig(), CLUSTER2, EDS_SERVICE_NAME2,
LRS_SERVER_NAME, locality3, "round_robin");
assertThat(target.policySelection.getProvider().getPolicyName()).isEqualTo("round_robin");
List<EquivalentAddressGroup> priorityAddrs1 =
AddressFilter.filter(childBalancer.addresses, priority1);
assertThat(priorityAddrs1).hasSize(2);
@ -290,9 +283,7 @@ public class ClusterResolverLoadBalancerTest {
assertThat(weightedTargetConfig.targets.keySet()).containsExactly(locality2.toString());
target = weightedTargetConfig.targets.get(locality2.toString());
assertThat(target.weight).isEqualTo(10);
assertThat(target.policySelection.getProvider().getPolicyName()).isEqualTo(LRS_POLICY_NAME);
assertLrsConfig((LrsConfig) target.policySelection.getConfig(), CLUSTER1, EDS_SERVICE_NAME1,
LRS_SERVER_NAME, locality2, "round_robin");
assertThat(target.policySelection.getProvider().getPolicyName()).isEqualTo("round_robin");
List<EquivalentAddressGroup> priorityAddrs3 =
AddressFilter.filter(childBalancer.addresses, priority3);
assertThat(priorityAddrs3).hasSize(1);
@ -458,10 +449,7 @@ public class ClusterResolverLoadBalancerTest {
ClusterImplConfig clusterImplConfig =
(ClusterImplConfig) priorityChildConfig.policySelection.getConfig();
assertClusterImplConfig(clusterImplConfig, CLUSTER_DNS, null, LRS_SERVER_NAME, 100L, null,
Collections.<DropOverload>emptyList(), LRS_POLICY_NAME);
LrsConfig lrsConfig = (LrsConfig) clusterImplConfig.childPolicy.getConfig();
assertLrsConfig(lrsConfig, CLUSTER_DNS, null, LRS_SERVER_NAME,
new Locality("", "", ""), "pick_first"); // hardcoded override
Collections.<DropOverload>emptyList(), "pick_first");
assertAddressesEqual(Arrays.asList(endpoint1, endpoint2), childBalancer.addresses);
}
@ -767,16 +755,6 @@ public class ClusterResolverLoadBalancerTest {
assertThat(config.childPolicy.getProvider().getPolicyName()).isEqualTo(childPolicy);
}
private static void assertLrsConfig(LrsConfig config, String cluster,
@Nullable String edsServiceName, String lrsServerName, Locality locality,
String childPolicy) {
assertThat(config.clusterName).isEqualTo(cluster);
assertThat(config.edsServiceName).isEqualTo(edsServiceName);
assertThat(config.lrsServerName).isEqualTo(lrsServerName);
assertThat(config.locality).isEqualTo(locality);
assertThat(config.childPolicy.getProvider().getPolicyName()).isEqualTo(childPolicy);
}
/** Asserts two list of EAGs contains same addresses, regardless of attributes. */
private static void assertAddressesEqual(
List<EquivalentAddressGroup> expected, List<EquivalentAddressGroup> actual) {

View File

@ -19,7 +19,6 @@ package io.grpc.xds;
import static com.google.common.truth.Truth.assertThat;
import static org.junit.Assert.assertEquals;
import static org.mockito.AdditionalAnswers.delegatesTo;
import static org.mockito.ArgumentMatchers.argThat;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.inOrder;
import static org.mockito.Mockito.mock;
@ -28,8 +27,6 @@ import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
import com.google.common.base.Stopwatch;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.util.concurrent.MoreExecutors;
@ -37,6 +34,8 @@ import com.google.protobuf.Struct;
import com.google.protobuf.Value;
import com.google.protobuf.util.Durations;
import io.envoyproxy.envoy.api.v2.core.Node;
import io.envoyproxy.envoy.api.v2.endpoint.ClusterStats;
import io.envoyproxy.envoy.api.v2.endpoint.UpstreamLocalityStats;
import io.envoyproxy.envoy.service.load_stats.v2.LoadReportingServiceGrpc;
import io.envoyproxy.envoy.service.load_stats.v2.LoadStatsRequest;
import io.envoyproxy.envoy.service.load_stats.v2.LoadStatsResponse;
@ -51,21 +50,11 @@ import io.grpc.internal.BackoffPolicy;
import io.grpc.internal.FakeClock;
import io.grpc.stub.StreamObserver;
import io.grpc.testing.GrpcCleanupRule;
import io.grpc.xds.EnvoyProtoData.ClusterStats;
import io.grpc.xds.EnvoyProtoData.ClusterStats.DroppedRequests;
import io.grpc.xds.EnvoyProtoData.Locality;
import io.grpc.xds.EnvoyProtoData.UpstreamLocalityStats;
import io.grpc.xds.LoadStatsManager.LoadStatsStore;
import io.grpc.xds.LoadStatsManager.LoadStatsStoreFactory;
import io.grpc.xds.LoadStatsManager2.ClusterDropStats;
import io.grpc.xds.LoadStatsManager2.ClusterLocalityStats;
import java.util.ArrayDeque;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.junit.After;
@ -75,7 +64,6 @@ import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
import org.mockito.ArgumentCaptor;
import org.mockito.ArgumentMatcher;
import org.mockito.Captor;
import org.mockito.InOrder;
import org.mockito.Mock;
@ -84,6 +72,7 @@ import org.mockito.MockitoAnnotations;
/**
* Unit tests for {@link LoadReportClient}.
*/
// TODO(chengyuanzhang): missing LRS V3 test.
@RunWith(JUnit4.class)
public class LoadReportClientTest {
// bootstrap node identifier
@ -94,6 +83,10 @@ public class LoadReportClientTest {
.build();
private static final String CLUSTER1 = "cluster-foo.googleapis.com";
private static final String CLUSTER2 = "cluster-bar.googleapis.com";
private static final String EDS_SERVICE_NAME1 = "backend-service-foo.googleapis.com";
private static final String EDS_SERVICE_NAME2 = "backend-service-bar.googleapis.com";
private static final Locality LOCALITY1 = new Locality("region1", "zone1", "subZone1");
private static final Locality LOCALITY2 = new Locality("region2", "zone2", "subZone2");
private static final FakeClock.TaskFilter LOAD_REPORTING_TASK_FILTER =
new FakeClock.TaskFilter() {
@Override
@ -124,14 +117,8 @@ public class LoadReportClientTest {
private final ArrayDeque<StreamObserver<LoadStatsRequest>> lrsRequestObservers =
new ArrayDeque<>();
private final AtomicBoolean callEnded = new AtomicBoolean(true);
private final LoadStatsManager loadStatsManager =
new LoadStatsManager(new LoadStatsStoreFactory() {
@Override
public LoadStatsStore newLoadStatsStore(String cluster, String clusterService) {
return new FakeLoadStatsStore(
cluster, clusterService, fakeClock.getStopwatchSupplier().get());
}
});
private final LoadStatsManager2 loadStatsManager =
new LoadStatsManager2(fakeClock.getStopwatchSupplier());
@Mock
private BackoffPolicy.Provider backoffPolicyProvider;
@ -142,6 +129,8 @@ public class LoadReportClientTest {
@Captor
private ArgumentCaptor<StreamObserver<LoadStatsResponse>> lrsResponseObserverCaptor;
@Captor
private ArgumentCaptor<LoadStatsRequest> requestCaptor;
@Captor
private ArgumentCaptor<Throwable> errorCaptor;
private LoadReportingServiceGrpc.LoadReportingServiceImplBase mockLoadReportingService;
@ -183,6 +172,7 @@ public class LoadReportClientTest {
.thenReturn(TimeUnit.SECONDS.toNanos(1L), TimeUnit.SECONDS.toNanos(10L));
when(backoffPolicy2.nextBackoffNanos())
.thenReturn(TimeUnit.SECONDS.toNanos(2L), TimeUnit.SECONDS.toNanos(20L));
addFakeStatsData();
lrsClient = new LoadReportClient(loadStatsManager, channel, false, NODE, syncContext,
fakeClock.getScheduledExecutorService(), backoffPolicyProvider,
fakeClock.getStopwatchSupplier());
@ -194,6 +184,28 @@ public class LoadReportClientTest {
});
}
private void addFakeStatsData() {
ClusterDropStats dropStats1 = loadStatsManager.getClusterDropStats(CLUSTER1, EDS_SERVICE_NAME1);
for (int i = 0; i < 52; i++) {
dropStats1.recordDroppedRequest("lb");
}
ClusterDropStats dropStats2 = loadStatsManager.getClusterDropStats(CLUSTER2, EDS_SERVICE_NAME2);
for (int i = 0; i < 23; i++) {
dropStats2.recordDroppedRequest("throttle");
}
ClusterLocalityStats localityStats1 =
loadStatsManager.getClusterLocalityStats(CLUSTER1, EDS_SERVICE_NAME1, LOCALITY1);
for (int i = 0; i < 31; i++) {
localityStats1.recordCallStarted();
}
ClusterLocalityStats localityStats2 =
loadStatsManager.getClusterLocalityStats(CLUSTER2, EDS_SERVICE_NAME2, LOCALITY2);
for (int i = 0; i < 45; i++) {
localityStats2.recordCallStarted();
}
localityStats2.recordCallFinished(Status.OK);
}
@After
public void tearDown() {
stopLoadReportingInSyncContext();
@ -206,95 +218,162 @@ public class LoadReportClientTest {
StreamObserver<LoadStatsResponse> responseObserver = lrsResponseObserverCaptor.getValue();
StreamObserver<LoadStatsRequest> requestObserver =
Iterables.getOnlyElement(lrsRequestObservers);
InOrder inOrder = inOrder(requestObserver);
inOrder.verify(requestObserver).onNext(eq(buildInitialRequest()));
FakeLoadStatsStore loadStatsStore1 =
(FakeLoadStatsStore) loadStatsManager.addLoadStats(CLUSTER1, null);
loadStatsStore1.refresh();
verify(requestObserver).onNext(eq(buildInitialRequest()));
// Management server asks to report loads for cluster1.
responseObserver.onNext(buildLrsResponse(ImmutableList.of(CLUSTER1), 1000));
responseObserver.onNext(LoadStatsResponse.newBuilder().addClusters(CLUSTER1)
.setLoadReportingInterval(Durations.fromSeconds(10L)).build());
fakeClock.forwardNanos(999);
inOrder.verifyNoMoreInteractions();
fakeClock.forwardNanos(1);
assertThat(loadStatsStore1.reported).hasSize(1);
ClusterStats report1 = loadStatsStore1.reported.poll();
assertThat(report1.getLoadReportIntervalNanos()).isEqualTo(1000);
inOrder.verify(requestObserver)
.onNext(argThat(new LoadStatsRequestMatcher(Collections.singletonList(report1))));
fakeClock.forwardTime(10L, TimeUnit.SECONDS);
verify(requestObserver, times(2)).onNext(requestCaptor.capture());
LoadStatsRequest request = requestCaptor.getValue();
ClusterStats clusterStats = Iterables.getOnlyElement(request.getClusterStatsList());
assertThat(clusterStats.getClusterName()).isEqualTo(CLUSTER1);
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME1);
assertThat(Durations.toSeconds(clusterStats.getLoadReportInterval())).isEqualTo(10L);
assertThat(Iterables.getOnlyElement(clusterStats.getDroppedRequestsList()).getCategory())
.isEqualTo("lb");
assertThat(Iterables.getOnlyElement(clusterStats.getDroppedRequestsList()).getDroppedCount())
.isEqualTo(52L);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(52L);
UpstreamLocalityStats localityStats =
Iterables.getOnlyElement(clusterStats.getUpstreamLocalityStatsList());
assertThat(localityStats.getLocality().getRegion()).isEqualTo("region1");
assertThat(localityStats.getLocality().getZone()).isEqualTo("zone1");
assertThat(localityStats.getLocality().getSubZone()).isEqualTo("subZone1");
assertThat(localityStats.getTotalIssuedRequests()).isEqualTo(31L);
assertThat(localityStats.getTotalSuccessfulRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalErrorRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalRequestsInProgress()).isEqualTo(31L);
loadStatsStore1.refresh();
fakeClock.forwardNanos(1000);
assertThat(loadStatsStore1.reported).hasSize(1);
report1 = loadStatsStore1.reported.poll();
assertThat(report1.getLoadReportIntervalNanos()).isEqualTo(1000);
inOrder.verify(requestObserver)
.onNext(argThat(new LoadStatsRequestMatcher(Collections.singletonList(report1))));
FakeLoadStatsStore loadStatsStore2 =
(FakeLoadStatsStore) loadStatsManager.addLoadStats(CLUSTER2, null);
loadStatsStore2.refresh();
fakeClock.forwardTime(10L, TimeUnit.SECONDS);
verify(requestObserver, times(3)).onNext(requestCaptor.capture());
request = requestCaptor.getValue();
clusterStats = Iterables.getOnlyElement(request.getClusterStatsList());
assertThat(clusterStats.getClusterName()).isEqualTo(CLUSTER1);
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME1);
assertThat(Durations.toSeconds(clusterStats.getLoadReportInterval())).isEqualTo(10L);
assertThat(clusterStats.getDroppedRequestsCount()).isEqualTo(0L);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(0L);
localityStats = Iterables.getOnlyElement(clusterStats.getUpstreamLocalityStatsList());
assertThat(localityStats.getLocality().getRegion()).isEqualTo("region1");
assertThat(localityStats.getLocality().getZone()).isEqualTo("zone1");
assertThat(localityStats.getLocality().getSubZone()).isEqualTo("subZone1");
assertThat(localityStats.getTotalIssuedRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalSuccessfulRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalErrorRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalRequestsInProgress()).isEqualTo(31L);
// Management server updates the interval of sending load reports, while still asking for
// loads to cluster1 only.
responseObserver.onNext(buildLrsResponse(ImmutableList.of(CLUSTER1), 2000));
responseObserver.onNext(LoadStatsResponse.newBuilder().addClusters(CLUSTER1)
.setLoadReportingInterval(Durations.fromSeconds(20L)).build());
fakeClock.forwardNanos(1000);
inOrder.verifyNoMoreInteractions();
fakeClock.forwardNanos(1000);
assertThat(loadStatsStore1.reported).hasSize(1);
report1 = loadStatsStore1.reported.poll();
assertThat(report1.getLoadReportIntervalNanos()).isEqualTo(2000);
assertThat(loadStatsStore2.reported).isEmpty();
inOrder.verify(requestObserver)
.onNext(argThat(new LoadStatsRequestMatcher(Collections.singletonList(report1))));
fakeClock.forwardTime(10L, TimeUnit.SECONDS);
verifyNoMoreInteractions(requestObserver);
fakeClock.forwardTime(10L, TimeUnit.SECONDS);
verify(requestObserver, times(4)).onNext(requestCaptor.capture());
request = requestCaptor.getValue();
clusterStats = Iterables.getOnlyElement(request.getClusterStatsList());
assertThat(clusterStats.getClusterName()).isEqualTo(CLUSTER1);
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME1);
assertThat(Durations.toSeconds(clusterStats.getLoadReportInterval())).isEqualTo(20L);
assertThat(clusterStats.getDroppedRequestsCount()).isEqualTo(0);
localityStats = Iterables.getOnlyElement(clusterStats.getUpstreamLocalityStatsList());
assertThat(localityStats.getLocality().getRegion()).isEqualTo("region1");
assertThat(localityStats.getLocality().getZone()).isEqualTo("zone1");
assertThat(localityStats.getLocality().getSubZone()).isEqualTo("subZone1");
assertThat(localityStats.getTotalIssuedRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalSuccessfulRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalErrorRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalRequestsInProgress()).isEqualTo(31L);
// Management server asks to report loads for all clusters.
responseObserver.onNext(
LoadStatsResponse.newBuilder()
.setSendAllClusters(true)
.setLoadReportingInterval(Durations.fromNanos(2000))
.build());
responseObserver.onNext(LoadStatsResponse.newBuilder().setSendAllClusters(true)
.setLoadReportingInterval(Durations.fromSeconds(20L)).build());
loadStatsStore1.refresh();
loadStatsStore2.refresh();
fakeClock.forwardNanos(2000);
assertThat(loadStatsStore1.reported).hasSize(1);
report1 = loadStatsStore1.reported.poll();
assertThat(loadStatsStore2.reported).hasSize(1);
ClusterStats report2 = loadStatsStore2.reported.poll();
assertThat(report1.getLoadReportIntervalNanos()).isEqualTo(2000);
assertThat(report2.getLoadReportIntervalNanos()).isEqualTo(2000 + 2000);
inOrder.verify(requestObserver)
.onNext(argThat(new LoadStatsRequestMatcher(Arrays.asList(report1, report2))));
fakeClock.forwardTime(20L, TimeUnit.SECONDS);
verify(requestObserver, times(5)).onNext(requestCaptor.capture());
request = requestCaptor.getValue();
assertThat(request.getClusterStatsCount()).isEqualTo(2);
ClusterStats clusterStats1 = findClusterStats(request.getClusterStatsList(), CLUSTER1);
assertThat(Durations.toSeconds(clusterStats1.getLoadReportInterval())).isEqualTo(20L);
assertThat(clusterStats1.getDroppedRequestsCount()).isEqualTo(0L);
assertThat(clusterStats1.getTotalDroppedRequests()).isEqualTo(0L);
UpstreamLocalityStats localityStats1 =
Iterables.getOnlyElement(clusterStats1.getUpstreamLocalityStatsList());
assertThat(localityStats1.getLocality().getRegion()).isEqualTo("region1");
assertThat(localityStats1.getLocality().getZone()).isEqualTo("zone1");
assertThat(localityStats1.getLocality().getSubZone()).isEqualTo("subZone1");
assertThat(localityStats1.getTotalIssuedRequests()).isEqualTo(0L);
assertThat(localityStats1.getTotalSuccessfulRequests()).isEqualTo(0L);
assertThat(localityStats1.getTotalErrorRequests()).isEqualTo(0L);
assertThat(localityStats1.getTotalRequestsInProgress()).isEqualTo(31L);
ClusterStats clusterStats2 = findClusterStats(request.getClusterStatsList(), CLUSTER2);
assertThat(Durations.toSeconds(clusterStats2.getLoadReportInterval()))
.isEqualTo(10L + 10L + 20L + 20L);
assertThat(Iterables.getOnlyElement(clusterStats2.getDroppedRequestsList()).getCategory())
.isEqualTo("throttle");
assertThat(Iterables.getOnlyElement(clusterStats2.getDroppedRequestsList()).getDroppedCount())
.isEqualTo(23L);
assertThat(clusterStats2.getTotalDroppedRequests()).isEqualTo(23L);
UpstreamLocalityStats localityStats2 =
Iterables.getOnlyElement(clusterStats2.getUpstreamLocalityStatsList());
assertThat(localityStats2.getLocality().getRegion()).isEqualTo("region2");
assertThat(localityStats2.getLocality().getZone()).isEqualTo("zone2");
assertThat(localityStats2.getLocality().getSubZone()).isEqualTo("subZone2");
assertThat(localityStats2.getTotalIssuedRequests()).isEqualTo(45L);
assertThat(localityStats2.getTotalSuccessfulRequests()).isEqualTo(1L);
assertThat(localityStats2.getTotalErrorRequests()).isEqualTo(0L);
assertThat(localityStats2.getTotalRequestsInProgress()).isEqualTo(45L - 1L);
// Load reports for cluster1 is no longer wanted.
responseObserver.onNext(buildLrsResponse(Collections.singletonList(CLUSTER2), 2000));
responseObserver.onNext(LoadStatsResponse.newBuilder().addClusters(CLUSTER2)
.setLoadReportingInterval(Durations.fromSeconds(10L)).build());
loadStatsStore1.refresh();
loadStatsStore2.refresh();
fakeClock.forwardNanos(2000);
assertThat(loadStatsStore1.reported).isEmpty();
assertThat(loadStatsStore2.reported).hasSize(1);
report2 = loadStatsStore2.reported.poll();
assertThat(report2.getLoadReportIntervalNanos()).isEqualTo(2000);
inOrder.verify(requestObserver)
.onNext(argThat(new LoadStatsRequestMatcher(Collections.singletonList(report2))));
fakeClock.forwardTime(10L, TimeUnit.SECONDS);
verify(requestObserver, times(6)).onNext(requestCaptor.capture());
request = requestCaptor.getValue();
clusterStats = Iterables.getOnlyElement(request.getClusterStatsList());
assertThat(clusterStats.getClusterName()).isEqualTo(CLUSTER2);
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME2);
assertThat(Durations.toSeconds(clusterStats.getLoadReportInterval())).isEqualTo(10L);
assertThat(clusterStats.getDroppedRequestsCount()).isEqualTo(0L);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(0L);
localityStats = Iterables.getOnlyElement(clusterStats.getUpstreamLocalityStatsList());
assertThat(localityStats.getLocality().getRegion()).isEqualTo("region2");
assertThat(localityStats.getLocality().getZone()).isEqualTo("zone2");
assertThat(localityStats.getLocality().getSubZone()).isEqualTo("subZone2");
assertThat(localityStats.getTotalIssuedRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalSuccessfulRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalErrorRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalRequestsInProgress()).isEqualTo(44L);
fakeClock.forwardTime(10L, TimeUnit.SECONDS);
verify(requestObserver, times(7)).onNext(requestCaptor.capture());
request = requestCaptor.getValue();
clusterStats = Iterables.getOnlyElement(request.getClusterStatsList());
assertThat(clusterStats.getClusterName()).isEqualTo(CLUSTER2);
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME2);
assertThat(Durations.toSeconds(clusterStats.getLoadReportInterval())).isEqualTo(10L);
assertThat(clusterStats.getDroppedRequestsCount()).isEqualTo(0L);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(0L);
localityStats = Iterables.getOnlyElement(clusterStats.getUpstreamLocalityStatsList());
assertThat(localityStats.getLocality().getRegion()).isEqualTo("region2");
assertThat(localityStats.getLocality().getZone()).isEqualTo("zone2");
assertThat(localityStats.getLocality().getSubZone()).isEqualTo("subZone2");
assertThat(localityStats.getTotalIssuedRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalSuccessfulRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalErrorRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalRequestsInProgress()).isEqualTo(44L);
// Management server asks loads for a cluster that client has no load data.
responseObserver
.onNext(buildLrsResponse(ImmutableList.of("cluster-unknown.googleapis.com"), 2000));
responseObserver.onNext(LoadStatsResponse.newBuilder().addClusters("unknown.googleapis.com")
.setLoadReportingInterval(Durations.fromSeconds(20L)).build());
fakeClock.forwardNanos(2000);
assertThat(loadStatsStore1.reported).isEmpty();
assertThat(loadStatsStore2.reported).isEmpty();
ArgumentCaptor<LoadStatsRequest> reportCaptor = ArgumentCaptor.forClass(null);
inOrder.verify(requestObserver).onNext(reportCaptor.capture());
assertThat(reportCaptor.getValue().getClusterStatsCount()).isEqualTo(0);
inOrder.verifyNoMoreInteractions();
fakeClock.forwardTime(20L, TimeUnit.SECONDS);
verify(requestObserver, times(8)).onNext(requestCaptor.capture());
assertThat(requestCaptor.getValue().getClusterStatsCount()).isEqualTo(0);
}
@Test
@ -306,12 +385,6 @@ public class LoadReportClientTest {
assertThat(lrsRequestObservers).hasSize(1);
StreamObserver<LoadStatsRequest> requestObserver = lrsRequestObservers.poll();
String clusterName = "cluster-foo.googleapis.com";
String clusterServiceName = "service-blade.googleapis.com";
FakeLoadStatsStore loadStatsStore =
(FakeLoadStatsStore) loadStatsManager.addLoadStats(clusterName, clusterServiceName);
loadStatsStore.refresh();
// First balancer RPC
verify(requestObserver).onNext(eq(buildInitialRequest()));
assertEquals(0, fakeClock.numPendingTasks(LRS_RPC_RETRY_TASK_FILTER));
@ -356,8 +429,8 @@ public class LoadReportClientTest {
assertEquals(0, fakeClock.numPendingTasks(LRS_RPC_RETRY_TASK_FILTER));
// Balancer sends a response asking for loads of the cluster.
responseObserver
.onNext(buildLrsResponse(ImmutableList.of(clusterName), 5));
responseObserver.onNext(LoadStatsResponse.newBuilder().addClusters(CLUSTER1)
.setLoadReportingInterval(Durations.fromNanos(5L)).build());
// Then breaks the RPC
responseObserver.onError(Status.UNAVAILABLE.asException());
@ -391,14 +464,29 @@ public class LoadReportClientTest {
// Load reporting back to normal.
responseObserver = lrsResponseObserverCaptor.getValue();
responseObserver
.onNext(buildLrsResponse(ImmutableList.of(clusterName), 10));
responseObserver.onNext(LoadStatsResponse.newBuilder().addClusters(CLUSTER1)
.setLoadReportingInterval(Durations.fromNanos(10L)).build());
fakeClock.forwardNanos(10);
ClusterStats report = Iterables.getOnlyElement(loadStatsStore.reported);
assertThat(report.getLoadReportIntervalNanos())
.isEqualTo(TimeUnit.SECONDS.toNanos(1 + 10 + 2) + 10);
verify(requestObserver)
.onNext(argThat(new LoadStatsRequestMatcher(Collections.singletonList(report))));
verify(requestObserver, times(2)).onNext(requestCaptor.capture());
LoadStatsRequest request = requestCaptor.getValue();
ClusterStats clusterStats = Iterables.getOnlyElement(request.getClusterStatsList());
assertThat(clusterStats.getClusterName()).isEqualTo(CLUSTER1);
assertThat(clusterStats.getClusterServiceName()).isEqualTo(EDS_SERVICE_NAME1);
assertThat(Durations.toSeconds(clusterStats.getLoadReportInterval())).isEqualTo(1L + 10L + 2L);
assertThat(Iterables.getOnlyElement(clusterStats.getDroppedRequestsList()).getCategory())
.isEqualTo("lb");
assertThat(Iterables.getOnlyElement(clusterStats.getDroppedRequestsList()).getDroppedCount())
.isEqualTo(52L);
assertThat(clusterStats.getTotalDroppedRequests()).isEqualTo(52L);
UpstreamLocalityStats localityStats =
Iterables.getOnlyElement(clusterStats.getUpstreamLocalityStatsList());
assertThat(localityStats.getLocality().getRegion()).isEqualTo("region1");
assertThat(localityStats.getLocality().getZone()).isEqualTo("zone1");
assertThat(localityStats.getLocality().getSubZone()).isEqualTo("subZone1");
assertThat(localityStats.getTotalIssuedRequests()).isEqualTo(31L);
assertThat(localityStats.getTotalSuccessfulRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalErrorRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalRequestsInProgress()).isEqualTo(31L);
// Wrapping up
verify(backoffPolicyProvider, times(2)).get();
@ -414,7 +502,8 @@ public class LoadReportClientTest {
Iterables.getOnlyElement(lrsRequestObservers);
verify(requestObserver).onNext(eq(buildInitialRequest()));
responseObserver.onNext(buildLrsResponse(Collections.singletonList(CLUSTER1), 1234));
responseObserver.onNext(LoadStatsResponse.newBuilder().addClusters(CLUSTER1)
.setLoadReportingInterval(Durations.fromNanos(1234L)).build());
assertEquals(1, fakeClock.numPendingTasks(LOAD_REPORTING_TASK_FILTER));
FakeClock.ScheduledTask scheduledTask =
Iterables.getOnlyElement(fakeClock.getPendingTasks(LOAD_REPORTING_TASK_FILTER));
@ -461,20 +550,14 @@ public class LoadReportClientTest {
assertThat(lrsRequestObservers).hasSize(1);
StreamObserver<LoadStatsRequest> requestObserver = lrsRequestObservers.poll();
String clusterName = "cluster-foo.googleapis.com";
String clusterServiceName = "service-blade.googleapis.com";
FakeLoadStatsStore loadStatsStore =
(FakeLoadStatsStore) loadStatsManager.addLoadStats(clusterName, clusterServiceName);
loadStatsStore.refresh();
// First balancer RPC
verify(requestObserver).onNext(eq(buildInitialRequest()));
assertEquals(0, fakeClock.numPendingTasks(LRS_RPC_RETRY_TASK_FILTER));
// Simulate receiving a response from traffic director.
assertEquals(0, fakeClock.numPendingTasks(LOAD_REPORTING_TASK_FILTER));
responseObserver
.onNext(buildLrsResponse(ImmutableList.of("namespace-foo:service-blade"), 1983));
responseObserver.onNext(LoadStatsResponse.newBuilder().addClusters(CLUSTER1)
.setLoadReportingInterval(Durations.fromNanos(1983L)).build());
// Load reporting task is scheduled
assertEquals(1, fakeClock.numPendingTasks(LOAD_REPORTING_TASK_FILTER));
FakeClock.ScheduledTask scheduledTask =
@ -504,16 +587,6 @@ public class LoadReportClientTest {
});
}
private static LoadStatsResponse buildLrsResponse(
List<String> clusterNames, long loadReportIntervalNanos) {
return
LoadStatsResponse
.newBuilder()
.addAllClusters(clusterNames)
.setLoadReportingInterval(Durations.fromNanos(loadReportIntervalNanos))
.build();
}
private static LoadStatsRequest buildInitialRequest() {
return
LoadStatsRequest.newBuilder()
@ -529,107 +602,12 @@ public class LoadReportClientTest {
.build();
}
/**
* For comparing LoadStatsRequest stats data regardless of .
*/
private static class LoadStatsRequestMatcher implements ArgumentMatcher<LoadStatsRequest> {
private final Map<String, ClusterStats> expectedStats = new HashMap<>();
LoadStatsRequestMatcher(Collection<ClusterStats> clusterStats) {
for (ClusterStats stats : clusterStats) {
expectedStats.put(stats.getClusterName(), stats);
private ClusterStats findClusterStats(List<ClusterStats> clusterStatsList, String clusterName) {
for (ClusterStats stats : clusterStatsList) {
if (stats.getClusterName().equals(clusterName)) {
return stats;
}
}
@Override
public boolean matches(LoadStatsRequest argument) {
if (argument.getClusterStatsCount() != expectedStats.size()) {
return false;
}
for (io.envoyproxy.envoy.api.v2.endpoint.ClusterStats stats
: argument.getClusterStatsList()) {
if (!stats.equals(expectedStats.get(stats.getClusterName()).toEnvoyProtoClusterStatsV2())) {
return false;
}
}
return true;
}
}
private static final class FakeLoadStatsStore implements LoadStatsStore {
private final String cluster;
private final String clusterService;
private final Stopwatch stopwatch;
private final Queue<ClusterStats> reported = new ArrayDeque<>();
private ClusterStats stats;
private FakeLoadStatsStore(String cluster, String clusterService, Stopwatch stopwatch) {
this.cluster = cluster;
this.clusterService = clusterService;
this.stopwatch = stopwatch;
stopwatch.reset().start();
refresh();
}
@Override
public ClusterStats generateLoadReport() {
ClusterStats report =
stats.toBuilder()
.setLoadReportIntervalNanos(stopwatch.elapsed(TimeUnit.NANOSECONDS))
.build();
stopwatch.reset().start();
reported.offer(report);
return report;
}
@Override
public ClientLoadCounter addLocality(EnvoyProtoData.Locality locality) {
throw new UnsupportedOperationException("should not used");
}
@Override
public void removeLocality(EnvoyProtoData.Locality locality) {
throw new UnsupportedOperationException("should not used");
}
@Override
public void recordDroppedRequest(String category) {
throw new UnsupportedOperationException("should not used");
}
@Override
public void recordDroppedRequest() {
throw new UnsupportedOperationException("should not used");
}
private void refresh() {
long callsInProgress = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
long callsSucceeded = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
long callsFailed = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
long callsIssued = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
long numLbDrops = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
long numThrottleDrops = ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
ClusterStats.Builder clusterStatsBuilder = ClusterStats.newBuilder();
clusterStatsBuilder.setClusterName(cluster);
if (clusterService != null) {
clusterStatsBuilder.setClusterServiceName(clusterService);
}
clusterStatsBuilder
.addUpstreamLocalityStats(UpstreamLocalityStats.newBuilder()
.setLocality(new Locality(
cluster + "-region-foo", cluster + "-zone-bar", cluster + "-subzone-baz"))
.setTotalRequestsInProgress(callsInProgress)
.setTotalSuccessfulRequests(callsSucceeded)
.setTotalErrorRequests(callsFailed)
.setTotalIssuedRequests(callsIssued)
.build())
.addDroppedRequests(
new DroppedRequests("lb",numLbDrops))
.addDroppedRequests(
new DroppedRequests("throttle", numThrottleDrops))
.setTotalDroppedRequests(numLbDrops + numThrottleDrops);
stats = clusterStatsBuilder.build();
}
return null;
}
}

View File

@ -0,0 +1,285 @@
/*
* Copyright 2021 The gRPC Authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.grpc.xds;
import static com.google.common.truth.Truth.assertThat;
import com.google.common.collect.Iterables;
import io.grpc.Status;
import io.grpc.internal.FakeClock;
import io.grpc.xds.EnvoyProtoData.ClusterStats;
import io.grpc.xds.EnvoyProtoData.ClusterStats.DroppedRequests;
import io.grpc.xds.EnvoyProtoData.Locality;
import io.grpc.xds.EnvoyProtoData.UpstreamLocalityStats;
import io.grpc.xds.LoadStatsManager2.ClusterDropStats;
import io.grpc.xds.LoadStatsManager2.ClusterLocalityStats;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
import javax.annotation.Nullable;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
/**
* Unit tests for {@link LoadStatsManager2}.
*/
@RunWith(JUnit4.class)
public class LoadStatsManager2Test {
private static final String CLUSTER_NAME1 = "cluster-foo.googleapis.com";
private static final String CLUSTER_NAME2 = "cluster-bar.googleapis.com";
private static final String EDS_SERVICE_NAME1 = "backend-service-foo.googleapis.com";
private static final String EDS_SERVICE_NAME2 = "backend-service-bar.googleapis.com";
private static final Locality LOCALITY1 =
new Locality("test_region1", "test_zone1", "test_subzone1");
private static final Locality LOCALITY2 =
new Locality("test_region2", "test_zone2", "test_subzone2");
private static final Locality LOCALITY3 =
new Locality("test_region3", "test_zone3", "test_subzone3");
private final FakeClock fakeClock = new FakeClock();
private final LoadStatsManager2 loadStatsManager =
new LoadStatsManager2(fakeClock.getStopwatchSupplier());
@Test
public void recordAndGetReport() {
ClusterDropStats dropCounter1 = loadStatsManager.getClusterDropStats(
CLUSTER_NAME1, EDS_SERVICE_NAME1);
ClusterDropStats dropCounter2 = loadStatsManager.getClusterDropStats(
CLUSTER_NAME1, EDS_SERVICE_NAME2);
ClusterLocalityStats loadCounter1 = loadStatsManager.getClusterLocalityStats(
CLUSTER_NAME1, EDS_SERVICE_NAME1, LOCALITY1);
ClusterLocalityStats loadCounter2 = loadStatsManager.getClusterLocalityStats(
CLUSTER_NAME1, EDS_SERVICE_NAME1, LOCALITY2);
ClusterLocalityStats loadCounter3 = loadStatsManager.getClusterLocalityStats(
CLUSTER_NAME2, null, LOCALITY3);
dropCounter1.recordDroppedRequest("lb");
dropCounter1.recordDroppedRequest("throttle");
for (int i = 0; i < 19; i++) {
loadCounter1.recordCallStarted();
}
fakeClock.forwardTime(5L, TimeUnit.SECONDS);
dropCounter2.recordDroppedRequest();
loadCounter1.recordCallFinished(Status.OK);
for (int i = 0; i < 9; i++) {
loadCounter2.recordCallStarted();
}
loadCounter2.recordCallFinished(Status.UNAVAILABLE);
fakeClock.forwardTime(10L, TimeUnit.SECONDS);
loadCounter3.recordCallStarted();
List<ClusterStats> allStats = loadStatsManager.getAllClusterStatsReports();
assertThat(allStats).hasSize(3); // three cluster:edsServiceName
ClusterStats stats1 = findClusterStats(allStats, CLUSTER_NAME1, EDS_SERVICE_NAME1);
assertThat(stats1.getLoadReportIntervalNanos()).isEqualTo(TimeUnit.SECONDS.toNanos(5L + 10L));
assertThat(stats1.getDroppedRequestsList()).hasSize(2);
assertThat(findDroppedRequestCount(stats1.getDroppedRequestsList(), "lb")).isEqualTo(1L);
assertThat(findDroppedRequestCount(stats1.getDroppedRequestsList(), "throttle")).isEqualTo(1L);
assertThat(stats1.getTotalDroppedRequests()).isEqualTo(1L + 1L);
assertThat(stats1.getUpstreamLocalityStatsList()).hasSize(2); // two localities
UpstreamLocalityStats loadStats1 =
findLocalityStats(stats1.getUpstreamLocalityStatsList(), LOCALITY1);
assertThat(loadStats1.getTotalIssuedRequests()).isEqualTo(19L);
assertThat(loadStats1.getTotalSuccessfulRequests()).isEqualTo(1L);
assertThat(loadStats1.getTotalErrorRequests()).isEqualTo(0L);
assertThat(loadStats1.getTotalRequestsInProgress()).isEqualTo(19L - 1L);
UpstreamLocalityStats loadStats2 =
findLocalityStats(stats1.getUpstreamLocalityStatsList(), LOCALITY2);
assertThat(loadStats2.getTotalIssuedRequests()).isEqualTo(9L);
assertThat(loadStats2.getTotalSuccessfulRequests()).isEqualTo(0L);
assertThat(loadStats2.getTotalErrorRequests()).isEqualTo(1L);
assertThat(loadStats2.getTotalRequestsInProgress()).isEqualTo(9L - 1L);
ClusterStats stats2 = findClusterStats(allStats, CLUSTER_NAME1, EDS_SERVICE_NAME2);
assertThat(stats2.getLoadReportIntervalNanos()).isEqualTo(TimeUnit.SECONDS.toNanos(5L + 10L));
assertThat(stats2.getDroppedRequestsList()).isEmpty(); // no categorized drops
assertThat(stats2.getTotalDroppedRequests()).isEqualTo(1L);
assertThat(stats2.getUpstreamLocalityStatsList()).isEmpty(); // no per-locality stats
ClusterStats stats3 = findClusterStats(allStats, CLUSTER_NAME2, null);
assertThat(stats3.getLoadReportIntervalNanos()).isEqualTo(TimeUnit.SECONDS.toNanos(5L + 10L));
assertThat(stats3.getDroppedRequestsList()).isEmpty();
assertThat(stats3.getTotalDroppedRequests()).isEqualTo(0L); // no drops recorded
assertThat(stats3.getUpstreamLocalityStatsList()).hasSize(1); // one localities
UpstreamLocalityStats loadStats3 =
Iterables.getOnlyElement(stats3.getUpstreamLocalityStatsList());
assertThat(loadStats3.getTotalIssuedRequests()).isEqualTo(1L);
assertThat(loadStats3.getTotalSuccessfulRequests()).isEqualTo(0L);
assertThat(loadStats3.getTotalErrorRequests()).isEqualTo(0L);
assertThat(loadStats3.getTotalRequestsInProgress()).isEqualTo(1L);
fakeClock.forwardTime(3L, TimeUnit.SECONDS);
List<ClusterStats> clusterStatsList = loadStatsManager.getClusterStatsReports(CLUSTER_NAME1);
assertThat(clusterStatsList).hasSize(2);
stats1 = findClusterStats(clusterStatsList, CLUSTER_NAME1, EDS_SERVICE_NAME1);
assertThat(stats1.getLoadReportIntervalNanos()).isEqualTo(TimeUnit.SECONDS.toNanos(3L));
assertThat(stats1.getDroppedRequestsList()).isEmpty();
assertThat(stats1.getTotalDroppedRequests()).isEqualTo(0L); // no new drops recorded
assertThat(stats1.getUpstreamLocalityStatsList()).hasSize(2); // two localities
loadStats1 = findLocalityStats(stats1.getUpstreamLocalityStatsList(), LOCALITY1);
assertThat(loadStats1.getTotalIssuedRequests()).isEqualTo(0L);
assertThat(loadStats1.getTotalSuccessfulRequests()).isEqualTo(0L);
assertThat(loadStats1.getTotalErrorRequests()).isEqualTo(0L);
assertThat(loadStats1.getTotalRequestsInProgress()).isEqualTo(18L); // still in-progress
loadStats2 = findLocalityStats(stats1.getUpstreamLocalityStatsList(), LOCALITY2);
assertThat(loadStats2.getTotalIssuedRequests()).isEqualTo(0L);
assertThat(loadStats2.getTotalSuccessfulRequests()).isEqualTo(0L);
assertThat(loadStats2.getTotalErrorRequests()).isEqualTo(0L);
assertThat(loadStats2.getTotalRequestsInProgress()).isEqualTo(8L); // still in-progress
stats2 = findClusterStats(clusterStatsList, CLUSTER_NAME1, EDS_SERVICE_NAME2);
assertThat(stats2.getLoadReportIntervalNanos()).isEqualTo(TimeUnit.SECONDS.toNanos(3L));
assertThat(stats2.getDroppedRequestsList()).isEmpty();
assertThat(stats2.getTotalDroppedRequests()).isEqualTo(0L); // no new drops recorded
assertThat(stats2.getUpstreamLocalityStatsList()).isEmpty(); // no per-locality stats
}
@Test
public void sharedDropCounterStatsAggregation() {
ClusterDropStats ref1 = loadStatsManager.getClusterDropStats(
CLUSTER_NAME1, EDS_SERVICE_NAME1);
ClusterDropStats ref2 = loadStatsManager.getClusterDropStats(
CLUSTER_NAME1, EDS_SERVICE_NAME1);
ref1.recordDroppedRequest("lb");
ref2.recordDroppedRequest("throttle");
ref1.recordDroppedRequest();
ref2.recordDroppedRequest();
ClusterStats stats = Iterables.getOnlyElement(
loadStatsManager.getClusterStatsReports(CLUSTER_NAME1));
assertThat(stats.getDroppedRequestsList()).hasSize(2);
assertThat(findDroppedRequestCount(stats.getDroppedRequestsList(), "lb")).isEqualTo(1L);
assertThat(findDroppedRequestCount(stats.getDroppedRequestsList(), "throttle")).isEqualTo(1L);
assertThat(stats.getTotalDroppedRequests()).isEqualTo(4L); // 2 cagetorized + 2 uncategoized
}
@Test
public void dropCounterDelayedDeletionAfterReported() {
ClusterDropStats counter = loadStatsManager.getClusterDropStats(
CLUSTER_NAME1, EDS_SERVICE_NAME1);
counter.recordDroppedRequest("lb");
ClusterStats stats = Iterables.getOnlyElement(
loadStatsManager.getClusterStatsReports(CLUSTER_NAME1));
assertThat(stats.getDroppedRequestsList()).hasSize(1);
assertThat(Iterables.getOnlyElement(stats.getDroppedRequestsList()).getDroppedCount())
.isEqualTo(1L);
assertThat(stats.getTotalDroppedRequests()).isEqualTo(1L);
counter.release();
stats = Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER_NAME1));
assertThat(stats.getDroppedRequestsList()).isEmpty();
assertThat(stats.getTotalDroppedRequests()).isEqualTo(0L);
assertThat(loadStatsManager.getClusterStatsReports(CLUSTER_NAME1)).isEmpty();
}
@Test
public void sharedLoadCounterStatsAggregation() {
ClusterLocalityStats ref1 = loadStatsManager.getClusterLocalityStats(
CLUSTER_NAME1, EDS_SERVICE_NAME1, LOCALITY1);
ClusterLocalityStats ref2 = loadStatsManager.getClusterLocalityStats(
CLUSTER_NAME1, EDS_SERVICE_NAME1, LOCALITY1);
ref1.recordCallStarted();
ref1.recordCallFinished(Status.OK);
ref2.recordCallStarted();
ref2.recordCallStarted();
ref2.recordCallFinished(Status.UNAVAILABLE);
ClusterStats stats = Iterables.getOnlyElement(
loadStatsManager.getClusterStatsReports(CLUSTER_NAME1));
UpstreamLocalityStats localityStats =
Iterables.getOnlyElement(stats.getUpstreamLocalityStatsList());
assertThat(localityStats.getTotalIssuedRequests()).isEqualTo(1L + 2L);
assertThat(localityStats.getTotalSuccessfulRequests()).isEqualTo(1L);
assertThat(localityStats.getTotalErrorRequests()).isEqualTo(1L);
assertThat(localityStats.getTotalRequestsInProgress()).isEqualTo(1L + 2L - 1L - 1L);
}
@Test
public void loadCounterDelayedDeletionAfterAllInProgressRequestsReported() {
ClusterLocalityStats counter = loadStatsManager.getClusterLocalityStats(
CLUSTER_NAME1, EDS_SERVICE_NAME1, LOCALITY1);
counter.recordCallStarted();
counter.recordCallStarted();
ClusterStats stats = Iterables.getOnlyElement(
loadStatsManager.getClusterStatsReports(CLUSTER_NAME1));
UpstreamLocalityStats localityStats =
Iterables.getOnlyElement(stats.getUpstreamLocalityStatsList());
assertThat(localityStats.getTotalIssuedRequests()).isEqualTo(2L);
assertThat(localityStats.getTotalSuccessfulRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalErrorRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalRequestsInProgress()).isEqualTo(2L);
// release the counter, but requests still in-flight
counter.release();
stats = Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER_NAME1));
localityStats = Iterables.getOnlyElement(stats.getUpstreamLocalityStatsList());
assertThat(localityStats.getTotalIssuedRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalSuccessfulRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalErrorRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalRequestsInProgress())
.isEqualTo(2L); // retained by in-flight calls
counter.recordCallFinished(Status.OK);
counter.recordCallFinished(Status.UNAVAILABLE);
stats = Iterables.getOnlyElement(loadStatsManager.getClusterStatsReports(CLUSTER_NAME1));
localityStats = Iterables.getOnlyElement(stats.getUpstreamLocalityStatsList());
assertThat(localityStats.getTotalIssuedRequests()).isEqualTo(0L);
assertThat(localityStats.getTotalSuccessfulRequests()).isEqualTo(1L);
assertThat(localityStats.getTotalErrorRequests()).isEqualTo(1L);
assertThat(localityStats.getTotalRequestsInProgress()).isEqualTo(0L);
assertThat(loadStatsManager.getClusterStatsReports(CLUSTER_NAME1)).isEmpty();
}
@Nullable
private static ClusterStats findClusterStats(
List<ClusterStats> statsList, String cluster, @Nullable String edsServiceName) {
for (ClusterStats stats : statsList) {
if (stats.getClusterName().equals(cluster)
&& Objects.equals(stats.getClusterServiceName(), edsServiceName)) {
return stats;
}
}
return null;
}
@Nullable
private static UpstreamLocalityStats findLocalityStats(
List<UpstreamLocalityStats> localityStatsList, Locality locality) {
for (UpstreamLocalityStats stats : localityStatsList) {
if (stats.getLocality().equals(locality)) {
return stats;
}
}
return null;
}
private static long findDroppedRequestCount(
List<DroppedRequests> droppedRequestsLists, String category) {
DroppedRequests drop = null;
for (DroppedRequests stats : droppedRequestsLists) {
if (stats.getCategory().equals(category)) {
drop = stats;
}
}
assertThat(drop).isNotNull();
return drop.getDroppedCount();
}
}

View File

@ -1,261 +0,0 @@
/*
* Copyright 2019 The gRPC Authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.grpc.xds;
import static com.google.common.truth.Truth.assertThat;
import com.google.common.base.Stopwatch;
import com.google.common.collect.ImmutableMap;
import io.grpc.internal.FakeClock;
import io.grpc.xds.ClientLoadCounter.MetricValue;
import io.grpc.xds.EnvoyProtoData.ClusterStats;
import io.grpc.xds.EnvoyProtoData.ClusterStats.DroppedRequests;
import io.grpc.xds.EnvoyProtoData.EndpointLoadMetricStats;
import io.grpc.xds.EnvoyProtoData.Locality;
import io.grpc.xds.EnvoyProtoData.UpstreamLocalityStats;
import io.grpc.xds.LoadStatsManager.LoadStatsStore;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import javax.annotation.Nullable;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
/** Unit tests for {@link LoadStatsStore}. */
@RunWith(JUnit4.class)
public class LoadStatsStoreImplTest {
private static final String CLUSTER_NAME = "cluster-test.googleapis.com";
private static final Locality LOCALITY1 =
new Locality("test_region1", "test_zone", "test_subzone");
private static final Locality LOCALITY2 =
new Locality("test_region2", "test_zone", "test_subzone");
private final FakeClock fakeClock = new FakeClock();
private LoadStatsStore loadStatsStore;
@Before
public void setUp() {
Stopwatch stopwatch = fakeClock.getStopwatchSupplier().get();
loadStatsStore = new LoadStatsStoreImpl(CLUSTER_NAME, null, stopwatch);
}
private static List<EndpointLoadMetricStats> buildEndpointLoadMetricStatsList(
Map<String, MetricValue> metrics) {
List<EndpointLoadMetricStats> res = new ArrayList<>();
for (Map.Entry<String, MetricValue> entry : metrics.entrySet()) {
res.add(EndpointLoadMetricStats.newBuilder()
.setMetricName(entry.getKey())
.setNumRequestsFinishedWithMetric(entry.getValue().getNumReports())
.setTotalMetricValue(entry.getValue().getTotalValue())
.build());
}
return res;
}
private static UpstreamLocalityStats buildUpstreamLocalityStats(
Locality locality,
long callsSucceed,
long callsInProgress,
long callsFailed,
long callsIssued,
@Nullable List<EndpointLoadMetricStats> metrics) {
UpstreamLocalityStats.Builder builder =
UpstreamLocalityStats.newBuilder()
.setLocality(locality)
.setTotalSuccessfulRequests(callsSucceed)
.setTotalErrorRequests(callsFailed)
.setTotalRequestsInProgress(callsInProgress)
.setTotalIssuedRequests(callsIssued);
if (metrics != null) {
builder.addAllLoadMetricStats(metrics);
}
return builder.build();
}
private static DroppedRequests buildDroppedRequests(String category, long counts) {
return new DroppedRequests(category, counts);
}
private static ClusterStats buildClusterStats(
@Nullable List<UpstreamLocalityStats> upstreamLocalityStatsList,
@Nullable List<DroppedRequests> droppedRequestsList, long totalDroppedRequests,
long intervalNano) {
ClusterStats.Builder clusterStatsBuilder = ClusterStats.newBuilder();
clusterStatsBuilder.setClusterName(CLUSTER_NAME);
if (upstreamLocalityStatsList != null) {
clusterStatsBuilder.addAllUpstreamLocalityStats(upstreamLocalityStatsList);
}
if (droppedRequestsList != null) {
for (DroppedRequests drop : droppedRequestsList) {
totalDroppedRequests += drop.getDroppedCount();
clusterStatsBuilder.addDroppedRequests(drop);
}
}
clusterStatsBuilder.setTotalDroppedRequests(totalDroppedRequests);
clusterStatsBuilder.setLoadReportIntervalNanos(intervalNano);
return clusterStatsBuilder.build();
}
private static void assertClusterStatsEqual(ClusterStats expected, ClusterStats actual) {
assertThat(actual.getClusterName()).isEqualTo(expected.getClusterName());
assertThat(actual.getLoadReportIntervalNanos())
.isEqualTo(expected.getLoadReportIntervalNanos());
assertThat(actual.getTotalDroppedRequests()).isEqualTo(expected.getTotalDroppedRequests());
assertThat(actual.getDroppedRequestsList()).hasSize(expected.getDroppedRequestsList().size());
assertThat(new HashSet<>(actual.getDroppedRequestsList()))
.isEqualTo(new HashSet<>(expected.getDroppedRequestsList()));
assertUpstreamLocalityStatsListsEqual(actual.getUpstreamLocalityStatsList(),
expected.getUpstreamLocalityStatsList());
}
private static void assertUpstreamLocalityStatsListsEqual(List<UpstreamLocalityStats> expected,
List<UpstreamLocalityStats> actual) {
assertThat(actual).hasSize(expected.size());
Map<Locality, UpstreamLocalityStats> expectedLocalityStats =
new HashMap<>();
for (UpstreamLocalityStats stats : expected) {
expectedLocalityStats.put(stats.getLocality(), stats);
}
for (UpstreamLocalityStats stats : actual) {
UpstreamLocalityStats expectedStats = expectedLocalityStats.get(stats.getLocality());
assertThat(expectedStats).isNotNull();
assertUpstreamLocalityStatsEqual(stats, expectedStats);
}
}
private static void assertUpstreamLocalityStatsEqual(UpstreamLocalityStats expected,
UpstreamLocalityStats actual) {
assertThat(actual.getLocality()).isEqualTo(expected.getLocality());
assertThat(actual.getTotalSuccessfulRequests())
.isEqualTo(expected.getTotalSuccessfulRequests());
assertThat(actual.getTotalRequestsInProgress())
.isEqualTo(expected.getTotalRequestsInProgress());
assertThat(actual.getTotalErrorRequests()).isEqualTo(expected.getTotalErrorRequests());
assertThat(new HashSet<>(actual.getLoadMetricStatsList()))
.isEqualTo(new HashSet<>(expected.getLoadMetricStatsList()));
}
@Test
public void removeInactiveCountersAfterGeneratingLoadReport() {
loadStatsStore.addLocality(LOCALITY1);
assertThat(loadStatsStore.generateLoadReport().getUpstreamLocalityStatsList()).hasSize(1);
loadStatsStore.removeLocality(LOCALITY1); // becomes inactive
assertThat(loadStatsStore.generateLoadReport().getUpstreamLocalityStatsList()).hasSize(1);
assertThat(loadStatsStore.generateLoadReport().getUpstreamLocalityStatsList()).isEmpty();
}
@Test
public void localityCountersReferenceCounted() {
loadStatsStore.addLocality(LOCALITY1);
loadStatsStore.addLocality(LOCALITY1);
loadStatsStore.removeLocality(LOCALITY1);
assertThat(loadStatsStore.generateLoadReport().getUpstreamLocalityStatsList()).hasSize(1);
assertThat(loadStatsStore.generateLoadReport().getUpstreamLocalityStatsList())
.hasSize(1); // still active
loadStatsStore.removeLocality(LOCALITY1); // becomes inactive
assertThat(loadStatsStore.generateLoadReport().getUpstreamLocalityStatsList()).hasSize(1);
assertThat(loadStatsStore.generateLoadReport().getUpstreamLocalityStatsList()).isEmpty();
}
@Test
public void recordCallAndMetricStats() {
ClientLoadCounter counter1 = loadStatsStore.addLocality(LOCALITY1);
counter1.setCallsSucceeded(4315);
counter1.setCallsInProgress(3421);
counter1.setCallsFailed(23);
counter1.setCallsIssued(593);
counter1.recordMetric("cpu_utilization", 0.3244);
counter1.recordMetric("mem_utilization", 0.01233);
counter1.recordMetric("named_cost_or_utilization", 3221.6543);
ClientLoadCounter counter2 = loadStatsStore.addLocality(LOCALITY2);
counter2.setCallsSucceeded(41234);
counter2.setCallsInProgress(432);
counter2.setCallsFailed(431);
counter2.setCallsIssued(702);
counter2.recordMetric("cpu_utilization", 0.6526);
counter2.recordMetric("mem_utilization", 0.3473);
counter2.recordMetric("named_cost_or_utilization", 87653.4234);
fakeClock.forwardNanos(1000L);
Map<String, MetricValue> metrics1 =
ImmutableMap.of(
"cpu_utilization", new MetricValue(1, 0.3244),
"mem_utilization", new MetricValue(1, 0.01233),
"named_cost_or_utilization", new MetricValue(1, 3221.6543));
Map<String, MetricValue> metrics2 =
ImmutableMap.of(
"cpu_utilization", new MetricValue(1, 0.6526),
"mem_utilization", new MetricValue(1, 0.3473),
"named_cost_or_utilization", new MetricValue(1, 87653.4234));
ClusterStats expectedReport =
buildClusterStats(
Arrays.asList(
buildUpstreamLocalityStats(LOCALITY1, 4315, 3421, 23, 593,
buildEndpointLoadMetricStatsList(metrics1)),
buildUpstreamLocalityStats(LOCALITY2, 41234, 432, 431, 702,
buildEndpointLoadMetricStatsList(metrics2))
),
null, 0L, 1000L);
assertClusterStatsEqual(expectedReport, loadStatsStore.generateLoadReport());
fakeClock.forwardNanos(2000L);
expectedReport =
buildClusterStats(
Arrays.asList(
buildUpstreamLocalityStats(LOCALITY1, 0, 3421, 0, 0, null),
buildUpstreamLocalityStats(LOCALITY2, 0, 432, 0, 0, null)
),
null, 0L, 2000L);
assertClusterStatsEqual(expectedReport, loadStatsStore.generateLoadReport());
}
@Test
public void recordDroppedRequests() {
int numLbDrop = 123;
int numThrottleDrop = 456;
int uncategorizedDrop = 789;
for (int i = 0; i < numLbDrop; i++) {
loadStatsStore.recordDroppedRequest("lb");
}
for (int i = 0; i < numThrottleDrop; i++) {
loadStatsStore.recordDroppedRequest("throttle");
}
for (int i = 0; i < uncategorizedDrop; i++) {
loadStatsStore.recordDroppedRequest();
}
fakeClock.forwardNanos(1000L);
ClusterStats expectedLoadReport =
buildClusterStats(null,
Arrays.asList(buildDroppedRequests("lb", numLbDrop),
buildDroppedRequests("throttle", numThrottleDrop)),
789L, 1000L);
assertClusterStatsEqual(expectedLoadReport, loadStatsStore.generateLoadReport());
fakeClock.forwardNanos(1000L);
expectedLoadReport =
buildClusterStats(null,
Arrays.asList(buildDroppedRequests("lb", 0L),
buildDroppedRequests("throttle", 0L)),
0L, 1000L);
assertClusterStatsEqual(expectedLoadReport, loadStatsStore.generateLoadReport());
}
}

View File

@ -1,334 +0,0 @@
/*
* Copyright 2020 The gRPC Authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.grpc.xds;
import static com.google.common.truth.Truth.assertThat;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import io.grpc.Attributes;
import io.grpc.ClientStreamTracer;
import io.grpc.ConnectivityState;
import io.grpc.EquivalentAddressGroup;
import io.grpc.LoadBalancer;
import io.grpc.LoadBalancer.Helper;
import io.grpc.LoadBalancer.PickResult;
import io.grpc.LoadBalancer.PickSubchannelArgs;
import io.grpc.LoadBalancer.ResolvedAddresses;
import io.grpc.LoadBalancer.Subchannel;
import io.grpc.LoadBalancer.SubchannelPicker;
import io.grpc.LoadBalancerProvider;
import io.grpc.Status;
import io.grpc.internal.ServiceConfigUtil.PolicySelection;
import io.grpc.xds.ClientLoadCounter.LoadRecordingStreamTracerFactory;
import io.grpc.xds.ClientLoadCounter.LoadRecordingSubchannelPicker;
import io.grpc.xds.EnvoyProtoData.ClusterStats;
import io.grpc.xds.EnvoyProtoData.Locality;
import io.grpc.xds.LoadStatsManager.LoadStatsStore;
import io.grpc.xds.LrsLoadBalancerProvider.LrsConfig;
import java.net.SocketAddress;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
import org.mockito.ArgumentCaptor;
import org.mockito.Mock;
import org.mockito.junit.MockitoJUnit;
import org.mockito.junit.MockitoRule;
/**
* Unit tests for {@link LrsLoadBalancer}.
*/
@RunWith(JUnit4.class)
public class LrsLoadBalancerTest {
@Rule
public final MockitoRule mockitoRule = MockitoJUnit.rule();
private static final String CLUSTER_NAME = "cluster-foo.googleapis.com";
private static final String EDS_SERVICE_NAME = "cluster-foo:service-blade";
private static final String LRS_SERVER_NAME = "trafficdirector.googleapis.com";
private static final Locality TEST_LOCALITY =
new Locality("test-region", "test-zone", "test-subzone");
private final LoadRecorder loadRecorder = new LoadRecorder();
private final Queue<LoadBalancer> childBalancers = new ArrayDeque<>();
@Mock
private Helper helper;
private LrsLoadBalancer loadBalancer;
@Before
public void setUp() {
loadBalancer = new LrsLoadBalancer(helper);
}
@After
public void tearDown() {
loadBalancer.shutdown();
}
@Test
public void subchannelPickerInterceptedWithLoadRecording() {
List<EquivalentAddressGroup> backendAddrs = createResolvedBackendAddresses(2);
deliverResolvedAddresses(backendAddrs, "round_robin");
FakeLoadBalancer childBalancer = (FakeLoadBalancer) childBalancers.poll();
NoopSubchannel subchannel = childBalancer.subchannels.values().iterator().next();
deliverSubchannelState(subchannel, ConnectivityState.READY);
assertThat(loadRecorder.recording).isTrue();
ArgumentCaptor<SubchannelPicker> pickerCaptor = ArgumentCaptor.forClass(null);
verify(helper).updateBalancingState(eq(ConnectivityState.READY), pickerCaptor.capture());
SubchannelPicker picker = pickerCaptor.getValue();
assertThat(picker).isInstanceOf(LoadRecordingSubchannelPicker.class);
PickResult result = picker.pickSubchannel(mock(PickSubchannelArgs.class));
ClientStreamTracer.Factory tracerFactory = result.getStreamTracerFactory();
assertThat(((LoadRecordingStreamTracerFactory) tracerFactory).getCounter())
.isSameInstanceAs(loadRecorder.counter);
loadBalancer.shutdown();
assertThat(childBalancer.shutdown).isTrue();
assertThat(loadRecorder.recording).isFalse();
}
@Test
public void updateChildPolicy() {
List<EquivalentAddressGroup> backendAddrs = createResolvedBackendAddresses(2);
deliverResolvedAddresses(backendAddrs, "round_robin");
FakeLoadBalancer childBalancer = (FakeLoadBalancer) childBalancers.poll();
assertThat(childBalancer.name).isEqualTo("round_robin");
deliverResolvedAddresses(backendAddrs, "pick_first");
assertThat(childBalancer.shutdown).isTrue();
childBalancer = (FakeLoadBalancer) childBalancers.poll();
assertThat(childBalancer.name).isEqualTo("pick_first");
loadBalancer.shutdown();
assertThat(childBalancer.shutdown).isTrue();
}
@Test
public void errorPropagation() {
loadBalancer.handleNameResolutionError(Status.UNKNOWN.withDescription("I failed"));
ArgumentCaptor<SubchannelPicker> pickerCaptor = ArgumentCaptor.forClass(null);
verify(helper)
.updateBalancingState(eq(ConnectivityState.TRANSIENT_FAILURE), pickerCaptor.capture());
Status status =
pickerCaptor.getValue().pickSubchannel(mock(PickSubchannelArgs.class)).getStatus();
assertThat(status.getDescription()).contains("I failed");
List<EquivalentAddressGroup> backendAddrs = createResolvedBackendAddresses(2);
deliverResolvedAddresses(backendAddrs, "round_robin");
// Error after child policy is created.
loadBalancer.handleNameResolutionError(Status.UNKNOWN.withDescription("I failed"));
verify(helper, times(2))
.updateBalancingState(eq(ConnectivityState.TRANSIENT_FAILURE), pickerCaptor.capture());
status = pickerCaptor.getValue().pickSubchannel(mock(PickSubchannelArgs.class)).getStatus();
assertThat(status.getDescription()).contains("I failed");
assertThat(status.getDescription()).contains("handled by downstream balancer");
}
private void deliverResolvedAddresses(
List<EquivalentAddressGroup> addresses, String childPolicy) {
PolicySelection childPolicyConfig =
new PolicySelection(new FakeLoadBalancerProvider(childPolicy), null);
LrsConfig config =
new LrsConfig(
CLUSTER_NAME, EDS_SERVICE_NAME, LRS_SERVER_NAME, TEST_LOCALITY, childPolicyConfig);
ResolvedAddresses resolvedAddresses =
ResolvedAddresses.newBuilder()
.setAddresses(addresses)
.setAttributes(
Attributes.newBuilder()
.set(InternalXdsAttributes.ATTR_CLUSTER_SERVICE_LOAD_STATS_STORE, loadRecorder)
.build())
.setLoadBalancingPolicyConfig(config)
.build();
loadBalancer.handleResolvedAddresses(resolvedAddresses);
}
private static List<EquivalentAddressGroup> createResolvedBackendAddresses(int n) {
List<EquivalentAddressGroup> list = new ArrayList<>();
for (int i = 0; i < n; i++) {
SocketAddress addr = new FakeSocketAddress("fake-address-" + i);
list.add(new EquivalentAddressGroup(addr));
}
return list;
}
private static void deliverSubchannelState(
final NoopSubchannel subchannel, ConnectivityState state) {
SubchannelPicker picker = new SubchannelPicker() {
@Override
public PickResult pickSubchannel(PickSubchannelArgs args) {
return PickResult.withSubchannel(subchannel);
}
};
subchannel.helper.updateBalancingState(state, picker);
}
private final class FakeLoadBalancerProvider extends LoadBalancerProvider {
private final String policyName;
FakeLoadBalancerProvider(String policyName) {
this.policyName = policyName;
}
@Override
public LoadBalancer newLoadBalancer(Helper helper) {
LoadBalancer balancer = new FakeLoadBalancer(helper, policyName);
childBalancers.add(balancer);
return balancer;
}
@Override
public boolean isAvailable() {
return true;
}
@Override
public int getPriority() {
return 0; // doesn't matter
}
@Override
public String getPolicyName() {
return policyName;
}
}
private static final class FakeLoadBalancer extends LoadBalancer {
private final Helper helper;
private final String name;
private boolean shutdown;
private final Map<EquivalentAddressGroup, NoopSubchannel> subchannels = new HashMap<>();
FakeLoadBalancer(Helper helper, String name) {
this.helper = helper;
this.name = name;
}
@Override
public void handleResolvedAddresses(ResolvedAddresses resolvedAddresses) {
List<EquivalentAddressGroup> addresses = resolvedAddresses.getAddresses();
for (EquivalentAddressGroup eag : addresses) {
subchannels.put(eag, new NoopSubchannel(helper));
}
}
@Override
public void handleNameResolutionError(final Status error) {
SubchannelPicker picker = new SubchannelPicker() {
@Override
public PickResult pickSubchannel(PickSubchannelArgs args) {
return PickResult.withError(error.augmentDescription("handled by downstream balancer"));
}
};
helper.updateBalancingState(ConnectivityState.TRANSIENT_FAILURE, picker);
}
@Override
public void shutdown() {
shutdown = true;
}
}
private static final class NoopSubchannel extends Subchannel {
final Helper helper;
NoopSubchannel(Helper helper) {
this.helper = helper;
}
@Override
public void shutdown() {
}
@Override
public void requestConnection() {
}
@Override
public Attributes getAttributes() {
return Attributes.EMPTY;
}
}
private static final class FakeSocketAddress extends SocketAddress {
final String name;
FakeSocketAddress(String name) {
this.name = name;
}
@Override
public String toString() {
return "FakeSocketAddress-" + name;
}
@Override
public boolean equals(Object other) {
if (other instanceof FakeSocketAddress) {
FakeSocketAddress otherAddr = (FakeSocketAddress) other;
return name.equals(otherAddr.name);
}
return false;
}
@Override
public int hashCode() {
return name.hashCode();
}
}
private static final class LoadRecorder implements LoadStatsStore {
private final ClientLoadCounter counter = new ClientLoadCounter();
private boolean recording = false;
@Override
public ClusterStats generateLoadReport() {
throw new UnsupportedOperationException("should not be called");
}
@Override
public ClientLoadCounter addLocality(Locality locality) {
assertThat(locality).isEqualTo(TEST_LOCALITY);
recording = true;
return counter;
}
@Override
public void removeLocality(Locality locality) {
assertThat(locality).isEqualTo(TEST_LOCALITY);
recording = false;
}
@Override
public void recordDroppedRequest(String category) {
throw new UnsupportedOperationException("should not be called");
}
@Override
public void recordDroppedRequest() {
throw new UnsupportedOperationException("should not be called");
}
}
}