xds: Initial implementation of a client using the v2 API (#3144)

This object will be used by a higher level xdsClient object, which will
provide the watch API used by the xds resolver and balancer
implementations.
This commit is contained in:
Easwar Swaminathan
2019-11-12 10:31:11 -08:00
committed by GitHub
parent 460b1d2ced
commit dc9615bb06
10 changed files with 2095 additions and 3 deletions

2
go.mod
View File

@ -3,7 +3,7 @@ module google.golang.org/grpc
go 1.11
require (
github.com/envoyproxy/go-control-plane v0.9.0
github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473
github.com/envoyproxy/protoc-gen-validate v0.1.0
github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b
github.com/golang/mock v1.1.1

4
go.sum
View File

@ -3,8 +3,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT
github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
github.com/envoyproxy/go-control-plane v0.9.0 h1:67WMNTvGrl7V1dWdKCeTwxDr7nio9clKoTlLhwIPnT4=
github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473 h1:4cmBvAEBNJaGARUEs3/suWRyfyBfhf7I60WBZq+bv2w=
github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
github.com/envoyproxy/protoc-gen-validate v0.1.0 h1:EQciDnbrYxy13PgWoY8AqoxGiPrpgBZ1R8UNe3ddc+A=
github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c=
github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58=

View File

@ -0,0 +1,156 @@
/*
*
* 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 fakexds provides a very basic fake implementation of the xDS server
// for unit testing purposes.
package fakexds
import (
"context"
"net"
"testing"
"time"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
discoverypb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
adsgrpc "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v2"
)
// TODO: Make this a var or a field in the server if there is a need to use a
// value other than this default.
const defaultChannelBufferSize = 50
// Request wraps an xDS request and error.
type Request struct {
Req *discoverypb.DiscoveryRequest
Err error
}
// Response wraps an xDS response and error.
type Response struct {
Resp *discoverypb.DiscoveryResponse
Err error
}
// Server is a very basic implementation of a fake xDS server. It provides a
// request and response channel for the user to control the requests that are
// expected and the responses that needs to be sent out.
type Server struct {
// RequestChan is a buffered channel on which the fake server writes the
// received requests onto.
RequestChan chan *Request
// ResponseChan is a buffered channel from which the fake server reads the
// responses that it must send out to the client.
ResponseChan chan *Response
}
// StartClientAndServer starts a fakexds.Server and creates a ClientConn
// talking to it. The returned cleanup function should be invoked by the caller
// once the test is done.
func StartClientAndServer(t *testing.T) (*Server, *grpc.ClientConn, func()) {
t.Helper()
var lis net.Listener
var err error
lis, err = net.Listen("tcp", "localhost:0")
if err != nil {
t.Fatalf("net.Listen() failed: %v", err)
}
server := grpc.NewServer()
fs := &Server{
RequestChan: make(chan *Request, defaultChannelBufferSize),
ResponseChan: make(chan *Response, defaultChannelBufferSize),
}
adsgrpc.RegisterAggregatedDiscoveryServiceServer(server, fs)
go server.Serve(lis)
t.Logf("Starting fake xDS server at %v...", lis.Addr().String())
defer func() {
if err != nil {
server.Stop()
lis.Close()
}
}()
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
var cc *grpc.ClientConn
cc, err = grpc.DialContext(ctx, lis.Addr().String(), grpc.WithInsecure(), grpc.WithBlock())
if err != nil {
t.Fatalf("grpc.DialContext(%s) failed: %v", lis.Addr().String(), err)
}
t.Log("Started xDS gRPC client...")
return fs, cc, func() {
server.Stop()
lis.Close()
}
}
// StreamAggregatedResources is the fake implementation to handle an ADS
// stream.
func (fs *Server) StreamAggregatedResources(s adsgrpc.AggregatedDiscoveryService_StreamAggregatedResourcesServer) error {
errCh := make(chan error, 2)
go func() {
for {
req, err := s.Recv()
if err != nil {
errCh <- err
return
}
fs.RequestChan <- &Request{req, err}
}
}()
go func() {
var retErr error
defer func() {
errCh <- retErr
}()
for {
select {
case r := <-fs.ResponseChan:
if r.Err != nil {
retErr = r.Err
return
}
if err := s.Send(r.Resp); err != nil {
retErr = err
return
}
case <-s.Context().Done():
retErr = s.Context().Err()
return
}
}
}()
if err := <-errCh; err != nil {
return err
}
return nil
}
// DeltaAggregatedResources helps implement the ADS service.
func (fs *Server) DeltaAggregatedResources(adsgrpc.AggregatedDiscoveryService_DeltaAggregatedResourcesServer) error {
return status.Error(codes.Unimplemented, "")
}

125
xds/internal/client/lds.go Normal file
View File

@ -0,0 +1,125 @@
/*
*
* 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 (
"fmt"
"github.com/golang/protobuf/ptypes"
"google.golang.org/grpc/grpclog"
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
httppb "github.com/envoyproxy/go-control-plane/envoy/config/filter/network/http_connection_manager/v2"
)
// newLDSRequest generates an LDS request proto for the provided target, to be
// sent out on the wire.
func (v2c *v2Client) newLDSRequest(target []string) *xdspb.DiscoveryRequest {
return &xdspb.DiscoveryRequest{
Node: v2c.nodeProto,
TypeUrl: listenerURL,
ResourceNames: target,
}
}
// sendLDS sends an LDS request for provided target on the provided stream.
func (v2c *v2Client) sendLDS(stream adsStream, target []string) bool {
if err := stream.Send(v2c.newLDSRequest(target)); err != nil {
grpclog.Warningf("xds: LDS request for resource %v failed: %v", target, err)
return false
}
return true
}
// 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[ldsResource]
if wi == nil {
return fmt.Errorf("xds: no LDS watcher found when handling LDS response: %+v", resp)
}
routeName := ""
for _, r := range resp.GetResources() {
var resource ptypes.DynamicAny
if err := ptypes.UnmarshalAny(r, &resource); err != nil {
return fmt.Errorf("xds: failed to unmarshal resource in LDS response: %v", err)
}
lis, ok := resource.Message.(*xdspb.Listener)
if !ok {
return fmt.Errorf("xds: unexpected resource type: %T in LDS response", resource.Message)
}
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 = getRouteConfigNameFromListener(lis)
if err != nil {
return err
}
}
var err error
if routeName == "" {
err = fmt.Errorf("xds: LDS target %s not found in received response %+v", wi.target, resp)
}
wi.expiryTimer.Stop()
wi.callback.(ldsCallback)(ldsUpdate{routeName: routeName}, err)
return nil
}
// getRouteConfigNameFromListener checks if the provided Listener proto meets
// the expected criteria. If so, it returns a non-empty routeConfigName.
func getRouteConfigNameFromListener(lis *xdspb.Listener) (string, error) {
if lis.GetApiListener() == nil {
return "", fmt.Errorf("xds: no api_listener field in LDS response %+v", lis)
}
var apiAny ptypes.DynamicAny
if err := ptypes.UnmarshalAny(lis.GetApiListener().GetApiListener(), &apiAny); err != nil {
return "", fmt.Errorf("xds: failed to unmarshal api_listner in LDS response: %v", err)
}
apiLis, ok := apiAny.Message.(*httppb.HttpConnectionManager)
if !ok {
return "", fmt.Errorf("xds: unexpected api_listener type: %T in LDS response", apiAny.Message)
}
switch apiLis.RouteSpecifier.(type) {
case *httppb.HttpConnectionManager_Rds:
name := apiLis.GetRds().GetRouteConfigName()
if name == "" {
return "", fmt.Errorf("xds: empty route_config_name in LDS response: %+v", lis)
}
return name, nil
case *httppb.HttpConnectionManager_RouteConfig:
// TODO: Add support for specifying the RouteConfiguration inline
// in the LDS response.
return "", fmt.Errorf("xds: LDS response contains RDS config inline. Not supported for now: %+v", apiLis)
case nil:
return "", fmt.Errorf("xds: no RouteSpecifier in received LDS response: %+v", apiLis)
default:
return "", fmt.Errorf("xds: unsupported type %T for RouteSpecifier in received LDS response", apiLis.RouteSpecifier)
}
}

View File

@ -0,0 +1,270 @@
/*
*
* 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"
"reflect"
"testing"
"time"
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
"google.golang.org/grpc/xds/internal/client/fakexds"
)
func TestGetRouteConfigNameFromListener(t *testing.T) {
tests := []struct {
name string
lis *xdspb.Listener
wantRoute string
wantErr bool
}{
{
name: "no-apiListener-field",
lis: &xdspb.Listener{},
wantRoute: "",
wantErr: true,
},
{
name: "badly-marshaled-apiListener",
lis: badAPIListener1,
wantRoute: "",
wantErr: true,
},
{
name: "wrong-type-in-apiListener",
lis: badResourceListener,
wantRoute: "",
wantErr: true,
},
{
name: "empty-httpConnMgr-in-apiListener",
lis: listenerWithEmptyHTTPConnMgr,
wantRoute: "",
wantErr: true,
},
{
name: "scopedRoutes-routeConfig-in-apiListener",
lis: listenerWithScopedRoutesRouteConfig,
wantRoute: "",
wantErr: true,
},
{
name: "goodListener1",
lis: goodListener1,
wantRoute: goodRouteName1,
wantErr: false,
},
}
for _, test := range tests {
gotRoute, err := getRouteConfigNameFromListener(test.lis)
if gotRoute != test.wantRoute {
t.Errorf("%s: getRouteConfigNameFromListener(%+v) = %v, want %v", test.name, test.lis, gotRoute, test.wantRoute)
}
if (err != nil) != test.wantErr {
t.Errorf("%s: getRouteConfigNameFromListener(%+v) = %v, want %v", test.name, test.lis, err, test.wantErr)
}
}
}
// TestHandleLDSResponse starts a fake xDS server, makes a ClientConn to it,
// and creates a v2Client using it. Then, it registers a watchLDS and tests
// different LDS responses.
func TestHandleLDSResponse(t *testing.T) {
fakeServer, client, cleanup := fakexds.StartClientAndServer(t)
defer cleanup()
v2c := newV2Client(client, goodNodeProto, func(int) time.Duration { return 0 })
tests := []struct {
name string
ldsResponse *xdspb.DiscoveryResponse
wantErr bool
wantUpdate *ldsUpdate
wantUpdateErr bool
}{
// Badly marshaled LDS response.
{
name: "badly-marshaled-response",
ldsResponse: badlyMarshaledLDSResponse,
wantErr: true,
wantUpdate: nil,
wantUpdateErr: false,
},
// Response does not contain Listener proto.
{
name: "no-listener-proto-in-response",
ldsResponse: badResourceTypeInLDSResponse,
wantErr: true,
wantUpdate: nil,
wantUpdateErr: false,
},
// No APIListener in the response. Just one test case here for a bad
// ApiListener, since the others are covered in
// TestGetRouteConfigNameFromListener.
{
name: "no-apiListener-in-response",
ldsResponse: noAPIListenerLDSResponse,
wantErr: true,
wantUpdate: nil,
wantUpdateErr: false,
},
// Response contains one listener and it is good.
{
name: "one-good-listener",
ldsResponse: goodLDSResponse1,
wantErr: false,
wantUpdate: &ldsUpdate{routeName: goodRouteName1},
wantUpdateErr: false,
},
// Response contains multiple good listeners, including the one we are
// interested in.
{
name: "multiple-good-listener",
ldsResponse: ldsResponseWithMultipleResources,
wantErr: false,
wantUpdate: &ldsUpdate{routeName: goodRouteName1},
wantUpdateErr: false,
},
// Response contains two good listeners (one interesting and one
// uninteresting), and one badly marshaled listener.
{
name: "good-bad-ugly-listeners",
ldsResponse: goodBadUglyLDSResponse,
wantErr: false,
wantUpdate: &ldsUpdate{routeName: goodRouteName1},
wantUpdateErr: false,
},
// Response contains one listener, but we are not interested in it.
{
name: "one-uninteresting-listener",
ldsResponse: goodLDSResponse2,
wantErr: false,
wantUpdate: &ldsUpdate{routeName: ""},
wantUpdateErr: true,
},
// Response constains no resources. This is the case where the server
// does not know about the target we are interested in.
{
name: "empty-response",
ldsResponse: emptyLDSResponse,
wantErr: false,
wantUpdate: &ldsUpdate{routeName: ""},
wantUpdateErr: true,
},
}
for _, test := range tests {
gotUpdateCh := make(chan ldsUpdate, 1)
gotUpdateErrCh := make(chan error, 1)
// Register a watcher, to trigger the v2Client to send an LDS request.
cancelWatch := v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
t.Logf("%s: in v2c.watchLDS callback, ldsUpdate: %+v, err: %v", test.name, u, err)
gotUpdateCh <- u
gotUpdateErrCh <- err
})
// Wait till the request makes it to the fakeServer. This ensures that
// the watch request has been processed by the v2Client.
<-fakeServer.RequestChan
// Directly push the response through a call to handleLDSResponse,
// thereby bypassing the fakeServer.
if err := v2c.handleLDSResponse(test.ldsResponse); (err != nil) != test.wantErr {
t.Fatalf("%s: v2c.handleLDSResponse() returned err: %v, wantErr: %v", test.name, err, test.wantErr)
}
// If the test needs the callback to be invoked, verify the update and
// error pushed to the callback.
if test.wantUpdate != nil {
timer := time.NewTimer(defaultTestTimeout)
select {
case <-timer.C:
t.Fatal("Timeout when expecting LDS update")
case gotUpdate := <-gotUpdateCh:
timer.Stop()
if !reflect.DeepEqual(gotUpdate, *test.wantUpdate) {
t.Fatalf("%s: got LDS update : %+v, want %+v", test.name, gotUpdate, *test.wantUpdate)
}
}
// Since the callback that we registered pushes to both channels at
// the same time, this channel read should return immediately.
gotUpdateErr := <-gotUpdateErrCh
if (gotUpdateErr != nil) != test.wantUpdateErr {
t.Fatalf("%s: got LDS update error {%v}, wantErr: %v", test.name, gotUpdateErr, test.wantUpdateErr)
}
}
cancelWatch()
}
}
// TestHandleLDSResponseWithoutWatch tests the case where the v2Client receives
// an LDS response without a registered watcher.
func TestHandleLDSResponseWithoutWatch(t *testing.T) {
_, client, cleanup := fakexds.StartClientAndServer(t)
defer cleanup()
v2c := newV2Client(client, goodNodeProto, func(int) time.Duration { return 0 })
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 TestLDSWatchExpiryTimer(t *testing.T) {
oldWatchExpiryTimeout := defaultWatchExpiryTimeout
defaultWatchExpiryTimeout = 1 * time.Second
defer func() {
defaultWatchExpiryTimeout = oldWatchExpiryTimeout
}()
fakeServer, client, cleanup := fakexds.StartClientAndServer(t)
defer cleanup()
v2c := newV2Client(client, goodNodeProto, func(int) time.Duration { return 0 })
// Wait till the request makes it to the fakeServer. This ensures that
// the watch request has been processed by the v2Client.
callbackCh := make(chan error, 1)
v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
t.Logf("in v2c.watchLDS callback, ldsUpdate: %+v, err: %v", u, err)
if u.routeName != "" {
callbackCh <- fmt.Errorf("received routeName %v in ldsCallback, wanted empty string", u.routeName)
}
if err == nil {
callbackCh <- errors.New("received nil error in ldsCallback")
}
callbackCh <- nil
})
<-fakeServer.RequestChan
timer := time.NewTimer(2 * time.Second)
select {
case <-timer.C:
t.Fatalf("Timeout expired when expecting LDS update")
case err := <-callbackCh:
timer.Stop()
if err != nil {
t.Fatal(err)
}
}
}

144
xds/internal/client/rds.go Normal file
View File

@ -0,0 +1,144 @@
/*
*
* 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 (
"fmt"
"net"
"github.com/golang/protobuf/ptypes"
"google.golang.org/grpc/grpclog"
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
)
// newRDSRequest generates an RDS request proto for the provided routeName, to
// be sent out on the wire.
func (v2c *v2Client) newRDSRequest(routeName []string) *xdspb.DiscoveryRequest {
return &xdspb.DiscoveryRequest{
Node: v2c.nodeProto,
TypeUrl: routeURL,
ResourceNames: routeName,
}
}
// sendRDS sends an RDS request for provided routeName on the provided stream.
func (v2c *v2Client) sendRDS(stream adsStream, routeName []string) bool {
if err := stream.Send(v2c.newRDSRequest(routeName)); err != nil {
grpclog.Infof("xds: RDS request for resource %v failed: %v", routeName, err)
return false
}
return true
}
// handleRDSResponse processes an RDS response received from the xDS server. On
// receipt of a good response, it caches validated resources and also invokes
// the registered watcher callback.
func (v2c *v2Client) handleRDSResponse(resp *xdspb.DiscoveryResponse) error {
v2c.mu.Lock()
defer v2c.mu.Unlock()
if v2c.watchMap[ldsResource] == nil {
return fmt.Errorf("xds: unexpected RDS response when no LDS watcher is registered: %+v", resp)
}
target := v2c.watchMap[ldsResource].target[0]
wi := v2c.watchMap[rdsResource]
if wi == nil {
return fmt.Errorf("xds: no RDS watcher found when handling RDS response: %+v", resp)
}
returnCluster := ""
localCache := make(map[string]string)
for _, r := range resp.GetResources() {
var resource ptypes.DynamicAny
if err := ptypes.UnmarshalAny(r, &resource); err != nil {
return fmt.Errorf("xds: failed to unmarshal resource in RDS response: %v", err)
}
rc, ok := resource.Message.(*xdspb.RouteConfiguration)
if !ok {
return fmt.Errorf("xds: unexpected resource type: %T in RDS response", resource.Message)
}
cluster := getClusterFromRouteConfiguration(rc, target)
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
if rc.GetName() == wi.target[0] {
returnCluster = 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.expiryTimer.Stop()
wi.callback.(rdsCallback)(rdsUpdate{clusterName: returnCluster}, nil)
}
return nil
}
// getClusterFromRouteConfiguration checks if the provided RouteConfiguration
// meets the expected criteria. If so, it returns a non-empty clusterName.
//
// A RouteConfiguration resource is considered valid when only if it contains a
// VirtualHost whose domain field matches the server name from the URI passed
// to the gRPC channel, and it contains a clusterName.
//
// The RouteConfiguration includes a list of VirtualHosts, which may have zero
// or more elements. We are interested in the element whose domains field
// matches the server name specified in the "xds:" URI (with port, if any,
// stripped off). The only field in the VirtualHost proto that the we are
// interested in is the list of routes. We only look at the last route in the
// list (the default route), whose match field must be empty and whose route
// field must be set. Inside that route message, the cluster field will
// contain the clusterName we are looking for.
func getClusterFromRouteConfiguration(rc *xdspb.RouteConfiguration, target string) string {
host, _, err := net.SplitHostPort(target)
if err != nil {
return ""
}
for _, vh := range rc.GetVirtualHosts() {
for _, domain := range vh.GetDomains() {
// TODO: Add support for wildcard matching here?
if domain != host || len(vh.GetRoutes()) == 0 {
continue
}
dr := vh.Routes[len(vh.Routes)-1]
if dr.GetMatch() == nil && dr.GetRoute() != nil {
return dr.GetRoute().GetCluster()
}
}
}
return ""
}

View File

@ -0,0 +1,492 @@
/*
*
* 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"
"reflect"
"testing"
"time"
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
routepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/route"
"google.golang.org/grpc/xds/internal/client/fakexds"
)
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 TestGetClusterFromRouteConfiguration(t *testing.T) {
tests := []struct {
name string
rc *xdspb.RouteConfiguration
wantCluster string
}{
{
name: "no-virtual-hosts-in-rc",
rc: emptyRouteConfig,
wantCluster: "",
},
{
name: "no-domains-in-rc",
rc: noDomainsInRouteConfig,
wantCluster: "",
},
{
name: "non-matching-domain-in-rc",
rc: &xdspb.RouteConfiguration{
VirtualHosts: []*routepb.VirtualHost{
{Domains: []string{uninterestingDomain}},
},
},
wantCluster: "",
},
{
name: "no-routes-in-rc",
rc: &xdspb.RouteConfiguration{
VirtualHosts: []*routepb.VirtualHost{
{Domains: []string{goodMatchingDomain}},
},
},
wantCluster: "",
},
{
name: "default-route-match-field-is-non-nil",
rc: &xdspb.RouteConfiguration{
VirtualHosts: []*routepb.VirtualHost{
{
Domains: []string{goodMatchingDomain},
Routes: []*routepb.Route{
{
Match: &routepb.RouteMatch{},
Action: &routepb.Route_Route{},
},
},
},
},
},
wantCluster: "",
},
{
name: "default-route-routeaction-field-is-nil",
rc: &xdspb.RouteConfiguration{
VirtualHosts: []*routepb.VirtualHost{
{
Domains: []string{goodMatchingDomain},
Routes: []*routepb.Route{{}},
},
},
},
wantCluster: "",
},
{
name: "default-route-cluster-field-is-empty",
rc: &xdspb.RouteConfiguration{
VirtualHosts: []*routepb.VirtualHost{
{
Domains: []string{goodMatchingDomain},
Routes: []*routepb.Route{
{
Action: &routepb.Route_Route{
Route: &routepb.RouteAction{
ClusterSpecifier: &routepb.RouteAction_ClusterHeader{},
},
},
},
},
},
},
},
wantCluster: "",
},
{
name: "good-route-config",
rc: goodRouteConfig1,
wantCluster: goodClusterName1,
},
}
for _, test := range tests {
if gotCluster := getClusterFromRouteConfiguration(test.rc, goodLDSTarget1); gotCluster != test.wantCluster {
t.Errorf("%s: getClusterFromRouteConfiguration(%+v, %v) = %v, want %v", test.name, test.rc, goodLDSTarget1, gotCluster, test.wantCluster)
}
}
}
// TestHandleRDSResponse 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 TestHandleRDSResponse(t *testing.T) {
fakeServer, client, cleanup := fakexds.StartClientAndServer(t)
defer cleanup()
v2c := newV2Client(client, goodNodeProto, func(int) time.Duration { return 0 })
// Register an LDS watcher, and wait till the request is sent out, the
// response is received and the callback is invoked.
cbCh := make(chan error, 1)
v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
t.Logf("v2c.watchLDS callback, ldsUpdate: %+v, err: %v", u, err)
cbCh <- err
})
<-fakeServer.RequestChan
fakeServer.ResponseChan <- &fakexds.Response{Resp: goodLDSResponse1}
if err := <-cbCh; err != nil {
t.Fatalf("v2c.watchLDS returned error in callback: %v", err)
}
tests := []struct {
name string
rdsResponse *xdspb.DiscoveryResponse
wantErr bool
wantUpdate *rdsUpdate
wantUpdateErr bool
}{
// Badly marshaled RDS response.
{
name: "badly-marshaled-response",
rdsResponse: badlyMarshaledRDSResponse,
wantErr: true,
wantUpdate: nil,
wantUpdateErr: false,
},
// Response does not contain RouteConfiguration proto.
{
name: "no-route-config-in-response",
rdsResponse: badResourceTypeInRDSResponse,
wantErr: true,
wantUpdate: nil,
wantUpdateErr: false,
},
// No VirtualHosts in the response. Just one test case here for a bad
// RouteConfiguration, since the others are covered in
// TestGetClusterFromRouteConfiguration.
{
name: "no-virtual-hosts-in-response",
rdsResponse: noVirtualHostsInRDSResponse,
wantErr: true,
wantUpdate: nil,
wantUpdateErr: false,
},
// Response contains one good RouteConfiguration, uninteresting though.
{
name: "one-uninteresting-route-config",
rdsResponse: goodRDSResponse2,
wantErr: false,
wantUpdate: nil,
wantUpdateErr: false,
},
// Response contains one good interesting RouteConfiguration.
{
name: "one-good-route-config",
rdsResponse: goodRDSResponse1,
wantErr: false,
wantUpdate: &rdsUpdate{clusterName: goodClusterName1},
wantUpdateErr: false,
},
}
for _, test := range tests {
gotUpdateCh := make(chan rdsUpdate, 1)
gotUpdateErrCh := make(chan error, 1)
// Register a watcher, to trigger the v2Client to send an RDS request.
cancelWatch := v2c.watchRDS(goodRouteName1, func(u rdsUpdate, err error) {
t.Logf("%s: in v2c.watchRDS callback, rdsUpdate: %+v, err: %v", test.name, u, err)
gotUpdateCh <- u
gotUpdateErrCh <- err
})
// Wait till the request makes it to the fakeServer. This ensures that
// the watch request has been processed by the v2Client.
<-fakeServer.RequestChan
// Directly push the response through a call to handleRDSResponse,
// thereby bypassing the fakeServer.
if err := v2c.handleRDSResponse(test.rdsResponse); (err != nil) != test.wantErr {
t.Fatalf("%s: v2c.handleRDSResponse() returned err: %v, wantErr: %v", test.name, err, test.wantErr)
}
// If the test needs the callback to be invoked, verify the update and
// error pushed to the callback.
if test.wantUpdate != nil {
timer := time.NewTimer(defaultTestTimeout)
select {
case <-timer.C:
t.Fatal("Timeout when expecting RDS update")
case gotUpdate := <-gotUpdateCh:
timer.Stop()
if !reflect.DeepEqual(gotUpdate, *test.wantUpdate) {
t.Fatalf("%s: got RDS update : %+v, want %+v", test.name, gotUpdate, *test.wantUpdate)
}
}
// Since the callback that we registered pushes to both channels at
// the same time, this channel read should return immediately.
gotUpdateErr := <-gotUpdateErrCh
if (gotUpdateErr != nil) != test.wantUpdateErr {
t.Fatalf("%s: got RDS update error {%v}, wantErr: %v", test.name, gotUpdateErr, test.wantUpdateErr)
}
}
cancelWatch()
}
}
// TestHandleRDSResponseWithoutLDSWatch tests the case where the v2Client
// receives an RDS response without a registered LDS watcher.
func TestHandleRDSResponseWithoutLDSWatch(t *testing.T) {
_, client, cleanup := fakexds.StartClientAndServer(t)
defer cleanup()
v2c := newV2Client(client, goodNodeProto, func(int) time.Duration { return 0 })
if v2c.handleRDSResponse(goodRDSResponse1) == nil {
t.Fatal("v2c.handleRDSResponse() succeeded, should have failed")
}
}
// TestHandleRDSResponseWithoutRDSWatch tests the case where the v2Client
// receives an RDS response without a registered RDS watcher.
func TestHandleRDSResponseWithoutRDSWatch(t *testing.T) {
fakeServer, client, cleanup := fakexds.StartClientAndServer(t)
defer cleanup()
v2c := newV2Client(client, goodNodeProto, func(int) time.Duration { return 0 })
// Register an LDS watcher, and wait till the request is sent out, the
// response is received and the callback is invoked.
cbCh := make(chan error, 1)
v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
t.Logf("v2c.watchLDS callback, ldsUpdate: %+v, err: %v", u, err)
cbCh <- err
})
<-fakeServer.RequestChan
fakeServer.ResponseChan <- &fakexds.Response{Resp: goodLDSResponse1}
if err := <-cbCh; err != nil {
t.Fatalf("v2c.watchLDS returned error in callback: %v", err)
}
if v2c.handleRDSResponse(goodRDSResponse1) == nil {
t.Fatal("v2c.handleRDSResponse() succeeded, should have failed")
}
}
// testOp contains all data related to one particular test operation. Not all
// fields make sense for all tests.
type testOp struct {
// target is the resource name to watch for.
target string
// responseToSend is the xDS response sent to the client
responseToSend *fakexds.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 testOps and returns error, if any, on the provided error
// channel. This is executed in a separate goroutine.
func testRDSCaching(t *testing.T, testOps []testOp, errCh chan error) {
t.Helper()
fakeServer, client, cleanup := fakexds.StartClientAndServer(t)
defer cleanup()
v2c := newV2Client(client, goodNodeProto, func(int) time.Duration { return 0 })
defer v2c.close()
t.Log("Started xds v2Client...")
// Register an LDS watcher, and wait till the request is sent out, the
// response is received and the callback is invoked.
cbCh := make(chan error, 1)
v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
t.Logf("v2c.watchLDS callback, ldsUpdate: %+v, err: %v", u, err)
cbCh <- err
})
<-fakeServer.RequestChan
fakeServer.ResponseChan <- &fakexds.Response{Resp: goodLDSResponse1}
if err := <-cbCh; err != nil {
errCh <- fmt.Errorf("v2c.watchLDS returned error in callback: %v", err)
return
}
callbackCh := make(chan struct{}, 1)
for _, testOp := range testOps {
// Register a watcher if required, and use a channel to signal the
// successful invocation of the callback.
if testOp.target != "" {
v2c.watchRDS(testOp.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 LDS target: %v...", testOp.target)
// Wait till the request makes it to the fakeServer. This ensures that
// the watch request has been processed by the v2Client.
<-fakeServer.RequestChan
t.Log("FakeServer received request...")
}
// Directly push the response through a call to handleRDSResponse,
// thereby bypassing the fakeServer.
if testOp.responseToSend != nil {
if err := v2c.handleRDSResponse(testOp.responseToSend.Resp); (err != nil) != testOp.wantOpErr {
errCh <- fmt.Errorf("v2c.handleRDSResponse() returned err: %v", 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 testOp.wantWatchCallback {
<-callbackCh
}
if !reflect.DeepEqual(v2c.cloneRDSCacheForTesting(), testOp.wantRDSCache) {
errCh <- fmt.Errorf("gotRDSCache: %v, wantRDSCache: %v", v2c.rdsCache, testOp.wantRDSCache)
return
}
}
t.Log("Completed all test ops successfully...")
errCh <- nil
}
// TestRDSCaching tests some end-to-end RDS flows using a fake xDS server, and
// verifies the RDS data cached at the v2Client.
func TestRDSCaching(t *testing.T) {
errCh := make(chan error, 1)
ops := []testOp{
// Add an RDS watch for a resource name (goodRouteName1), which returns one
// matching resource in the response.
{
target: goodRouteName1,
responseToSend: &fakexds.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: &fakexds.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: &fakexds.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,
},
}
go testRDSCaching(t, ops, errCh)
timer := time.NewTimer(defaultTestTimeout)
select {
case <-timer.C:
t.Fatal("Timeout when expecting RDS update")
case err := <-errCh:
timer.Stop()
if err != nil {
t.Fatal(err)
}
}
}
func TestRDSWatchExpiryTimer(t *testing.T) {
oldWatchExpiryTimeout := defaultWatchExpiryTimeout
defaultWatchExpiryTimeout = 1 * time.Second
defer func() {
defaultWatchExpiryTimeout = oldWatchExpiryTimeout
}()
fakeServer, client, cleanup := fakexds.StartClientAndServer(t)
defer cleanup()
v2c := newV2Client(client, goodNodeProto, func(int) time.Duration { return 0 })
defer v2c.close()
t.Log("Started xds v2Client...")
// Register an LDS watcher, and wait till the request is sent out, the
// response is received and the callback is invoked.
ldsCallbackCh := make(chan struct{})
v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
t.Logf("v2c.watchLDS callback, ldsUpdate: %+v, err: %v", u, err)
close(ldsCallbackCh)
})
<-fakeServer.RequestChan
fakeServer.ResponseChan <- &fakexds.Response{Resp: goodLDSResponse1}
<-ldsCallbackCh
// Wait till the request makes it to the fakeServer. This ensures that
// the watch request has been processed by the v2Client.
rdsCallbackCh := make(chan error, 1)
v2c.watchRDS(goodRouteName1, func(u rdsUpdate, err error) {
t.Logf("Received callback with rdsUpdate {%+v} and error {%v}", u, err)
if u.clusterName != "" {
rdsCallbackCh <- fmt.Errorf("received clusterName %v in rdsCallback, wanted empty string", u.clusterName)
}
if err == nil {
rdsCallbackCh <- errors.New("received nil error in rdsCallback")
}
rdsCallbackCh <- nil
})
<-fakeServer.RequestChan
timer := time.NewTimer(2 * time.Second)
select {
case <-timer.C:
t.Fatalf("Timeout expired when expecting RDS update")
case err := <-rdsCallbackCh:
timer.Stop()
if err != nil {
t.Fatal(err)
}
}
}

View File

@ -0,0 +1,82 @@
/*
*
* 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 (
listenerURL = "type.googleapis.com/envoy.api.v2.Listener"
routeURL = "type.googleapis.com/envoy.api.v2.RouteConfiguration"
clusterURL = "type.googleapis.com/envoy.api.v2.Cluster"
endpointURL = "type.googleapis.com/envoy.api.v2.ClusterLoadAssignment"
)
// resourceType is an enum to represent the different xDS resources.
type resourceType int
const (
ldsResource resourceType = iota
rdsResource
cdsResource
edsResource
)
// 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 {
wType resourceType
target []string
state watchState
callback interface{}
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()
}
}
type ldsUpdate struct {
routeName string
}
type ldsCallback func(ldsUpdate, error)
type rdsUpdate struct {
clusterName string
}
type rdsCallback func(rdsUpdate, error)

View File

@ -0,0 +1,326 @@
/*
*
* 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 (
"context"
"fmt"
"sync"
"time"
"google.golang.org/grpc"
"google.golang.org/grpc/grpclog"
"google.golang.org/grpc/internal/buffer"
corepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
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
// 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.
// 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
// ClientConn to the xDS gRPC server. Owned by the parent xdsClient.
cc *grpc.ClientConn
nodeProto *corepb.Node
backoff func(int) time.Duration
// watchCh in the channel onto which watchInfo objects are pushed by the
// watch API, and it is read and acted upon by the send() goroutine.
watchCh *buffer.Unbounded
mu sync.Mutex
// Message specific watch infos, protected by the above mutex. These are
// written to, after successfully reading from the update channel, and are
// read from when recovering from a broken stream to resend the xDS
// 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[resourceType]*watchInfo
// 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.
rdsCache map[string]string
}
// newV2Client creates a new v2Client initialized with the passed arguments.
func newV2Client(cc *grpc.ClientConn, nodeProto *corepb.Node, backoff func(int) time.Duration) *v2Client {
v2c := &v2Client{
cc: cc,
nodeProto: nodeProto,
backoff: backoff,
watchCh: buffer.NewUnbounded(),
watchMap: make(map[resourceType]*watchInfo),
rdsCache: make(map[string]string),
}
v2c.ctx, v2c.cancelCtx = context.WithCancel(context.Background())
go v2c.run()
return v2c
}
// close cleans up resources and goroutines allocated by this client.
func (v2c *v2Client) close() {
v2c.cancelCtx()
}
// run starts an ADS stream (and backs off exponentially, if the previous
// stream failed without receiving a single reply) and runs the sender and
// receiver routines to send and receive data from the stream respectively.
func (v2c *v2Client) run() {
retries := 0
for {
select {
case <-v2c.ctx.Done():
return
default:
}
if retries != 0 {
t := time.NewTimer(v2c.backoff(retries))
select {
case <-t.C:
case <-v2c.ctx.Done():
if !t.Stop() {
<-t.C
}
return
}
}
retries++
cli := adsgrpc.NewAggregatedDiscoveryServiceClient(v2c.cc)
stream, err := cli.StreamAggregatedResources(v2c.ctx, grpc.WaitForReady(true))
if err != nil {
grpclog.Infof("xds: ADS stream creation failed: %v", err)
continue
}
// send() could be blocked on reading updates from the different update
// channels when it is not actually sending out messages. So, we need a
// way to break out of send() when recv() returns. This done channel is
// used to for that purpose.
done := make(chan struct{})
go v2c.send(stream, done)
if v2c.recv(stream) {
retries = 0
}
close(done)
}
}
// sendExisting sends out xDS requests for registered watchers when recovering
// from a broken stream.
//
// We call stream.Send() here with the lock being held. It should be OK to do
// that here because the stream has just started and Send() usually returns
// quickly (once it pushes the message onto the transport layer) and is only
// ever blocked if we don't have enough flow control quota.
func (v2c *v2Client) sendExisting(stream adsStream) bool {
v2c.mu.Lock()
defer v2c.mu.Unlock()
for wType, wi := range v2c.watchMap {
switch wType {
case ldsResource:
if !v2c.sendLDS(stream, wi.target) {
return false
}
case rdsResource:
if !v2c.sendRDS(stream, wi.target) {
return false
}
}
}
return true
}
// send reads watch infos from update channel and sends out actual xDS requests
// on the provided ADS stream.
func (v2c *v2Client) send(stream adsStream, done chan struct{}) {
if !v2c.sendExisting(stream) {
return
}
for {
select {
case <-v2c.ctx.Done():
return
case u := <-v2c.watchCh.Get():
v2c.watchCh.Load()
wi := u.(*watchInfo)
v2c.mu.Lock()
if wi.state == watchCancelled {
v2c.mu.Unlock()
continue
}
wi.state = watchStarted
target := wi.target
v2c.checkCacheAndUpdateWatchMap(wi)
v2c.mu.Unlock()
switch wi.wType {
case ldsResource:
if !v2c.sendLDS(stream, target) {
return
}
case rdsResource:
if !v2c.sendRDS(stream, target) {
return
}
}
case <-done:
return
}
}
}
// recv receives xDS responses on the provided ADS stream and branches out to
// message specific handlers.
func (v2c *v2Client) recv(stream adsStream) bool {
success := false
for {
resp, err := stream.Recv()
if err != nil {
grpclog.Warningf("xds: ADS stream recv failed: %v", err)
return success
}
switch resp.GetTypeUrl() {
case listenerURL:
if err := v2c.handleLDSResponse(resp); err != nil {
grpclog.Warningf("xds: LDS response handler failed: %v", err)
return success
}
case routeURL:
if err := v2c.handleRDSResponse(resp); err != nil {
grpclog.Warningf("xds: RDS response handler failed: %v", err)
return success
}
default:
grpclog.Warningf("xds: unknown response URL type: %v", resp.GetTypeUrl())
}
success = true
}
}
// 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 ldsCallback) (cancel func()) {
wi := &watchInfo{wType: ldsResource, target: []string{target}, callback: ldsCb}
v2c.watchCh.Put(wi)
return func() {
v2c.mu.Lock()
defer v2c.mu.Unlock()
if wi.state == watchEnqueued {
wi.state = watchCancelled
return
}
v2c.watchMap[ldsResource].cancel()
delete(v2c.watchMap, ldsResource)
}
}
// 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 rdsCallback) (cancel func()) {
wi := &watchInfo{wType: rdsResource, target: []string{routeName}, callback: rdsCb}
v2c.watchCh.Put(wi)
return func() {
v2c.mu.Lock()
defer v2c.mu.Unlock()
if wi.state == watchEnqueued {
wi.state = watchCancelled
return
}
v2c.watchMap[rdsResource].cancel()
delete(v2c.watchMap, rdsResource)
// 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.
}
}
// 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.wType]; existing != nil {
existing.cancel()
}
v2c.watchMap[wi.wType] = wi
switch wi.wType {
case ldsResource:
wi.expiryTimer = time.AfterFunc(defaultWatchExpiryTimeout, func() {
// We need to grab the lock here because we are accessing the
// watchInfo (which is now stored in the watchMap) from this
// method which will be called when the timer fires.
v2c.mu.Lock()
wi.callback.(ldsCallback)(ldsUpdate{routeName: ""}, fmt.Errorf("xds: LDS target %s not found", wi.target))
v2c.mu.Unlock()
})
case rdsResource:
routeName := wi.target[0]
if cluster := v2c.rdsCache[routeName]; cluster != "" {
// Invoke the callback now, since we found the entry in the cache.
var err error
if v2c.watchMap[ldsResource] == nil {
cluster = ""
err = fmt.Errorf("xds: no LDS watcher found when handling RDS watch for route {%v} from cache", routeName)
}
wi.callback.(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() {
// We need to grab the lock here because we are accessing the
// watchInfo (which is now stored in the watchMap) from this
// method which will be called when the timer fires.
v2c.mu.Lock()
wi.callback.(rdsCallback)(rdsUpdate{clusterName: ""}, fmt.Errorf("xds: RDS target %s not found", wi.target))
v2c.mu.Unlock()
})
}
}

View File

@ -0,0 +1,497 @@
/*
*
* 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"
"testing"
"time"
"github.com/golang/protobuf/proto"
"google.golang.org/grpc/xds/internal/client/fakexds"
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
basepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
routepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/route"
httppb "github.com/envoyproxy/go-control-plane/envoy/config/filter/network/http_connection_manager/v2"
listenerpb "github.com/envoyproxy/go-control-plane/envoy/config/listener/v2"
anypb "github.com/golang/protobuf/ptypes/any"
structpb "github.com/golang/protobuf/ptypes/struct"
)
const (
defaultTestTimeout = 2 * time.Second
goodLDSTarget1 = "lds.target.good:1111"
goodLDSTarget2 = "lds.target.good:2222"
goodRouteName1 = "GoodRouteConfig1"
goodRouteName2 = "GoodRouteConfig2"
uninterestingRouteName = "UninterestingRouteName"
goodMatchingDomain = "lds.target.good"
uninterestingDomain = "uninteresting.domain"
goodClusterName1 = "GoodClusterName1"
goodClusterName2 = "GoodClusterName2"
uninterestingClusterName = "UninterestingClusterName"
httpConnManagerURL = "type.googleapis.com/envoy.config.filter.network.http_connection_manager.v2.HttpConnectionManager"
)
var (
goodNodeProto = &basepb.Node{
Id: "ENVOY_NODE_ID",
Metadata: &structpb.Struct{
Fields: map[string]*structpb.Value{
"TRAFFICDIRECTOR_GRPC_HOSTNAME": {
Kind: &structpb.Value_StringValue{StringValue: "trafficdirector"},
},
},
},
}
goodLDSRequest = &xdspb.DiscoveryRequest{
Node: goodNodeProto,
TypeUrl: listenerURL,
ResourceNames: []string{goodLDSTarget1},
}
goodHTTPConnManager1 = &httppb.HttpConnectionManager{
RouteSpecifier: &httppb.HttpConnectionManager_Rds{
Rds: &httppb.Rds{
RouteConfigName: goodRouteName1,
},
},
}
marshaledConnMgr1, _ = proto.Marshal(goodHTTPConnManager1)
emptyHTTPConnManager = &httppb.HttpConnectionManager{
RouteSpecifier: &httppb.HttpConnectionManager_Rds{
Rds: &httppb.Rds{},
},
}
emptyMarshaledConnMgr, _ = proto.Marshal(emptyHTTPConnManager)
connMgrWithScopedRoutes = &httppb.HttpConnectionManager{
RouteSpecifier: &httppb.HttpConnectionManager_ScopedRoutes{},
}
marshaledConnMgrWithScopedRoutes, _ = proto.Marshal(connMgrWithScopedRoutes)
goodListener1 = &xdspb.Listener{
Name: goodLDSTarget1,
ApiListener: &listenerpb.ApiListener{
ApiListener: &anypb.Any{
TypeUrl: httpConnManagerURL,
Value: marshaledConnMgr1,
},
},
}
marshaledListener1, _ = proto.Marshal(goodListener1)
goodListener2 = &xdspb.Listener{
Name: goodLDSTarget2,
ApiListener: &listenerpb.ApiListener{
ApiListener: &anypb.Any{
TypeUrl: httpConnManagerURL,
Value: marshaledConnMgr1,
},
},
}
marshaledListener2, _ = proto.Marshal(goodListener2)
noAPIListener = &xdspb.Listener{Name: goodLDSTarget1}
marshaledNoAPIListener, _ = proto.Marshal(noAPIListener)
badAPIListener1 = &xdspb.Listener{
Name: goodLDSTarget1,
ApiListener: &listenerpb.ApiListener{
ApiListener: &anypb.Any{
TypeUrl: httpConnManagerURL,
Value: []byte{1, 2, 3, 4},
},
},
}
badAPIListener2 = &xdspb.Listener{
Name: goodLDSTarget2,
ApiListener: &listenerpb.ApiListener{
ApiListener: &anypb.Any{
TypeUrl: httpConnManagerURL,
Value: []byte{1, 2, 3, 4},
},
},
}
badlyMarshaledAPIListener2, _ = proto.Marshal(badAPIListener2)
badResourceListener = &xdspb.Listener{
Name: goodLDSTarget1,
ApiListener: &listenerpb.ApiListener{
ApiListener: &anypb.Any{
TypeUrl: listenerURL,
Value: marshaledListener1,
},
},
}
listenerWithEmptyHTTPConnMgr = &xdspb.Listener{
Name: goodLDSTarget1,
ApiListener: &listenerpb.ApiListener{
ApiListener: &anypb.Any{
TypeUrl: httpConnManagerURL,
Value: emptyMarshaledConnMgr,
},
},
}
listenerWithScopedRoutesRouteConfig = &xdspb.Listener{
Name: goodLDSTarget1,
ApiListener: &listenerpb.ApiListener{
ApiListener: &anypb.Any{
TypeUrl: httpConnManagerURL,
Value: marshaledConnMgrWithScopedRoutes,
},
},
}
goodLDSResponse1 = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: listenerURL,
Value: marshaledListener1,
},
},
TypeUrl: listenerURL,
}
goodLDSResponse2 = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: listenerURL,
Value: marshaledListener2,
},
},
TypeUrl: listenerURL,
}
emptyLDSResponse = &xdspb.DiscoveryResponse{TypeUrl: listenerURL}
badlyMarshaledLDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: listenerURL,
Value: []byte{1, 2, 3, 4},
},
},
TypeUrl: listenerURL,
}
badResourceTypeInLDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: listenerURL,
Value: marshaledConnMgr1,
},
},
TypeUrl: listenerURL,
}
ldsResponseWithMultipleResources = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: listenerURL,
Value: marshaledListener2,
},
{
TypeUrl: listenerURL,
Value: marshaledListener1,
},
},
TypeUrl: listenerURL,
}
noAPIListenerLDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: listenerURL,
Value: marshaledNoAPIListener,
},
},
TypeUrl: listenerURL,
}
goodBadUglyLDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: listenerURL,
Value: marshaledListener2,
},
{
TypeUrl: listenerURL,
Value: marshaledListener1,
},
{
TypeUrl: listenerURL,
Value: badlyMarshaledAPIListener2,
},
},
TypeUrl: listenerURL,
}
badlyMarshaledRDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: routeURL,
Value: []byte{1, 2, 3, 4},
},
},
TypeUrl: routeURL,
}
badResourceTypeInRDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: routeURL,
Value: marshaledConnMgr1,
},
},
TypeUrl: routeURL,
}
emptyRouteConfig = &xdspb.RouteConfiguration{}
marshaledEmptyRouteConfig, _ = proto.Marshal(emptyRouteConfig)
noDomainsInRouteConfig = &xdspb.RouteConfiguration{
VirtualHosts: []*routepb.VirtualHost{{}},
}
noVirtualHostsInRDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: routeURL,
Value: marshaledEmptyRouteConfig,
},
},
TypeUrl: routeURL,
}
goodRouteConfig1 = &xdspb.RouteConfiguration{
Name: goodRouteName1,
VirtualHosts: []*routepb.VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*routepb.Route{
{
Action: &routepb.Route_Route{
Route: &routepb.RouteAction{
ClusterSpecifier: &routepb.RouteAction_Cluster{Cluster: uninterestingClusterName},
},
},
},
},
},
{
Domains: []string{goodMatchingDomain},
Routes: []*routepb.Route{
{
Action: &routepb.Route_Route{
Route: &routepb.RouteAction{
ClusterSpecifier: &routepb.RouteAction_Cluster{Cluster: goodClusterName1},
},
},
},
},
},
},
}
marshaledGoodRouteConfig1, _ = proto.Marshal(goodRouteConfig1)
goodRouteConfig2 = &xdspb.RouteConfiguration{
Name: goodRouteName2,
VirtualHosts: []*routepb.VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*routepb.Route{
{
Action: &routepb.Route_Route{
Route: &routepb.RouteAction{
ClusterSpecifier: &routepb.RouteAction_Cluster{Cluster: uninterestingClusterName},
},
},
},
},
},
{
Domains: []string{goodMatchingDomain},
Routes: []*routepb.Route{
{
Action: &routepb.Route_Route{
Route: &routepb.RouteAction{
ClusterSpecifier: &routepb.RouteAction_Cluster{Cluster: goodClusterName2},
},
},
},
},
},
},
}
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{
Resources: []*anypb.Any{
{
TypeUrl: routeURL,
Value: marshaledGoodRouteConfig1,
},
},
TypeUrl: routeURL,
}
goodRDSResponse2 = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: routeURL,
Value: marshaledGoodRouteConfig2,
},
},
TypeUrl: routeURL,
}
uninterestingRDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: routeURL,
Value: marshaledUninterestingRouteConfig,
},
},
TypeUrl: routeURL,
}
)
// TestV2ClientBackoffAfterRecvError verifies if the v2Client backoffs when it
// encounters a Recv error while receiving an LDS response.
func TestV2ClientBackoffAfterRecvError(t *testing.T) {
fakeServer, client, cleanup := fakexds.StartClientAndServer(t)
defer cleanup()
// Override the v2Client backoff function with this, so that we can verify
// that a backoff actually was triggerred.
boCh := make(chan int, 1)
clientBackoff := func(v int) time.Duration {
boCh <- v
return 0
}
v2c := newV2Client(client, goodNodeProto, clientBackoff)
defer v2c.close()
t.Log("Started xds v2Client...")
callbackCh := make(chan struct{})
v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
close(callbackCh)
})
<-fakeServer.RequestChan
t.Log("FakeServer received request...")
fakeServer.ResponseChan <- &fakexds.Response{Err: errors.New("RPC error")}
t.Log("Bad LDS response pushed to fakeServer...")
timer := time.NewTimer(defaultTestTimeout)
select {
case <-timer.C:
t.Fatal("Timeout when expecting LDS update")
case <-boCh:
timer.Stop()
t.Log("v2Client backed off before retrying...")
case <-callbackCh:
t.Fatal("Received unexpected LDS callback")
}
}
// TestV2ClientRetriesAfterBrokenStream verifies the case where a stream
// encountered a Recv() error, and is expected to send out xDS requests for
// registered watchers once it comes back up again.
func TestV2ClientRetriesAfterBrokenStream(t *testing.T) {
fakeServer, client, cleanup := fakexds.StartClientAndServer(t)
defer cleanup()
v2c := newV2Client(client, goodNodeProto, func(int) time.Duration { return 0 })
defer v2c.close()
t.Log("Started xds v2Client...")
callbackCh := make(chan struct{}, 1)
v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
t.Logf("Received LDS callback with ldsUpdate {%+v} and error {%v}", u, err)
callbackCh <- struct{}{}
})
<-fakeServer.RequestChan
t.Log("FakeServer received request...")
fakeServer.ResponseChan <- &fakexds.Response{Resp: goodLDSResponse1}
t.Log("Good LDS response pushed to fakeServer...")
timer := time.NewTimer(defaultTestTimeout)
select {
case <-timer.C:
t.Fatal("Timeout when expecting LDS update")
case <-callbackCh:
timer.Stop()
}
fakeServer.ResponseChan <- &fakexds.Response{Err: errors.New("RPC error")}
t.Log("Bad LDS response pushed to fakeServer...")
timer = time.NewTimer(defaultTestTimeout)
select {
case <-timer.C:
t.Fatal("Timeout when expecting LDS update")
case gotRequest := <-fakeServer.RequestChan:
timer.Stop()
t.Log("received LDS request after stream re-creation")
if !proto.Equal(gotRequest.Req, goodLDSRequest) {
t.Fatalf("gotRequest: %+v, wantRequest: %+v", gotRequest.Req, goodLDSRequest)
}
}
}
// TestV2ClientCancelWatch verifies that the registered watch callback is not
// invoked if a response is received after the watcher is cancelled.
func TestV2ClientCancelWatch(t *testing.T) {
fakeServer, client, cleanup := fakexds.StartClientAndServer(t)
defer cleanup()
v2c := newV2Client(client, goodNodeProto, func(int) time.Duration { return 0 })
defer v2c.close()
t.Log("Started xds v2Client...")
callbackCh := make(chan struct{}, 1)
cancelFunc := v2c.watchLDS(goodLDSTarget1, func(u ldsUpdate, err error) {
t.Logf("Received LDS callback with ldsUpdate {%+v} and error {%v}", u, err)
callbackCh <- struct{}{}
})
<-fakeServer.RequestChan
t.Log("FakeServer received request...")
fakeServer.ResponseChan <- &fakexds.Response{Resp: goodLDSResponse1}
t.Log("Good LDS response pushed to fakeServer...")
timer := time.NewTimer(defaultTestTimeout)
select {
case <-timer.C:
t.Fatal("Timeout when expecting LDS update")
case <-callbackCh:
timer.Stop()
}
cancelFunc()
fakeServer.ResponseChan <- &fakexds.Response{Resp: goodLDSResponse1}
t.Log("Another good LDS response pushed to fakeServer...")
timer = time.NewTimer(defaultTestTimeout)
select {
case <-timer.C:
case <-callbackCh:
timer.Stop()
t.Fatalf("Watch callback invoked after the watcher was cancelled")
}
}