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:
Menghan Li
2020-04-15 14:32:31 -07:00
committed by GitHub
parent 6e001bea42
commit ff40ef4227
38 changed files with 2917 additions and 1765 deletions

View File

@ -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

View File

@ -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)

View File

@ -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 {

View File

@ -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()

View File

@ -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{}

View File

@ -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".

View File

@ -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.
//

View File

@ -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,
}
)

View File

@ -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)
}

View 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
}
}
}

View File

@ -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)
}
}

View 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.
}
}
}
}

View 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)
}

View 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")
}
}

View 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)
}

View 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")
}
}

View 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)
}

View 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)
}
}

View 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)
}

View 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)
}
}

View 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
}
}

View 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)
}
}

View File

@ -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)
}

View File

@ -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)

View File

@ -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
}

View File

@ -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++
}

View File

@ -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

View 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,
}
)

View File

@ -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
}

View File

@ -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)
}

View File

@ -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.

View 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
}

View File

@ -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)
}

View File

@ -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
}

View File

@ -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",

View File

@ -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

View File

@ -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()