xds: refactor xds_client (#3477)
This PR refactors xds_client to support multiples watches. Those watches can be for the same type and same resource_name. There's upper level `Client` and lower level `v2client`. Before this change, all logic was in `v2client`, and `Client` was a thin wrapper. This PR moves some of the functionality from `v2client` to `Client`. New layers: - Upper level `Client` - keeps a list of watchers - provides method `func WatchXXX() (cancel func())` - has `WatchService()` which involves `LDS` and `RDS` - handles resources from the xDS responses and dispatch to the watchers - including multiple watchers for the same resource_name - keeps cache - and checks cache for new watches - Lower level `v2client` - is a dumb client that - manages ADS stream - sends a new xDS request when add/remove watch - parses xDS responses - It doesn't call watchers, but forwards all parsed results to upper Client - handles ACK/NACK - supports `addWatch(type, name)` and `removeWatch(type, name)` - instead of `func watchCDS() func()`, which is now moved up to upper `Client` Also includes other changes: - Corresponding test changes (some tests for `v2client` were moved to `Client`) - Method and type renaming - CDS/EDS -> Cluster/Endpoints - callback functions all accept updates as non-pointers
This commit is contained in:
@ -106,7 +106,7 @@ func (cdsBB) ParseConfig(c json.RawMessage) (serviceconfig.LoadBalancingConfig,
|
||||
// xdsClientInterface contains methods from xdsClient.Client which are used by
|
||||
// the cdsBalancer. This will be faked out in unittests.
|
||||
type xdsClientInterface interface {
|
||||
WatchCluster(string, func(xdsclient.CDSUpdate, error)) func()
|
||||
WatchCluster(string, func(xdsclient.ClusterUpdate, error)) func()
|
||||
Close()
|
||||
}
|
||||
|
||||
@ -132,7 +132,7 @@ type scUpdate struct {
|
||||
// results in creating a new edsBalancer (if one doesn't already exist) and
|
||||
// pushing the update to it.
|
||||
type watchUpdate struct {
|
||||
cds xdsclient.CDSUpdate
|
||||
cds xdsclient.ClusterUpdate
|
||||
err error
|
||||
}
|
||||
|
||||
@ -274,7 +274,7 @@ func (b *cdsBalancer) run() {
|
||||
|
||||
// handleClusterUpdate is the CDS watch API callback. It simply pushes the
|
||||
// received information on to the update channel for run() to pick it up.
|
||||
func (b *cdsBalancer) handleClusterUpdate(cu xdsclient.CDSUpdate, err error) {
|
||||
func (b *cdsBalancer) handleClusterUpdate(cu xdsclient.ClusterUpdate, err error) {
|
||||
if b.isClosed() {
|
||||
b.logger.Warningf("xds: received cluster update {%+v} after cdsBalancer was closed", cu)
|
||||
return
|
||||
|
@ -59,7 +59,7 @@ type testClientConn struct {
|
||||
|
||||
// cdsWatchInfo wraps the update and the error sent in a CDS watch callback.
|
||||
type cdsWatchInfo struct {
|
||||
update xdsclient.CDSUpdate
|
||||
update xdsclient.ClusterUpdate
|
||||
err error
|
||||
}
|
||||
|
||||
@ -369,18 +369,18 @@ func (s) TestHandleClusterUpdate(t *testing.T) {
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
cdsUpdate xdsclient.CDSUpdate
|
||||
cdsUpdate xdsclient.ClusterUpdate
|
||||
updateErr error
|
||||
wantCCS balancer.ClientConnState
|
||||
}{
|
||||
{
|
||||
name: "happy-case-with-lrs",
|
||||
cdsUpdate: xdsclient.CDSUpdate{ServiceName: serviceName, EnableLRS: true},
|
||||
cdsUpdate: xdsclient.ClusterUpdate{ServiceName: serviceName, EnableLRS: true},
|
||||
wantCCS: edsCCS(serviceName, true, xdsC),
|
||||
},
|
||||
{
|
||||
name: "happy-case-without-lrs",
|
||||
cdsUpdate: xdsclient.CDSUpdate{ServiceName: serviceName},
|
||||
cdsUpdate: xdsclient.ClusterUpdate{ServiceName: serviceName},
|
||||
wantCCS: edsCCS(serviceName, false, xdsC),
|
||||
},
|
||||
{
|
||||
@ -408,7 +408,7 @@ func (s) TestResolverError(t *testing.T) {
|
||||
cdsB.Close()
|
||||
}()
|
||||
|
||||
cdsUpdate := xdsclient.CDSUpdate{ServiceName: serviceName}
|
||||
cdsUpdate := xdsclient.ClusterUpdate{ServiceName: serviceName}
|
||||
wantCCS := edsCCS(serviceName, false, xdsC)
|
||||
if err := invokeWatchCbAndWait(xdsC, cdsWatchInfo{cdsUpdate, nil}, wantCCS, edsB); err != nil {
|
||||
t.Fatal(err)
|
||||
@ -433,7 +433,7 @@ func (s) TestUpdateSubConnState(t *testing.T) {
|
||||
cdsB.Close()
|
||||
}()
|
||||
|
||||
cdsUpdate := xdsclient.CDSUpdate{ServiceName: serviceName}
|
||||
cdsUpdate := xdsclient.ClusterUpdate{ServiceName: serviceName}
|
||||
wantCCS := edsCCS(serviceName, false, xdsC)
|
||||
if err := invokeWatchCbAndWait(xdsC, cdsWatchInfo{cdsUpdate, nil}, wantCCS, edsB); err != nil {
|
||||
t.Fatal(err)
|
||||
@ -453,7 +453,7 @@ func (s) TestClose(t *testing.T) {
|
||||
xdsC, cdsB, edsB, cancel := setupWithWatch(t)
|
||||
defer cancel()
|
||||
|
||||
cdsUpdate := xdsclient.CDSUpdate{ServiceName: serviceName}
|
||||
cdsUpdate := xdsclient.ClusterUpdate{ServiceName: serviceName}
|
||||
wantCCS := edsCCS(serviceName, false, xdsC)
|
||||
if err := invokeWatchCbAndWait(xdsC, cdsWatchInfo{cdsUpdate, nil}, wantCCS, edsB); err != nil {
|
||||
t.Fatal(err)
|
||||
|
@ -95,7 +95,7 @@ func (b *edsBalancerBuilder) ParseConfig(c json.RawMessage) (serviceconfig.LoadB
|
||||
// TODO: none of the methods in this interface needs to be exported.
|
||||
type edsBalancerImplInterface interface {
|
||||
// HandleEDSResponse passes the received EDS message from traffic director to eds balancer.
|
||||
HandleEDSResponse(edsResp *xdsclient.EDSUpdate)
|
||||
HandleEDSResponse(edsResp xdsclient.EndpointsUpdate)
|
||||
// HandleChildPolicy updates the eds balancer the intra-cluster load balancing policy to use.
|
||||
HandleChildPolicy(name string, config json.RawMessage)
|
||||
// HandleSubConnStateChange handles state change for SubConn.
|
||||
@ -196,9 +196,9 @@ func (x *edsBalancer) handleGRPCUpdate(update interface{}) {
|
||||
|
||||
func (x *edsBalancer) handleXDSClientUpdate(update interface{}) {
|
||||
switch u := update.(type) {
|
||||
// TODO: this func should accept (*xdsclient.EDSUpdate, error), and process
|
||||
// TODO: this func should accept (xdsclient.EndpointsUpdate, error), and process
|
||||
// the error, instead of having a separate loseContact signal.
|
||||
case *xdsclient.EDSUpdate:
|
||||
case xdsclient.EndpointsUpdate:
|
||||
x.edsImpl.HandleEDSResponse(u)
|
||||
case *loseContact:
|
||||
// loseContact can be useful for going into fallback.
|
||||
@ -246,7 +246,7 @@ func (x *edsBalancer) UpdateClientConnState(s balancer.ClientConnState) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *edsBalancer) handleEDSUpdate(resp *xdsclient.EDSUpdate) error {
|
||||
func (x *edsBalancer) handleEDSUpdate(resp xdsclient.EndpointsUpdate) error {
|
||||
// TODO: this function should take (resp, error), and send them together on
|
||||
// the channel. There doesn't need to be a separate `loseContact` function.
|
||||
select {
|
||||
|
@ -173,7 +173,7 @@ func (edsImpl *edsBalancerImpl) updateDrops(dropConfig []xdsclient.OverloadDropC
|
||||
// SubConns. It also handles drops.
|
||||
//
|
||||
// HandleChildPolicy and HandleEDSResponse must be called by the same goroutine.
|
||||
func (edsImpl *edsBalancerImpl) HandleEDSResponse(edsResp *xdsclient.EDSUpdate) {
|
||||
func (edsImpl *edsBalancerImpl) HandleEDSResponse(edsResp xdsclient.EndpointsUpdate) {
|
||||
// TODO: Unhandled fields from EDS response:
|
||||
// - edsResp.GetPolicy().GetOverprovisioningFactor()
|
||||
// - locality.GetPriority()
|
||||
|
@ -109,7 +109,7 @@ func (f *fakeEDSBalancer) HandleChildPolicy(name string, config json.RawMessage)
|
||||
}
|
||||
|
||||
func (f *fakeEDSBalancer) Close() {}
|
||||
func (f *fakeEDSBalancer) HandleEDSResponse(edsResp *xdsclient.EDSUpdate) {}
|
||||
func (f *fakeEDSBalancer) HandleEDSResponse(edsResp xdsclient.EndpointsUpdate) {}
|
||||
func (f *fakeEDSBalancer) updateState(priority priorityType, s balancer.State) {}
|
||||
|
||||
func (f *fakeEDSBalancer) waitForChildPolicy(wantPolicy *loadBalancingConfig) error {
|
||||
@ -254,7 +254,7 @@ func (s) TestXDSConfigBalancerNameUpdate(t *testing.T) {
|
||||
})
|
||||
|
||||
xdsC := waitForNewXDSClientWithEDSWatch(t, xdsClientCh, balancerName)
|
||||
xdsC.InvokeWatchEDSCallback(&xdsclient.EDSUpdate{}, nil)
|
||||
xdsC.InvokeWatchEDSCallback(xdsclient.EndpointsUpdate{}, nil)
|
||||
}
|
||||
}
|
||||
|
||||
@ -335,7 +335,7 @@ func (s) TestXDSConnfigChildPolicyUpdate(t *testing.T) {
|
||||
},
|
||||
})
|
||||
xdsC := waitForNewXDSClientWithEDSWatch(t, xdsClientCh, testBalancerNameFooBar)
|
||||
xdsC.InvokeWatchEDSCallback(&xdsclient.EDSUpdate{}, nil)
|
||||
xdsC.InvokeWatchEDSCallback(xdsclient.EndpointsUpdate{}, nil)
|
||||
edsLB := waitForNewEDSLB(t, edsLBCh)
|
||||
edsLB.waitForChildPolicy(&loadBalancingConfig{
|
||||
Name: string(fakeBalancerA),
|
||||
@ -384,7 +384,7 @@ func (s) TestXDSSubConnStateChange(t *testing.T) {
|
||||
})
|
||||
|
||||
xdsC := waitForNewXDSClientWithEDSWatch(t, xdsClientCh, testBalancerNameFooBar)
|
||||
xdsC.InvokeWatchEDSCallback(&xdsclient.EDSUpdate{}, nil)
|
||||
xdsC.InvokeWatchEDSCallback(xdsclient.EndpointsUpdate{}, nil)
|
||||
edsLB := waitForNewEDSLB(t, edsLBCh)
|
||||
|
||||
fsc := &fakeSubConn{}
|
||||
|
@ -32,7 +32,7 @@ import (
|
||||
// xdsClientInterface contains only the xds_client methods needed by EDS
|
||||
// balancer. It's defined so we can override xdsclientNew function in tests.
|
||||
type xdsClientInterface interface {
|
||||
WatchEndpoints(clusterName string, edsCb func(*xdsclient.EDSUpdate, error)) (cancel func())
|
||||
WatchEndpoints(clusterName string, edsCb func(xdsclient.EndpointsUpdate, error)) (cancel func())
|
||||
ReportLoad(server string, clusterName string, loadStore lrs.Store) (cancel func())
|
||||
Close()
|
||||
}
|
||||
@ -50,7 +50,7 @@ var (
|
||||
type xdsclientWrapper struct {
|
||||
logger *grpclog.PrefixLogger
|
||||
|
||||
newEDSUpdate func(*xdsclient.EDSUpdate) error
|
||||
newEDSUpdate func(xdsclient.EndpointsUpdate) error
|
||||
loseContact func()
|
||||
bbo balancer.BuildOptions
|
||||
loadStore lrs.Store
|
||||
@ -78,7 +78,7 @@ type xdsclientWrapper struct {
|
||||
//
|
||||
// The given callbacks won't be called until the underlying xds_client is
|
||||
// working and sends updates.
|
||||
func newXDSClientWrapper(newEDSUpdate func(*xdsclient.EDSUpdate) error, loseContact func(), bbo balancer.BuildOptions, loadStore lrs.Store, logger *grpclog.PrefixLogger) *xdsclientWrapper {
|
||||
func newXDSClientWrapper(newEDSUpdate func(xdsclient.EndpointsUpdate) error, loseContact func(), bbo balancer.BuildOptions, loadStore lrs.Store, logger *grpclog.PrefixLogger) *xdsclientWrapper {
|
||||
return &xdsclientWrapper{
|
||||
logger: logger,
|
||||
newEDSUpdate: newEDSUpdate,
|
||||
@ -184,7 +184,10 @@ func (c *xdsclientWrapper) startEndpointsWatch(nameToWatch string) {
|
||||
}
|
||||
|
||||
c.edsServiceName = nameToWatch
|
||||
cancelEDSWatch := c.xdsclient.WatchEndpoints(c.edsServiceName, func(update *xdsclient.EDSUpdate, err error) {
|
||||
if c.cancelEndpointsWatch != nil {
|
||||
c.cancelEndpointsWatch()
|
||||
}
|
||||
cancelEDSWatch := c.xdsclient.WatchEndpoints(c.edsServiceName, func(update xdsclient.EndpointsUpdate, err error) {
|
||||
if err != nil {
|
||||
// TODO: this should trigger a call to `c.loseContact`, when the
|
||||
// error indicates "lose contact".
|
||||
|
@ -21,6 +21,7 @@ package edsbalancer
|
||||
import (
|
||||
"errors"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
|
||||
corepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
|
||||
@ -109,9 +110,25 @@ func (s) TestClientWrapperWatchEDS(t *testing.T) {
|
||||
EDSServiceName: test.edsServiceName,
|
||||
}, nil)
|
||||
|
||||
req, err := fakeServer.XDSRequestChan.Receive()
|
||||
if err != nil {
|
||||
t.Fatalf("EDS RPC failed with err: %v", err)
|
||||
var req interface{}
|
||||
for i := 0; i < 2; i++ {
|
||||
// Each new watch will first cancel the previous watch, and then
|
||||
// start a new watch. The cancel will trigger a request as well.
|
||||
// This loop receives the two requests and keeps the last.
|
||||
r, err := fakeServer.XDSRequestChan.TimedReceive(time.Millisecond * 100)
|
||||
if err != nil {
|
||||
t.Fatalf("i: %v, expected xDS request, but got error: %v", i, err)
|
||||
}
|
||||
req = r
|
||||
// If edsServiceName is empty string, the client doesn't cancel
|
||||
// and resend request. The request from channel was from client
|
||||
// init, and we don't expect a second request.
|
||||
if test.edsServiceName == "" {
|
||||
break
|
||||
}
|
||||
}
|
||||
if r, err := fakeServer.XDSRequestChan.TimedReceive(time.Millisecond * 100); err == nil {
|
||||
t.Fatalf("Expected req channel recv timeout, but got request: %v", r)
|
||||
}
|
||||
edsReq := req.(*fakeserver.Request)
|
||||
if edsReq.Err != nil {
|
||||
@ -142,7 +159,7 @@ func (s) TestClientWrapperWatchEDS(t *testing.T) {
|
||||
// edsBalancer with the received error.
|
||||
func (s) TestClientWrapperHandleUpdateError(t *testing.T) {
|
||||
edsRespChan := testutils.NewChannel()
|
||||
newEDS := func(update *xdsclient.EDSUpdate) error {
|
||||
newEDS := func(update xdsclient.EndpointsUpdate) error {
|
||||
edsRespChan.Send(update)
|
||||
return nil
|
||||
}
|
||||
@ -159,7 +176,7 @@ func (s) TestClientWrapperHandleUpdateError(t *testing.T) {
|
||||
if gotCluster != testEDSClusterName {
|
||||
t.Fatalf("xdsClient.WatchEndpoints() called with cluster: %v, want %v", gotCluster, testEDSClusterName)
|
||||
}
|
||||
xdsC.InvokeWatchEDSCallback(nil, errors.New("EDS watch callback error"))
|
||||
xdsC.InvokeWatchEDSCallback(xdsclient.EndpointsUpdate{}, errors.New("EDS watch callback error"))
|
||||
|
||||
// The callback is called with an error, expect no update from edsRespChan.
|
||||
//
|
||||
|
@ -1,487 +0,0 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2019 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 client
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
discoverypb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
|
||||
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
|
||||
corepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
|
||||
"github.com/golang/protobuf/proto"
|
||||
anypb "github.com/golang/protobuf/ptypes/any"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/testutils/fakeserver"
|
||||
)
|
||||
|
||||
const (
|
||||
clusterName1 = "foo-cluster"
|
||||
clusterName2 = "bar-cluster"
|
||||
serviceName1 = "foo-service"
|
||||
serviceName2 = "bar-service"
|
||||
)
|
||||
|
||||
func (v2c *v2Client) cloneCDSCacheForTesting() map[string]CDSUpdate {
|
||||
v2c.mu.Lock()
|
||||
defer v2c.mu.Unlock()
|
||||
|
||||
cloneCache := make(map[string]CDSUpdate)
|
||||
for k, v := range v2c.cdsCache {
|
||||
cloneCache[k] = v
|
||||
}
|
||||
return cloneCache
|
||||
}
|
||||
|
||||
func (s) TestValidateCluster(t *testing.T) {
|
||||
emptyUpdate := CDSUpdate{ServiceName: "", EnableLRS: false}
|
||||
tests := []struct {
|
||||
name string
|
||||
cluster *xdspb.Cluster
|
||||
wantUpdate CDSUpdate
|
||||
wantErr bool
|
||||
}{
|
||||
{
|
||||
name: "non-eds-cluster-type",
|
||||
cluster: &xdspb.Cluster{
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_STATIC},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{
|
||||
EdsConfig: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Ads{
|
||||
Ads: &corepb.AggregatedConfigSource{},
|
||||
},
|
||||
},
|
||||
},
|
||||
LbPolicy: xdspb.Cluster_LEAST_REQUEST,
|
||||
},
|
||||
wantUpdate: emptyUpdate,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "no-eds-config",
|
||||
cluster: &xdspb.Cluster{
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
LbPolicy: xdspb.Cluster_ROUND_ROBIN,
|
||||
},
|
||||
wantUpdate: emptyUpdate,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "no-ads-config-source",
|
||||
cluster: &xdspb.Cluster{
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{},
|
||||
LbPolicy: xdspb.Cluster_ROUND_ROBIN,
|
||||
},
|
||||
wantUpdate: emptyUpdate,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "non-round-robin-lb-policy",
|
||||
cluster: &xdspb.Cluster{
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{
|
||||
EdsConfig: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Ads{
|
||||
Ads: &corepb.AggregatedConfigSource{},
|
||||
},
|
||||
},
|
||||
},
|
||||
LbPolicy: xdspb.Cluster_LEAST_REQUEST,
|
||||
},
|
||||
wantUpdate: emptyUpdate,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "happy-case-no-service-name-no-lrs",
|
||||
cluster: &xdspb.Cluster{
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{
|
||||
EdsConfig: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Ads{
|
||||
Ads: &corepb.AggregatedConfigSource{},
|
||||
},
|
||||
},
|
||||
},
|
||||
LbPolicy: xdspb.Cluster_ROUND_ROBIN,
|
||||
},
|
||||
wantUpdate: emptyUpdate,
|
||||
},
|
||||
{
|
||||
name: "happy-case-no-lrs",
|
||||
cluster: &xdspb.Cluster{
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{
|
||||
EdsConfig: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Ads{
|
||||
Ads: &corepb.AggregatedConfigSource{},
|
||||
},
|
||||
},
|
||||
ServiceName: serviceName1,
|
||||
},
|
||||
LbPolicy: xdspb.Cluster_ROUND_ROBIN,
|
||||
},
|
||||
wantUpdate: CDSUpdate{ServiceName: serviceName1, EnableLRS: false},
|
||||
},
|
||||
{
|
||||
name: "happiest-case",
|
||||
cluster: goodCluster1,
|
||||
wantUpdate: CDSUpdate{ServiceName: serviceName1, EnableLRS: true},
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
gotUpdate, gotErr := validateCluster(test.cluster)
|
||||
if (gotErr != nil) != test.wantErr {
|
||||
t.Errorf("validateCluster(%+v) returned error: %v, wantErr: %v", test.cluster, gotErr, test.wantErr)
|
||||
}
|
||||
if !cmp.Equal(gotUpdate, test.wantUpdate) {
|
||||
t.Errorf("validateCluster(%+v) = %v, want: %v", test.cluster, gotUpdate, test.wantUpdate)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// TestCDSHandleResponse starts a fake xDS server, makes a ClientConn to it,
|
||||
// and creates a v2Client using it. Then, it registers a CDS watcher and tests
|
||||
// different CDS responses.
|
||||
func (s) TestCDSHandleResponse(t *testing.T) {
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
cdsResponse *xdspb.DiscoveryResponse
|
||||
wantErr bool
|
||||
wantUpdate *CDSUpdate
|
||||
wantUpdateErr bool
|
||||
}{
|
||||
// Badly marshaled CDS response.
|
||||
{
|
||||
name: "badly-marshaled-response",
|
||||
cdsResponse: badlyMarshaledCDSResponse,
|
||||
wantErr: true,
|
||||
wantUpdate: nil,
|
||||
wantUpdateErr: false,
|
||||
},
|
||||
// Response does not contain Cluster proto.
|
||||
{
|
||||
name: "no-cluster-proto-in-response",
|
||||
cdsResponse: badResourceTypeInLDSResponse,
|
||||
wantErr: true,
|
||||
wantUpdate: nil,
|
||||
wantUpdateErr: false,
|
||||
},
|
||||
// Response contains no clusters.
|
||||
{
|
||||
name: "no-cluster",
|
||||
cdsResponse: &xdspb.DiscoveryResponse{},
|
||||
wantErr: false,
|
||||
wantUpdate: &CDSUpdate{},
|
||||
wantUpdateErr: true,
|
||||
},
|
||||
// Response contains one good cluster we are not interested in.
|
||||
{
|
||||
name: "one-uninteresting-cluster",
|
||||
cdsResponse: goodCDSResponse2,
|
||||
wantErr: false,
|
||||
wantUpdate: &CDSUpdate{},
|
||||
wantUpdateErr: true,
|
||||
},
|
||||
// Response contains one cluster and it is good.
|
||||
{
|
||||
name: "one-good-cluster",
|
||||
cdsResponse: goodCDSResponse1,
|
||||
wantErr: false,
|
||||
wantUpdate: &CDSUpdate{ServiceName: serviceName1, EnableLRS: true},
|
||||
wantUpdateErr: false,
|
||||
},
|
||||
}
|
||||
for _, test := range tests {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
testWatchHandle(t, &watchHandleTestcase{
|
||||
responseToHandle: test.cdsResponse,
|
||||
wantHandleErr: test.wantErr,
|
||||
wantUpdate: test.wantUpdate,
|
||||
wantUpdateErr: test.wantUpdateErr,
|
||||
|
||||
cdsWatch: v2c.watchCDS,
|
||||
watchReqChan: fakeServer.XDSRequestChan,
|
||||
handleXDSResp: v2c.handleCDSResponse,
|
||||
})
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// TestCDSHandleResponseWithoutWatch tests the case where the v2Client receives
|
||||
// a CDS response without a registered watcher.
|
||||
func (s) TestCDSHandleResponseWithoutWatch(t *testing.T) {
|
||||
_, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
|
||||
if v2c.handleCDSResponse(goodCDSResponse1) == nil {
|
||||
t.Fatal("v2c.handleCDSResponse() succeeded, should have failed")
|
||||
}
|
||||
}
|
||||
|
||||
// cdsTestOp contains all data related to one particular test operation. Not
|
||||
// all fields make sense for all tests.
|
||||
type cdsTestOp struct {
|
||||
// target is the resource name to watch for.
|
||||
target string
|
||||
// responseToSend is the xDS response sent to the client
|
||||
responseToSend *fakeserver.Response
|
||||
// wantOpErr specfies whether the main operation should return an error.
|
||||
wantOpErr bool
|
||||
// wantCDSCache is the expected rdsCache at the end of an operation.
|
||||
wantCDSCache map[string]CDSUpdate
|
||||
// wantWatchCallback specifies if the watch callback should be invoked.
|
||||
wantWatchCallback bool
|
||||
}
|
||||
|
||||
// testCDSCaching is a helper function which starts a fake xDS server, makes a
|
||||
// ClientConn to it, creates a v2Client using it. It then reads a bunch of
|
||||
// test operations to be performed from cdsTestOps and returns error, if any,
|
||||
// on the provided error channel. This is executed in a separate goroutine.
|
||||
func testCDSCaching(t *testing.T, cdsTestOps []cdsTestOp, errCh *testutils.Channel) {
|
||||
t.Helper()
|
||||
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
|
||||
callbackCh := make(chan struct{}, 1)
|
||||
for _, cdsTestOp := range cdsTestOps {
|
||||
// Register a watcher if required, and use a channel to signal the
|
||||
// successful invocation of the callback.
|
||||
if cdsTestOp.target != "" {
|
||||
v2c.watchCDS(cdsTestOp.target, func(u CDSUpdate, err error) {
|
||||
t.Logf("Received callback with CDSUpdate {%+v} and error {%v}", u, err)
|
||||
callbackCh <- struct{}{}
|
||||
})
|
||||
t.Logf("Registered a watcher for CDS target: %v...", cdsTestOp.target)
|
||||
|
||||
// Wait till the request makes it to the fakeServer. This ensures that
|
||||
// the watch request has been processed by the v2Client.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
errCh.Send(fmt.Errorf("Timeout waiting for CDS request: %v", err))
|
||||
return
|
||||
}
|
||||
t.Log("FakeServer received request...")
|
||||
}
|
||||
|
||||
// Directly push the response through a call to handleCDSResponse,
|
||||
// thereby bypassing the fakeServer.
|
||||
if cdsTestOp.responseToSend != nil {
|
||||
resp := cdsTestOp.responseToSend.Resp.(*discoverypb.DiscoveryResponse)
|
||||
if err := v2c.handleCDSResponse(resp); (err != nil) != cdsTestOp.wantOpErr {
|
||||
errCh.Send(fmt.Errorf("v2c.handleRDSResponse(%+v) returned err: %v", resp, err))
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
// If the test needs the callback to be invoked, just verify that
|
||||
// it was invoked. Since we verify the contents of the cache, it's
|
||||
// ok not to verify the contents of the callback.
|
||||
if cdsTestOp.wantWatchCallback {
|
||||
<-callbackCh
|
||||
}
|
||||
|
||||
if !cmp.Equal(v2c.cloneCDSCacheForTesting(), cdsTestOp.wantCDSCache) {
|
||||
errCh.Send(fmt.Errorf("gotCDSCache: %v, wantCDSCache: %v", v2c.rdsCache, cdsTestOp.wantCDSCache))
|
||||
return
|
||||
}
|
||||
}
|
||||
t.Log("Completed all test ops successfully...")
|
||||
errCh.Send(nil)
|
||||
}
|
||||
|
||||
// TestCDSCaching tests some end-to-end CDS flows using a fake xDS server, and
|
||||
// verifies the CDS data cached at the v2Client.
|
||||
func (s) TestCDSCaching(t *testing.T) {
|
||||
ops := []cdsTestOp{
|
||||
// Add an CDS watch for a cluster name (clusterName1), which returns one
|
||||
// matching resource in the response.
|
||||
{
|
||||
target: clusterName1,
|
||||
responseToSend: &fakeserver.Response{Resp: goodCDSResponse1},
|
||||
wantCDSCache: map[string]CDSUpdate{
|
||||
clusterName1: {serviceName1, true},
|
||||
},
|
||||
wantWatchCallback: true,
|
||||
},
|
||||
// Push an CDS response which contains a new resource (apart from the
|
||||
// one received in the previous response). This should be cached.
|
||||
{
|
||||
responseToSend: &fakeserver.Response{Resp: cdsResponseWithMultipleResources},
|
||||
wantCDSCache: map[string]CDSUpdate{
|
||||
clusterName1: {serviceName1, true},
|
||||
clusterName2: {serviceName2, false},
|
||||
},
|
||||
wantWatchCallback: true,
|
||||
},
|
||||
// Switch the watch target to clusterName2, which was already cached. No
|
||||
// response is received from the server (as expected), but we want the
|
||||
// callback to be invoked with the new serviceName.
|
||||
{
|
||||
target: clusterName2,
|
||||
wantCDSCache: map[string]CDSUpdate{
|
||||
clusterName1: {serviceName1, true},
|
||||
clusterName2: {serviceName2, false},
|
||||
},
|
||||
wantWatchCallback: true,
|
||||
},
|
||||
// Push an empty CDS response. This should clear the cache.
|
||||
{
|
||||
responseToSend: &fakeserver.Response{Resp: &xdspb.DiscoveryResponse{TypeUrl: cdsURL}},
|
||||
wantOpErr: false,
|
||||
wantCDSCache: map[string]CDSUpdate{},
|
||||
wantWatchCallback: true,
|
||||
},
|
||||
}
|
||||
errCh := testutils.NewChannel()
|
||||
go testCDSCaching(t, ops, errCh)
|
||||
waitForNilErr(t, errCh)
|
||||
}
|
||||
|
||||
// TestCDSWatchExpiryTimer tests the case where the client does not receive an
|
||||
// CDS response for the request that it sends out. We want the watch callback
|
||||
// to be invoked with an error once the watchExpiryTimer fires.
|
||||
func (s) TestCDSWatchExpiryTimer(t *testing.T) {
|
||||
oldWatchExpiryTimeout := defaultWatchExpiryTimeout
|
||||
defaultWatchExpiryTimeout = 500 * time.Millisecond
|
||||
defer func() {
|
||||
defaultWatchExpiryTimeout = oldWatchExpiryTimeout
|
||||
}()
|
||||
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
v2c.watchCDS(clusterName1, func(u CDSUpdate, err error) {
|
||||
t.Logf("Received callback with CDSUpdate {%+v} and error {%v}", u, err)
|
||||
if u.ServiceName != "" {
|
||||
callbackCh.Send(fmt.Errorf("received serviceName %v in cdsCallback, wanted empty string", u.ServiceName))
|
||||
}
|
||||
if err == nil {
|
||||
callbackCh.Send(errors.New("received nil error in cdsCallback"))
|
||||
}
|
||||
callbackCh.Send(nil)
|
||||
})
|
||||
|
||||
// Wait till the request makes it to the fakeServer. This ensures that
|
||||
// the watch request has been processed by the v2Client.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an CDS request")
|
||||
}
|
||||
waitForNilErr(t, callbackCh)
|
||||
}
|
||||
|
||||
var (
|
||||
badlyMarshaledCDSResponse = &xdspb.DiscoveryResponse{
|
||||
Resources: []*anypb.Any{
|
||||
{
|
||||
TypeUrl: cdsURL,
|
||||
Value: []byte{1, 2, 3, 4},
|
||||
},
|
||||
},
|
||||
TypeUrl: cdsURL,
|
||||
}
|
||||
goodCluster1 = &xdspb.Cluster{
|
||||
Name: clusterName1,
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{
|
||||
EdsConfig: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Ads{
|
||||
Ads: &corepb.AggregatedConfigSource{},
|
||||
},
|
||||
},
|
||||
ServiceName: serviceName1,
|
||||
},
|
||||
LbPolicy: xdspb.Cluster_ROUND_ROBIN,
|
||||
LrsServer: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Self{
|
||||
Self: &corepb.SelfConfigSource{},
|
||||
},
|
||||
},
|
||||
}
|
||||
marshaledCluster1, _ = proto.Marshal(goodCluster1)
|
||||
goodCluster2 = &xdspb.Cluster{
|
||||
Name: clusterName2,
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{
|
||||
EdsConfig: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Ads{
|
||||
Ads: &corepb.AggregatedConfigSource{},
|
||||
},
|
||||
},
|
||||
ServiceName: serviceName2,
|
||||
},
|
||||
LbPolicy: xdspb.Cluster_ROUND_ROBIN,
|
||||
}
|
||||
marshaledCluster2, _ = proto.Marshal(goodCluster2)
|
||||
goodCDSResponse1 = &xdspb.DiscoveryResponse{
|
||||
Resources: []*anypb.Any{
|
||||
{
|
||||
TypeUrl: cdsURL,
|
||||
Value: marshaledCluster1,
|
||||
},
|
||||
},
|
||||
TypeUrl: cdsURL,
|
||||
}
|
||||
goodCDSResponse2 = &xdspb.DiscoveryResponse{
|
||||
Resources: []*anypb.Any{
|
||||
{
|
||||
TypeUrl: cdsURL,
|
||||
Value: marshaledCluster2,
|
||||
},
|
||||
},
|
||||
TypeUrl: cdsURL,
|
||||
}
|
||||
cdsResponseWithMultipleResources = &xdspb.DiscoveryResponse{
|
||||
Resources: []*anypb.Any{
|
||||
{
|
||||
TypeUrl: cdsURL,
|
||||
Value: marshaledCluster1,
|
||||
},
|
||||
{
|
||||
TypeUrl: cdsURL,
|
||||
Value: marshaledCluster2,
|
||||
},
|
||||
},
|
||||
TypeUrl: cdsURL,
|
||||
}
|
||||
)
|
@ -26,9 +26,12 @@ import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
corepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/internal/backoff"
|
||||
"google.golang.org/grpc/internal/buffer"
|
||||
"google.golang.org/grpc/internal/grpclog"
|
||||
"google.golang.org/grpc/internal/grpcsync"
|
||||
"google.golang.org/grpc/keepalive"
|
||||
"google.golang.org/grpc/xds/internal/client/bootstrap"
|
||||
)
|
||||
@ -45,6 +48,18 @@ type Options struct {
|
||||
TargetName string
|
||||
}
|
||||
|
||||
// Interface to be overridden in tests.
|
||||
type xdsv2Client interface {
|
||||
addWatch(resourceType, resourceName string)
|
||||
removeWatch(resourceType, resourceName string)
|
||||
close()
|
||||
}
|
||||
|
||||
// Function to be overridden in tests.
|
||||
var newXDSV2Client = func(parent *Client, cc *grpc.ClientConn, nodeProto *corepb.Node, backoff func(int) time.Duration, logger *grpclog.PrefixLogger) xdsv2Client {
|
||||
return newV2Client(parent, cc, nodeProto, backoff, logger)
|
||||
}
|
||||
|
||||
// Client is a full fledged gRPC client which queries a set of discovery APIs
|
||||
// (collectively termed as xDS) on a remote management server, to discover
|
||||
// various dynamic resources.
|
||||
@ -53,16 +68,23 @@ type Options struct {
|
||||
// implementations. But the same client can only be shared by the same parent
|
||||
// ClientConn.
|
||||
type Client struct {
|
||||
done *grpcsync.Event
|
||||
opts Options
|
||||
cc *grpc.ClientConn // Connection to the xDS server
|
||||
v2c *v2Client // Actual xDS client implementation using the v2 API
|
||||
v2c xdsv2Client // Actual xDS client implementation using the v2 API
|
||||
|
||||
logger *grpclog.PrefixLogger
|
||||
|
||||
mu sync.Mutex
|
||||
serviceCallback func(ServiceUpdate, error)
|
||||
ldsCancel func()
|
||||
rdsCancel func()
|
||||
updateCh *buffer.Unbounded // chan *watcherInfoWithUpdate
|
||||
mu sync.Mutex
|
||||
ldsWatchers map[string]map[*watchInfo]bool
|
||||
ldsCache map[string]ldsUpdate
|
||||
rdsWatchers map[string]map[*watchInfo]bool
|
||||
rdsCache map[string]rdsUpdate
|
||||
cdsWatchers map[string]map[*watchInfo]bool
|
||||
cdsCache map[string]ClusterUpdate
|
||||
edsWatchers map[string]map[*watchInfo]bool
|
||||
edsCache map[string]EndpointsUpdate
|
||||
}
|
||||
|
||||
// New returns a new xdsClient configured with opts.
|
||||
@ -85,7 +107,20 @@ func New(opts Options) (*Client, error) {
|
||||
}
|
||||
dopts = append(dopts, opts.DialOpts...)
|
||||
|
||||
c := &Client{opts: opts}
|
||||
c := &Client{
|
||||
done: grpcsync.NewEvent(),
|
||||
opts: opts,
|
||||
|
||||
updateCh: buffer.NewUnbounded(),
|
||||
ldsWatchers: make(map[string]map[*watchInfo]bool),
|
||||
ldsCache: make(map[string]ldsUpdate),
|
||||
rdsWatchers: make(map[string]map[*watchInfo]bool),
|
||||
rdsCache: make(map[string]rdsUpdate),
|
||||
cdsWatchers: make(map[string]map[*watchInfo]bool),
|
||||
cdsCache: make(map[string]ClusterUpdate),
|
||||
edsWatchers: make(map[string]map[*watchInfo]bool),
|
||||
edsCache: make(map[string]EndpointsUpdate),
|
||||
}
|
||||
|
||||
cc, err := grpc.Dial(opts.Config.BalancerName, dopts...)
|
||||
if err != nil {
|
||||
@ -96,92 +131,42 @@ func New(opts Options) (*Client, error) {
|
||||
c.logger = grpclog.NewPrefixLogger(loggingPrefix(c))
|
||||
c.logger.Infof("Created ClientConn to xDS server: %s", opts.Config.BalancerName)
|
||||
|
||||
c.v2c = newV2Client(cc, opts.Config.NodeProto, backoff.DefaultExponential.Backoff, c.logger)
|
||||
c.v2c = newXDSV2Client(c, cc, opts.Config.NodeProto, backoff.DefaultExponential.Backoff, c.logger)
|
||||
c.logger.Infof("Created")
|
||||
go c.run()
|
||||
return c, nil
|
||||
}
|
||||
|
||||
// run is a goroutine for all the callbacks.
|
||||
//
|
||||
// Callback can be called in watch(), if an item is found in cache. Without this
|
||||
// goroutine, the callback will be called inline, which might cause a deadlock
|
||||
// in user's code. Callbacks also cannot be simple `go callback()` because the
|
||||
// order matters.
|
||||
func (c *Client) run() {
|
||||
for {
|
||||
select {
|
||||
case t := <-c.updateCh.Get():
|
||||
c.updateCh.Load()
|
||||
if c.done.HasFired() {
|
||||
return
|
||||
}
|
||||
c.callCallback(t.(*watcherInfoWithUpdate))
|
||||
case <-c.done.Done():
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Close closes the gRPC connection to the xDS server.
|
||||
func (c *Client) Close() {
|
||||
if c.done.HasFired() {
|
||||
return
|
||||
}
|
||||
c.done.Fire()
|
||||
// TODO: Should we invoke the registered callbacks here with an error that
|
||||
// the client is closed?
|
||||
c.v2c.close()
|
||||
c.cc.Close()
|
||||
c.logger.Infof("Shutdown")
|
||||
}
|
||||
|
||||
// ServiceUpdate contains update about the service.
|
||||
type ServiceUpdate struct {
|
||||
Cluster string
|
||||
}
|
||||
|
||||
// handleLDSUpdate is the LDS watcher callback we registered with the v2Client.
|
||||
func (c *Client) handleLDSUpdate(u ldsUpdate, err error) {
|
||||
c.logger.Infof("xds: client received LDS update: %+v, err: %v", u, err)
|
||||
if err != nil {
|
||||
c.mu.Lock()
|
||||
if c.serviceCallback != nil {
|
||||
c.serviceCallback(ServiceUpdate{}, err)
|
||||
}
|
||||
c.mu.Unlock()
|
||||
return
|
||||
}
|
||||
|
||||
c.mu.Lock()
|
||||
c.rdsCancel = c.v2c.watchRDS(u.routeName, c.handleRDSUpdate)
|
||||
c.mu.Unlock()
|
||||
}
|
||||
|
||||
// handleRDSUpdate is the RDS watcher callback we registered with the v2Client.
|
||||
func (c *Client) handleRDSUpdate(u rdsUpdate, err error) {
|
||||
c.logger.Infof("xds: client received RDS update: %+v, err: %v", u, err)
|
||||
if err != nil {
|
||||
c.mu.Lock()
|
||||
if c.serviceCallback != nil {
|
||||
c.serviceCallback(ServiceUpdate{}, err)
|
||||
}
|
||||
c.mu.Unlock()
|
||||
return
|
||||
}
|
||||
|
||||
c.mu.Lock()
|
||||
if c.serviceCallback != nil {
|
||||
c.serviceCallback(ServiceUpdate{Cluster: u.clusterName}, nil)
|
||||
}
|
||||
c.mu.Unlock()
|
||||
}
|
||||
|
||||
// WatchService uses LDS and RDS protocols to discover information about the
|
||||
// provided serviceName.
|
||||
func (c *Client) WatchService(serviceName string, callback func(ServiceUpdate, error)) (cancel func()) {
|
||||
// TODO: Error out early if the client is closed. Ideally, this should
|
||||
// never be called after the client is closed though.
|
||||
c.mu.Lock()
|
||||
c.serviceCallback = callback
|
||||
c.ldsCancel = c.v2c.watchLDS(serviceName, c.handleLDSUpdate)
|
||||
c.mu.Unlock()
|
||||
|
||||
return func() {
|
||||
c.mu.Lock()
|
||||
c.serviceCallback = nil
|
||||
if c.ldsCancel != nil {
|
||||
c.ldsCancel()
|
||||
}
|
||||
if c.rdsCancel != nil {
|
||||
c.rdsCancel()
|
||||
}
|
||||
c.mu.Unlock()
|
||||
}
|
||||
}
|
||||
|
||||
// WatchCluster uses CDS to discover information about the provided
|
||||
// clusterName.
|
||||
func (c *Client) WatchCluster(clusterName string, cdsCb func(CDSUpdate, error)) (cancel func()) {
|
||||
return c.v2c.watchCDS(clusterName, cdsCb)
|
||||
}
|
||||
|
||||
// WatchEndpoints uses EDS to discover information about the endpoints in the
|
||||
// provided clusterName.
|
||||
func (c *Client) WatchEndpoints(clusterName string, edsCb func(*EDSUpdate, error)) (cancel func()) {
|
||||
return c.v2c.watchEDS(clusterName, edsCb)
|
||||
}
|
||||
|
162
xds/internal/client/client_callback.go
Normal file
162
xds/internal/client/client_callback.go
Normal file
@ -0,0 +1,162 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2020 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 client
|
||||
|
||||
type watcherInfoWithUpdate struct {
|
||||
wi *watchInfo
|
||||
update interface{}
|
||||
err error
|
||||
}
|
||||
|
||||
func (c *Client) scheduleCallback(wi *watchInfo, update interface{}, err error) {
|
||||
c.updateCh.Put(&watcherInfoWithUpdate{
|
||||
wi: wi,
|
||||
update: update,
|
||||
err: err,
|
||||
})
|
||||
}
|
||||
|
||||
func (c *Client) callCallback(wiu *watcherInfoWithUpdate) {
|
||||
c.mu.Lock()
|
||||
// Use a closure to capture the callback and type assertion, to save one
|
||||
// more switch case.
|
||||
//
|
||||
// The callback must be called without c.mu. Otherwise if the callback calls
|
||||
// another watch() inline, it will cause a deadlock. This leaves a small
|
||||
// window that a watcher's callback could be called after the watcher is
|
||||
// canceled, and the user needs to take care of it.
|
||||
var ccb func()
|
||||
switch wiu.wi.typeURL {
|
||||
case ldsURL:
|
||||
if s, ok := c.ldsWatchers[wiu.wi.target]; ok && s[wiu.wi] {
|
||||
ccb = func() { wiu.wi.ldsCallback(wiu.update.(ldsUpdate), wiu.err) }
|
||||
}
|
||||
case rdsURL:
|
||||
if s, ok := c.rdsWatchers[wiu.wi.target]; ok && s[wiu.wi] {
|
||||
ccb = func() { wiu.wi.rdsCallback(wiu.update.(rdsUpdate), wiu.err) }
|
||||
}
|
||||
case cdsURL:
|
||||
if s, ok := c.cdsWatchers[wiu.wi.target]; ok && s[wiu.wi] {
|
||||
ccb = func() { wiu.wi.cdsCallback(wiu.update.(ClusterUpdate), wiu.err) }
|
||||
}
|
||||
case edsURL:
|
||||
if s, ok := c.edsWatchers[wiu.wi.target]; ok && s[wiu.wi] {
|
||||
ccb = func() { wiu.wi.edsCallback(wiu.update.(EndpointsUpdate), wiu.err) }
|
||||
}
|
||||
}
|
||||
c.mu.Unlock()
|
||||
|
||||
if ccb != nil {
|
||||
ccb()
|
||||
}
|
||||
}
|
||||
|
||||
// newLDSUpdate is called by the underlying xdsv2Client when it receives an xDS
|
||||
// response.
|
||||
//
|
||||
// A response can contain multiple resources. They will be parsed and put in a
|
||||
// map from resource name to the resource content.
|
||||
func (c *Client) newLDSUpdate(d map[string]ldsUpdate) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
for name, update := range d {
|
||||
if s, ok := c.ldsWatchers[name]; ok {
|
||||
for wi := range s {
|
||||
c.scheduleCallback(wi, update, nil)
|
||||
}
|
||||
// Sync cache.
|
||||
c.logger.Debugf("LDS resource with name %v, value %+v added to cache", name, update)
|
||||
c.ldsCache[name] = update
|
||||
}
|
||||
}
|
||||
// TODO: handle removing resources, which means if a resource exists in the
|
||||
// previous update, but not in the new update. This needs the balancers and
|
||||
// resolvers to handle errors correctly.
|
||||
|
||||
// TODO: remove item from cache and remove corresponding RDS cached data.
|
||||
}
|
||||
|
||||
// newRDSUpdate is called by the underlying xdsv2Client when it receives an xDS
|
||||
// response.
|
||||
//
|
||||
// A response can contain multiple resources. They will be parsed and put in a
|
||||
// map from resource name to the resource content.
|
||||
func (c *Client) newRDSUpdate(d map[string]rdsUpdate) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
for name, update := range d {
|
||||
if s, ok := c.rdsWatchers[name]; ok {
|
||||
for wi := range s {
|
||||
c.scheduleCallback(wi, update, nil)
|
||||
}
|
||||
// Sync cache.
|
||||
c.logger.Debugf("RDS resource with name %v, value %+v added to cache", name, update)
|
||||
c.rdsCache[name] = update
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// newCDSUpdate is called by the underlying xdsv2Client when it receives an xDS
|
||||
// response.
|
||||
//
|
||||
// A response can contain multiple resources. They will be parsed and put in a
|
||||
// map from resource name to the resource content.
|
||||
func (c *Client) newCDSUpdate(d map[string]ClusterUpdate) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
for name, update := range d {
|
||||
if s, ok := c.cdsWatchers[name]; ok {
|
||||
for wi := range s {
|
||||
c.scheduleCallback(wi, update, nil)
|
||||
}
|
||||
// Sync cache.
|
||||
c.logger.Debugf("CDS resource with name %v, value %+v added to cache", name, update)
|
||||
c.cdsCache[name] = update
|
||||
}
|
||||
}
|
||||
// TODO: handle removing resources, which means if a resource exists in the
|
||||
// previous update, but not in the new update. This needs the balancers and
|
||||
// resolvers to handle errors correctly.
|
||||
|
||||
// TODO: remove item from cache and remove corresponding EDS cached data.
|
||||
}
|
||||
|
||||
// newEDSUpdate is called by the underlying xdsv2Client when it receives an xDS
|
||||
// response.
|
||||
//
|
||||
// A response can contain multiple resources. They will be parsed and put in a
|
||||
// map from resource name to the resource content.
|
||||
func (c *Client) newEDSUpdate(d map[string]EndpointsUpdate) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
for name, update := range d {
|
||||
if s, ok := c.edsWatchers[name]; ok {
|
||||
for wi := range s {
|
||||
c.scheduleCallback(wi, update, nil)
|
||||
}
|
||||
// Sync cache.
|
||||
c.logger.Debugf("EDS resource with name %v, value %+v added to cache", name, update)
|
||||
c.edsCache[name] = update
|
||||
}
|
||||
}
|
||||
}
|
@ -19,12 +19,11 @@
|
||||
package client
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/internal/grpclog"
|
||||
"google.golang.org/grpc/internal/grpctest"
|
||||
"google.golang.org/grpc/xds/internal/client/bootstrap"
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
@ -41,6 +40,16 @@ func Test(t *testing.T) {
|
||||
grpctest.RunSubTests(t, s{})
|
||||
}
|
||||
|
||||
const (
|
||||
testXDSServer = "xds-server"
|
||||
chanRecvTimeout = 100 * time.Millisecond
|
||||
|
||||
testLDSName = "test-lds"
|
||||
testRDSName = "test-rds"
|
||||
testCDSName = "test-cds"
|
||||
testEDSName = "test-eds"
|
||||
)
|
||||
|
||||
func clientOpts(balancerName string) Options {
|
||||
return Options{
|
||||
Config: bootstrap.Config{
|
||||
@ -48,9 +57,6 @@ func clientOpts(balancerName string) Options {
|
||||
Creds: grpc.WithInsecure(),
|
||||
NodeProto: &corepb.Node{},
|
||||
},
|
||||
// WithTimeout is deprecated. But we are OK to call it here from the
|
||||
// test, so we clearly know that the dial failed.
|
||||
DialOpts: []grpc.DialOption{grpc.WithTimeout(5 * time.Second), grpc.WithBlock()},
|
||||
}
|
||||
}
|
||||
|
||||
@ -117,185 +123,88 @@ func (s) TestNew(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
// TestWatchService tests the happy case of registering a watcher for
|
||||
// service updates and receiving a good update.
|
||||
func (s) TestWatchService(t *testing.T) {
|
||||
fakeServer, cleanup, err := fakeserver.StartServer()
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to start fake xDS server: %v", err)
|
||||
}
|
||||
defer cleanup()
|
||||
type testXDSV2Client struct {
|
||||
r updateHandler
|
||||
|
||||
xdsClient, err := New(clientOpts(fakeServer.Address))
|
||||
if err != nil {
|
||||
t.Fatalf("New returned error: %v", err)
|
||||
}
|
||||
defer xdsClient.Close()
|
||||
t.Log("Created an xdsClient...")
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
cancelWatch := xdsClient.WatchService(goodLDSTarget1, func(su ServiceUpdate, err error) {
|
||||
if err != nil {
|
||||
callbackCh.Send(fmt.Errorf("xdsClient.WatchService returned error: %v", err))
|
||||
return
|
||||
}
|
||||
if su.Cluster != goodClusterName1 {
|
||||
callbackCh.Send(fmt.Errorf("got clusterName: %+v, want clusterName: %+v", su.Cluster, goodClusterName1))
|
||||
return
|
||||
}
|
||||
callbackCh.Send(nil)
|
||||
})
|
||||
defer cancelWatch()
|
||||
t.Log("Registered a watcher for service updates...")
|
||||
|
||||
// Make the fakeServer send LDS response.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an LDS request")
|
||||
}
|
||||
fakeServer.XDSResponseChan <- &fakeserver.Response{Resp: goodLDSResponse1}
|
||||
|
||||
// Make the fakeServer send RDS response.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an RDS request")
|
||||
}
|
||||
fakeServer.XDSResponseChan <- &fakeserver.Response{Resp: goodRDSResponse1}
|
||||
waitForNilErr(t, callbackCh)
|
||||
addWatches map[string]chan string
|
||||
removeWatches map[string]chan string
|
||||
}
|
||||
|
||||
// TestWatchServiceWithNoResponseFromServer tests the case where the
|
||||
// xDS server does not respond to the requests being sent out as part of
|
||||
// registering a service update watcher. The underlying v2Client will timeout
|
||||
// and will send us an error.
|
||||
func (s) TestWatchServiceWithNoResponseFromServer(t *testing.T) {
|
||||
fakeServer, cleanup, err := fakeserver.StartServer()
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to start fake xDS server: %v", err)
|
||||
func overrideNewXDSV2Client() (<-chan *testXDSV2Client, func()) {
|
||||
oldNewXDSV2Client := newXDSV2Client
|
||||
ch := make(chan *testXDSV2Client, 1)
|
||||
newXDSV2Client = func(parent *Client, cc *grpc.ClientConn, nodeProto *corepb.Node, backoff func(int) time.Duration, logger *grpclog.PrefixLogger) xdsv2Client {
|
||||
ret := newTestXDSV2Client(parent)
|
||||
ch <- ret
|
||||
return ret
|
||||
}
|
||||
defer cleanup()
|
||||
|
||||
xdsClient, err := New(clientOpts(fakeServer.Address))
|
||||
if err != nil {
|
||||
t.Fatalf("New returned error: %v", err)
|
||||
}
|
||||
defer xdsClient.Close()
|
||||
t.Log("Created an xdsClient...")
|
||||
|
||||
oldWatchExpiryTimeout := defaultWatchExpiryTimeout
|
||||
defaultWatchExpiryTimeout = 500 * time.Millisecond
|
||||
defer func() {
|
||||
defaultWatchExpiryTimeout = oldWatchExpiryTimeout
|
||||
}()
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
cancelWatch := xdsClient.WatchService(goodLDSTarget1, func(su ServiceUpdate, err error) {
|
||||
if su.Cluster != "" {
|
||||
callbackCh.Send(fmt.Errorf("got clusterName: %+v, want empty clusterName", su.Cluster))
|
||||
return
|
||||
}
|
||||
if err == nil {
|
||||
callbackCh.Send(errors.New("xdsClient.WatchService returned error non-nil error"))
|
||||
return
|
||||
}
|
||||
callbackCh.Send(nil)
|
||||
})
|
||||
defer cancelWatch()
|
||||
t.Log("Registered a watcher for service updates...")
|
||||
|
||||
// Wait for one request from the client, but send no reponses.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an LDS request")
|
||||
}
|
||||
waitForNilErr(t, callbackCh)
|
||||
return ch, func() { newXDSV2Client = oldNewXDSV2Client }
|
||||
}
|
||||
|
||||
// TestWatchServiceEmptyRDS tests the case where the underlying
|
||||
// v2Client receives an empty RDS response.
|
||||
func (s) TestWatchServiceEmptyRDS(t *testing.T) {
|
||||
fakeServer, cleanup, err := fakeserver.StartServer()
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to start fake xDS server: %v", err)
|
||||
func newTestXDSV2Client(r updateHandler) *testXDSV2Client {
|
||||
addWatches := make(map[string]chan string)
|
||||
addWatches[ldsURL] = make(chan string, 10)
|
||||
addWatches[rdsURL] = make(chan string, 10)
|
||||
addWatches[cdsURL] = make(chan string, 10)
|
||||
addWatches[edsURL] = make(chan string, 10)
|
||||
removeWatches := make(map[string]chan string)
|
||||
removeWatches[ldsURL] = make(chan string, 10)
|
||||
removeWatches[rdsURL] = make(chan string, 10)
|
||||
removeWatches[cdsURL] = make(chan string, 10)
|
||||
removeWatches[edsURL] = make(chan string, 10)
|
||||
return &testXDSV2Client{
|
||||
r: r,
|
||||
addWatches: addWatches,
|
||||
removeWatches: removeWatches,
|
||||
}
|
||||
defer cleanup()
|
||||
|
||||
xdsClient, err := New(clientOpts(fakeServer.Address))
|
||||
if err != nil {
|
||||
t.Fatalf("New returned error: %v", err)
|
||||
}
|
||||
defer xdsClient.Close()
|
||||
t.Log("Created an xdsClient...")
|
||||
|
||||
oldWatchExpiryTimeout := defaultWatchExpiryTimeout
|
||||
defaultWatchExpiryTimeout = 500 * time.Millisecond
|
||||
defer func() {
|
||||
defaultWatchExpiryTimeout = oldWatchExpiryTimeout
|
||||
}()
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
cancelWatch := xdsClient.WatchService(goodLDSTarget1, func(su ServiceUpdate, err error) {
|
||||
if su.Cluster != "" {
|
||||
callbackCh.Send(fmt.Errorf("got clusterName: %+v, want empty clusterName", su.Cluster))
|
||||
return
|
||||
}
|
||||
if err == nil {
|
||||
callbackCh.Send(errors.New("xdsClient.WatchService returned error non-nil error"))
|
||||
return
|
||||
}
|
||||
callbackCh.Send(nil)
|
||||
})
|
||||
defer cancelWatch()
|
||||
t.Log("Registered a watcher for service updates...")
|
||||
|
||||
// Make the fakeServer send LDS response.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an LDS request")
|
||||
}
|
||||
fakeServer.XDSResponseChan <- &fakeserver.Response{Resp: goodLDSResponse1}
|
||||
|
||||
// Make the fakeServer send an empty RDS response.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an RDS request")
|
||||
}
|
||||
fakeServer.XDSResponseChan <- &fakeserver.Response{Resp: noVirtualHostsInRDSResponse}
|
||||
waitForNilErr(t, callbackCh)
|
||||
}
|
||||
|
||||
// TestWatchServiceWithClientClose tests the case where xDS responses are
|
||||
// received after the client is closed, and we make sure that the registered
|
||||
// watcher callback is not invoked.
|
||||
func (s) TestWatchServiceWithClientClose(t *testing.T) {
|
||||
fakeServer, cleanup, err := fakeserver.StartServer()
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to start fake xDS server: %v", err)
|
||||
}
|
||||
func (c *testXDSV2Client) addWatch(resourceType, resourceName string) {
|
||||
c.addWatches[resourceType] <- resourceName
|
||||
}
|
||||
|
||||
func (c *testXDSV2Client) removeWatch(resourceType, resourceName string) {
|
||||
c.removeWatches[resourceType] <- resourceName
|
||||
}
|
||||
|
||||
func (c *testXDSV2Client) close() {}
|
||||
|
||||
// TestWatchCallAnotherWatch covers the case where watch() is called inline by a
|
||||
// callback. It makes sure it doesn't cause a deadlock.
|
||||
func (s) TestWatchCallAnotherWatch(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
xdsClient, err := New(clientOpts(fakeServer.Address))
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("New returned error: %v", err)
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer xdsClient.Close()
|
||||
t.Log("Created an xdsClient...")
|
||||
defer c.Close()
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
cancelWatch := xdsClient.WatchService(goodLDSTarget1, func(su ServiceUpdate, err error) {
|
||||
callbackCh.Send(errors.New("watcher callback invoked after client close"))
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
clusterUpdateCh := testutils.NewChannel()
|
||||
c.WatchCluster(testCDSName, func(update ClusterUpdate, err error) {
|
||||
clusterUpdateCh.Send(clusterUpdateErr{u: update, err: err})
|
||||
// Calls another watch inline, to ensure there's deadlock.
|
||||
c.WatchCluster("another-random-name", func(ClusterUpdate, error) {})
|
||||
})
|
||||
defer cancelWatch()
|
||||
t.Log("Registered a watcher for service updates...")
|
||||
|
||||
// Make the fakeServer send LDS response.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an LDS request")
|
||||
}
|
||||
fakeServer.XDSResponseChan <- &fakeserver.Response{Resp: goodLDSResponse1}
|
||||
wantUpdate := ClusterUpdate{ServiceName: testEDSName}
|
||||
v2Client.r.newCDSUpdate(map[string]ClusterUpdate{
|
||||
testCDSName: wantUpdate,
|
||||
})
|
||||
|
||||
xdsClient.Close()
|
||||
t.Log("Closing the xdsClient...")
|
||||
|
||||
// Push an RDS response from the fakeserver
|
||||
fakeServer.XDSResponseChan <- &fakeserver.Response{Resp: goodRDSResponse1}
|
||||
if cbErr, err := callbackCh.Receive(); err != testutils.ErrRecvTimeout {
|
||||
t.Fatal(cbErr)
|
||||
if u, err := clusterUpdateCh.Receive(); err != nil || u != (clusterUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected clusterUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
wantUpdate2 := ClusterUpdate{ServiceName: testEDSName + "2"}
|
||||
v2Client.r.newCDSUpdate(map[string]ClusterUpdate{
|
||||
testCDSName: wantUpdate2,
|
||||
})
|
||||
|
||||
if u, err := clusterUpdateCh.Receive(); err != nil || u != (clusterUpdateErr{wantUpdate2, nil}) {
|
||||
t.Errorf("unexpected clusterUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
}
|
||||
|
124
xds/internal/client/client_watchers.go
Normal file
124
xds/internal/client/client_watchers.go
Normal file
@ -0,0 +1,124 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2020 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 client
|
||||
|
||||
import (
|
||||
"time"
|
||||
)
|
||||
|
||||
// The value chosen here is based on the default value of the
|
||||
// initial_fetch_timeout field in corepb.ConfigSource proto.
|
||||
var defaultWatchExpiryTimeout = 15 * time.Second
|
||||
|
||||
const (
|
||||
ldsURL = "type.googleapis.com/envoy.api.v2.Listener"
|
||||
rdsURL = "type.googleapis.com/envoy.api.v2.RouteConfiguration"
|
||||
cdsURL = "type.googleapis.com/envoy.api.v2.Cluster"
|
||||
edsURL = "type.googleapis.com/envoy.api.v2.ClusterLoadAssignment"
|
||||
)
|
||||
|
||||
// watchInfo holds all the information from a watch() call.
|
||||
type watchInfo struct {
|
||||
typeURL string
|
||||
target string
|
||||
|
||||
ldsCallback ldsCallbackFunc
|
||||
rdsCallback rdsCallbackFunc
|
||||
cdsCallback func(ClusterUpdate, error)
|
||||
edsCallback func(EndpointsUpdate, error)
|
||||
expiryTimer *time.Timer
|
||||
}
|
||||
|
||||
func (c *Client) watch(wi *watchInfo) (cancel func()) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
c.logger.Debugf("new watch for type %v, resource name %v", wi.typeURL, wi.target)
|
||||
var watchers map[string]map[*watchInfo]bool
|
||||
switch wi.typeURL {
|
||||
case ldsURL:
|
||||
watchers = c.ldsWatchers
|
||||
case rdsURL:
|
||||
watchers = c.rdsWatchers
|
||||
case cdsURL:
|
||||
watchers = c.cdsWatchers
|
||||
case edsURL:
|
||||
watchers = c.edsWatchers
|
||||
}
|
||||
|
||||
resourceName := wi.target
|
||||
s, ok := watchers[wi.target]
|
||||
if !ok {
|
||||
// If this is a new watcher, will ask lower level to send a new request with
|
||||
// the resource name.
|
||||
//
|
||||
// If this type+name is already being watched, will not notify the
|
||||
// underlying xdsv2Client.
|
||||
c.logger.Debugf("first watch for type %v, resource name %v, will send a new xDS request", wi.typeURL, wi.target)
|
||||
s = make(map[*watchInfo]bool)
|
||||
watchers[resourceName] = s
|
||||
c.v2c.addWatch(wi.typeURL, resourceName)
|
||||
}
|
||||
// No matter what, add the new watcher to the set, so it's callback will be
|
||||
// call for new responses.
|
||||
s[wi] = true
|
||||
|
||||
// If the resource is in cache, call the callback with the value.
|
||||
switch wi.typeURL {
|
||||
case ldsURL:
|
||||
if v, ok := c.ldsCache[resourceName]; ok {
|
||||
c.logger.Debugf("LDS resource with name %v found in cache: %+v", wi.target, v)
|
||||
c.scheduleCallback(wi, v, nil)
|
||||
}
|
||||
case rdsURL:
|
||||
if v, ok := c.rdsCache[resourceName]; ok {
|
||||
c.logger.Debugf("RDS resource with name %v found in cache: %+v", wi.target, v)
|
||||
c.scheduleCallback(wi, v, nil)
|
||||
}
|
||||
case cdsURL:
|
||||
if v, ok := c.cdsCache[resourceName]; ok {
|
||||
c.logger.Debugf("CDS resource with name %v found in cache: %+v", wi.target, v)
|
||||
c.scheduleCallback(wi, v, nil)
|
||||
}
|
||||
case edsURL:
|
||||
if v, ok := c.edsCache[resourceName]; ok {
|
||||
c.logger.Debugf("EDS resource with name %v found in cache: %+v", wi.target, v)
|
||||
c.scheduleCallback(wi, v, nil)
|
||||
}
|
||||
}
|
||||
|
||||
return func() {
|
||||
c.logger.Debugf("watch for type %v, resource name %v canceled", wi.typeURL, wi.target)
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
if s := watchers[resourceName]; s != nil {
|
||||
wi.expiryTimer.Stop()
|
||||
// Remove this watcher, so it's callback will not be called in the
|
||||
// future.
|
||||
delete(s, wi)
|
||||
if len(s) == 0 {
|
||||
c.logger.Debugf("last watch for type %v, resource name %v canceled, will send a new xDS request", wi.typeURL, wi.target)
|
||||
// If this was the last watcher, also tell xdsv2Client to stop
|
||||
// watching this resource.
|
||||
delete(watchers, resourceName)
|
||||
c.v2c.removeWatch(wi.typeURL, resourceName)
|
||||
// TODO: remove item from cache.
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
56
xds/internal/client/client_watchers_cluster.go
Normal file
56
xds/internal/client/client_watchers_cluster.go
Normal file
@ -0,0 +1,56 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2020 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 client
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
)
|
||||
|
||||
// ClusterUpdate contains information from a received CDS response, which is of
|
||||
// interest to the registered CDS watcher.
|
||||
type ClusterUpdate struct {
|
||||
// ServiceName is the service name corresponding to the clusterName which
|
||||
// is being watched for through CDS.
|
||||
ServiceName string
|
||||
// EnableLRS indicates whether or not load should be reported through LRS.
|
||||
EnableLRS bool
|
||||
}
|
||||
|
||||
// WatchCluster uses CDS to discover information about the provided
|
||||
// clusterName.
|
||||
//
|
||||
// WatchCluster can be called multiple times, with same or different
|
||||
// clusterNames. Each call will start an independent watcher for the resource.
|
||||
//
|
||||
// Note that during race (e.g. an xDS response is received while the user is
|
||||
// calling cancel()), there's a small window where the callback can be called
|
||||
// after the watcher is canceled. The caller needs to handle this case.
|
||||
func (c *Client) WatchCluster(clusterName string, cb func(ClusterUpdate, error)) (cancel func()) {
|
||||
wi := &watchInfo{
|
||||
typeURL: cdsURL,
|
||||
target: clusterName,
|
||||
cdsCallback: cb,
|
||||
}
|
||||
|
||||
wi.expiryTimer = time.AfterFunc(defaultWatchExpiryTimeout, func() {
|
||||
c.scheduleCallback(wi, ClusterUpdate{}, fmt.Errorf("xds: CDS target %s not found, watcher timeout", clusterName))
|
||||
})
|
||||
return c.watch(wi)
|
||||
}
|
276
xds/internal/client/client_watchers_cluster_test.go
Normal file
276
xds/internal/client/client_watchers_cluster_test.go
Normal file
@ -0,0 +1,276 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2020 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 client
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
)
|
||||
|
||||
type clusterUpdateErr struct {
|
||||
u ClusterUpdate
|
||||
err error
|
||||
}
|
||||
|
||||
// TestClusterWatch covers the cases:
|
||||
// - an update is received after a watch()
|
||||
// - an update for another resource name (which doesn't trigger callback)
|
||||
// - an upate is received after cancel()
|
||||
func (s) TestClusterWatch(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
clusterUpdateCh := testutils.NewChannel()
|
||||
cancelWatch := c.WatchCluster(testCDSName, func(update ClusterUpdate, err error) {
|
||||
clusterUpdateCh.Send(clusterUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate := ClusterUpdate{ServiceName: testEDSName}
|
||||
// This is calling v2Client.r to send the update, but r is set to Client, so
|
||||
// this is same as calling Client to update. The one thing this covers is
|
||||
// that `NewXDSV2Client` is called with the right parent.
|
||||
//
|
||||
// TODO: in a future cleanup, this (and the same thing in other tests) can
|
||||
// be changed call Client directly.
|
||||
v2Client.r.newCDSUpdate(map[string]ClusterUpdate{
|
||||
testCDSName: wantUpdate,
|
||||
})
|
||||
|
||||
if u, err := clusterUpdateCh.Receive(); err != nil || u != (clusterUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected clusterUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
// Another update for a different resource name.
|
||||
v2Client.r.newCDSUpdate(map[string]ClusterUpdate{
|
||||
"randomName": {},
|
||||
})
|
||||
|
||||
if u, err := clusterUpdateCh.TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected clusterUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
|
||||
// Cancel watch, and send update again.
|
||||
cancelWatch()
|
||||
v2Client.r.newCDSUpdate(map[string]ClusterUpdate{
|
||||
testCDSName: wantUpdate,
|
||||
})
|
||||
|
||||
if u, err := clusterUpdateCh.TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected clusterUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestClusterTwoWatchSameResourceName covers the case where an update is received
|
||||
// after two watch() for the same resource name.
|
||||
func (s) TestClusterTwoWatchSameResourceName(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
var clusterUpdateChs []*testutils.Channel
|
||||
const count = 2
|
||||
|
||||
var cancelLastWatch func()
|
||||
|
||||
for i := 0; i < count; i++ {
|
||||
clusterUpdateCh := testutils.NewChannel()
|
||||
clusterUpdateChs = append(clusterUpdateChs, clusterUpdateCh)
|
||||
cancelLastWatch = c.WatchCluster(testCDSName, func(update ClusterUpdate, err error) {
|
||||
clusterUpdateCh.Send(clusterUpdateErr{u: update, err: err})
|
||||
})
|
||||
}
|
||||
|
||||
wantUpdate := ClusterUpdate{ServiceName: testEDSName}
|
||||
v2Client.r.newCDSUpdate(map[string]ClusterUpdate{
|
||||
testCDSName: wantUpdate,
|
||||
})
|
||||
|
||||
for i := 0; i < count; i++ {
|
||||
if u, err := clusterUpdateChs[i].Receive(); err != nil || u != (clusterUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("i=%v, unexpected clusterUpdate: %v, error receiving from channel: %v", i, u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// Cancel the last watch, and send update again.
|
||||
cancelLastWatch()
|
||||
v2Client.r.newCDSUpdate(map[string]ClusterUpdate{
|
||||
testCDSName: wantUpdate,
|
||||
})
|
||||
|
||||
for i := 0; i < count-1; i++ {
|
||||
if u, err := clusterUpdateChs[i].Receive(); err != nil || u != (clusterUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("i=%v, unexpected clusterUpdate: %v, error receiving from channel: %v", i, u, err)
|
||||
}
|
||||
}
|
||||
|
||||
if u, err := clusterUpdateChs[count-1].TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected clusterUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestClusterThreeWatchDifferentResourceName covers the case where an update is
|
||||
// received after three watch() for different resource names.
|
||||
func (s) TestClusterThreeWatchDifferentResourceName(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
var clusterUpdateChs []*testutils.Channel
|
||||
const count = 2
|
||||
|
||||
// Two watches for the same name.
|
||||
for i := 0; i < count; i++ {
|
||||
clusterUpdateCh := testutils.NewChannel()
|
||||
clusterUpdateChs = append(clusterUpdateChs, clusterUpdateCh)
|
||||
c.WatchCluster(testCDSName+"1", func(update ClusterUpdate, err error) {
|
||||
clusterUpdateCh.Send(clusterUpdateErr{u: update, err: err})
|
||||
})
|
||||
}
|
||||
|
||||
// Third watch for a different name.
|
||||
clusterUpdateCh2 := testutils.NewChannel()
|
||||
c.WatchCluster(testCDSName+"2", func(update ClusterUpdate, err error) {
|
||||
clusterUpdateCh2.Send(clusterUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate1 := ClusterUpdate{ServiceName: testEDSName + "1"}
|
||||
wantUpdate2 := ClusterUpdate{ServiceName: testEDSName + "2"}
|
||||
v2Client.r.newCDSUpdate(map[string]ClusterUpdate{
|
||||
testCDSName + "1": wantUpdate1,
|
||||
testCDSName + "2": wantUpdate2,
|
||||
})
|
||||
|
||||
for i := 0; i < count; i++ {
|
||||
if u, err := clusterUpdateChs[i].Receive(); err != nil || u != (clusterUpdateErr{wantUpdate1, nil}) {
|
||||
t.Errorf("i=%v, unexpected clusterUpdate: %v, error receiving from channel: %v", i, u, err)
|
||||
}
|
||||
}
|
||||
|
||||
if u, err := clusterUpdateCh2.Receive(); err != nil || u != (clusterUpdateErr{wantUpdate2, nil}) {
|
||||
t.Errorf("unexpected clusterUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestClusterWatchAfterCache covers the case where watch is called after the update
|
||||
// is in cache.
|
||||
func (s) TestClusterWatchAfterCache(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
clusterUpdateCh := testutils.NewChannel()
|
||||
c.WatchCluster(testCDSName, func(update ClusterUpdate, err error) {
|
||||
clusterUpdateCh.Send(clusterUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate := ClusterUpdate{ServiceName: testEDSName}
|
||||
v2Client.r.newCDSUpdate(map[string]ClusterUpdate{
|
||||
testCDSName: wantUpdate,
|
||||
})
|
||||
|
||||
if u, err := clusterUpdateCh.Receive(); err != nil || u != (clusterUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected clusterUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
// Another watch for the resource in cache.
|
||||
clusterUpdateCh2 := testutils.NewChannel()
|
||||
c.WatchCluster(testCDSName, func(update ClusterUpdate, err error) {
|
||||
clusterUpdateCh2.Send(clusterUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
// New watch should receives the update.
|
||||
if u, err := clusterUpdateCh2.Receive(); err != nil || u != (clusterUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected clusterUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
// Old watch should see nothing.
|
||||
if u, err := clusterUpdateCh.TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected clusterUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestClusterWatchExpiryTimer tests the case where the client does not receive
|
||||
// an CDS response for the request that it sends out. We want the watch callback
|
||||
// to be invoked with an error once the watchExpiryTimer fires.
|
||||
func (s) TestClusterWatchExpiryTimer(t *testing.T) {
|
||||
oldWatchExpiryTimeout := defaultWatchExpiryTimeout
|
||||
defaultWatchExpiryTimeout = 500 * time.Millisecond
|
||||
defer func() {
|
||||
defaultWatchExpiryTimeout = oldWatchExpiryTimeout
|
||||
}()
|
||||
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
<-v2ClientCh
|
||||
|
||||
clusterUpdateCh := testutils.NewChannel()
|
||||
c.WatchCluster(testCDSName, func(u ClusterUpdate, err error) {
|
||||
clusterUpdateCh.Send(clusterUpdateErr{u: u, err: err})
|
||||
})
|
||||
|
||||
u, err := clusterUpdateCh.TimedReceive(defaultWatchExpiryTimeout * 2)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to get clusterUpdate: %v", err)
|
||||
}
|
||||
uu := u.(clusterUpdateErr)
|
||||
if uu.u != (ClusterUpdate{}) {
|
||||
t.Errorf("unexpected clusterUpdate: %v, want %v", uu.u, ClusterUpdate{})
|
||||
}
|
||||
if uu.err == nil {
|
||||
t.Errorf("unexpected clusterError: <nil>, want error watcher timeout")
|
||||
}
|
||||
}
|
93
xds/internal/client/client_watchers_endpoints.go
Normal file
93
xds/internal/client/client_watchers_endpoints.go
Normal file
@ -0,0 +1,93 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2020 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 client
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc/xds/internal"
|
||||
)
|
||||
|
||||
// OverloadDropConfig contains the config to drop overloads.
|
||||
type OverloadDropConfig struct {
|
||||
Category string
|
||||
Numerator uint32
|
||||
Denominator uint32
|
||||
}
|
||||
|
||||
// EndpointHealthStatus represents the health status of an endpoint.
|
||||
type EndpointHealthStatus int32
|
||||
|
||||
const (
|
||||
// EndpointHealthStatusUnknown represents HealthStatus UNKNOWN.
|
||||
EndpointHealthStatusUnknown EndpointHealthStatus = iota
|
||||
// EndpointHealthStatusHealthy represents HealthStatus HEALTHY.
|
||||
EndpointHealthStatusHealthy
|
||||
// EndpointHealthStatusUnhealthy represents HealthStatus UNHEALTHY.
|
||||
EndpointHealthStatusUnhealthy
|
||||
// EndpointHealthStatusDraining represents HealthStatus DRAINING.
|
||||
EndpointHealthStatusDraining
|
||||
// EndpointHealthStatusTimeout represents HealthStatus TIMEOUT.
|
||||
EndpointHealthStatusTimeout
|
||||
// EndpointHealthStatusDegraded represents HealthStatus DEGRADED.
|
||||
EndpointHealthStatusDegraded
|
||||
)
|
||||
|
||||
// Endpoint contains information of an endpoint.
|
||||
type Endpoint struct {
|
||||
Address string
|
||||
HealthStatus EndpointHealthStatus
|
||||
Weight uint32
|
||||
}
|
||||
|
||||
// Locality contains information of a locality.
|
||||
type Locality struct {
|
||||
Endpoints []Endpoint
|
||||
ID internal.LocalityID
|
||||
Priority uint32
|
||||
Weight uint32
|
||||
}
|
||||
|
||||
// EndpointsUpdate contains an EDS update.
|
||||
type EndpointsUpdate struct {
|
||||
Drops []OverloadDropConfig
|
||||
Localities []Locality
|
||||
}
|
||||
|
||||
// WatchEndpoints uses EDS to discover endpoints in the provided clusterName.
|
||||
//
|
||||
// WatchEndpoints can be called multiple times, with same or different
|
||||
// clusterNames. Each call will start an independent watcher for the resource.
|
||||
//
|
||||
// Note that during race (e.g. an xDS response is received while the user is
|
||||
// calling cancel()), there's a small window where the callback can be called
|
||||
// after the watcher is canceled. The caller needs to handle this case.
|
||||
func (c *Client) WatchEndpoints(clusterName string, cb func(EndpointsUpdate, error)) (cancel func()) {
|
||||
wi := &watchInfo{
|
||||
typeURL: edsURL,
|
||||
target: clusterName,
|
||||
edsCallback: cb,
|
||||
}
|
||||
|
||||
wi.expiryTimer = time.AfterFunc(defaultWatchExpiryTimeout, func() {
|
||||
c.scheduleCallback(wi, EndpointsUpdate{}, fmt.Errorf("xds: EDS target %s not found, watcher timeout", clusterName))
|
||||
})
|
||||
return c.watch(wi)
|
||||
}
|
289
xds/internal/client/client_watchers_endpoints_test.go
Normal file
289
xds/internal/client/client_watchers_endpoints_test.go
Normal file
@ -0,0 +1,289 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2020 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 client
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"google.golang.org/grpc/xds/internal"
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
)
|
||||
|
||||
var (
|
||||
testLocalities = []Locality{
|
||||
{
|
||||
Endpoints: []Endpoint{{Address: "addr1:314"}},
|
||||
ID: internal.LocalityID{SubZone: "locality-1"},
|
||||
Priority: 1,
|
||||
Weight: 1,
|
||||
},
|
||||
{
|
||||
Endpoints: []Endpoint{{Address: "addr2:159"}},
|
||||
ID: internal.LocalityID{SubZone: "locality-2"},
|
||||
Priority: 0,
|
||||
Weight: 1,
|
||||
},
|
||||
}
|
||||
)
|
||||
|
||||
type endpointsUpdateErr struct {
|
||||
u EndpointsUpdate
|
||||
err error
|
||||
}
|
||||
|
||||
// TestEndpointsWatch covers the cases:
|
||||
// - an update is received after a watch()
|
||||
// - an update for another resource name (which doesn't trigger callback)
|
||||
// - an upate is received after cancel()
|
||||
func (s) TestEndpointsWatch(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
endpointsUpdateCh := testutils.NewChannel()
|
||||
cancelWatch := c.WatchEndpoints(testCDSName, func(update EndpointsUpdate, err error) {
|
||||
endpointsUpdateCh.Send(endpointsUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate := EndpointsUpdate{Localities: []Locality{testLocalities[0]}}
|
||||
v2Client.r.newEDSUpdate(map[string]EndpointsUpdate{
|
||||
testCDSName: wantUpdate,
|
||||
})
|
||||
|
||||
if u, err := endpointsUpdateCh.Receive(); err != nil || !cmp.Equal(u, endpointsUpdateErr{wantUpdate, nil}, cmp.AllowUnexported(endpointsUpdateErr{})) {
|
||||
t.Errorf("unexpected endpointsUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
// Another update for a different resource name.
|
||||
v2Client.r.newEDSUpdate(map[string]EndpointsUpdate{
|
||||
"randomName": {},
|
||||
})
|
||||
|
||||
if u, err := endpointsUpdateCh.TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected endpointsUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
|
||||
// Cancel watch, and send update again.
|
||||
cancelWatch()
|
||||
v2Client.r.newEDSUpdate(map[string]EndpointsUpdate{
|
||||
testCDSName: wantUpdate,
|
||||
})
|
||||
|
||||
if u, err := endpointsUpdateCh.TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected endpointsUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestEndpointsTwoWatchSameResourceName covers the case where an update is received
|
||||
// after two watch() for the same resource name.
|
||||
func (s) TestEndpointsTwoWatchSameResourceName(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
var endpointsUpdateChs []*testutils.Channel
|
||||
const count = 2
|
||||
|
||||
var cancelLastWatch func()
|
||||
|
||||
for i := 0; i < count; i++ {
|
||||
endpointsUpdateCh := testutils.NewChannel()
|
||||
endpointsUpdateChs = append(endpointsUpdateChs, endpointsUpdateCh)
|
||||
cancelLastWatch = c.WatchEndpoints(testCDSName, func(update EndpointsUpdate, err error) {
|
||||
endpointsUpdateCh.Send(endpointsUpdateErr{u: update, err: err})
|
||||
})
|
||||
}
|
||||
|
||||
wantUpdate := EndpointsUpdate{Localities: []Locality{testLocalities[0]}}
|
||||
v2Client.r.newEDSUpdate(map[string]EndpointsUpdate{
|
||||
testCDSName: wantUpdate,
|
||||
})
|
||||
|
||||
for i := 0; i < count; i++ {
|
||||
if u, err := endpointsUpdateChs[i].Receive(); err != nil || !cmp.Equal(u, endpointsUpdateErr{wantUpdate, nil}, cmp.AllowUnexported(endpointsUpdateErr{})) {
|
||||
t.Errorf("i=%v, unexpected endpointsUpdate: %v, error receiving from channel: %v", i, u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// Cancel the last watch, and send update again.
|
||||
cancelLastWatch()
|
||||
v2Client.r.newEDSUpdate(map[string]EndpointsUpdate{
|
||||
testCDSName: wantUpdate,
|
||||
})
|
||||
|
||||
for i := 0; i < count-1; i++ {
|
||||
if u, err := endpointsUpdateChs[i].Receive(); err != nil || !cmp.Equal(u, endpointsUpdateErr{wantUpdate, nil}, cmp.AllowUnexported(endpointsUpdateErr{})) {
|
||||
t.Errorf("i=%v, unexpected endpointsUpdate: %v, error receiving from channel: %v", i, u, err)
|
||||
}
|
||||
}
|
||||
|
||||
if u, err := endpointsUpdateChs[count-1].TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected endpointsUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestEndpointsThreeWatchDifferentResourceName covers the case where an update is
|
||||
// received after three watch() for different resource names.
|
||||
func (s) TestEndpointsThreeWatchDifferentResourceName(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
var endpointsUpdateChs []*testutils.Channel
|
||||
const count = 2
|
||||
|
||||
// Two watches for the same name.
|
||||
for i := 0; i < count; i++ {
|
||||
endpointsUpdateCh := testutils.NewChannel()
|
||||
endpointsUpdateChs = append(endpointsUpdateChs, endpointsUpdateCh)
|
||||
c.WatchEndpoints(testCDSName+"1", func(update EndpointsUpdate, err error) {
|
||||
endpointsUpdateCh.Send(endpointsUpdateErr{u: update, err: err})
|
||||
})
|
||||
}
|
||||
|
||||
// Third watch for a different name.
|
||||
endpointsUpdateCh2 := testutils.NewChannel()
|
||||
c.WatchEndpoints(testCDSName+"2", func(update EndpointsUpdate, err error) {
|
||||
endpointsUpdateCh2.Send(endpointsUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate1 := EndpointsUpdate{Localities: []Locality{testLocalities[0]}}
|
||||
wantUpdate2 := EndpointsUpdate{Localities: []Locality{testLocalities[1]}}
|
||||
v2Client.r.newEDSUpdate(map[string]EndpointsUpdate{
|
||||
testCDSName + "1": wantUpdate1,
|
||||
testCDSName + "2": wantUpdate2,
|
||||
})
|
||||
|
||||
for i := 0; i < count; i++ {
|
||||
if u, err := endpointsUpdateChs[i].Receive(); err != nil || !cmp.Equal(u, endpointsUpdateErr{wantUpdate1, nil}, cmp.AllowUnexported(endpointsUpdateErr{})) {
|
||||
t.Errorf("i=%v, unexpected endpointsUpdate: %v, error receiving from channel: %v", i, u, err)
|
||||
}
|
||||
}
|
||||
|
||||
if u, err := endpointsUpdateCh2.Receive(); err != nil || !cmp.Equal(u, endpointsUpdateErr{wantUpdate2, nil}, cmp.AllowUnexported(endpointsUpdateErr{})) {
|
||||
t.Errorf("unexpected endpointsUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestEndpointsWatchAfterCache covers the case where watch is called after the update
|
||||
// is in cache.
|
||||
func (s) TestEndpointsWatchAfterCache(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
endpointsUpdateCh := testutils.NewChannel()
|
||||
c.WatchEndpoints(testCDSName, func(update EndpointsUpdate, err error) {
|
||||
endpointsUpdateCh.Send(endpointsUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate := EndpointsUpdate{Localities: []Locality{testLocalities[0]}}
|
||||
v2Client.r.newEDSUpdate(map[string]EndpointsUpdate{
|
||||
testCDSName: wantUpdate,
|
||||
})
|
||||
|
||||
if u, err := endpointsUpdateCh.Receive(); err != nil || !cmp.Equal(u, endpointsUpdateErr{wantUpdate, nil}, cmp.AllowUnexported(endpointsUpdateErr{})) {
|
||||
t.Errorf("unexpected endpointsUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
// Another watch for the resource in cache.
|
||||
endpointsUpdateCh2 := testutils.NewChannel()
|
||||
c.WatchEndpoints(testCDSName, func(update EndpointsUpdate, err error) {
|
||||
endpointsUpdateCh2.Send(endpointsUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
// New watch should receives the update.
|
||||
if u, err := endpointsUpdateCh2.Receive(); err != nil || !cmp.Equal(u, endpointsUpdateErr{wantUpdate, nil}, cmp.AllowUnexported(endpointsUpdateErr{})) {
|
||||
t.Errorf("unexpected endpointsUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
// Old watch should see nothing.
|
||||
if u, err := endpointsUpdateCh.TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected endpointsUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestEndpointsWatchExpiryTimer tests the case where the client does not receive
|
||||
// an CDS response for the request that it sends out. We want the watch callback
|
||||
// to be invoked with an error once the watchExpiryTimer fires.
|
||||
func (s) TestEndpointsWatchExpiryTimer(t *testing.T) {
|
||||
oldWatchExpiryTimeout := defaultWatchExpiryTimeout
|
||||
defaultWatchExpiryTimeout = 500 * time.Millisecond
|
||||
defer func() {
|
||||
defaultWatchExpiryTimeout = oldWatchExpiryTimeout
|
||||
}()
|
||||
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
<-v2ClientCh
|
||||
|
||||
endpointsUpdateCh := testutils.NewChannel()
|
||||
c.WatchEndpoints(testCDSName, func(update EndpointsUpdate, err error) {
|
||||
endpointsUpdateCh.Send(endpointsUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
u, err := endpointsUpdateCh.TimedReceive(defaultWatchExpiryTimeout * 2)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to get endpointsUpdate: %v", err)
|
||||
}
|
||||
uu := u.(endpointsUpdateErr)
|
||||
if !cmp.Equal(uu.u, EndpointsUpdate{}, cmp.AllowUnexported(endpointsUpdateErr{})) {
|
||||
t.Errorf("unexpected endpointsUpdate: %v, want %v", uu.u, EndpointsUpdate{})
|
||||
}
|
||||
if uu.err == nil {
|
||||
t.Errorf("unexpected endpointsError: <nil>, want error watcher timeout")
|
||||
}
|
||||
}
|
47
xds/internal/client/client_watchers_lds.go
Normal file
47
xds/internal/client/client_watchers_lds.go
Normal file
@ -0,0 +1,47 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2020 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 client
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
)
|
||||
|
||||
type ldsUpdate struct {
|
||||
routeName string
|
||||
}
|
||||
type ldsCallbackFunc func(ldsUpdate, error)
|
||||
|
||||
// watchLDS starts a listener watcher for the service..
|
||||
//
|
||||
// Note that during race (e.g. an xDS response is received while the user is
|
||||
// calling cancel()), there's a small window where the callback can be called
|
||||
// after the watcher is canceled. The caller needs to handle this case.
|
||||
func (c *Client) watchLDS(serviceName string, cb ldsCallbackFunc) (cancel func()) {
|
||||
wi := &watchInfo{
|
||||
typeURL: ldsURL,
|
||||
target: serviceName,
|
||||
ldsCallback: cb,
|
||||
}
|
||||
|
||||
wi.expiryTimer = time.AfterFunc(defaultWatchExpiryTimeout, func() {
|
||||
c.scheduleCallback(wi, ldsUpdate{}, fmt.Errorf("xds: LDS target %s not found, watcher timeout", serviceName))
|
||||
})
|
||||
return c.watch(wi)
|
||||
}
|
230
xds/internal/client/client_watchers_lds_test.go
Normal file
230
xds/internal/client/client_watchers_lds_test.go
Normal file
@ -0,0 +1,230 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2020 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 client
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
)
|
||||
|
||||
type ldsUpdateErr struct {
|
||||
u ldsUpdate
|
||||
err error
|
||||
}
|
||||
|
||||
// TestLDSWatch covers the cases:
|
||||
// - an update is received after a watch()
|
||||
// - an update for another resource name (which doesn't trigger callback)
|
||||
// - an upate is received after cancel()
|
||||
func (s) TestLDSWatch(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
ldsUpdateCh := testutils.NewChannel()
|
||||
cancelWatch := c.watchLDS(testLDSName, func(update ldsUpdate, err error) {
|
||||
ldsUpdateCh.Send(ldsUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate := ldsUpdate{routeName: testRDSName}
|
||||
v2Client.r.newLDSUpdate(map[string]ldsUpdate{
|
||||
testLDSName: wantUpdate,
|
||||
})
|
||||
|
||||
if u, err := ldsUpdateCh.Receive(); err != nil || u != (ldsUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected ldsUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
// Another update for a different resource name.
|
||||
v2Client.r.newLDSUpdate(map[string]ldsUpdate{
|
||||
"randomName": {},
|
||||
})
|
||||
|
||||
if u, err := ldsUpdateCh.TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected ldsUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
|
||||
// Cancel watch, and send update again.
|
||||
cancelWatch()
|
||||
v2Client.r.newLDSUpdate(map[string]ldsUpdate{
|
||||
testLDSName: wantUpdate,
|
||||
})
|
||||
|
||||
if u, err := ldsUpdateCh.TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected ldsUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestLDSTwoWatchSameResourceName covers the case where an update is received
|
||||
// after two watch() for the same resource name.
|
||||
func (s) TestLDSTwoWatchSameResourceName(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
var ldsUpdateChs []*testutils.Channel
|
||||
const count = 2
|
||||
|
||||
var cancelLastWatch func()
|
||||
|
||||
for i := 0; i < count; i++ {
|
||||
ldsUpdateCh := testutils.NewChannel()
|
||||
ldsUpdateChs = append(ldsUpdateChs, ldsUpdateCh)
|
||||
cancelLastWatch = c.watchLDS(testLDSName, func(update ldsUpdate, err error) {
|
||||
ldsUpdateCh.Send(ldsUpdateErr{u: update, err: err})
|
||||
})
|
||||
}
|
||||
|
||||
wantUpdate := ldsUpdate{routeName: testRDSName}
|
||||
v2Client.r.newLDSUpdate(map[string]ldsUpdate{
|
||||
testLDSName: wantUpdate,
|
||||
})
|
||||
|
||||
for i := 0; i < count; i++ {
|
||||
if u, err := ldsUpdateChs[i].Receive(); err != nil || u != (ldsUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("i=%v, unexpected ldsUpdate: %v, error receiving from channel: %v", i, u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// Cancel the last watch, and send update again.
|
||||
cancelLastWatch()
|
||||
v2Client.r.newLDSUpdate(map[string]ldsUpdate{
|
||||
testLDSName: wantUpdate,
|
||||
})
|
||||
|
||||
for i := 0; i < count-1; i++ {
|
||||
if u, err := ldsUpdateChs[i].Receive(); err != nil || u != (ldsUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("i=%v, unexpected ldsUpdate: %v, error receiving from channel: %v", i, u, err)
|
||||
}
|
||||
}
|
||||
|
||||
if u, err := ldsUpdateChs[count-1].TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected ldsUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestLDSThreeWatchDifferentResourceName covers the case where an update is
|
||||
// received after three watch() for different resource names.
|
||||
func (s) TestLDSThreeWatchDifferentResourceName(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
var ldsUpdateChs []*testutils.Channel
|
||||
const count = 2
|
||||
|
||||
// Two watches for the same name.
|
||||
for i := 0; i < count; i++ {
|
||||
ldsUpdateCh := testutils.NewChannel()
|
||||
ldsUpdateChs = append(ldsUpdateChs, ldsUpdateCh)
|
||||
c.watchLDS(testLDSName+"1", func(update ldsUpdate, err error) {
|
||||
ldsUpdateCh.Send(ldsUpdateErr{u: update, err: err})
|
||||
})
|
||||
}
|
||||
|
||||
// Third watch for a different name.
|
||||
ldsUpdateCh2 := testutils.NewChannel()
|
||||
c.watchLDS(testLDSName+"2", func(update ldsUpdate, err error) {
|
||||
ldsUpdateCh2.Send(ldsUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate1 := ldsUpdate{routeName: testRDSName + "1"}
|
||||
wantUpdate2 := ldsUpdate{routeName: testRDSName + "2"}
|
||||
v2Client.r.newLDSUpdate(map[string]ldsUpdate{
|
||||
testLDSName + "1": wantUpdate1,
|
||||
testLDSName + "2": wantUpdate2,
|
||||
})
|
||||
|
||||
for i := 0; i < count; i++ {
|
||||
if u, err := ldsUpdateChs[i].Receive(); err != nil || u != (ldsUpdateErr{wantUpdate1, nil}) {
|
||||
t.Errorf("i=%v, unexpected ldsUpdate: %v, error receiving from channel: %v", i, u, err)
|
||||
}
|
||||
}
|
||||
|
||||
if u, err := ldsUpdateCh2.Receive(); err != nil || u != (ldsUpdateErr{wantUpdate2, nil}) {
|
||||
t.Errorf("unexpected ldsUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestLDSWatchAfterCache covers the case where watch is called after the update
|
||||
// is in cache.
|
||||
func (s) TestLDSWatchAfterCache(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
ldsUpdateCh := testutils.NewChannel()
|
||||
c.watchLDS(testLDSName, func(update ldsUpdate, err error) {
|
||||
ldsUpdateCh.Send(ldsUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate := ldsUpdate{routeName: testRDSName}
|
||||
v2Client.r.newLDSUpdate(map[string]ldsUpdate{
|
||||
testLDSName: wantUpdate,
|
||||
})
|
||||
|
||||
if u, err := ldsUpdateCh.Receive(); err != nil || u != (ldsUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected ldsUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
// Another watch for the resource in cache.
|
||||
ldsUpdateCh2 := testutils.NewChannel()
|
||||
c.watchLDS(testLDSName, func(update ldsUpdate, err error) {
|
||||
ldsUpdateCh2.Send(ldsUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
// New watch should receives the update.
|
||||
if u, err := ldsUpdateCh2.Receive(); err != nil || u != (ldsUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected ldsUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
// Old watch should see nothing.
|
||||
if u, err := ldsUpdateCh.TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected ldsUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
}
|
47
xds/internal/client/client_watchers_rds.go
Normal file
47
xds/internal/client/client_watchers_rds.go
Normal file
@ -0,0 +1,47 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2020 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 client
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
)
|
||||
|
||||
type rdsUpdate struct {
|
||||
clusterName string
|
||||
}
|
||||
type rdsCallbackFunc func(rdsUpdate, error)
|
||||
|
||||
// watchRDS starts a listener watcher for the service..
|
||||
//
|
||||
// Note that during race (e.g. an xDS response is received while the user is
|
||||
// calling cancel()), there's a small window where the callback can be called
|
||||
// after the watcher is canceled. The caller needs to handle this case.
|
||||
func (c *Client) watchRDS(routeName string, cb rdsCallbackFunc) (cancel func()) {
|
||||
wi := &watchInfo{
|
||||
typeURL: rdsURL,
|
||||
target: routeName,
|
||||
rdsCallback: cb,
|
||||
}
|
||||
|
||||
wi.expiryTimer = time.AfterFunc(defaultWatchExpiryTimeout, func() {
|
||||
c.scheduleCallback(wi, rdsUpdate{}, fmt.Errorf("xds: RDS target %s not found, watcher timeout", routeName))
|
||||
})
|
||||
return c.watch(wi)
|
||||
}
|
230
xds/internal/client/client_watchers_rds_test.go
Normal file
230
xds/internal/client/client_watchers_rds_test.go
Normal file
@ -0,0 +1,230 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2020 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 client
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
)
|
||||
|
||||
type rdsUpdateErr struct {
|
||||
u rdsUpdate
|
||||
err error
|
||||
}
|
||||
|
||||
// TestRDSWatch covers the cases:
|
||||
// - an update is received after a watch()
|
||||
// - an update for another resource name (which doesn't trigger callback)
|
||||
// - an upate is received after cancel()
|
||||
func (s) TestRDSWatch(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
rdsUpdateCh := testutils.NewChannel()
|
||||
cancelWatch := c.watchRDS(testRDSName, func(update rdsUpdate, err error) {
|
||||
rdsUpdateCh.Send(rdsUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate := rdsUpdate{clusterName: testCDSName}
|
||||
v2Client.r.newRDSUpdate(map[string]rdsUpdate{
|
||||
testRDSName: wantUpdate,
|
||||
})
|
||||
|
||||
if u, err := rdsUpdateCh.Receive(); err != nil || u != (rdsUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected rdsUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
// Another update for a different resource name.
|
||||
v2Client.r.newRDSUpdate(map[string]rdsUpdate{
|
||||
"randomName": {},
|
||||
})
|
||||
|
||||
if u, err := rdsUpdateCh.TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected rdsUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
|
||||
// Cancel watch, and send update again.
|
||||
cancelWatch()
|
||||
v2Client.r.newRDSUpdate(map[string]rdsUpdate{
|
||||
testRDSName: wantUpdate,
|
||||
})
|
||||
|
||||
if u, err := rdsUpdateCh.TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected rdsUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestRDSTwoWatchSameResourceName covers the case where an update is received
|
||||
// after two watch() for the same resource name.
|
||||
func (s) TestRDSTwoWatchSameResourceName(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
var rdsUpdateChs []*testutils.Channel
|
||||
const count = 2
|
||||
|
||||
var cancelLastWatch func()
|
||||
|
||||
for i := 0; i < count; i++ {
|
||||
rdsUpdateCh := testutils.NewChannel()
|
||||
rdsUpdateChs = append(rdsUpdateChs, rdsUpdateCh)
|
||||
cancelLastWatch = c.watchRDS(testRDSName, func(update rdsUpdate, err error) {
|
||||
rdsUpdateCh.Send(rdsUpdateErr{u: update, err: err})
|
||||
})
|
||||
}
|
||||
|
||||
wantUpdate := rdsUpdate{clusterName: testCDSName}
|
||||
v2Client.r.newRDSUpdate(map[string]rdsUpdate{
|
||||
testRDSName: wantUpdate,
|
||||
})
|
||||
|
||||
for i := 0; i < count; i++ {
|
||||
if u, err := rdsUpdateChs[i].Receive(); err != nil || u != (rdsUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("i=%v, unexpected rdsUpdate: %v, error receiving from channel: %v", i, u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// Cancel the last watch, and send update again.
|
||||
cancelLastWatch()
|
||||
v2Client.r.newRDSUpdate(map[string]rdsUpdate{
|
||||
testRDSName: wantUpdate,
|
||||
})
|
||||
|
||||
for i := 0; i < count-1; i++ {
|
||||
if u, err := rdsUpdateChs[i].Receive(); err != nil || u != (rdsUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("i=%v, unexpected rdsUpdate: %v, error receiving from channel: %v", i, u, err)
|
||||
}
|
||||
}
|
||||
|
||||
if u, err := rdsUpdateChs[count-1].TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected rdsUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestRDSThreeWatchDifferentResourceName covers the case where an update is
|
||||
// received after three watch() for different resource names.
|
||||
func (s) TestRDSThreeWatchDifferentResourceName(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
var rdsUpdateChs []*testutils.Channel
|
||||
const count = 2
|
||||
|
||||
// Two watches for the same name.
|
||||
for i := 0; i < count; i++ {
|
||||
rdsUpdateCh := testutils.NewChannel()
|
||||
rdsUpdateChs = append(rdsUpdateChs, rdsUpdateCh)
|
||||
c.watchRDS(testRDSName+"1", func(update rdsUpdate, err error) {
|
||||
rdsUpdateCh.Send(rdsUpdateErr{u: update, err: err})
|
||||
})
|
||||
}
|
||||
|
||||
// Third watch for a different name.
|
||||
rdsUpdateCh2 := testutils.NewChannel()
|
||||
c.watchRDS(testRDSName+"2", func(update rdsUpdate, err error) {
|
||||
rdsUpdateCh2.Send(rdsUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate1 := rdsUpdate{clusterName: testCDSName + "1"}
|
||||
wantUpdate2 := rdsUpdate{clusterName: testCDSName + "2"}
|
||||
v2Client.r.newRDSUpdate(map[string]rdsUpdate{
|
||||
testRDSName + "1": wantUpdate1,
|
||||
testRDSName + "2": wantUpdate2,
|
||||
})
|
||||
|
||||
for i := 0; i < count; i++ {
|
||||
if u, err := rdsUpdateChs[i].Receive(); err != nil || u != (rdsUpdateErr{wantUpdate1, nil}) {
|
||||
t.Errorf("i=%v, unexpected rdsUpdate: %v, error receiving from channel: %v", i, u, err)
|
||||
}
|
||||
}
|
||||
|
||||
if u, err := rdsUpdateCh2.Receive(); err != nil || u != (rdsUpdateErr{wantUpdate2, nil}) {
|
||||
t.Errorf("unexpected rdsUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestRDSWatchAfterCache covers the case where watch is called after the update
|
||||
// is in cache.
|
||||
func (s) TestRDSWatchAfterCache(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
rdsUpdateCh := testutils.NewChannel()
|
||||
c.watchRDS(testRDSName, func(update rdsUpdate, err error) {
|
||||
rdsUpdateCh.Send(rdsUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate := rdsUpdate{clusterName: testCDSName}
|
||||
v2Client.r.newRDSUpdate(map[string]rdsUpdate{
|
||||
testRDSName: wantUpdate,
|
||||
})
|
||||
|
||||
if u, err := rdsUpdateCh.Receive(); err != nil || u != (rdsUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected rdsUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
// Another watch for the resource in cache.
|
||||
rdsUpdateCh2 := testutils.NewChannel()
|
||||
c.watchRDS(testRDSName, func(update rdsUpdate, err error) {
|
||||
rdsUpdateCh2.Send(rdsUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
// New watch should receives the update.
|
||||
if u, err := rdsUpdateCh2.Receive(); err != nil || u != (rdsUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected rdsUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
// Old watch should see nothing.
|
||||
if u, err := rdsUpdateCh.TimedReceive(chanRecvTimeout); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected rdsUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
}
|
114
xds/internal/client/client_watchers_service.go
Normal file
114
xds/internal/client/client_watchers_service.go
Normal file
@ -0,0 +1,114 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2020 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 client
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
)
|
||||
|
||||
// ServiceUpdate contains update about the service.
|
||||
type ServiceUpdate struct {
|
||||
Cluster string
|
||||
}
|
||||
|
||||
// WatchService uses LDS and RDS to discover information about the provided
|
||||
// serviceName.
|
||||
//
|
||||
// WatchService can only be called once. The second call will not start a
|
||||
// watcher and the callback will get an error. It's this case because an xDS
|
||||
// client is expected to be used only by one ClientConn.
|
||||
//
|
||||
// Note that during race (e.g. an xDS response is received while the user is
|
||||
// calling cancel()), there's a small window where the callback can be called
|
||||
// after the watcher is canceled. The caller needs to handle this case.
|
||||
func (c *Client) WatchService(serviceName string, cb func(ServiceUpdate, error)) (cancel func()) {
|
||||
c.mu.Lock()
|
||||
if len(c.ldsWatchers) != 0 {
|
||||
go cb(ServiceUpdate{}, fmt.Errorf("unexpected WatchService when there's another service being watched"))
|
||||
c.mu.Unlock()
|
||||
return func() {}
|
||||
}
|
||||
c.mu.Unlock()
|
||||
|
||||
w := &serviceUpdateWatcher{c: c, serviceCb: cb}
|
||||
w.ldsCancel = c.watchLDS(serviceName, w.handleLDSResp)
|
||||
|
||||
return w.close
|
||||
}
|
||||
|
||||
// serviceUpdateWatcher handles LDS and RDS response, and calls the service
|
||||
// callback at the right time.
|
||||
type serviceUpdateWatcher struct {
|
||||
c *Client
|
||||
ldsCancel func()
|
||||
serviceCb func(ServiceUpdate, error)
|
||||
|
||||
mu sync.Mutex
|
||||
closed bool
|
||||
rdsCancel func()
|
||||
}
|
||||
|
||||
func (w *serviceUpdateWatcher) handleLDSResp(update ldsUpdate, err error) {
|
||||
w.c.logger.Infof("xds: client received LDS update: %+v, err: %v", update, err)
|
||||
w.mu.Lock()
|
||||
defer w.mu.Unlock()
|
||||
if w.closed {
|
||||
return
|
||||
}
|
||||
// TODO: this error case returns early, without canceling the existing RDS
|
||||
// watch. If we decided to stop the RDS watch when LDS errors, move this
|
||||
// after rdsCancel(). We may also need to check the error type and do
|
||||
// different things based on that (e.g. cancel RDS watch only on
|
||||
// resourceRemovedError, but not on connectionError).
|
||||
if err != nil {
|
||||
w.serviceCb(ServiceUpdate{}, err)
|
||||
return
|
||||
}
|
||||
|
||||
if w.rdsCancel != nil {
|
||||
w.rdsCancel()
|
||||
}
|
||||
w.rdsCancel = w.c.watchRDS(update.routeName, w.handleRDSResp)
|
||||
}
|
||||
|
||||
func (w *serviceUpdateWatcher) handleRDSResp(update rdsUpdate, err error) {
|
||||
w.c.logger.Infof("xds: client received RDS update: %+v, err: %v", update, err)
|
||||
w.mu.Lock()
|
||||
defer w.mu.Unlock()
|
||||
if w.closed {
|
||||
return
|
||||
}
|
||||
if err != nil {
|
||||
w.serviceCb(ServiceUpdate{}, err)
|
||||
return
|
||||
}
|
||||
w.serviceCb(ServiceUpdate{Cluster: update.clusterName}, nil)
|
||||
}
|
||||
|
||||
func (w *serviceUpdateWatcher) close() {
|
||||
w.mu.Lock()
|
||||
defer w.mu.Unlock()
|
||||
w.closed = true
|
||||
w.ldsCancel()
|
||||
if w.rdsCancel != nil {
|
||||
w.rdsCancel()
|
||||
w.rdsCancel = nil
|
||||
}
|
||||
}
|
339
xds/internal/client/client_watchers_service_test.go
Normal file
339
xds/internal/client/client_watchers_service_test.go
Normal file
@ -0,0 +1,339 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2020 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 client
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/testutils/fakeserver"
|
||||
)
|
||||
|
||||
type serviceUpdateErr struct {
|
||||
u ServiceUpdate
|
||||
err error
|
||||
}
|
||||
|
||||
// TestServiceWatch covers the cases:
|
||||
// - an update is received after a watch()
|
||||
// - an update for another resource name (which doesn't trigger callback)
|
||||
// - an upate is received after cancel()
|
||||
func (s) TestServiceWatch(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
serviceUpdateCh := testutils.NewChannel()
|
||||
c.WatchService(testLDSName, func(update ServiceUpdate, err error) {
|
||||
serviceUpdateCh.Send(serviceUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate := ServiceUpdate{Cluster: testCDSName}
|
||||
|
||||
<-v2Client.addWatches[ldsURL]
|
||||
v2Client.r.newLDSUpdate(map[string]ldsUpdate{
|
||||
testLDSName: {routeName: testRDSName},
|
||||
})
|
||||
<-v2Client.addWatches[rdsURL]
|
||||
v2Client.r.newRDSUpdate(map[string]rdsUpdate{
|
||||
testRDSName: {clusterName: testCDSName},
|
||||
})
|
||||
|
||||
if u, err := serviceUpdateCh.Receive(); err != nil || u != (serviceUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected serviceUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestServiceWatchLDSUpdate covers the case that after first LDS and first RDS
|
||||
// response, the second LDS response trigger an new RDS watch, and an update of
|
||||
// the old RDS watch doesn't trigger update to service callback.
|
||||
func (s) TestServiceWatchLDSUpdate(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
serviceUpdateCh := testutils.NewChannel()
|
||||
c.WatchService(testLDSName, func(update ServiceUpdate, err error) {
|
||||
serviceUpdateCh.Send(serviceUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate := ServiceUpdate{Cluster: testCDSName}
|
||||
|
||||
<-v2Client.addWatches[ldsURL]
|
||||
v2Client.r.newLDSUpdate(map[string]ldsUpdate{
|
||||
testLDSName: {routeName: testRDSName},
|
||||
})
|
||||
<-v2Client.addWatches[rdsURL]
|
||||
v2Client.r.newRDSUpdate(map[string]rdsUpdate{
|
||||
testRDSName: {clusterName: testCDSName},
|
||||
})
|
||||
|
||||
if u, err := serviceUpdateCh.Receive(); err != nil || u != (serviceUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected serviceUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
// Another LDS update with a different RDS_name.
|
||||
v2Client.r.newLDSUpdate(map[string]ldsUpdate{
|
||||
testLDSName: {routeName: testRDSName + "2"},
|
||||
})
|
||||
<-v2Client.addWatches[rdsURL]
|
||||
|
||||
// Another update for the old name.
|
||||
v2Client.r.newRDSUpdate(map[string]rdsUpdate{
|
||||
testRDSName: {clusterName: testCDSName},
|
||||
})
|
||||
|
||||
if u, err := serviceUpdateCh.Receive(); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected serviceUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
|
||||
wantUpdate2 := ServiceUpdate{Cluster: testCDSName + "2"}
|
||||
// RDS update for the new name.
|
||||
v2Client.r.newRDSUpdate(map[string]rdsUpdate{
|
||||
testRDSName + "2": {clusterName: testCDSName + "2"},
|
||||
})
|
||||
|
||||
if u, err := serviceUpdateCh.Receive(); err != nil || u != (serviceUpdateErr{wantUpdate2, nil}) {
|
||||
t.Errorf("unexpected serviceUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestServiceWatchSecond covers the case where a second WatchService() gets an
|
||||
// error (because only one is allowed). But the first watch still receives
|
||||
// updates.
|
||||
func (s) TestServiceWatchSecond(t *testing.T) {
|
||||
v2ClientCh, cleanup := overrideNewXDSV2Client()
|
||||
defer cleanup()
|
||||
|
||||
c, err := New(clientOpts(testXDSServer))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to create client: %v", err)
|
||||
}
|
||||
defer c.Close()
|
||||
|
||||
v2Client := <-v2ClientCh
|
||||
|
||||
serviceUpdateCh := testutils.NewChannel()
|
||||
c.WatchService(testLDSName, func(update ServiceUpdate, err error) {
|
||||
serviceUpdateCh.Send(serviceUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
wantUpdate := ServiceUpdate{Cluster: testCDSName}
|
||||
|
||||
<-v2Client.addWatches[ldsURL]
|
||||
v2Client.r.newLDSUpdate(map[string]ldsUpdate{
|
||||
testLDSName: {routeName: testRDSName},
|
||||
})
|
||||
<-v2Client.addWatches[rdsURL]
|
||||
v2Client.r.newRDSUpdate(map[string]rdsUpdate{
|
||||
testRDSName: {clusterName: testCDSName},
|
||||
})
|
||||
|
||||
if u, err := serviceUpdateCh.Receive(); err != nil || u != (serviceUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected serviceUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
serviceUpdateCh2 := testutils.NewChannel()
|
||||
// Call WatchService() again, with the same or different name.
|
||||
c.WatchService(testLDSName, func(update ServiceUpdate, err error) {
|
||||
serviceUpdateCh2.Send(serviceUpdateErr{u: update, err: err})
|
||||
})
|
||||
|
||||
u, err := serviceUpdateCh2.Receive()
|
||||
if err != nil {
|
||||
t.Fatalf("failed to get serviceUpdate: %v", err)
|
||||
}
|
||||
uu := u.(serviceUpdateErr)
|
||||
if uu.u != (ServiceUpdate{}) {
|
||||
t.Errorf("unexpected serviceUpdate: %v, want %v", uu.u, ServiceUpdate{})
|
||||
}
|
||||
if uu.err == nil {
|
||||
t.Errorf("unexpected serviceError: <nil>, want error watcher timeout")
|
||||
}
|
||||
|
||||
// Send update again, first callback should be called, second should
|
||||
// timeout.
|
||||
v2Client.r.newLDSUpdate(map[string]ldsUpdate{
|
||||
testLDSName: {routeName: testRDSName},
|
||||
})
|
||||
v2Client.r.newRDSUpdate(map[string]rdsUpdate{
|
||||
testRDSName: {clusterName: testCDSName},
|
||||
})
|
||||
|
||||
if u, err := serviceUpdateCh.Receive(); err != nil || u != (serviceUpdateErr{wantUpdate, nil}) {
|
||||
t.Errorf("unexpected serviceUpdate: %v, error receiving from channel: %v", u, err)
|
||||
}
|
||||
|
||||
if u, err := serviceUpdateCh2.Receive(); err != testutils.ErrRecvTimeout {
|
||||
t.Errorf("unexpected serviceUpdate: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestServiceWatchWithNoResponseFromServer tests the case where the xDS server
|
||||
// does not respond to the requests being sent out as part of registering a
|
||||
// service update watcher. The callback will get an error.
|
||||
func (s) TestServiceWatchWithNoResponseFromServer(t *testing.T) {
|
||||
fakeServer, cleanup, err := fakeserver.StartServer()
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to start fake xDS server: %v", err)
|
||||
}
|
||||
defer cleanup()
|
||||
|
||||
xdsClient, err := New(clientOpts(fakeServer.Address))
|
||||
if err != nil {
|
||||
t.Fatalf("New returned error: %v", err)
|
||||
}
|
||||
defer xdsClient.Close()
|
||||
t.Log("Created an xdsClient...")
|
||||
|
||||
oldWatchExpiryTimeout := defaultWatchExpiryTimeout
|
||||
defaultWatchExpiryTimeout = 500 * time.Millisecond
|
||||
defer func() {
|
||||
defaultWatchExpiryTimeout = oldWatchExpiryTimeout
|
||||
}()
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
cancelWatch := xdsClient.WatchService(goodLDSTarget1, func(su ServiceUpdate, err error) {
|
||||
if su.Cluster != "" {
|
||||
callbackCh.Send(fmt.Errorf("got clusterName: %+v, want empty clusterName", su.Cluster))
|
||||
return
|
||||
}
|
||||
if err == nil {
|
||||
callbackCh.Send(errors.New("xdsClient.WatchService returned error non-nil error"))
|
||||
return
|
||||
}
|
||||
callbackCh.Send(nil)
|
||||
})
|
||||
defer cancelWatch()
|
||||
t.Log("Registered a watcher for service updates...")
|
||||
|
||||
// Wait for one request from the client, but send no reponses.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an LDS request")
|
||||
}
|
||||
waitForNilErr(t, callbackCh)
|
||||
}
|
||||
|
||||
// TestServiceWatchEmptyRDS tests the case where the underlying v2Client
|
||||
// receives an empty RDS response. The callback will get an error.
|
||||
func (s) TestServiceWatchEmptyRDS(t *testing.T) {
|
||||
fakeServer, cleanup, err := fakeserver.StartServer()
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to start fake xDS server: %v", err)
|
||||
}
|
||||
defer cleanup()
|
||||
|
||||
xdsClient, err := New(clientOpts(fakeServer.Address))
|
||||
if err != nil {
|
||||
t.Fatalf("New returned error: %v", err)
|
||||
}
|
||||
defer xdsClient.Close()
|
||||
t.Log("Created an xdsClient...")
|
||||
|
||||
oldWatchExpiryTimeout := defaultWatchExpiryTimeout
|
||||
defaultWatchExpiryTimeout = 500 * time.Millisecond
|
||||
defer func() {
|
||||
defaultWatchExpiryTimeout = oldWatchExpiryTimeout
|
||||
}()
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
cancelWatch := xdsClient.WatchService(goodLDSTarget1, func(su ServiceUpdate, err error) {
|
||||
if su.Cluster != "" {
|
||||
callbackCh.Send(fmt.Errorf("got clusterName: %+v, want empty clusterName", su.Cluster))
|
||||
return
|
||||
}
|
||||
if err == nil {
|
||||
callbackCh.Send(errors.New("xdsClient.WatchService returned error non-nil error"))
|
||||
return
|
||||
}
|
||||
callbackCh.Send(nil)
|
||||
})
|
||||
defer cancelWatch()
|
||||
t.Log("Registered a watcher for service updates...")
|
||||
|
||||
// Make the fakeServer send LDS response.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an LDS request")
|
||||
}
|
||||
fakeServer.XDSResponseChan <- &fakeserver.Response{Resp: goodLDSResponse1}
|
||||
|
||||
// Make the fakeServer send an empty RDS response.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an RDS request")
|
||||
}
|
||||
fakeServer.XDSResponseChan <- &fakeserver.Response{Resp: noVirtualHostsInRDSResponse}
|
||||
waitForNilErr(t, callbackCh)
|
||||
}
|
||||
|
||||
// TestServiceWatchWithClientClose tests the case where xDS responses are
|
||||
// received after the client is closed, and we make sure that the registered
|
||||
// watcher callback is not invoked.
|
||||
func (s) TestServiceWatchWithClientClose(t *testing.T) {
|
||||
fakeServer, cleanup, err := fakeserver.StartServer()
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to start fake xDS server: %v", err)
|
||||
}
|
||||
defer cleanup()
|
||||
|
||||
xdsClient, err := New(clientOpts(fakeServer.Address))
|
||||
if err != nil {
|
||||
t.Fatalf("New returned error: %v", err)
|
||||
}
|
||||
defer xdsClient.Close()
|
||||
t.Log("Created an xdsClient...")
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
cancelWatch := xdsClient.WatchService(goodLDSTarget1, func(su ServiceUpdate, err error) {
|
||||
callbackCh.Send(errors.New("watcher callback invoked after client close"))
|
||||
})
|
||||
defer cancelWatch()
|
||||
t.Log("Registered a watcher for service updates...")
|
||||
|
||||
// Make the fakeServer send LDS response.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an LDS request")
|
||||
}
|
||||
fakeServer.XDSResponseChan <- &fakeserver.Response{Resp: goodLDSResponse1}
|
||||
|
||||
xdsClient.Close()
|
||||
t.Log("Closing the xdsClient...")
|
||||
|
||||
// Push an RDS response from the fakeserver
|
||||
fakeServer.XDSResponseChan <- &fakeserver.Response{Resp: goodRDSResponse1}
|
||||
if cbErr, err := callbackCh.Receive(); err != testutils.ErrRecvTimeout {
|
||||
t.Fatal(cbErr)
|
||||
}
|
||||
}
|
@ -20,6 +20,7 @@ package client
|
||||
import (
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
@ -29,19 +30,53 @@ import (
|
||||
)
|
||||
|
||||
type watchHandleTestcase struct {
|
||||
typeURL string
|
||||
resourceName string
|
||||
|
||||
responseToHandle *xdspb.DiscoveryResponse
|
||||
wantHandleErr bool
|
||||
wantUpdate interface{}
|
||||
wantUpdateErr bool
|
||||
}
|
||||
|
||||
// Only one of the following should be non-nil. The one corresponding with
|
||||
// typeURL will be called.
|
||||
ldsWatch func(target string, ldsCb ldsCallbackFunc) (cancel func())
|
||||
rdsWatch func(routeName string, rdsCb rdsCallbackFunc) (cancel func())
|
||||
cdsWatch func(clusterName string, cdsCb cdsCallbackFunc) (cancel func())
|
||||
edsWatch func(clusterName string, edsCb edsCallbackFunc) (cancel func())
|
||||
watchReqChan *testutils.Channel // The request sent for watch will be sent to this channel.
|
||||
handleXDSResp func(response *xdspb.DiscoveryResponse) error
|
||||
type testUpdateReceiver struct {
|
||||
f func(typeURL string, d map[string]interface{})
|
||||
}
|
||||
|
||||
func (t *testUpdateReceiver) newLDSUpdate(d map[string]ldsUpdate) {
|
||||
dd := make(map[string]interface{})
|
||||
for k, v := range d {
|
||||
dd[k] = v
|
||||
}
|
||||
t.newUpdate(ldsURL, dd)
|
||||
}
|
||||
|
||||
func (t *testUpdateReceiver) newRDSUpdate(d map[string]rdsUpdate) {
|
||||
dd := make(map[string]interface{})
|
||||
for k, v := range d {
|
||||
dd[k] = v
|
||||
}
|
||||
t.newUpdate(rdsURL, dd)
|
||||
}
|
||||
|
||||
func (t *testUpdateReceiver) newCDSUpdate(d map[string]ClusterUpdate) {
|
||||
dd := make(map[string]interface{})
|
||||
for k, v := range d {
|
||||
dd[k] = v
|
||||
}
|
||||
t.newUpdate(cdsURL, dd)
|
||||
}
|
||||
|
||||
func (t *testUpdateReceiver) newEDSUpdate(d map[string]EndpointsUpdate) {
|
||||
dd := make(map[string]interface{})
|
||||
for k, v := range d {
|
||||
dd[k] = v
|
||||
}
|
||||
t.newUpdate(edsURL, dd)
|
||||
}
|
||||
|
||||
func (t *testUpdateReceiver) newUpdate(typeURL string, d map[string]interface{}) {
|
||||
t.f(typeURL, d)
|
||||
}
|
||||
|
||||
// testWatchHandle is called to test response handling for each xDS.
|
||||
@ -51,44 +86,38 @@ type watchHandleTestcase struct {
|
||||
// handleXDSResp with responseToHandle (if it's set). It then compares the
|
||||
// update received by watch callback with the expected results.
|
||||
func testWatchHandle(t *testing.T, test *watchHandleTestcase) {
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
type updateErr struct {
|
||||
u interface{}
|
||||
err error
|
||||
}
|
||||
gotUpdateCh := testutils.NewChannel()
|
||||
|
||||
var cancelWatch func()
|
||||
v2c := newV2Client(&testUpdateReceiver{
|
||||
f: func(typeURL string, d map[string]interface{}) {
|
||||
if typeURL == test.typeURL {
|
||||
if u, ok := d[test.resourceName]; ok {
|
||||
gotUpdateCh.Send(updateErr{u, nil})
|
||||
}
|
||||
}
|
||||
},
|
||||
}, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
|
||||
// RDS needs an existin LDS watch for the hostname.
|
||||
if test.typeURL == rdsURL {
|
||||
doLDS(t, v2c, fakeServer)
|
||||
}
|
||||
|
||||
// Register the watcher, this will also trigger the v2Client to send the xDS
|
||||
// request.
|
||||
switch {
|
||||
case test.ldsWatch != nil:
|
||||
cancelWatch = test.ldsWatch(goodLDSTarget1, func(u ldsUpdate, err error) {
|
||||
t.Logf("in v2c.watchLDS callback, ldsUpdate: %+v, err: %v", u, err)
|
||||
gotUpdateCh.Send(updateErr{u, err})
|
||||
})
|
||||
case test.rdsWatch != nil:
|
||||
cancelWatch = test.rdsWatch(goodRouteName1, func(u rdsUpdate, err error) {
|
||||
t.Logf("in v2c.watchRDS callback, rdsUpdate: %+v, err: %v", u, err)
|
||||
gotUpdateCh.Send(updateErr{u, err})
|
||||
})
|
||||
case test.cdsWatch != nil:
|
||||
cancelWatch = test.cdsWatch(clusterName1, func(u CDSUpdate, err error) {
|
||||
t.Logf("in v2c.watchCDS callback, cdsUpdate: %+v, err: %v", u, err)
|
||||
gotUpdateCh.Send(updateErr{u, err})
|
||||
})
|
||||
case test.edsWatch != nil:
|
||||
cancelWatch = test.edsWatch(goodEDSName, func(u *EDSUpdate, err error) {
|
||||
t.Logf("in v2c.watchEDS callback, edsUpdate: %+v, err: %v", u, err)
|
||||
gotUpdateCh.Send(updateErr{*u, err})
|
||||
})
|
||||
default:
|
||||
t.Fatalf("no watch() is set")
|
||||
}
|
||||
defer cancelWatch()
|
||||
v2c.addWatch(test.typeURL, test.resourceName)
|
||||
|
||||
// Wait till the request makes it to the fakeServer. This ensures that
|
||||
// the watch request has been processed by the v2Client.
|
||||
if _, err := test.watchReqChan.Receive(); err != nil {
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout waiting for an xDS request: %v", err)
|
||||
}
|
||||
|
||||
@ -98,7 +127,18 @@ func testWatchHandle(t *testing.T, test *watchHandleTestcase) {
|
||||
//
|
||||
// Also note that this won't trigger ACK, so there's no need to clear the
|
||||
// request channel afterwards.
|
||||
if err := test.handleXDSResp(test.responseToHandle); (err != nil) != test.wantHandleErr {
|
||||
var handleXDSResp func(response *xdspb.DiscoveryResponse) error
|
||||
switch test.typeURL {
|
||||
case ldsURL:
|
||||
handleXDSResp = v2c.handleLDSResponse
|
||||
case rdsURL:
|
||||
handleXDSResp = v2c.handleRDSResponse
|
||||
case cdsURL:
|
||||
handleXDSResp = v2c.handleCDSResponse
|
||||
case edsURL:
|
||||
handleXDSResp = v2c.handleEDSResponse
|
||||
}
|
||||
if err := handleXDSResp(test.responseToHandle); (err != nil) != test.wantHandleErr {
|
||||
t.Fatalf("v2c.handleRDSResponse() returned err: %v, wantErr: %v", err, test.wantHandleErr)
|
||||
}
|
||||
|
||||
@ -121,7 +161,7 @@ func testWatchHandle(t *testing.T, test *watchHandleTestcase) {
|
||||
t.Fatal("Timeout expecting xDS update")
|
||||
}
|
||||
gotUpdate := uErr.(updateErr).u
|
||||
opt := cmp.AllowUnexported(rdsUpdate{}, ldsUpdate{}, CDSUpdate{}, EDSUpdate{})
|
||||
opt := cmp.AllowUnexported(rdsUpdate{}, ldsUpdate{}, ClusterUpdate{}, EndpointsUpdate{})
|
||||
if diff := cmp.Diff(gotUpdate, wantUpdate, opt); diff != "" {
|
||||
t.Fatalf("got update : %+v, want %+v, diff: %s", gotUpdate, wantUpdate, diff)
|
||||
}
|
||||
|
@ -1,104 +0,0 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2019 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 client
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
adsgrpc "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v2"
|
||||
)
|
||||
|
||||
type adsStream adsgrpc.AggregatedDiscoveryService_StreamAggregatedResourcesClient
|
||||
|
||||
const (
|
||||
ldsURL = "type.googleapis.com/envoy.api.v2.Listener"
|
||||
rdsURL = "type.googleapis.com/envoy.api.v2.RouteConfiguration"
|
||||
cdsURL = "type.googleapis.com/envoy.api.v2.Cluster"
|
||||
edsURL = "type.googleapis.com/envoy.api.v2.ClusterLoadAssignment"
|
||||
)
|
||||
|
||||
// watchState is an enum to represent the state of a watch call.
|
||||
type watchState int
|
||||
|
||||
const (
|
||||
watchEnqueued watchState = iota
|
||||
watchCancelled
|
||||
watchStarted
|
||||
)
|
||||
|
||||
// watchInfo holds all the information about a watch call.
|
||||
type watchInfo struct {
|
||||
typeURL string
|
||||
target []string
|
||||
state watchState
|
||||
|
||||
ldsCallback ldsCallbackFunc
|
||||
rdsCallback rdsCallbackFunc
|
||||
cdsCallback cdsCallbackFunc
|
||||
edsCallback edsCallbackFunc
|
||||
expiryTimer *time.Timer
|
||||
}
|
||||
|
||||
// cancel marks the state as cancelled, and also stops the expiry timer.
|
||||
func (wi *watchInfo) cancel() {
|
||||
wi.state = watchCancelled
|
||||
if wi.expiryTimer != nil {
|
||||
wi.expiryTimer.Stop()
|
||||
}
|
||||
}
|
||||
|
||||
// stopTimer stops the expiry timer without cancelling the watch.
|
||||
func (wi *watchInfo) stopTimer() {
|
||||
if wi.expiryTimer != nil {
|
||||
wi.expiryTimer.Stop()
|
||||
}
|
||||
}
|
||||
|
||||
type ackInfo struct {
|
||||
typeURL string
|
||||
version string // NACK if version is an empty string.
|
||||
nonce string
|
||||
// ACK/NACK are tagged with the stream it's for. When the stream is down,
|
||||
// all the ACK/NACK for this stream will be dropped, and the version/nonce
|
||||
// won't be updated.
|
||||
stream adsStream
|
||||
}
|
||||
|
||||
type ldsUpdate struct {
|
||||
routeName string
|
||||
}
|
||||
type ldsCallbackFunc func(ldsUpdate, error)
|
||||
|
||||
type rdsUpdate struct {
|
||||
clusterName string
|
||||
}
|
||||
type rdsCallbackFunc func(rdsUpdate, error)
|
||||
|
||||
// CDSUpdate contains information from a received CDS response, which is of
|
||||
// interest to the registered CDS watcher.
|
||||
type CDSUpdate struct {
|
||||
// ServiceName is the service name corresponding to the clusterName which
|
||||
// is being watched for through CDS.
|
||||
ServiceName string
|
||||
// EnableLRS indicates whether or not load should be reported through LRS.
|
||||
EnableLRS bool
|
||||
}
|
||||
type cdsCallbackFunc func(CDSUpdate, error)
|
||||
|
||||
type edsCallbackFunc func(*EDSUpdate, error)
|
@ -20,7 +20,6 @@ package client
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
@ -33,19 +32,37 @@ import (
|
||||
adsgrpc "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v2"
|
||||
)
|
||||
|
||||
// The value chosen here is based on the default value of the
|
||||
// initial_fetch_timeout field in corepb.ConfigSource proto.
|
||||
var defaultWatchExpiryTimeout = 15 * time.Second
|
||||
type adsStream adsgrpc.AggregatedDiscoveryService_StreamAggregatedResourcesClient
|
||||
|
||||
var _ xdsv2Client = &v2Client{}
|
||||
|
||||
// updateHandler handles the update (parsed from xds responses). It's
|
||||
// implemented by the upper level Client.
|
||||
//
|
||||
// It's an interface to be overridden in test.
|
||||
type updateHandler interface {
|
||||
newLDSUpdate(d map[string]ldsUpdate)
|
||||
newRDSUpdate(d map[string]rdsUpdate)
|
||||
newCDSUpdate(d map[string]ClusterUpdate)
|
||||
newEDSUpdate(d map[string]EndpointsUpdate)
|
||||
}
|
||||
|
||||
// v2Client performs the actual xDS RPCs using the xDS v2 API. It creates a
|
||||
// single ADS stream on which the different types of xDS requests and responses
|
||||
// are multiplexed.
|
||||
//
|
||||
// This client's main purpose is to make the RPC, build/parse proto messages,
|
||||
// and do ACK/NACK. It's a naive implementation that sends whatever the upper
|
||||
// layer tells it to send. It will call the callback with everything in every
|
||||
// response. It doesn't keep a cache of responses, or check for duplicates.
|
||||
//
|
||||
// The reason for splitting this out from the top level xdsClient object is
|
||||
// because there is already an xDS v3Aplha API in development. If and when we
|
||||
// want to switch to that, this separation will ease that process.
|
||||
type v2Client struct {
|
||||
ctx context.Context
|
||||
cancelCtx context.CancelFunc
|
||||
parent updateHandler
|
||||
|
||||
// ClientConn to the xDS gRPC server. Owned by the parent xdsClient.
|
||||
cc *grpc.ClientConn
|
||||
@ -54,8 +71,11 @@ type v2Client struct {
|
||||
|
||||
logger *grpclog.PrefixLogger
|
||||
|
||||
// streamCh is the channel where new ADS streams are pushed to (when the old
|
||||
// stream is broken). It's monitored by the sending goroutine, so requests
|
||||
// are sent to the most up-to-date stream.
|
||||
streamCh chan adsStream
|
||||
// sendCh in the channel onto which watchInfo objects are pushed by the
|
||||
// sendCh is the channel onto which watchAction objects are pushed by the
|
||||
// watch API, and it is read and acted upon by the send() goroutine.
|
||||
sendCh *buffer.Unbounded
|
||||
|
||||
@ -66,7 +86,7 @@ type v2Client struct {
|
||||
// messages. When the user of this client object cancels a watch call,
|
||||
// these are set to nil. All accesses to the map protected and any value
|
||||
// inside the map should be protected with the above mutex.
|
||||
watchMap map[string]*watchInfo
|
||||
watchMap map[string]map[string]bool
|
||||
// versionMap contains the version that was acked (the version in the ack
|
||||
// request that was sent on wire). The key is typeURL, the value is the
|
||||
// version string, becaues the versions for different resource types should
|
||||
@ -74,33 +94,19 @@ type v2Client struct {
|
||||
versionMap map[string]string
|
||||
// nonceMap contains the nonce from the most recent received response.
|
||||
nonceMap map[string]string
|
||||
// rdsCache maintains a mapping of {routeConfigName --> clusterName} from
|
||||
// validated route configurations received in RDS responses. We cache all
|
||||
// valid route configurations, whether or not we are interested in them
|
||||
// when we received them (because we could become interested in them in the
|
||||
// future and the server wont send us those resources again).
|
||||
// Protected by the above mutex.
|
||||
// hostname is the LDS resource_name to watch. It is set to the first LDS
|
||||
// resource_name to watch, and removed when the LDS watch is canceled.
|
||||
//
|
||||
// TODO: remove RDS cache. The updated spec says client can ignore
|
||||
// unrequested resources.
|
||||
// https://github.com/envoyproxy/envoy/blob/master/api/xds_protocol.rst#resource-hints
|
||||
rdsCache map[string]string
|
||||
// rdsCache maintains a mapping of {clusterName --> CDSUpdate} from
|
||||
// validated cluster configurations received in CDS responses. We cache all
|
||||
// valid cluster configurations, whether or not we are interested in them
|
||||
// when we received them (because we could become interested in them in the
|
||||
// future and the server wont send us those resources again). This is only
|
||||
// to support legacy management servers that do not honor the
|
||||
// resource_names field. As per the latest spec, the server should resend
|
||||
// the response when the request changes, even if it had sent the same
|
||||
// resource earlier (when not asked for). Protected by the above mutex.
|
||||
cdsCache map[string]CDSUpdate
|
||||
// It's from the dial target of the parent ClientConn. RDS resource
|
||||
// processing needs this to do the host matching.
|
||||
hostname string
|
||||
}
|
||||
|
||||
// newV2Client creates a new v2Client initialized with the passed arguments.
|
||||
func newV2Client(cc *grpc.ClientConn, nodeProto *corepb.Node, backoff func(int) time.Duration, logger *grpclog.PrefixLogger) *v2Client {
|
||||
func newV2Client(parent updateHandler, cc *grpc.ClientConn, nodeProto *corepb.Node, backoff func(int) time.Duration, logger *grpclog.PrefixLogger) *v2Client {
|
||||
v2c := &v2Client{
|
||||
cc: cc,
|
||||
parent: parent,
|
||||
nodeProto: nodeProto,
|
||||
backoff: backoff,
|
||||
|
||||
@ -109,11 +115,9 @@ func newV2Client(cc *grpc.ClientConn, nodeProto *corepb.Node, backoff func(int)
|
||||
streamCh: make(chan adsStream, 1),
|
||||
sendCh: buffer.NewUnbounded(),
|
||||
|
||||
watchMap: make(map[string]*watchInfo),
|
||||
watchMap: make(map[string]map[string]bool),
|
||||
versionMap: make(map[string]string),
|
||||
nonceMap: make(map[string]string),
|
||||
rdsCache: make(map[string]string),
|
||||
cdsCache: make(map[string]CDSUpdate),
|
||||
}
|
||||
v2c.ctx, v2c.cancelCtx = context.WithCancel(context.Background())
|
||||
|
||||
@ -214,8 +218,8 @@ func (v2c *v2Client) sendExisting(stream adsStream) bool {
|
||||
v2c.versionMap = make(map[string]string)
|
||||
v2c.nonceMap = make(map[string]string)
|
||||
|
||||
for typeURL, wi := range v2c.watchMap {
|
||||
if !v2c.sendRequest(stream, wi.target, typeURL, "", "") {
|
||||
for typeURL, s := range v2c.watchMap {
|
||||
if !v2c.sendRequest(stream, mapToSlice(s), typeURL, "", "") {
|
||||
return false
|
||||
}
|
||||
}
|
||||
@ -223,45 +227,79 @@ func (v2c *v2Client) sendExisting(stream adsStream) bool {
|
||||
return true
|
||||
}
|
||||
|
||||
// processWatchInfo pulls the fields needed by the request from a watchInfo.
|
||||
type watchAction struct {
|
||||
typeURL string
|
||||
remove bool // Whether this is to remove watch for the resource.
|
||||
resource string
|
||||
}
|
||||
|
||||
// processWatchInfo pulls the fields needed by the request from a watchAction.
|
||||
//
|
||||
// It also calls callback with cached response, and updates the watch map in
|
||||
// v2c.
|
||||
//
|
||||
// If the watch was already canceled, it returns false for send
|
||||
func (v2c *v2Client) processWatchInfo(t *watchInfo) (target []string, typeURL, version, nonce string, send bool) {
|
||||
// It also updates the watch map in v2c.
|
||||
func (v2c *v2Client) processWatchInfo(t *watchAction) (target []string, typeURL, version, nonce string, send bool) {
|
||||
v2c.mu.Lock()
|
||||
defer v2c.mu.Unlock()
|
||||
if t.state == watchCancelled {
|
||||
return // This returns all zero values, and false for send.
|
||||
|
||||
var current map[string]bool
|
||||
current, ok := v2c.watchMap[t.typeURL]
|
||||
if !ok {
|
||||
current = make(map[string]bool)
|
||||
v2c.watchMap[t.typeURL] = current
|
||||
}
|
||||
t.state = watchStarted
|
||||
|
||||
if t.remove {
|
||||
delete(current, t.resource)
|
||||
if len(current) == 0 {
|
||||
delete(v2c.watchMap, t.typeURL)
|
||||
}
|
||||
} else {
|
||||
current[t.resource] = true
|
||||
}
|
||||
|
||||
// Special handling for LDS, because RDS needs the LDS resource_name for
|
||||
// response host matching.
|
||||
if t.typeURL == ldsURL {
|
||||
// Set hostname to the first LDS resource_name, and reset it when the
|
||||
// last LDS watch is removed. The upper level Client isn't expected to
|
||||
// watchLDS more than once.
|
||||
if l := len(current); l == 1 {
|
||||
v2c.hostname = t.resource
|
||||
} else if l == 0 {
|
||||
v2c.hostname = ""
|
||||
}
|
||||
}
|
||||
|
||||
send = true
|
||||
|
||||
typeURL = t.typeURL
|
||||
target = t.target
|
||||
v2c.checkCacheAndUpdateWatchMap(t)
|
||||
// TODO: if watch is called again with the same resource names,
|
||||
// there's no need to send another request.
|
||||
|
||||
target = mapToSlice(current)
|
||||
// We don't reset version or nonce when a new watch is started. The version
|
||||
// and nonce from previous response are carried by the request unless the
|
||||
// stream is recreated.
|
||||
version = v2c.versionMap[typeURL]
|
||||
nonce = v2c.nonceMap[typeURL]
|
||||
return
|
||||
return target, typeURL, version, nonce, send
|
||||
}
|
||||
|
||||
// processAckInfo pulls the fields needed by the ack request from a ackInfo.
|
||||
type ackAction struct {
|
||||
typeURL string
|
||||
version string // NACK if version is an empty string.
|
||||
nonce string
|
||||
// ACK/NACK are tagged with the stream it's for. When the stream is down,
|
||||
// all the ACK/NACK for this stream will be dropped, and the version/nonce
|
||||
// won't be updated.
|
||||
stream adsStream
|
||||
}
|
||||
|
||||
// processAckInfo pulls the fields needed by the ack request from a ackAction.
|
||||
//
|
||||
// If no active watch is found for this ack, it returns false for send.
|
||||
func (v2c *v2Client) processAckInfo(t *ackInfo, stream adsStream) (target []string, typeURL, version, nonce string, send bool) {
|
||||
func (v2c *v2Client) processAckInfo(t *ackAction, stream adsStream) (target []string, typeURL, version, nonce string, send bool) {
|
||||
if t.stream != stream {
|
||||
// If ACK's stream isn't the current sending stream, this means the ACK
|
||||
// was pushed to queue before the old stream broke, and a new stream has
|
||||
// been started since. Return immediately here so we don't update the
|
||||
// nonce for the new stream.
|
||||
return
|
||||
return nil, "", "", "", false
|
||||
}
|
||||
typeURL = t.typeURL
|
||||
|
||||
@ -274,16 +312,18 @@ func (v2c *v2Client) processAckInfo(t *ackInfo, stream adsStream) (target []stri
|
||||
nonce = t.nonce
|
||||
v2c.nonceMap[typeURL] = nonce
|
||||
|
||||
wi, ok := v2c.watchMap[typeURL]
|
||||
if !ok {
|
||||
s, ok := v2c.watchMap[typeURL]
|
||||
if !ok || len(s) == 0 {
|
||||
// We don't send the request ack if there's no active watch (this can be
|
||||
// either the server sends responses before any request, or the watch is
|
||||
// canceled while the ackInfo is in queue), because there's no resource
|
||||
// canceled while the ackAction is in queue), because there's no resource
|
||||
// name. And if we send a request with empty resource name list, the
|
||||
// server may treat it as a wild card and send us everything.
|
||||
return nil, "", "", "", false
|
||||
}
|
||||
send = true
|
||||
target = mapToSlice(s)
|
||||
|
||||
version = t.version
|
||||
if version == "" {
|
||||
// This is a nack, get the previous acked version.
|
||||
@ -294,7 +334,6 @@ func (v2c *v2Client) processAckInfo(t *ackInfo, stream adsStream) (target []stri
|
||||
} else {
|
||||
v2c.versionMap[typeURL] = version
|
||||
}
|
||||
target = wi.target
|
||||
return target, typeURL, version, nonce, send
|
||||
}
|
||||
|
||||
@ -303,7 +342,7 @@ func (v2c *v2Client) processAckInfo(t *ackInfo, stream adsStream) (target []stri
|
||||
// It watches the stream channel for new streams, and the request channel for
|
||||
// new requests to send on the stream.
|
||||
//
|
||||
// For each new request (watchInfo), it's
|
||||
// For each new request (watchAction), it's
|
||||
// - processed and added to the watch map
|
||||
// - so resend will pick them up when there are new streams)
|
||||
// - sent on the current stream if there's one
|
||||
@ -320,8 +359,7 @@ func (v2c *v2Client) send() {
|
||||
select {
|
||||
case <-v2c.ctx.Done():
|
||||
return
|
||||
case newStream := <-v2c.streamCh:
|
||||
stream = newStream
|
||||
case stream = <-v2c.streamCh:
|
||||
if !v2c.sendExisting(stream) {
|
||||
// send failed, clear the current stream.
|
||||
stream = nil
|
||||
@ -335,9 +373,9 @@ func (v2c *v2Client) send() {
|
||||
send bool
|
||||
)
|
||||
switch t := u.(type) {
|
||||
case *watchInfo:
|
||||
case *watchAction:
|
||||
target, typeURL, version, nonce, send = v2c.processWatchInfo(t)
|
||||
case *ackInfo:
|
||||
case *ackAction:
|
||||
target, typeURL, version, nonce, send = v2c.processAckInfo(t, stream)
|
||||
}
|
||||
if !send {
|
||||
@ -388,7 +426,7 @@ func (v2c *v2Client) recv(stream adsStream) bool {
|
||||
|
||||
typeURL := resp.GetTypeUrl()
|
||||
if respHandleErr != nil {
|
||||
v2c.sendCh.Put(&ackInfo{
|
||||
v2c.sendCh.Put(&ackAction{
|
||||
typeURL: typeURL,
|
||||
version: "",
|
||||
nonce: resp.GetNonce(),
|
||||
@ -397,7 +435,7 @@ func (v2c *v2Client) recv(stream adsStream) bool {
|
||||
v2c.logger.Warningf("Sending NACK for response type: %v, version: %v, nonce: %v, reason: %v", typeURL, resp.GetVersionInfo(), resp.GetNonce(), respHandleErr)
|
||||
continue
|
||||
}
|
||||
v2c.sendCh.Put(&ackInfo{
|
||||
v2c.sendCh.Put(&ackAction{
|
||||
typeURL: typeURL,
|
||||
version: resp.GetVersionInfo(),
|
||||
nonce: resp.GetNonce(),
|
||||
@ -408,145 +446,25 @@ func (v2c *v2Client) recv(stream adsStream) bool {
|
||||
}
|
||||
}
|
||||
|
||||
// watchLDS registers an LDS watcher for the provided target. Updates
|
||||
// corresponding to received LDS responses will be pushed to the provided
|
||||
// callback. The caller can cancel the watch by invoking the returned cancel
|
||||
// function.
|
||||
// The provided callback should not block or perform any expensive operations
|
||||
// or call other methods of the v2Client object.
|
||||
func (v2c *v2Client) watchLDS(target string, ldsCb ldsCallbackFunc) (cancel func()) {
|
||||
return v2c.watch(&watchInfo{
|
||||
typeURL: ldsURL,
|
||||
target: []string{target},
|
||||
ldsCallback: ldsCb,
|
||||
func (v2c *v2Client) addWatch(resourceType, resourceName string) {
|
||||
v2c.sendCh.Put(&watchAction{
|
||||
typeURL: resourceType,
|
||||
remove: false,
|
||||
resource: resourceName,
|
||||
})
|
||||
}
|
||||
|
||||
// watchRDS registers an RDS watcher for the provided routeName. Updates
|
||||
// corresponding to received RDS responses will be pushed to the provided
|
||||
// callback. The caller can cancel the watch by invoking the returned cancel
|
||||
// function.
|
||||
// The provided callback should not block or perform any expensive operations
|
||||
// or call other methods of the v2Client object.
|
||||
func (v2c *v2Client) watchRDS(routeName string, rdsCb rdsCallbackFunc) (cancel func()) {
|
||||
return v2c.watch(&watchInfo{
|
||||
typeURL: rdsURL,
|
||||
target: []string{routeName},
|
||||
rdsCallback: rdsCb,
|
||||
})
|
||||
// TODO: Once a registered RDS watch is cancelled, we should send an RDS
|
||||
// request with no resources. This will let the server know that we are no
|
||||
// longer interested in this resource.
|
||||
}
|
||||
|
||||
// watchCDS registers an CDS watcher for the provided clusterName. Updates
|
||||
// corresponding to received CDS responses will be pushed to the provided
|
||||
// callback. The caller can cancel the watch by invoking the returned cancel
|
||||
// function.
|
||||
// The provided callback should not block or perform any expensive operations
|
||||
// or call other methods of the v2Client object.
|
||||
func (v2c *v2Client) watchCDS(clusterName string, cdsCb cdsCallbackFunc) (cancel func()) {
|
||||
return v2c.watch(&watchInfo{
|
||||
typeURL: cdsURL,
|
||||
target: []string{clusterName},
|
||||
cdsCallback: cdsCb,
|
||||
func (v2c *v2Client) removeWatch(resourceType, resourceName string) {
|
||||
v2c.sendCh.Put(&watchAction{
|
||||
typeURL: resourceType,
|
||||
remove: true,
|
||||
resource: resourceName,
|
||||
})
|
||||
}
|
||||
|
||||
// watchEDS registers an EDS watcher for the provided clusterName. Updates
|
||||
// corresponding to received EDS responses will be pushed to the provided
|
||||
// callback. The caller can cancel the watch by invoking the returned cancel
|
||||
// function.
|
||||
// The provided callback should not block or perform any expensive operations
|
||||
// or call other methods of the v2Client object.
|
||||
func (v2c *v2Client) watchEDS(clusterName string, edsCb edsCallbackFunc) (cancel func()) {
|
||||
return v2c.watch(&watchInfo{
|
||||
typeURL: edsURL,
|
||||
target: []string{clusterName},
|
||||
edsCallback: edsCb,
|
||||
})
|
||||
// TODO: Once a registered EDS watch is cancelled, we should send an EDS
|
||||
// request with no resources. This will let the server know that we are no
|
||||
// longer interested in this resource.
|
||||
}
|
||||
|
||||
func (v2c *v2Client) watch(wi *watchInfo) (cancel func()) {
|
||||
v2c.sendCh.Put(wi)
|
||||
v2c.logger.Infof("Sending ADS request for new watch of type: %v, resource names: %v", wi.typeURL, wi.target)
|
||||
return func() {
|
||||
v2c.mu.Lock()
|
||||
defer v2c.mu.Unlock()
|
||||
if wi.state == watchEnqueued {
|
||||
wi.state = watchCancelled
|
||||
return
|
||||
}
|
||||
v2c.watchMap[wi.typeURL].cancel()
|
||||
delete(v2c.watchMap, wi.typeURL)
|
||||
// TODO: should we reset ack version string when cancelling the watch?
|
||||
}
|
||||
}
|
||||
|
||||
// checkCacheAndUpdateWatchMap is called when a new watch call is handled in
|
||||
// send(). If an existing watcher is found, its expiry timer is stopped. If the
|
||||
// watchInfo to be added to the watchMap is found in the cache, the watcher
|
||||
// callback is immediately invoked.
|
||||
//
|
||||
// Caller should hold v2c.mu
|
||||
func (v2c *v2Client) checkCacheAndUpdateWatchMap(wi *watchInfo) {
|
||||
if existing := v2c.watchMap[wi.typeURL]; existing != nil {
|
||||
existing.cancel()
|
||||
}
|
||||
|
||||
v2c.watchMap[wi.typeURL] = wi
|
||||
switch wi.typeURL {
|
||||
// We need to grab the lock inside of the expiryTimer's afterFunc because
|
||||
// we need to access the watchInfo, which is stored in the watchMap.
|
||||
case ldsURL:
|
||||
wi.expiryTimer = time.AfterFunc(defaultWatchExpiryTimeout, func() {
|
||||
v2c.mu.Lock()
|
||||
wi.ldsCallback(ldsUpdate{}, fmt.Errorf("xds: LDS target %s not found, watcher timeout", wi.target))
|
||||
v2c.mu.Unlock()
|
||||
})
|
||||
case rdsURL:
|
||||
routeName := wi.target[0]
|
||||
if cluster := v2c.rdsCache[routeName]; cluster != "" {
|
||||
var err error
|
||||
if v2c.watchMap[ldsURL] == nil {
|
||||
cluster = ""
|
||||
err = fmt.Errorf("xds: no LDS watcher found when handling RDS watch for route {%v} from cache", routeName)
|
||||
}
|
||||
v2c.logger.Infof("Resource with name %v, type %v found in cache", routeName, wi.typeURL)
|
||||
wi.rdsCallback(rdsUpdate{clusterName: cluster}, err)
|
||||
return
|
||||
}
|
||||
// Add the watch expiry timer only for new watches we don't find in
|
||||
// the cache, and return from here.
|
||||
wi.expiryTimer = time.AfterFunc(defaultWatchExpiryTimeout, func() {
|
||||
v2c.mu.Lock()
|
||||
wi.rdsCallback(rdsUpdate{}, fmt.Errorf("xds: RDS target %s not found, watcher timeout", wi.target))
|
||||
v2c.mu.Unlock()
|
||||
})
|
||||
case cdsURL:
|
||||
clusterName := wi.target[0]
|
||||
if update, ok := v2c.cdsCache[clusterName]; ok {
|
||||
var err error
|
||||
if v2c.watchMap[cdsURL] == nil {
|
||||
err = fmt.Errorf("xds: no CDS watcher found when handling CDS watch for cluster {%v} from cache", clusterName)
|
||||
}
|
||||
v2c.logger.Infof("Resource with name %v, type %v found in cache", clusterName, wi.typeURL)
|
||||
wi.cdsCallback(update, err)
|
||||
return
|
||||
}
|
||||
wi.expiryTimer = time.AfterFunc(defaultWatchExpiryTimeout, func() {
|
||||
v2c.mu.Lock()
|
||||
wi.cdsCallback(CDSUpdate{}, fmt.Errorf("xds: CDS target %s not found, watcher timeout", wi.target))
|
||||
v2c.mu.Unlock()
|
||||
})
|
||||
case edsURL:
|
||||
wi.expiryTimer = time.AfterFunc(defaultWatchExpiryTimeout, func() {
|
||||
v2c.mu.Lock()
|
||||
wi.edsCallback(nil, fmt.Errorf("xds: EDS target %s not found, watcher timeout", wi.target))
|
||||
v2c.mu.Unlock()
|
||||
})
|
||||
func mapToSlice(m map[string]bool) (ret []string) {
|
||||
for i := range m {
|
||||
ret = append(ret, i)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
@ -27,10 +27,43 @@ import (
|
||||
"github.com/golang/protobuf/proto"
|
||||
anypb "github.com/golang/protobuf/ptypes/any"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/testutils/fakeserver"
|
||||
)
|
||||
|
||||
func startXDSV2Client(t *testing.T, cc *grpc.ClientConn) (v2c *v2Client, cbLDS, cbRDS, cbCDS, cbEDS *testutils.Channel, cleanup func()) {
|
||||
cbLDS = testutils.NewChannel()
|
||||
cbRDS = testutils.NewChannel()
|
||||
cbCDS = testutils.NewChannel()
|
||||
cbEDS = testutils.NewChannel()
|
||||
v2c = newV2Client(&testUpdateReceiver{
|
||||
f: func(typeURL string, d map[string]interface{}) {
|
||||
t.Logf("Received %s callback with {%+v}", typeURL, d)
|
||||
switch typeURL {
|
||||
case ldsURL:
|
||||
if _, ok := d[goodLDSTarget1]; ok {
|
||||
cbLDS.Send(struct{}{})
|
||||
}
|
||||
case rdsURL:
|
||||
if _, ok := d[goodRouteName1]; ok {
|
||||
cbRDS.Send(struct{}{})
|
||||
}
|
||||
case cdsURL:
|
||||
if _, ok := d[goodClusterName1]; ok {
|
||||
cbCDS.Send(struct{}{})
|
||||
}
|
||||
case edsURL:
|
||||
if _, ok := d[goodEDSName]; ok {
|
||||
cbEDS.Send(struct{}{})
|
||||
}
|
||||
}
|
||||
},
|
||||
}, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
t.Log("Started xds v2Client...")
|
||||
return v2c, cbLDS, cbRDS, cbCDS, cbEDS, v2c.close
|
||||
}
|
||||
|
||||
// compareXDSRequest reads requests from channel, compare it with want.
|
||||
func compareXDSRequest(ch *testutils.Channel, want *xdspb.DiscoveryRequest, version, nonce string) error {
|
||||
val, err := ch.Receive()
|
||||
@ -61,36 +94,30 @@ func sendXDSRespWithVersion(ch chan<- *fakeserver.Response, respWithoutVersion *
|
||||
|
||||
// startXDS calls watch to send the first request. It then sends a good response
|
||||
// and checks for ack.
|
||||
func startXDS(t *testing.T, xdsname string, v2c *v2Client, reqChan *testutils.Channel, req *xdspb.DiscoveryRequest, preVersion string, preNonce string) *testutils.Channel {
|
||||
callbackCh := testutils.NewChannel()
|
||||
func startXDS(t *testing.T, xdsname string, v2c *v2Client, reqChan *testutils.Channel, req *xdspb.DiscoveryRequest, preVersion string, preNonce string) {
|
||||
var (
|
||||
nameToWatch, typeURLToWatch string
|
||||
)
|
||||
switch xdsname {
|
||||
case "LDS":
|
||||
v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
|
||||
t.Logf("Received %s callback with ldsUpdate {%+v} and error {%v}", xdsname, u, err)
|
||||
callbackCh.Send(struct{}{})
|
||||
})
|
||||
typeURLToWatch = ldsURL
|
||||
nameToWatch = goodLDSTarget1
|
||||
case "RDS":
|
||||
v2c.watchRDS(goodRouteName1, func(u rdsUpdate, err error) {
|
||||
t.Logf("Received %s callback with ldsUpdate {%+v} and error {%v}", xdsname, u, err)
|
||||
callbackCh.Send(struct{}{})
|
||||
})
|
||||
typeURLToWatch = rdsURL
|
||||
nameToWatch = goodRouteName1
|
||||
case "CDS":
|
||||
v2c.watchCDS(goodClusterName1, func(u CDSUpdate, err error) {
|
||||
t.Logf("Received %s callback with ldsUpdate {%+v} and error {%v}", xdsname, u, err)
|
||||
callbackCh.Send(struct{}{})
|
||||
})
|
||||
typeURLToWatch = cdsURL
|
||||
nameToWatch = goodClusterName1
|
||||
case "EDS":
|
||||
v2c.watchEDS(goodEDSName, func(u *EDSUpdate, err error) {
|
||||
t.Logf("Received %s callback with ldsUpdate {%+v} and error {%v}", xdsname, u, err)
|
||||
callbackCh.Send(struct{}{})
|
||||
})
|
||||
typeURLToWatch = edsURL
|
||||
nameToWatch = goodEDSName
|
||||
}
|
||||
v2c.addWatch(typeURLToWatch, nameToWatch)
|
||||
|
||||
if err := compareXDSRequest(reqChan, req, preVersion, preNonce); err != nil {
|
||||
t.Fatalf("Failed to receive %s request: %v", xdsname, err)
|
||||
}
|
||||
t.Logf("FakeServer received %s request...", xdsname)
|
||||
return callbackCh
|
||||
}
|
||||
|
||||
// sendGoodResp sends the good response, with the given version, and a random
|
||||
@ -107,12 +134,12 @@ func sendGoodResp(t *testing.T, xdsname string, fakeServer *fakeserver.Server, v
|
||||
t.Logf("Good %s response pushed to fakeServer...", xdsname)
|
||||
|
||||
if err := compareXDSRequest(fakeServer.XDSRequestChan, wantReq, strconv.Itoa(version), nonce); err != nil {
|
||||
t.Errorf("Failed to receive %s request: %v", xdsname, err)
|
||||
t.Fatalf("Failed to receive %s request: %v", xdsname, err)
|
||||
}
|
||||
t.Logf("Good %s response acked", xdsname)
|
||||
|
||||
if _, err := callbackCh.Receive(); err != nil {
|
||||
t.Errorf("Timeout when expecting %s update", xdsname)
|
||||
t.Fatalf("Timeout when expecting %s update", xdsname)
|
||||
}
|
||||
t.Logf("Good %s response callback executed", xdsname)
|
||||
return
|
||||
@ -140,7 +167,7 @@ func sendBadResp(t *testing.T, xdsname string, fakeServer *fakeserver.Server, ve
|
||||
}, version)
|
||||
t.Logf("Bad %s response pushed to fakeServer...", xdsname)
|
||||
if err := compareXDSRequest(fakeServer.XDSRequestChan, wantReq, strconv.Itoa(version-1), nonce); err != nil {
|
||||
t.Errorf("Failed to receive %s request: %v", xdsname, err)
|
||||
t.Fatalf("Failed to receive %s request: %v", xdsname, err)
|
||||
}
|
||||
t.Logf("Bad %s response nacked", xdsname)
|
||||
}
|
||||
@ -159,21 +186,21 @@ func (s) TestV2ClientAck(t *testing.T) {
|
||||
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
|
||||
v2c, cbLDS, cbRDS, cbCDS, cbEDS, v2cCleanup := startXDSV2Client(t, cc)
|
||||
defer v2cCleanup()
|
||||
|
||||
// Start the watch, send a good response, and check for ack.
|
||||
cbLDS := startXDS(t, "LDS", v2c, fakeServer.XDSRequestChan, goodLDSRequest, "", "")
|
||||
startXDS(t, "LDS", v2c, fakeServer.XDSRequestChan, goodLDSRequest, "", "")
|
||||
sendGoodResp(t, "LDS", fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS)
|
||||
versionLDS++
|
||||
cbRDS := startXDS(t, "RDS", v2c, fakeServer.XDSRequestChan, goodRDSRequest, "", "")
|
||||
startXDS(t, "RDS", v2c, fakeServer.XDSRequestChan, goodRDSRequest, "", "")
|
||||
sendGoodResp(t, "RDS", fakeServer, versionRDS, goodRDSResponse1, goodRDSRequest, cbRDS)
|
||||
versionRDS++
|
||||
cbCDS := startXDS(t, "CDS", v2c, fakeServer.XDSRequestChan, goodCDSRequest, "", "")
|
||||
startXDS(t, "CDS", v2c, fakeServer.XDSRequestChan, goodCDSRequest, "", "")
|
||||
sendGoodResp(t, "CDS", fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS)
|
||||
versionCDS++
|
||||
cbEDS := startXDS(t, "EDS", v2c, fakeServer.XDSRequestChan, goodEDSRequest, "", "")
|
||||
startXDS(t, "EDS", v2c, fakeServer.XDSRequestChan, goodEDSRequest, "", "")
|
||||
sendGoodResp(t, "EDS", fakeServer, versionEDS, goodEDSResponse1, goodEDSRequest, cbEDS)
|
||||
versionEDS++
|
||||
|
||||
@ -205,12 +232,12 @@ func (s) TestV2ClientAckFirstIsNack(t *testing.T) {
|
||||
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
|
||||
v2c, cbLDS, _, _, _, v2cCleanup := startXDSV2Client(t, cc)
|
||||
defer v2cCleanup()
|
||||
|
||||
// Start the watch, send a good response, and check for ack.
|
||||
cbLDS := startXDS(t, "LDS", v2c, fakeServer.XDSRequestChan, goodLDSRequest, "", "")
|
||||
startXDS(t, "LDS", v2c, fakeServer.XDSRequestChan, goodLDSRequest, "", "")
|
||||
|
||||
nonce := sendXDSRespWithVersion(fakeServer.XDSResponseChan, &xdspb.DiscoveryResponse{
|
||||
Resources: []*anypb.Any{{}},
|
||||
@ -237,17 +264,17 @@ func (s) TestV2ClientAckNackAfterNewWatch(t *testing.T) {
|
||||
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
|
||||
v2c, cbLDS, _, _, _, v2cCleanup := startXDSV2Client(t, cc)
|
||||
defer v2cCleanup()
|
||||
|
||||
// Start the watch, send a good response, and check for ack.
|
||||
cbLDS := startXDS(t, "LDS", v2c, fakeServer.XDSRequestChan, goodLDSRequest, "", "")
|
||||
startXDS(t, "LDS", v2c, fakeServer.XDSRequestChan, goodLDSRequest, "", "")
|
||||
nonce := sendGoodResp(t, "LDS", fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS)
|
||||
|
||||
// Start a new watch. The version in the new request should be the version
|
||||
// from the previous response, thus versionLDS before ++.
|
||||
cbLDS = startXDS(t, "LDS", v2c, fakeServer.XDSRequestChan, goodLDSRequest, strconv.Itoa(versionLDS), nonce)
|
||||
startXDS(t, "LDS", v2c, fakeServer.XDSRequestChan, goodLDSRequest, strconv.Itoa(versionLDS), nonce)
|
||||
versionLDS++
|
||||
|
||||
// This is an invalid response after the new watch.
|
||||
@ -275,16 +302,12 @@ func (s) TestV2ClientAckNewWatchAfterCancel(t *testing.T) {
|
||||
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
|
||||
v2c, _, _, cbCDS, _, v2cCleanup := startXDSV2Client(t, cc)
|
||||
defer v2cCleanup()
|
||||
|
||||
// Start a CDS watch.
|
||||
callbackCh := testutils.NewChannel()
|
||||
cancel := v2c.watchCDS(goodClusterName1, func(u CDSUpdate, err error) {
|
||||
t.Logf("Received %s callback with ldsUpdate {%+v} and error {%v}", "CDS", u, err)
|
||||
callbackCh.Send(struct{}{})
|
||||
})
|
||||
v2c.addWatch(cdsURL, goodClusterName1)
|
||||
if err := compareXDSRequest(fakeServer.XDSRequestChan, goodCDSRequest, "", ""); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
@ -292,15 +315,19 @@ func (s) TestV2ClientAckNewWatchAfterCancel(t *testing.T) {
|
||||
|
||||
// Send a good CDS response, this function waits for the ACK with the right
|
||||
// version.
|
||||
nonce := sendGoodResp(t, "CDS", fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, callbackCh)
|
||||
nonce := sendGoodResp(t, "CDS", fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS)
|
||||
|
||||
// Cancel the CDS watch, and start a new one. The new watch should have the
|
||||
// version from the response above.
|
||||
cancel()
|
||||
v2c.watchCDS(goodClusterName1, func(u CDSUpdate, err error) {
|
||||
t.Logf("Received %s callback with ldsUpdate {%+v} and error {%v}", "CDS", u, err)
|
||||
callbackCh.Send(struct{}{})
|
||||
})
|
||||
v2c.removeWatch(cdsURL, goodClusterName1)
|
||||
// Wait for a request with no resource names, because the only watch was
|
||||
// removed.
|
||||
emptyReq := &xdspb.DiscoveryRequest{Node: goodNodeProto, TypeUrl: cdsURL}
|
||||
if err := compareXDSRequest(fakeServer.XDSRequestChan, emptyReq, strconv.Itoa(versionCDS), nonce); err != nil {
|
||||
t.Fatalf("Failed to receive %s request: %v", "CDS", err)
|
||||
}
|
||||
v2c.addWatch(cdsURL, goodClusterName1)
|
||||
// Wait for a request with correct resource names and version.
|
||||
if err := compareXDSRequest(fakeServer.XDSRequestChan, goodCDSRequest, strconv.Itoa(versionCDS), nonce); err != nil {
|
||||
t.Fatalf("Failed to receive %s request: %v", "CDS", err)
|
||||
}
|
||||
@ -311,7 +338,7 @@ func (s) TestV2ClientAckNewWatchAfterCancel(t *testing.T) {
|
||||
versionCDS++
|
||||
|
||||
// send another good response, and check for ack, with the new version.
|
||||
sendGoodResp(t, "CDS", fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, callbackCh)
|
||||
sendGoodResp(t, "CDS", fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS)
|
||||
versionCDS++
|
||||
}
|
||||
|
||||
@ -324,44 +351,51 @@ func (s) TestV2ClientAckCancelResponseRace(t *testing.T) {
|
||||
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
|
||||
v2c, _, _, cbCDS, _, v2cCleanup := startXDSV2Client(t, cc)
|
||||
defer v2cCleanup()
|
||||
|
||||
// Start a CDS watch.
|
||||
callbackCh := testutils.NewChannel()
|
||||
cancel := v2c.watchCDS(goodClusterName1, func(u CDSUpdate, err error) {
|
||||
t.Logf("Received %s callback with ldsUpdate {%+v} and error {%v}", "CDS", u, err)
|
||||
callbackCh.Send(struct{}{})
|
||||
})
|
||||
v2c.addWatch(cdsURL, goodClusterName1)
|
||||
if err := compareXDSRequest(fakeServer.XDSRequestChan, goodCDSRequest, "", ""); err != nil {
|
||||
t.Fatalf("Failed to receive %s request: %v", "CDS", err)
|
||||
}
|
||||
t.Logf("FakeServer received %s request...", "CDS")
|
||||
|
||||
// send another good response, and check for ack, with the new version.
|
||||
sendGoodResp(t, "CDS", fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, callbackCh)
|
||||
versionCDS++
|
||||
// send a good response, and check for ack, with the new version.
|
||||
nonce := sendGoodResp(t, "CDS", fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS)
|
||||
|
||||
// Cancel the watch before the next response is sent. This mimics the case
|
||||
// watch is canceled while response is on wire.
|
||||
cancel()
|
||||
v2c.removeWatch(cdsURL, goodClusterName1)
|
||||
// Wait for a request with no resource names, because the only watch was
|
||||
// removed.
|
||||
emptyReq := &xdspb.DiscoveryRequest{Node: goodNodeProto, TypeUrl: cdsURL}
|
||||
if err := compareXDSRequest(fakeServer.XDSRequestChan, emptyReq, strconv.Itoa(versionCDS), nonce); err != nil {
|
||||
t.Fatalf("Failed to receive %s request: %v", "CDS", err)
|
||||
}
|
||||
versionCDS++
|
||||
|
||||
if req, err := fakeServer.XDSRequestChan.Receive(); err != testutils.ErrRecvTimeout {
|
||||
t.Fatalf("Got unexpected xds request after watch is canceled: %v", req)
|
||||
}
|
||||
|
||||
// Send a good response.
|
||||
nonce := sendXDSRespWithVersion(fakeServer.XDSResponseChan, goodCDSResponse1, versionCDS)
|
||||
nonce = sendXDSRespWithVersion(fakeServer.XDSResponseChan, goodCDSResponse1, versionCDS)
|
||||
t.Logf("Good %s response pushed to fakeServer...", "CDS")
|
||||
|
||||
// Expect no ACK because watch was canceled.
|
||||
if req, err := fakeServer.XDSRequestChan.Receive(); err != testutils.ErrRecvTimeout {
|
||||
t.Fatalf("Got unexpected xds request after watch is canceled: %v", req)
|
||||
}
|
||||
// Still expected an callback update, because response was good.
|
||||
if _, err := cbCDS.Receive(); err != nil {
|
||||
t.Fatalf("Timeout when expecting %s update", "CDS")
|
||||
}
|
||||
|
||||
// Start a new watch. The new watch should have the nonce from the response
|
||||
// above, and version from the first good response.
|
||||
v2c.watchCDS(goodClusterName1, func(u CDSUpdate, err error) {
|
||||
t.Logf("Received %s callback with ldsUpdate {%+v} and error {%v}", "CDS", u, err)
|
||||
callbackCh.Send(struct{}{})
|
||||
})
|
||||
v2c.addWatch(cdsURL, goodClusterName1)
|
||||
if err := compareXDSRequest(fakeServer.XDSRequestChan, goodCDSRequest, strconv.Itoa(versionCDS-1), nonce); err != nil {
|
||||
t.Fatalf("Failed to receive %s request: %v", "CDS", err)
|
||||
}
|
||||
@ -371,6 +405,6 @@ func (s) TestV2ClientAckCancelResponseRace(t *testing.T) {
|
||||
versionCDS++
|
||||
|
||||
// send another good response, and check for ack, with the new version.
|
||||
sendGoodResp(t, "CDS", fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, callbackCh)
|
||||
sendGoodResp(t, "CDS", fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS)
|
||||
versionCDS++
|
||||
}
|
||||
|
@ -28,16 +28,7 @@ import (
|
||||
// handleCDSResponse processes an CDS response received from the xDS server. On
|
||||
// receipt of a good response, it also invokes the registered watcher callback.
|
||||
func (v2c *v2Client) handleCDSResponse(resp *xdspb.DiscoveryResponse) error {
|
||||
v2c.mu.Lock()
|
||||
defer v2c.mu.Unlock()
|
||||
|
||||
wi := v2c.watchMap[cdsURL]
|
||||
if wi == nil {
|
||||
return fmt.Errorf("xds: no CDS watcher found when handling CDS response: %+v", resp)
|
||||
}
|
||||
|
||||
var returnUpdate CDSUpdate
|
||||
localCache := make(map[string]CDSUpdate)
|
||||
returnUpdate := make(map[string]ClusterUpdate)
|
||||
for _, r := range resp.GetResources() {
|
||||
var resource ptypes.DynamicAny
|
||||
if err := ptypes.UnmarshalAny(r, &resource); err != nil {
|
||||
@ -58,25 +49,16 @@ func (v2c *v2Client) handleCDSResponse(resp *xdspb.DiscoveryResponse) error {
|
||||
if update.ServiceName == "" {
|
||||
update.ServiceName = cluster.GetName()
|
||||
}
|
||||
localCache[cluster.GetName()] = update
|
||||
v2c.logger.Debugf("Resource with name %v, type %T, value %+v added to cache", cluster.GetName(), update, update)
|
||||
if cluster.GetName() == wi.target[0] {
|
||||
returnUpdate = update
|
||||
}
|
||||
returnUpdate[cluster.GetName()] = update
|
||||
}
|
||||
v2c.cdsCache = localCache
|
||||
|
||||
var err error
|
||||
if returnUpdate.ServiceName == "" {
|
||||
err = fmt.Errorf("xds: CDS target %s not found in received response %+v", wi.target, resp)
|
||||
}
|
||||
wi.stopTimer()
|
||||
wi.cdsCallback(returnUpdate, err)
|
||||
v2c.parent.newCDSUpdate(returnUpdate)
|
||||
return nil
|
||||
}
|
||||
|
||||
func validateCluster(cluster *xdspb.Cluster) (CDSUpdate, error) {
|
||||
emptyUpdate := CDSUpdate{ServiceName: "", EnableLRS: false}
|
||||
func validateCluster(cluster *xdspb.Cluster) (ClusterUpdate, error) {
|
||||
emptyUpdate := ClusterUpdate{ServiceName: "", EnableLRS: false}
|
||||
switch {
|
||||
case cluster.GetType() != xdspb.Cluster_EDS:
|
||||
return emptyUpdate, fmt.Errorf("xds: unexpected cluster type %v in response: %+v", cluster.GetType(), cluster)
|
||||
@ -86,7 +68,7 @@ func validateCluster(cluster *xdspb.Cluster) (CDSUpdate, error) {
|
||||
return emptyUpdate, fmt.Errorf("xds: unexpected lbPolicy %v in response: %+v", cluster.GetLbPolicy(), cluster)
|
||||
}
|
||||
|
||||
return CDSUpdate{
|
||||
return ClusterUpdate{
|
||||
ServiceName: cluster.GetEdsClusterConfig().GetServiceName(),
|
||||
EnableLRS: cluster.GetLrsServer().GetSelf() != nil,
|
||||
}, nil
|
295
xds/internal/client/v2client_cds_test.go
Normal file
295
xds/internal/client/v2client_cds_test.go
Normal file
@ -0,0 +1,295 @@
|
||||
/*
|
||||
*
|
||||
* Copyright 2019 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 client
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
|
||||
corepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
|
||||
"github.com/golang/protobuf/proto"
|
||||
anypb "github.com/golang/protobuf/ptypes/any"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
)
|
||||
|
||||
const (
|
||||
serviceName1 = "foo-service"
|
||||
serviceName2 = "bar-service"
|
||||
)
|
||||
|
||||
func (s) TestValidateCluster(t *testing.T) {
|
||||
emptyUpdate := ClusterUpdate{ServiceName: "", EnableLRS: false}
|
||||
tests := []struct {
|
||||
name string
|
||||
cluster *xdspb.Cluster
|
||||
wantUpdate ClusterUpdate
|
||||
wantErr bool
|
||||
}{
|
||||
{
|
||||
name: "non-eds-cluster-type",
|
||||
cluster: &xdspb.Cluster{
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_STATIC},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{
|
||||
EdsConfig: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Ads{
|
||||
Ads: &corepb.AggregatedConfigSource{},
|
||||
},
|
||||
},
|
||||
},
|
||||
LbPolicy: xdspb.Cluster_LEAST_REQUEST,
|
||||
},
|
||||
wantUpdate: emptyUpdate,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "no-eds-config",
|
||||
cluster: &xdspb.Cluster{
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
LbPolicy: xdspb.Cluster_ROUND_ROBIN,
|
||||
},
|
||||
wantUpdate: emptyUpdate,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "no-ads-config-source",
|
||||
cluster: &xdspb.Cluster{
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{},
|
||||
LbPolicy: xdspb.Cluster_ROUND_ROBIN,
|
||||
},
|
||||
wantUpdate: emptyUpdate,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "non-round-robin-lb-policy",
|
||||
cluster: &xdspb.Cluster{
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{
|
||||
EdsConfig: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Ads{
|
||||
Ads: &corepb.AggregatedConfigSource{},
|
||||
},
|
||||
},
|
||||
},
|
||||
LbPolicy: xdspb.Cluster_LEAST_REQUEST,
|
||||
},
|
||||
wantUpdate: emptyUpdate,
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
name: "happy-case-no-service-name-no-lrs",
|
||||
cluster: &xdspb.Cluster{
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{
|
||||
EdsConfig: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Ads{
|
||||
Ads: &corepb.AggregatedConfigSource{},
|
||||
},
|
||||
},
|
||||
},
|
||||
LbPolicy: xdspb.Cluster_ROUND_ROBIN,
|
||||
},
|
||||
wantUpdate: emptyUpdate,
|
||||
},
|
||||
{
|
||||
name: "happy-case-no-lrs",
|
||||
cluster: &xdspb.Cluster{
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{
|
||||
EdsConfig: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Ads{
|
||||
Ads: &corepb.AggregatedConfigSource{},
|
||||
},
|
||||
},
|
||||
ServiceName: serviceName1,
|
||||
},
|
||||
LbPolicy: xdspb.Cluster_ROUND_ROBIN,
|
||||
},
|
||||
wantUpdate: ClusterUpdate{ServiceName: serviceName1, EnableLRS: false},
|
||||
},
|
||||
{
|
||||
name: "happiest-case",
|
||||
cluster: goodCluster1,
|
||||
wantUpdate: ClusterUpdate{ServiceName: serviceName1, EnableLRS: true},
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
gotUpdate, gotErr := validateCluster(test.cluster)
|
||||
if (gotErr != nil) != test.wantErr {
|
||||
t.Errorf("validateCluster(%+v) returned error: %v, wantErr: %v", test.cluster, gotErr, test.wantErr)
|
||||
}
|
||||
if !cmp.Equal(gotUpdate, test.wantUpdate) {
|
||||
t.Errorf("validateCluster(%+v) = %v, want: %v", test.cluster, gotUpdate, test.wantUpdate)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// TestCDSHandleResponse starts a fake xDS server, makes a ClientConn to it,
|
||||
// and creates a v2Client using it. Then, it registers a CDS watcher and tests
|
||||
// different CDS responses.
|
||||
func (s) TestCDSHandleResponse(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
cdsResponse *xdspb.DiscoveryResponse
|
||||
wantErr bool
|
||||
wantUpdate *ClusterUpdate
|
||||
wantUpdateErr bool
|
||||
}{
|
||||
// Badly marshaled CDS response.
|
||||
{
|
||||
name: "badly-marshaled-response",
|
||||
cdsResponse: badlyMarshaledCDSResponse,
|
||||
wantErr: true,
|
||||
wantUpdate: nil,
|
||||
wantUpdateErr: false,
|
||||
},
|
||||
// Response does not contain Cluster proto.
|
||||
{
|
||||
name: "no-cluster-proto-in-response",
|
||||
cdsResponse: badResourceTypeInLDSResponse,
|
||||
wantErr: true,
|
||||
wantUpdate: nil,
|
||||
wantUpdateErr: false,
|
||||
},
|
||||
// Response contains no clusters.
|
||||
{
|
||||
name: "no-cluster",
|
||||
cdsResponse: &xdspb.DiscoveryResponse{},
|
||||
wantErr: false,
|
||||
wantUpdate: nil,
|
||||
wantUpdateErr: false,
|
||||
},
|
||||
// Response contains one good cluster we are not interested in.
|
||||
{
|
||||
name: "one-uninteresting-cluster",
|
||||
cdsResponse: goodCDSResponse2,
|
||||
wantErr: false,
|
||||
wantUpdate: nil,
|
||||
wantUpdateErr: false,
|
||||
},
|
||||
// Response contains one cluster and it is good.
|
||||
{
|
||||
name: "one-good-cluster",
|
||||
cdsResponse: goodCDSResponse1,
|
||||
wantErr: false,
|
||||
wantUpdate: &ClusterUpdate{ServiceName: serviceName1, EnableLRS: true},
|
||||
wantUpdateErr: false,
|
||||
},
|
||||
}
|
||||
for _, test := range tests {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
testWatchHandle(t, &watchHandleTestcase{
|
||||
typeURL: cdsURL,
|
||||
resourceName: goodClusterName1,
|
||||
|
||||
responseToHandle: test.cdsResponse,
|
||||
wantHandleErr: test.wantErr,
|
||||
wantUpdate: test.wantUpdate,
|
||||
wantUpdateErr: test.wantUpdateErr,
|
||||
})
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// TestCDSHandleResponseWithoutWatch tests the case where the v2Client receives
|
||||
// a CDS response without a registered watcher.
|
||||
func (s) TestCDSHandleResponseWithoutWatch(t *testing.T) {
|
||||
_, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(&testUpdateReceiver{
|
||||
f: func(string, map[string]interface{}) {},
|
||||
}, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
|
||||
if v2c.handleCDSResponse(badResourceTypeInLDSResponse) == nil {
|
||||
t.Fatal("v2c.handleCDSResponse() succeeded, should have failed")
|
||||
}
|
||||
|
||||
if v2c.handleCDSResponse(goodCDSResponse1) != nil {
|
||||
t.Fatal("v2c.handleCDSResponse() succeeded, should have failed")
|
||||
}
|
||||
}
|
||||
|
||||
var (
|
||||
badlyMarshaledCDSResponse = &xdspb.DiscoveryResponse{
|
||||
Resources: []*anypb.Any{
|
||||
{
|
||||
TypeUrl: cdsURL,
|
||||
Value: []byte{1, 2, 3, 4},
|
||||
},
|
||||
},
|
||||
TypeUrl: cdsURL,
|
||||
}
|
||||
goodCluster1 = &xdspb.Cluster{
|
||||
Name: goodClusterName1,
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{
|
||||
EdsConfig: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Ads{
|
||||
Ads: &corepb.AggregatedConfigSource{},
|
||||
},
|
||||
},
|
||||
ServiceName: serviceName1,
|
||||
},
|
||||
LbPolicy: xdspb.Cluster_ROUND_ROBIN,
|
||||
LrsServer: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Self{
|
||||
Self: &corepb.SelfConfigSource{},
|
||||
},
|
||||
},
|
||||
}
|
||||
marshaledCluster1, _ = proto.Marshal(goodCluster1)
|
||||
goodCluster2 = &xdspb.Cluster{
|
||||
Name: goodClusterName2,
|
||||
ClusterDiscoveryType: &xdspb.Cluster_Type{Type: xdspb.Cluster_EDS},
|
||||
EdsClusterConfig: &xdspb.Cluster_EdsClusterConfig{
|
||||
EdsConfig: &corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &corepb.ConfigSource_Ads{
|
||||
Ads: &corepb.AggregatedConfigSource{},
|
||||
},
|
||||
},
|
||||
ServiceName: serviceName2,
|
||||
},
|
||||
LbPolicy: xdspb.Cluster_ROUND_ROBIN,
|
||||
}
|
||||
marshaledCluster2, _ = proto.Marshal(goodCluster2)
|
||||
goodCDSResponse1 = &xdspb.DiscoveryResponse{
|
||||
Resources: []*anypb.Any{
|
||||
{
|
||||
TypeUrl: cdsURL,
|
||||
Value: marshaledCluster1,
|
||||
},
|
||||
},
|
||||
TypeUrl: cdsURL,
|
||||
}
|
||||
goodCDSResponse2 = &xdspb.DiscoveryResponse{
|
||||
Resources: []*anypb.Any{
|
||||
{
|
||||
TypeUrl: cdsURL,
|
||||
Value: marshaledCluster2,
|
||||
},
|
||||
},
|
||||
TypeUrl: cdsURL,
|
||||
}
|
||||
)
|
@ -30,52 +30,6 @@ import (
|
||||
"google.golang.org/grpc/xds/internal"
|
||||
)
|
||||
|
||||
// OverloadDropConfig contains the config to drop overloads.
|
||||
type OverloadDropConfig struct {
|
||||
Category string
|
||||
Numerator uint32
|
||||
Denominator uint32
|
||||
}
|
||||
|
||||
// EndpointHealthStatus represents the health status of an endpoint.
|
||||
type EndpointHealthStatus int32
|
||||
|
||||
const (
|
||||
// EndpointHealthStatusUnknown represents HealthStatus UNKNOWN.
|
||||
EndpointHealthStatusUnknown EndpointHealthStatus = iota
|
||||
// EndpointHealthStatusHealthy represents HealthStatus HEALTHY.
|
||||
EndpointHealthStatusHealthy
|
||||
// EndpointHealthStatusUnhealthy represents HealthStatus UNHEALTHY.
|
||||
EndpointHealthStatusUnhealthy
|
||||
// EndpointHealthStatusDraining represents HealthStatus DRAINING.
|
||||
EndpointHealthStatusDraining
|
||||
// EndpointHealthStatusTimeout represents HealthStatus TIMEOUT.
|
||||
EndpointHealthStatusTimeout
|
||||
// EndpointHealthStatusDegraded represents HealthStatus DEGRADED.
|
||||
EndpointHealthStatusDegraded
|
||||
)
|
||||
|
||||
// Endpoint contains information of an endpoint.
|
||||
type Endpoint struct {
|
||||
Address string
|
||||
HealthStatus EndpointHealthStatus
|
||||
Weight uint32
|
||||
}
|
||||
|
||||
// Locality contains information of a locality.
|
||||
type Locality struct {
|
||||
Endpoints []Endpoint
|
||||
ID internal.LocalityID
|
||||
Priority uint32
|
||||
Weight uint32
|
||||
}
|
||||
|
||||
// EDSUpdate contains an EDS update.
|
||||
type EDSUpdate struct {
|
||||
Drops []OverloadDropConfig
|
||||
Localities []Locality
|
||||
}
|
||||
|
||||
func parseAddress(socketAddress *corepb.SocketAddress) string {
|
||||
return net.JoinHostPort(socketAddress.GetAddress(), strconv.Itoa(int(socketAddress.GetPortValue())))
|
||||
}
|
||||
@ -113,12 +67,12 @@ func parseEndpoints(lbEndpoints []*endpointpb.LbEndpoint) []Endpoint {
|
||||
return endpoints
|
||||
}
|
||||
|
||||
// ParseEDSRespProto turns EDS response proto message to EDSUpdate.
|
||||
// ParseEDSRespProto turns EDS response proto message to EndpointsUpdate.
|
||||
//
|
||||
// This is temporarily exported to be used in eds balancer, before it switches
|
||||
// to use xds client. TODO: unexport.
|
||||
func ParseEDSRespProto(m *xdspb.ClusterLoadAssignment) (*EDSUpdate, error) {
|
||||
ret := &EDSUpdate{}
|
||||
func ParseEDSRespProto(m *xdspb.ClusterLoadAssignment) (EndpointsUpdate, error) {
|
||||
ret := EndpointsUpdate{}
|
||||
for _, dropPolicy := range m.GetPolicy().GetDropOverloads() {
|
||||
ret.Drops = append(ret.Drops, parseDropPolicy(dropPolicy))
|
||||
}
|
||||
@ -126,7 +80,7 @@ func ParseEDSRespProto(m *xdspb.ClusterLoadAssignment) (*EDSUpdate, error) {
|
||||
for _, locality := range m.Endpoints {
|
||||
l := locality.GetLocality()
|
||||
if l == nil {
|
||||
return nil, fmt.Errorf("EDS response contains a locality without ID, locality: %+v", locality)
|
||||
return EndpointsUpdate{}, fmt.Errorf("EDS response contains a locality without ID, locality: %+v", locality)
|
||||
}
|
||||
lid := internal.LocalityID{
|
||||
Region: l.Region,
|
||||
@ -144,7 +98,7 @@ func ParseEDSRespProto(m *xdspb.ClusterLoadAssignment) (*EDSUpdate, error) {
|
||||
}
|
||||
for i := 0; i < len(priorities); i++ {
|
||||
if _, ok := priorities[uint32(i)]; !ok {
|
||||
return nil, fmt.Errorf("priority %v missing (with different priorities %v received)", i, priorities)
|
||||
return EndpointsUpdate{}, fmt.Errorf("priority %v missing (with different priorities %v received)", i, priorities)
|
||||
}
|
||||
}
|
||||
return ret, nil
|
||||
@ -153,9 +107,9 @@ func ParseEDSRespProto(m *xdspb.ClusterLoadAssignment) (*EDSUpdate, error) {
|
||||
// ParseEDSRespProtoForTesting parses EDS response, and panic if parsing fails.
|
||||
// This is used by EDS balancer tests.
|
||||
//
|
||||
// TODO: delete this. The EDS balancer tests should build an EDSUpdate directly,
|
||||
// TODO: delete this. The EDS balancer tests should build an EndpointsUpdate directly,
|
||||
// instead of building and parsing a proto message.
|
||||
func ParseEDSRespProtoForTesting(m *xdspb.ClusterLoadAssignment) *EDSUpdate {
|
||||
func ParseEDSRespProtoForTesting(m *xdspb.ClusterLoadAssignment) EndpointsUpdate {
|
||||
u, err := ParseEDSRespProto(m)
|
||||
if err != nil {
|
||||
panic(err.Error())
|
||||
@ -164,15 +118,7 @@ func ParseEDSRespProtoForTesting(m *xdspb.ClusterLoadAssignment) *EDSUpdate {
|
||||
}
|
||||
|
||||
func (v2c *v2Client) handleEDSResponse(resp *xdspb.DiscoveryResponse) error {
|
||||
v2c.mu.Lock()
|
||||
defer v2c.mu.Unlock()
|
||||
|
||||
wi := v2c.watchMap[edsURL]
|
||||
if wi == nil {
|
||||
return fmt.Errorf("xds: no EDS watcher found when handling EDS response: %+v", resp)
|
||||
}
|
||||
|
||||
var returnUpdate *EDSUpdate
|
||||
returnUpdate := make(map[string]EndpointsUpdate)
|
||||
for _, r := range resp.GetResources() {
|
||||
var resource ptypes.DynamicAny
|
||||
if err := ptypes.UnmarshalAny(r, &resource); err != nil {
|
||||
@ -184,29 +130,14 @@ func (v2c *v2Client) handleEDSResponse(resp *xdspb.DiscoveryResponse) error {
|
||||
}
|
||||
v2c.logger.Infof("Resource with name: %v, type: %T, contains: %v", cla.GetClusterName(), cla, cla)
|
||||
|
||||
if cla.GetClusterName() != wi.target[0] {
|
||||
// We won't validate the remaining resources. If one of the
|
||||
// uninteresting ones is invalid, we will still ACK the response.
|
||||
continue
|
||||
}
|
||||
|
||||
u, err := ParseEDSRespProto(cla)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
returnUpdate = u
|
||||
// Break from the loop because the request resource is found. But
|
||||
// this also means we won't validate the remaining resources. If one
|
||||
// of the uninteresting ones is invalid, we will still ACK the
|
||||
// response.
|
||||
break
|
||||
}
|
||||
|
||||
if returnUpdate != nil {
|
||||
wi.stopTimer()
|
||||
wi.edsCallback(returnUpdate, nil)
|
||||
returnUpdate[cla.GetClusterName()] = u
|
||||
}
|
||||
|
||||
v2c.parent.newEDSUpdate(returnUpdate)
|
||||
return nil
|
||||
}
|
@ -18,8 +18,6 @@
|
||||
package client
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
@ -29,14 +27,13 @@ import (
|
||||
anypb "github.com/golang/protobuf/ptypes/any"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"google.golang.org/grpc/xds/internal"
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
)
|
||||
|
||||
func (s) TestEDSParseRespProto(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
m *xdspb.ClusterLoadAssignment
|
||||
want *EDSUpdate
|
||||
want EndpointsUpdate
|
||||
wantErr bool
|
||||
}{
|
||||
{
|
||||
@ -47,7 +44,7 @@ func (s) TestEDSParseRespProto(t *testing.T) {
|
||||
clab0.AddLocality("locality-2", 1, 2, []string{"addr2:159"}, nil)
|
||||
return clab0.Build()
|
||||
}(),
|
||||
want: nil,
|
||||
want: EndpointsUpdate{},
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
@ -57,7 +54,7 @@ func (s) TestEDSParseRespProto(t *testing.T) {
|
||||
clab0.AddLocality("", 1, 0, []string{"addr1:314"}, nil)
|
||||
return clab0.Build()
|
||||
}(),
|
||||
want: nil,
|
||||
want: EndpointsUpdate{},
|
||||
wantErr: true,
|
||||
},
|
||||
{
|
||||
@ -74,7 +71,7 @@ func (s) TestEDSParseRespProto(t *testing.T) {
|
||||
})
|
||||
return clab0.Build()
|
||||
}(),
|
||||
want: &EDSUpdate{
|
||||
want: EndpointsUpdate{
|
||||
Drops: nil,
|
||||
Localities: []Locality{
|
||||
{
|
||||
@ -162,17 +159,11 @@ var (
|
||||
)
|
||||
|
||||
func (s) TestEDSHandleResponse(t *testing.T) {
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
edsResponse *xdspb.DiscoveryResponse
|
||||
wantErr bool
|
||||
wantUpdate *EDSUpdate
|
||||
wantUpdate *EndpointsUpdate
|
||||
wantUpdateErr bool
|
||||
}{
|
||||
// Any in resource is badly marshaled.
|
||||
@ -204,7 +195,7 @@ func (s) TestEDSHandleResponse(t *testing.T) {
|
||||
name: "one-good-assignment",
|
||||
edsResponse: goodEDSResponse1,
|
||||
wantErr: false,
|
||||
wantUpdate: &EDSUpdate{
|
||||
wantUpdate: &EndpointsUpdate{
|
||||
Localities: []Locality{
|
||||
{
|
||||
Endpoints: []Endpoint{{Address: "addr1:314"}},
|
||||
@ -226,14 +217,12 @@ func (s) TestEDSHandleResponse(t *testing.T) {
|
||||
for _, test := range tests {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
testWatchHandle(t, &watchHandleTestcase{
|
||||
typeURL: edsURL,
|
||||
resourceName: goodEDSName,
|
||||
responseToHandle: test.edsResponse,
|
||||
wantHandleErr: test.wantErr,
|
||||
wantUpdate: test.wantUpdate,
|
||||
wantUpdateErr: test.wantUpdateErr,
|
||||
|
||||
edsWatch: v2c.watchEDS,
|
||||
watchReqChan: fakeServer.XDSRequestChan,
|
||||
handleXDSResp: v2c.handleEDSResponse,
|
||||
})
|
||||
})
|
||||
}
|
||||
@ -245,44 +234,16 @@ func (s) TestEDSHandleResponseWithoutWatch(t *testing.T) {
|
||||
_, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
v2c := newV2Client(&testUpdateReceiver{
|
||||
f: func(string, map[string]interface{}) {},
|
||||
}, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
|
||||
if v2c.handleEDSResponse(goodEDSResponse1) == nil {
|
||||
if v2c.handleEDSResponse(badResourceTypeInEDSResponse) == nil {
|
||||
t.Fatal("v2c.handleEDSResponse() succeeded, should have failed")
|
||||
}
|
||||
|
||||
if v2c.handleEDSResponse(goodEDSResponse1) != nil {
|
||||
t.Fatal("v2c.handleEDSResponse() succeeded, should have failed")
|
||||
}
|
||||
}
|
||||
|
||||
func (s) TestEDSWatchExpiryTimer(t *testing.T) {
|
||||
oldWatchExpiryTimeout := defaultWatchExpiryTimeout
|
||||
defaultWatchExpiryTimeout = 500 * time.Millisecond
|
||||
defer func() {
|
||||
defaultWatchExpiryTimeout = oldWatchExpiryTimeout
|
||||
}()
|
||||
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
v2c.watchEDS(goodRouteName1, func(u *EDSUpdate, err error) {
|
||||
t.Logf("Received callback with edsUpdate {%+v} and error {%v}", u, err)
|
||||
if u != nil {
|
||||
callbackCh.Send(fmt.Errorf("received EDSUpdate %v in edsCallback, wanted nil", u))
|
||||
}
|
||||
if err == nil {
|
||||
callbackCh.Send(errors.New("received nil error in edsCallback"))
|
||||
}
|
||||
callbackCh.Send(nil)
|
||||
})
|
||||
|
||||
// Wait till the request makes it to the fakeServer. This ensures that
|
||||
// the watch request has been processed by the v2Client.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an CDS request")
|
||||
}
|
||||
waitForNilErr(t, callbackCh)
|
||||
}
|
@ -17,7 +17,7 @@
|
||||
|
||||
// All structs/functions in this file should be unexported. They are used in EDS
|
||||
// balancer tests now, to generate test inputs. Eventually, EDS balancer tests
|
||||
// should generate EDSUpdate directly, instead of generating and parsing the
|
||||
// should generate EndpointsUpdate directly, instead of generating and parsing the
|
||||
// proto message.
|
||||
// TODO: unexported everything in this file.
|
||||
|
@ -29,15 +29,7 @@ import (
|
||||
// handleLDSResponse processes an LDS response received from the xDS server. On
|
||||
// receipt of a good response, it also invokes the registered watcher callback.
|
||||
func (v2c *v2Client) handleLDSResponse(resp *xdspb.DiscoveryResponse) error {
|
||||
v2c.mu.Lock()
|
||||
defer v2c.mu.Unlock()
|
||||
|
||||
wi := v2c.watchMap[ldsURL]
|
||||
if wi == nil {
|
||||
return fmt.Errorf("xds: no LDS watcher found when handling LDS response: %+v", resp)
|
||||
}
|
||||
|
||||
routeName := ""
|
||||
returnUpdate := make(map[string]ldsUpdate)
|
||||
for _, r := range resp.GetResources() {
|
||||
var resource ptypes.DynamicAny
|
||||
if err := ptypes.UnmarshalAny(r, &resource); err != nil {
|
||||
@ -48,27 +40,14 @@ func (v2c *v2Client) handleLDSResponse(resp *xdspb.DiscoveryResponse) error {
|
||||
return fmt.Errorf("xds: unexpected resource type: %T in LDS response", resource.Message)
|
||||
}
|
||||
v2c.logger.Infof("Resource with name: %v, type: %T, contains: %v", lis.GetName(), lis, lis)
|
||||
if lis.GetName() != wi.target[0] {
|
||||
// We ignore listeners we are not watching for because LDS is
|
||||
// special in the sense that there is only one resource we are
|
||||
// interested in, and this resource does not change over the
|
||||
// lifetime of the v2Client. So, we don't have to cache other
|
||||
// listeners which we are not interested in.
|
||||
continue
|
||||
}
|
||||
var err error
|
||||
routeName, err = v2c.getRouteConfigNameFromListener(lis)
|
||||
routeName, err := v2c.getRouteConfigNameFromListener(lis)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
returnUpdate[lis.GetName()] = ldsUpdate{routeName: routeName}
|
||||
}
|
||||
|
||||
var err error
|
||||
if routeName == "" {
|
||||
err = fmt.Errorf("xds: LDS target %s not found in received response %+v", wi.target, resp)
|
||||
}
|
||||
wi.stopTimer()
|
||||
wi.ldsCallback(ldsUpdate{routeName: routeName}, err)
|
||||
v2c.parent.newLDSUpdate(returnUpdate)
|
||||
return nil
|
||||
}
|
||||
|
@ -19,13 +19,10 @@
|
||||
package client
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
)
|
||||
|
||||
func (s) TestLDSGetRouteConfig(t *testing.T) {
|
||||
@ -74,7 +71,7 @@ func (s) TestLDSGetRouteConfig(t *testing.T) {
|
||||
}
|
||||
_, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
v2c := newV2Client(nil, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
for _, test := range tests {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
@ -93,12 +90,6 @@ func (s) TestLDSGetRouteConfig(t *testing.T) {
|
||||
// and creates a v2Client using it. Then, it registers a watchLDS and tests
|
||||
// different LDS responses.
|
||||
func (s) TestLDSHandleResponse(t *testing.T) {
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
ldsResponse *xdspb.DiscoveryResponse
|
||||
@ -150,12 +141,13 @@ func (s) TestLDSHandleResponse(t *testing.T) {
|
||||
wantUpdateErr: false,
|
||||
},
|
||||
// Response contains two good listeners (one interesting and one
|
||||
// uninteresting), and one badly marshaled listener.
|
||||
// uninteresting), and one badly marshaled listener. This will cause a
|
||||
// nack because the uninteresting listener will still be parsed.
|
||||
{
|
||||
name: "good-bad-ugly-listeners",
|
||||
ldsResponse: goodBadUglyLDSResponse,
|
||||
wantErr: false,
|
||||
wantUpdate: &ldsUpdate{routeName: goodRouteName1},
|
||||
wantErr: true,
|
||||
wantUpdate: nil,
|
||||
wantUpdateErr: false,
|
||||
},
|
||||
// Response contains one listener, but we are not interested in it.
|
||||
@ -163,8 +155,8 @@ func (s) TestLDSHandleResponse(t *testing.T) {
|
||||
name: "one-uninteresting-listener",
|
||||
ldsResponse: goodLDSResponse2,
|
||||
wantErr: false,
|
||||
wantUpdate: &ldsUpdate{routeName: ""},
|
||||
wantUpdateErr: true,
|
||||
wantUpdate: nil,
|
||||
wantUpdateErr: false,
|
||||
},
|
||||
// Response constains no resources. This is the case where the server
|
||||
// does not know about the target we are interested in.
|
||||
@ -172,22 +164,20 @@ func (s) TestLDSHandleResponse(t *testing.T) {
|
||||
name: "empty-response",
|
||||
ldsResponse: emptyLDSResponse,
|
||||
wantErr: false,
|
||||
wantUpdate: &ldsUpdate{routeName: ""},
|
||||
wantUpdateErr: true,
|
||||
wantUpdate: nil,
|
||||
wantUpdateErr: false,
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
testWatchHandle(t, &watchHandleTestcase{
|
||||
typeURL: ldsURL,
|
||||
resourceName: goodLDSTarget1,
|
||||
responseToHandle: test.ldsResponse,
|
||||
wantHandleErr: test.wantErr,
|
||||
wantUpdate: test.wantUpdate,
|
||||
wantUpdateErr: test.wantUpdateErr,
|
||||
|
||||
ldsWatch: v2c.watchLDS,
|
||||
watchReqChan: fakeServer.XDSRequestChan,
|
||||
handleXDSResp: v2c.handleLDSResponse,
|
||||
})
|
||||
})
|
||||
}
|
||||
@ -199,46 +189,16 @@ func (s) TestLDSHandleResponseWithoutWatch(t *testing.T) {
|
||||
_, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
v2c := newV2Client(&testUpdateReceiver{
|
||||
f: func(string, map[string]interface{}) {},
|
||||
}, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
|
||||
if v2c.handleLDSResponse(goodLDSResponse1) == nil {
|
||||
if v2c.handleLDSResponse(badResourceTypeInLDSResponse) == nil {
|
||||
t.Fatal("v2c.handleLDSResponse() succeeded, should have failed")
|
||||
}
|
||||
|
||||
if v2c.handleLDSResponse(goodLDSResponse1) != nil {
|
||||
t.Fatal("v2c.handleLDSResponse() succeeded, should have failed")
|
||||
}
|
||||
}
|
||||
|
||||
// TestLDSWatchExpiryTimer tests the case where the client does not receive an
|
||||
// LDS response for the request that it sends out. We want the watch callback
|
||||
// to be invoked with an error once the watchExpiryTimer fires.
|
||||
func (s) TestLDSWatchExpiryTimer(t *testing.T) {
|
||||
oldWatchExpiryTimeout := defaultWatchExpiryTimeout
|
||||
defaultWatchExpiryTimeout = 500 * time.Millisecond
|
||||
defer func() {
|
||||
defaultWatchExpiryTimeout = oldWatchExpiryTimeout
|
||||
}()
|
||||
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
|
||||
t.Logf("in v2c.watchLDS callback, ldsUpdate: %+v, err: %v", u, err)
|
||||
if u.routeName != "" {
|
||||
callbackCh.Send(fmt.Errorf("received routeName %v in ldsCallback, wanted empty string", u.routeName))
|
||||
}
|
||||
if err == nil {
|
||||
callbackCh.Send(errors.New("received nil error in ldsCallback"))
|
||||
}
|
||||
callbackCh.Send(nil)
|
||||
})
|
||||
|
||||
// Wait till the request makes it to the fakeServer. This ensures that
|
||||
// the watch request has been processed by the v2Client.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an LDS request")
|
||||
}
|
||||
waitForNilErr(t, callbackCh)
|
||||
}
|
@ -32,20 +32,10 @@ import (
|
||||
// the registered watcher callback.
|
||||
func (v2c *v2Client) handleRDSResponse(resp *xdspb.DiscoveryResponse) error {
|
||||
v2c.mu.Lock()
|
||||
defer v2c.mu.Unlock()
|
||||
hostname := v2c.hostname
|
||||
v2c.mu.Unlock()
|
||||
|
||||
if v2c.watchMap[ldsURL] == nil {
|
||||
return fmt.Errorf("xds: unexpected RDS response when no LDS watcher is registered: %+v", resp)
|
||||
}
|
||||
target := v2c.watchMap[ldsURL].target[0]
|
||||
|
||||
wi := v2c.watchMap[rdsURL]
|
||||
if wi == nil {
|
||||
return fmt.Errorf("xds: no RDS watcher found when handling RDS response: %+v", resp)
|
||||
}
|
||||
|
||||
returnCluster := ""
|
||||
localCache := make(map[string]string)
|
||||
returnUpdate := make(map[string]rdsUpdate)
|
||||
for _, r := range resp.GetResources() {
|
||||
var resource ptypes.DynamicAny
|
||||
if err := ptypes.UnmarshalAny(r, &resource); err != nil {
|
||||
@ -55,40 +45,19 @@ func (v2c *v2Client) handleRDSResponse(resp *xdspb.DiscoveryResponse) error {
|
||||
if !ok {
|
||||
return fmt.Errorf("xds: unexpected resource type: %T in RDS response", resource.Message)
|
||||
}
|
||||
v2c.logger.Infof("Resource with name: %v, type: %T, contains: %v", rc.GetName(), rc, rc)
|
||||
cluster := getClusterFromRouteConfiguration(rc, target)
|
||||
v2c.logger.Infof("Resource with name: %v, type: %T, contains: %v. Picking routes for current watching hostname %v", rc.GetName(), rc, rc, v2c.hostname)
|
||||
|
||||
// Use the hostname (resourceName for LDS) to find the routes.
|
||||
cluster := getClusterFromRouteConfiguration(rc, hostname)
|
||||
if cluster == "" {
|
||||
return fmt.Errorf("xds: received invalid RouteConfiguration in RDS response: %+v", rc)
|
||||
}
|
||||
|
||||
// If we get here, it means that this resource was a good one.
|
||||
localCache[rc.GetName()] = cluster
|
||||
v2c.logger.Debugf("Resource with name %v, type %T, value %+v added to cache", rc.GetName(), cluster, cluster)
|
||||
|
||||
// TODO: remove cache, and only process resources that are interesting.
|
||||
if rc.GetName() == wi.target[0] {
|
||||
returnCluster = cluster
|
||||
}
|
||||
returnUpdate[rc.GetName()] = rdsUpdate{clusterName: cluster}
|
||||
}
|
||||
|
||||
// Update the cache in the v2Client only after we have confirmed that all
|
||||
// resources in the received response were good.
|
||||
for k, v := range localCache {
|
||||
// TODO: Need to handle deletion of entries from the cache based on LDS
|
||||
// watch calls. Not handling it does not affect correctness, but leads
|
||||
// to unnecessary memory consumption.
|
||||
v2c.rdsCache[k] = v
|
||||
}
|
||||
|
||||
if returnCluster != "" {
|
||||
// We stop the expiry timer and invoke the callback only when we have
|
||||
// received the resource that we are watching for. Since RDS is an
|
||||
// incremental protocol, the fact that we did not receive the resource
|
||||
// that we are watching for in this response does not mean that the
|
||||
// server does not know about it.
|
||||
wi.stopTimer()
|
||||
wi.rdsCallback(rdsUpdate{clusterName: returnCluster}, nil)
|
||||
}
|
||||
v2c.parent.newRDSUpdate(returnUpdate)
|
||||
return nil
|
||||
}
|
||||
|
@ -19,31 +19,16 @@
|
||||
package client
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
discoverypb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
|
||||
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
|
||||
routepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/route"
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/testutils/fakeserver"
|
||||
)
|
||||
|
||||
func (v2c *v2Client) cloneRDSCacheForTesting() map[string]string {
|
||||
v2c.mu.Lock()
|
||||
defer v2c.mu.Unlock()
|
||||
|
||||
cloneCache := make(map[string]string)
|
||||
for k, v := range v2c.rdsCache {
|
||||
cloneCache[k] = v
|
||||
}
|
||||
return cloneCache
|
||||
}
|
||||
|
||||
func (s) TestRDSGetClusterFromRouteConfiguration(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
@ -169,38 +154,16 @@ func (s) TestRDSGetClusterFromRouteConfiguration(t *testing.T) {
|
||||
// pre-requirement for RDS, and RDS handle would fail without an existing LDS
|
||||
// watch.
|
||||
func doLDS(t *testing.T, v2c *v2Client, fakeServer *fakeserver.Server) {
|
||||
// Register an LDS watcher, and wait till the request is sent out, the
|
||||
// response is received and the callback is invoked.
|
||||
cbCh := testutils.NewChannel()
|
||||
v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
|
||||
t.Logf("v2c.watchLDS callback, ldsUpdate: %+v, err: %v", u, err)
|
||||
cbCh.Send(err)
|
||||
})
|
||||
|
||||
v2c.addWatch(ldsURL, goodLDSTarget1)
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout waiting for LDS request: %v", err)
|
||||
}
|
||||
|
||||
fakeServer.XDSResponseChan <- &fakeserver.Response{Resp: goodLDSResponse1}
|
||||
waitForNilErr(t, cbCh)
|
||||
|
||||
// Read the LDS ack, to clear RequestChan for following tests.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout waiting for LDS ACK: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestRDSHandleResponse starts a fake xDS server, makes a ClientConn to it,
|
||||
// and creates a v2Client using it. Then, it registers an LDS and RDS watcher
|
||||
// and tests different RDS responses.
|
||||
func (s) TestRDSHandleResponse(t *testing.T) {
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
doLDS(t, v2c, fakeServer)
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
rdsResponse *xdspb.DiscoveryResponse
|
||||
@ -251,18 +214,15 @@ func (s) TestRDSHandleResponse(t *testing.T) {
|
||||
wantUpdateErr: false,
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
testWatchHandle(t, &watchHandleTestcase{
|
||||
typeURL: rdsURL,
|
||||
resourceName: goodRouteName1,
|
||||
responseToHandle: test.rdsResponse,
|
||||
wantHandleErr: test.wantErr,
|
||||
wantUpdate: test.wantUpdate,
|
||||
wantUpdateErr: test.wantUpdateErr,
|
||||
|
||||
rdsWatch: v2c.watchRDS,
|
||||
watchReqChan: fakeServer.XDSRequestChan,
|
||||
handleXDSResp: v2c.handleRDSResponse,
|
||||
})
|
||||
})
|
||||
}
|
||||
@ -274,7 +234,9 @@ func (s) TestRDSHandleResponseWithoutLDSWatch(t *testing.T) {
|
||||
_, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
v2c := newV2Client(&testUpdateReceiver{
|
||||
f: func(string, map[string]interface{}) {},
|
||||
}, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
|
||||
if v2c.handleRDSResponse(goodRDSResponse1) == nil {
|
||||
@ -288,181 +250,22 @@ func (s) TestRDSHandleResponseWithoutRDSWatch(t *testing.T) {
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
v2c := newV2Client(&testUpdateReceiver{
|
||||
f: func(string, map[string]interface{}) {},
|
||||
}, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
doLDS(t, v2c, fakeServer)
|
||||
|
||||
if v2c.handleRDSResponse(goodRDSResponse1) == nil {
|
||||
if v2c.handleRDSResponse(badResourceTypeInRDSResponse) == nil {
|
||||
t.Fatal("v2c.handleRDSResponse() succeeded, should have failed")
|
||||
}
|
||||
|
||||
if v2c.handleRDSResponse(goodRDSResponse1) != nil {
|
||||
t.Fatal("v2c.handleRDSResponse() succeeded, should have failed")
|
||||
}
|
||||
}
|
||||
|
||||
// rdsTestOp contains all data related to one particular test operation. Not
|
||||
// all fields make sense for all tests.
|
||||
type rdsTestOp struct {
|
||||
// target is the resource name to watch for.
|
||||
target string
|
||||
// responseToSend is the xDS response sent to the client
|
||||
responseToSend *fakeserver.Response
|
||||
// wantOpErr specfies whether the main operation should return an error.
|
||||
wantOpErr bool
|
||||
// wantRDSCache is the expected rdsCache at the end of an operation.
|
||||
wantRDSCache map[string]string
|
||||
// wantWatchCallback specifies if the watch callback should be invoked.
|
||||
wantWatchCallback bool
|
||||
}
|
||||
|
||||
// testRDSCaching is a helper function which starts a fake xDS server, makes a
|
||||
// ClientConn to it, creates a v2Client using it, registers an LDS watcher and
|
||||
// pushes a good LDS response. It then reads a bunch of test operations to be
|
||||
// performed from rdsTestOps and returns error, if any, on the provided error
|
||||
// channel. This is executed in a separate goroutine.
|
||||
func testRDSCaching(t *testing.T, rdsTestOps []rdsTestOp, errCh *testutils.Channel) {
|
||||
t.Helper()
|
||||
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
doLDS(t, v2c, fakeServer)
|
||||
|
||||
callbackCh := make(chan struct{}, 1)
|
||||
for _, rdsTestOp := range rdsTestOps {
|
||||
// Register a watcher if required, and use a channel to signal the
|
||||
// successful invocation of the callback.
|
||||
if rdsTestOp.target != "" {
|
||||
v2c.watchRDS(rdsTestOp.target, func(u rdsUpdate, err error) {
|
||||
t.Logf("Received callback with rdsUpdate {%+v} and error {%v}", u, err)
|
||||
callbackCh <- struct{}{}
|
||||
})
|
||||
t.Logf("Registered a watcher for RDS target: %v...", rdsTestOp.target)
|
||||
|
||||
// Wait till the request makes it to the fakeServer. This ensures that
|
||||
// the watch request has been processed by the v2Client.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
errCh.Send(fmt.Errorf("Timeout waiting for RDS request: %v", err))
|
||||
}
|
||||
t.Log("FakeServer received request...")
|
||||
}
|
||||
|
||||
// Directly push the response through a call to handleRDSResponse,
|
||||
// thereby bypassing the fakeServer.
|
||||
if rdsTestOp.responseToSend != nil {
|
||||
resp := rdsTestOp.responseToSend.Resp.(*discoverypb.DiscoveryResponse)
|
||||
if err := v2c.handleRDSResponse(resp); (err != nil) != rdsTestOp.wantOpErr {
|
||||
errCh.Send(fmt.Errorf("v2c.handleRDSResponse(%+v) returned err: %v", resp, err))
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
// If the test needs the callback to be invoked, just verify that
|
||||
// it was invoked. Since we verify the contents of the cache, it's
|
||||
// ok not to verify the contents of the callback.
|
||||
if rdsTestOp.wantWatchCallback {
|
||||
<-callbackCh
|
||||
}
|
||||
|
||||
if !cmp.Equal(v2c.cloneRDSCacheForTesting(), rdsTestOp.wantRDSCache) {
|
||||
errCh.Send(fmt.Errorf("gotRDSCache: %v, wantRDSCache: %v", v2c.rdsCache, rdsTestOp.wantRDSCache))
|
||||
return
|
||||
}
|
||||
}
|
||||
t.Log("Completed all test ops successfully...")
|
||||
errCh.Send(nil)
|
||||
}
|
||||
|
||||
// TestRDSCaching tests some end-to-end RDS flows using a fake xDS server, and
|
||||
// verifies the RDS data cached at the v2Client.
|
||||
func (s) TestRDSCaching(t *testing.T) {
|
||||
ops := []rdsTestOp{
|
||||
// Add an RDS watch for a resource name (goodRouteName1), which returns one
|
||||
// matching resource in the response.
|
||||
{
|
||||
target: goodRouteName1,
|
||||
responseToSend: &fakeserver.Response{Resp: goodRDSResponse1},
|
||||
wantRDSCache: map[string]string{goodRouteName1: goodClusterName1},
|
||||
wantWatchCallback: true,
|
||||
},
|
||||
// Push an RDS response with a new resource. This resource is considered
|
||||
// good because its domain field matches our LDS watch target, but the
|
||||
// routeConfigName does not match our RDS watch (so the watch callback will
|
||||
// not be invoked). But this should still be cached.
|
||||
{
|
||||
responseToSend: &fakeserver.Response{Resp: goodRDSResponse2},
|
||||
wantRDSCache: map[string]string{
|
||||
goodRouteName1: goodClusterName1,
|
||||
goodRouteName2: goodClusterName2,
|
||||
},
|
||||
},
|
||||
// Push an uninteresting RDS response. This should cause handleRDSResponse
|
||||
// to return an error. But the watch callback should not be invoked, and
|
||||
// the cache should not be updated.
|
||||
{
|
||||
responseToSend: &fakeserver.Response{Resp: uninterestingRDSResponse},
|
||||
wantOpErr: true,
|
||||
wantRDSCache: map[string]string{
|
||||
goodRouteName1: goodClusterName1,
|
||||
goodRouteName2: goodClusterName2,
|
||||
},
|
||||
},
|
||||
// Switch the watch target to goodRouteName2, which was already cached. No
|
||||
// response is received from the server (as expected), but we want the
|
||||
// callback to be invoked with the new clusterName.
|
||||
{
|
||||
target: goodRouteName2,
|
||||
wantRDSCache: map[string]string{
|
||||
goodRouteName1: goodClusterName1,
|
||||
goodRouteName2: goodClusterName2,
|
||||
},
|
||||
wantWatchCallback: true,
|
||||
},
|
||||
}
|
||||
errCh := testutils.NewChannel()
|
||||
go testRDSCaching(t, ops, errCh)
|
||||
waitForNilErr(t, errCh)
|
||||
}
|
||||
|
||||
// TestRDSWatchExpiryTimer tests the case where the client does not receive an
|
||||
// RDS response for the request that it sends out. We want the watch callback
|
||||
// to be invoked with an error once the watchExpiryTimer fires.
|
||||
func (s) TestRDSWatchExpiryTimer(t *testing.T) {
|
||||
oldWatchExpiryTimeout := defaultWatchExpiryTimeout
|
||||
defaultWatchExpiryTimeout = 500 * time.Millisecond
|
||||
defer func() {
|
||||
defaultWatchExpiryTimeout = oldWatchExpiryTimeout
|
||||
}()
|
||||
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
doLDS(t, v2c, fakeServer)
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
v2c.watchRDS(goodRouteName1, func(u rdsUpdate, err error) {
|
||||
t.Logf("Received callback with rdsUpdate {%+v} and error {%v}", u, err)
|
||||
if u.clusterName != "" {
|
||||
callbackCh.Send(fmt.Errorf("received clusterName %v in rdsCallback, wanted empty string", u.clusterName))
|
||||
}
|
||||
if err == nil {
|
||||
callbackCh.Send(errors.New("received nil error in rdsCallback"))
|
||||
}
|
||||
callbackCh.Send(nil)
|
||||
})
|
||||
|
||||
// Wait till the request makes it to the fakeServer. This ensures that
|
||||
// the watch request has been processed by the v2Client.
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an RDS request")
|
||||
}
|
||||
waitForNilErr(t, callbackCh)
|
||||
}
|
||||
|
||||
func TestMatchTypeForDomain(t *testing.T) {
|
||||
func (s) TestMatchTypeForDomain(t *testing.T) {
|
||||
tests := []struct {
|
||||
d string
|
||||
want domainMatchType
|
||||
@ -481,7 +284,7 @@ func TestMatchTypeForDomain(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestMatch(t *testing.T) {
|
||||
func (s) TestMatch(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
domain string
|
||||
@ -508,7 +311,7 @@ func TestMatch(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestFindBestMatchingVirtualHost(t *testing.T) {
|
||||
func (s) TestFindBestMatchingVirtualHost(t *testing.T) {
|
||||
var (
|
||||
oneExactMatch = &routepb.VirtualHost{
|
||||
Name: "one-exact-match",
|
@ -343,25 +343,7 @@ var (
|
||||
},
|
||||
}
|
||||
marshaledGoodRouteConfig2, _ = proto.Marshal(goodRouteConfig2)
|
||||
uninterestingRouteConfig = &xdspb.RouteConfiguration{
|
||||
Name: uninterestingRouteName,
|
||||
VirtualHosts: []*routepb.VirtualHost{
|
||||
{
|
||||
Domains: []string{uninterestingDomain},
|
||||
Routes: []*routepb.Route{
|
||||
{
|
||||
Action: &routepb.Route_Route{
|
||||
Route: &routepb.RouteAction{
|
||||
ClusterSpecifier: &routepb.RouteAction_Cluster{Cluster: uninterestingClusterName},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
marshaledUninterestingRouteConfig, _ = proto.Marshal(uninterestingRouteConfig)
|
||||
goodRDSResponse1 = &xdspb.DiscoveryResponse{
|
||||
goodRDSResponse1 = &xdspb.DiscoveryResponse{
|
||||
Resources: []*anypb.Any{
|
||||
{
|
||||
TypeUrl: rdsURL,
|
||||
@ -379,15 +361,6 @@ var (
|
||||
},
|
||||
TypeUrl: rdsURL,
|
||||
}
|
||||
uninterestingRDSResponse = &xdspb.DiscoveryResponse{
|
||||
Resources: []*anypb.Any{
|
||||
{
|
||||
TypeUrl: rdsURL,
|
||||
Value: marshaledUninterestingRouteConfig,
|
||||
},
|
||||
},
|
||||
TypeUrl: rdsURL,
|
||||
}
|
||||
)
|
||||
|
||||
// TestV2ClientBackoffAfterRecvError verifies if the v2Client backoffs when it
|
||||
@ -404,14 +377,15 @@ func (s) TestV2ClientBackoffAfterRecvError(t *testing.T) {
|
||||
return 0
|
||||
}
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, clientBackoff, nil)
|
||||
callbackCh := make(chan struct{})
|
||||
v2c := newV2Client(&testUpdateReceiver{
|
||||
f: func(string, map[string]interface{}) { close(callbackCh) },
|
||||
}, cc, goodNodeProto, clientBackoff, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
|
||||
callbackCh := make(chan struct{})
|
||||
v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
|
||||
close(callbackCh)
|
||||
})
|
||||
// v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {})
|
||||
v2c.addWatch(ldsURL, goodLDSTarget1)
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an LDS request")
|
||||
}
|
||||
@ -439,15 +413,21 @@ func (s) TestV2ClientRetriesAfterBrokenStream(t *testing.T) {
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
callbackCh := testutils.NewChannel()
|
||||
v2c := newV2Client(&testUpdateReceiver{
|
||||
f: func(typeURL string, d map[string]interface{}) {
|
||||
if typeURL == ldsURL {
|
||||
if u, ok := d[goodLDSTarget1]; ok {
|
||||
t.Logf("Received LDS callback with ldsUpdate {%+v}", u)
|
||||
callbackCh.Send(struct{}{})
|
||||
}
|
||||
}
|
||||
},
|
||||
}, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
|
||||
t.Logf("Received LDS callback with ldsUpdate {%+v} and error {%v}", u, err)
|
||||
callbackCh.Send(struct{}{})
|
||||
})
|
||||
v2c.addWatch(ldsURL, goodLDSTarget1)
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an LDS request")
|
||||
}
|
||||
@ -478,43 +458,11 @@ func (s) TestV2ClientRetriesAfterBrokenStream(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
// TestV2ClientCancelWatch verifies that the registered watch callback is not
|
||||
// invoked if a response is received after the watcher is cancelled.
|
||||
func (s) TestV2ClientCancelWatch(t *testing.T) {
|
||||
fakeServer, cc, cleanup := startServerAndGetCC(t)
|
||||
defer cleanup()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
cancelFunc := v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
|
||||
t.Logf("Received LDS callback with ldsUpdate {%+v} and error {%v}", u, err)
|
||||
callbackCh.Send(struct{}{})
|
||||
})
|
||||
if _, err := fakeServer.XDSRequestChan.Receive(); err != nil {
|
||||
t.Fatalf("Timeout expired when expecting an LDS request")
|
||||
}
|
||||
t.Log("FakeServer received request...")
|
||||
|
||||
fakeServer.XDSResponseChan <- &fakeserver.Response{Resp: goodLDSResponse1}
|
||||
t.Log("Good LDS response pushed to fakeServer...")
|
||||
|
||||
if _, err := callbackCh.Receive(); err != nil {
|
||||
t.Fatal("Timeout when expecting LDS update")
|
||||
}
|
||||
|
||||
cancelFunc()
|
||||
|
||||
fakeServer.XDSResponseChan <- &fakeserver.Response{Resp: goodLDSResponse1}
|
||||
t.Log("Another good LDS response pushed to fakeServer...")
|
||||
|
||||
if _, err := callbackCh.Receive(); err != testutils.ErrRecvTimeout {
|
||||
t.Fatalf("Watch callback invoked after the watcher was cancelled")
|
||||
}
|
||||
}
|
||||
|
||||
// TestV2ClientWatchWithoutStream verifies the case where a watch is started
|
||||
// when the xds stream is not created. The watcher should not receive any update
|
||||
// (because there won't be any xds response, and timeout is done at a upper
|
||||
// level). And when the stream is re-created, the watcher should get future
|
||||
// updates.
|
||||
func (s) TestV2ClientWatchWithoutStream(t *testing.T) {
|
||||
oldWatchExpiryTimeout := defaultWatchExpiryTimeout
|
||||
defaultWatchExpiryTimeout = 500 * time.Millisecond
|
||||
@ -538,25 +486,27 @@ func (s) TestV2ClientWatchWithoutStream(t *testing.T) {
|
||||
}
|
||||
defer cc.Close()
|
||||
|
||||
v2c := newV2Client(cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
callbackCh := testutils.NewChannel()
|
||||
v2c := newV2Client(&testUpdateReceiver{
|
||||
f: func(typeURL string, d map[string]interface{}) {
|
||||
if typeURL == ldsURL {
|
||||
if u, ok := d[goodLDSTarget1]; ok {
|
||||
t.Logf("Received LDS callback with ldsUpdate {%+v}", u)
|
||||
callbackCh.Send(u)
|
||||
}
|
||||
}
|
||||
},
|
||||
}, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
|
||||
defer v2c.close()
|
||||
t.Log("Started xds v2Client...")
|
||||
|
||||
callbackCh := testutils.NewChannel()
|
||||
// This watch is started when the xds-ClientConn is in Transient Failure,
|
||||
// and no xds stream is created.
|
||||
v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
|
||||
t.Logf("Received LDS callback with ldsUpdate {%+v} and error {%v}", u, err)
|
||||
if err != nil {
|
||||
callbackCh.Send(err)
|
||||
}
|
||||
callbackCh.Send(u)
|
||||
})
|
||||
v2c.addWatch(ldsURL, goodLDSTarget1)
|
||||
|
||||
// The watcher should receive an update, with a timeout error in it.
|
||||
if v, err := callbackCh.TimedReceive(time.Second); err != nil {
|
||||
t.Fatal("Timeout when expecting LDS update")
|
||||
} else if _, ok := v.(error); !ok {
|
||||
t.Fatalf("Expect an error from watcher, got %v", v)
|
||||
if v, err := callbackCh.TimedReceive(100 * time.Millisecond); err == nil {
|
||||
t.Fatalf("Expect an timeout error from watcher, got %v", v)
|
||||
}
|
||||
|
||||
// Send the real server address to the ClientConn, the stream should be
|
||||
|
@ -42,8 +42,8 @@ type Client struct {
|
||||
|
||||
mu sync.Mutex
|
||||
serviceCb func(xdsclient.ServiceUpdate, error)
|
||||
cdsCb func(xdsclient.CDSUpdate, error)
|
||||
edsCb func(*xdsclient.EDSUpdate, error)
|
||||
cdsCb func(xdsclient.ClusterUpdate, error)
|
||||
edsCb func(xdsclient.EndpointsUpdate, error)
|
||||
}
|
||||
|
||||
// WatchService registers a LDS/RDS watch.
|
||||
@ -77,7 +77,7 @@ func (xdsC *Client) InvokeWatchServiceCallback(cluster string, err error) {
|
||||
}
|
||||
|
||||
// WatchCluster registers a CDS watch.
|
||||
func (xdsC *Client) WatchCluster(clusterName string, callback func(xdsclient.CDSUpdate, error)) func() {
|
||||
func (xdsC *Client) WatchCluster(clusterName string, callback func(xdsclient.ClusterUpdate, error)) func() {
|
||||
xdsC.mu.Lock()
|
||||
defer xdsC.mu.Unlock()
|
||||
|
||||
@ -99,7 +99,7 @@ func (xdsC *Client) WaitForWatchCluster() (string, error) {
|
||||
}
|
||||
|
||||
// InvokeWatchClusterCallback invokes the registered cdsWatch callback.
|
||||
func (xdsC *Client) InvokeWatchClusterCallback(update xdsclient.CDSUpdate, err error) {
|
||||
func (xdsC *Client) InvokeWatchClusterCallback(update xdsclient.ClusterUpdate, err error) {
|
||||
xdsC.mu.Lock()
|
||||
defer xdsC.mu.Unlock()
|
||||
|
||||
@ -114,7 +114,7 @@ func (xdsC *Client) WaitForCancelClusterWatch() error {
|
||||
}
|
||||
|
||||
// WatchEndpoints registers an EDS watch for provided clusterName.
|
||||
func (xdsC *Client) WatchEndpoints(clusterName string, callback func(*xdsclient.EDSUpdate, error)) (cancel func()) {
|
||||
func (xdsC *Client) WatchEndpoints(clusterName string, callback func(xdsclient.EndpointsUpdate, error)) (cancel func()) {
|
||||
xdsC.mu.Lock()
|
||||
defer xdsC.mu.Unlock()
|
||||
|
||||
@ -136,7 +136,7 @@ func (xdsC *Client) WaitForWatchEDS() (string, error) {
|
||||
}
|
||||
|
||||
// InvokeWatchEDSCallback invokes the registered edsWatch callback.
|
||||
func (xdsC *Client) InvokeWatchEDSCallback(update *xdsclient.EDSUpdate, err error) {
|
||||
func (xdsC *Client) InvokeWatchEDSCallback(update xdsclient.EndpointsUpdate, err error) {
|
||||
xdsC.mu.Lock()
|
||||
defer xdsC.mu.Unlock()
|
||||
|
||||
|
Reference in New Issue
Block a user