xdsclient: resource agnostic API implementation (#5776)

This commit is contained in:
Easwar Swaminathan 2022-12-21 13:53:03 -08:00 committed by GitHub
parent 07ac97c355
commit 08479c5e2e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
59 changed files with 3379 additions and 7563 deletions

View File

@ -25,55 +25,58 @@ package csds
import (
"context"
"fmt"
"io"
"sync"
"github.com/golang/protobuf/proto"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/grpclog"
internalgrpclog "google.golang.org/grpc/internal/grpclog"
"google.golang.org/grpc/status"
"google.golang.org/grpc/xds/internal/xdsclient"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/protobuf/types/known/timestamppb"
v3adminpb "github.com/envoyproxy/go-control-plane/envoy/admin/v3"
v2corepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
v3statusgrpc "github.com/envoyproxy/go-control-plane/envoy/service/status/v3"
v3statuspb "github.com/envoyproxy/go-control-plane/envoy/service/status/v3"
"github.com/golang/protobuf/proto"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/grpclog"
"google.golang.org/grpc/status"
"google.golang.org/grpc/xds/internal/xdsclient"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/protobuf/types/known/timestamppb"
_ "google.golang.org/grpc/xds/internal/xdsclient/controller/version/v2" // Register v2 xds_client.
_ "google.golang.org/grpc/xds/internal/xdsclient/controller/version/v3" // Register v3 xds_client.
)
var (
logger = grpclog.Component("xds")
newXDSClient = func() xdsclient.XDSClient {
c, err := xdsclient.New()
if err != nil {
logger.Warningf("failed to create xds client: %v", err)
return nil
}
return c
}
)
var logger = grpclog.Component("xds")
const (
listenerTypeURL = "envoy.config.listener.v3.Listener"
routeConfigTypeURL = "envoy.config.route.v3.RouteConfiguration"
clusterTypeURL = "envoy.config.cluster.v3.Cluster"
endpointsTypeURL = "envoy.config.endpoint.v3.ClusterLoadAssignment"
)
const prefix = "[csds-server %p] "
// ClientStatusDiscoveryServer implementations interface ClientStatusDiscoveryServiceServer.
func prefixLogger(s *ClientStatusDiscoveryServer) *internalgrpclog.PrefixLogger {
return internalgrpclog.NewPrefixLogger(logger, fmt.Sprintf(prefix, s))
}
// ClientStatusDiscoveryServer provides an implementation of the Client Status
// Discovery Service (CSDS) for exposing the xDS config of a given client. See
// https://github.com/envoyproxy/envoy/blob/main/api/envoy/service/status/v3/csds.proto.
//
// For more details about the gRPC implementation of CSDS, refer to gRPC A40 at:
// https://github.com/grpc/proposal/blob/master/A40-csds-support.md.
type ClientStatusDiscoveryServer struct {
// xdsClient will always be the same in practice. But we keep a copy in each
// server instance for testing.
logger *internalgrpclog.PrefixLogger
mu sync.Mutex
xdsClient xdsclient.XDSClient
}
// NewClientStatusDiscoveryServer returns an implementation of the CSDS server that can be
// registered on a gRPC server.
// NewClientStatusDiscoveryServer returns an implementation of the CSDS server
// that can be registered on a gRPC server.
func NewClientStatusDiscoveryServer() (*ClientStatusDiscoveryServer, error) {
return &ClientStatusDiscoveryServer{xdsClient: newXDSClient()}, nil
c, err := xdsclient.New()
if err != nil {
logger.Warningf("Failed to create xDS client: %v", err)
}
s := &ClientStatusDiscoveryServer{xdsClient: c}
s.logger = prefixLogger(s)
s.logger.Infof("Created CSDS server, with xdsClient %p", c)
return s, nil
}
// StreamClientStatus implementations interface ClientStatusDiscoveryServiceServer.
@ -106,6 +109,9 @@ func (s *ClientStatusDiscoveryServer) FetchClientStatus(_ context.Context, req *
//
// If it returns an error, the error is a status error.
func (s *ClientStatusDiscoveryServer) buildClientStatusRespForReq(req *v3statuspb.ClientStatusRequest) (*v3statuspb.ClientStatusResponse, error) {
s.mu.Lock()
defer s.mu.Unlock()
if s.xdsClient == nil {
return &v3statuspb.ClientStatusResponse{}, nil
}
@ -115,21 +121,12 @@ func (s *ClientStatusDiscoveryServer) buildClientStatusRespForReq(req *v3statusp
return nil, status.Errorf(codes.InvalidArgument, "node_matchers are not supported, request contains node_matchers: %v", req.NodeMatchers)
}
lds := dumpToGenericXdsConfig(listenerTypeURL, s.xdsClient.DumpLDS)
rds := dumpToGenericXdsConfig(routeConfigTypeURL, s.xdsClient.DumpRDS)
cds := dumpToGenericXdsConfig(clusterTypeURL, s.xdsClient.DumpCDS)
eds := dumpToGenericXdsConfig(endpointsTypeURL, s.xdsClient.DumpEDS)
configs := make([]*v3statuspb.ClientConfig_GenericXdsConfig, 0, len(lds)+len(rds)+len(cds)+len(eds))
configs = append(configs, lds...)
configs = append(configs, rds...)
configs = append(configs, cds...)
configs = append(configs, eds...)
dump := s.xdsClient.DumpResources()
ret := &v3statuspb.ClientStatusResponse{
Config: []*v3statuspb.ClientConfig{
{
Node: nodeProtoToV3(s.xdsClient.BootstrapConfig().XDSServer.NodeProto),
GenericXdsConfigs: configs,
Node: nodeProtoToV3(s.xdsClient.BootstrapConfig().XDSServer.NodeProto, s.logger),
GenericXdsConfigs: dumpToGenericXdsConfig(dump),
},
},
}
@ -138,9 +135,11 @@ func (s *ClientStatusDiscoveryServer) buildClientStatusRespForReq(req *v3statusp
// Close cleans up the resources.
func (s *ClientStatusDiscoveryServer) Close() {
s.mu.Lock()
if s.xdsClient != nil {
s.xdsClient.Close()
}
s.mu.Unlock()
}
// nodeProtoToV3 converts the given proto into a v3.Node. n is from bootstrap
@ -153,7 +152,7 @@ func (s *ClientStatusDiscoveryServer) Close() {
// The default case (not v2 or v3) is nil, instead of error, because the
// resources in the response are more important than the node. The worst case is
// that the user will receive no Node info, but will still get resources.
func nodeProtoToV3(n proto.Message) *v3corepb.Node {
func nodeProtoToV3(n proto.Message, logger *internalgrpclog.PrefixLogger) *v3corepb.Node {
var node *v3corepb.Node
switch nn := n.(type) {
case *v3corepb.Node:
@ -174,26 +173,27 @@ func nodeProtoToV3(n proto.Message) *v3corepb.Node {
return node
}
func dumpToGenericXdsConfig(typeURL string, dumpF func() map[string]xdsresource.UpdateWithMD) []*v3statuspb.ClientConfig_GenericXdsConfig {
dump := dumpF()
ret := make([]*v3statuspb.ClientConfig_GenericXdsConfig, 0, len(dump))
for name, d := range dump {
config := &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: typeURL,
Name: name,
VersionInfo: d.MD.Version,
XdsConfig: d.Raw,
LastUpdated: timestamppb.New(d.MD.Timestamp),
ClientStatus: serviceStatusToProto(d.MD.Status),
}
if errState := d.MD.ErrState; errState != nil {
config.ErrorState = &v3adminpb.UpdateFailureState{
LastUpdateAttempt: timestamppb.New(errState.Timestamp),
Details: errState.Err.Error(),
VersionInfo: errState.Version,
func dumpToGenericXdsConfig(dump map[string]map[string]xdsresource.UpdateWithMD) []*v3statuspb.ClientConfig_GenericXdsConfig {
var ret []*v3statuspb.ClientConfig_GenericXdsConfig
for typeURL, updates := range dump {
for name, update := range updates {
config := &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: typeURL,
Name: name,
VersionInfo: update.MD.Version,
XdsConfig: update.Raw,
LastUpdated: timestamppb.New(update.MD.Timestamp),
ClientStatus: serviceStatusToProto(update.MD.Status),
}
if errState := update.MD.ErrState; errState != nil {
config.ErrorState = &v3adminpb.UpdateFailureState{
LastUpdateAttempt: timestamppb.New(errState.Timestamp),
Details: errState.Err.Error(),
VersionInfo: errState.Version,
}
}
ret = append(ret, config)
}
ret = append(ret, config)
}
return ret
}

434
xds/csds/csds_e2e_test.go Normal file
View File

@ -0,0 +1,434 @@
/*
*
* Copyright 2021 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 csds_test
import (
"context"
"fmt"
"io"
"sort"
"strings"
"testing"
"time"
"github.com/golang/protobuf/proto"
"github.com/google/go-cmp/cmp"
"github.com/google/uuid"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
"google.golang.org/grpc/internal/grpctest"
"google.golang.org/grpc/internal/testutils"
"google.golang.org/grpc/internal/testutils/xds/bootstrap"
"google.golang.org/grpc/internal/testutils/xds/e2e"
"google.golang.org/grpc/xds/csds"
"google.golang.org/grpc/xds/internal/xdsclient"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/protobuf/testing/protocmp"
"google.golang.org/protobuf/types/known/anypb"
v3adminpb "github.com/envoyproxy/go-control-plane/envoy/admin/v3"
v3clusterpb "github.com/envoyproxy/go-control-plane/envoy/config/cluster/v3"
v3endpointpb "github.com/envoyproxy/go-control-plane/envoy/config/endpoint/v3"
v3listenerpb "github.com/envoyproxy/go-control-plane/envoy/config/listener/v3"
v3routepb "github.com/envoyproxy/go-control-plane/envoy/config/route/v3"
v3statuspb "github.com/envoyproxy/go-control-plane/envoy/service/status/v3"
v3statuspbgrpc "github.com/envoyproxy/go-control-plane/envoy/service/status/v3"
_ "google.golang.org/grpc/xds/internal/httpfilter/router" // Register the router filter
)
const defaultTestTimeout = 5 * time.Second
var cmpOpts = cmp.Options{
cmp.Transformer("sort", func(in []*v3statuspb.ClientConfig_GenericXdsConfig) []*v3statuspb.ClientConfig_GenericXdsConfig {
out := append([]*v3statuspb.ClientConfig_GenericXdsConfig(nil), in...)
sort.Slice(out, func(i, j int) bool {
a, b := out[i], out[j]
if a == nil {
return true
}
if b == nil {
return false
}
if strings.Compare(a.TypeUrl, b.TypeUrl) == 0 {
return strings.Compare(a.Name, b.Name) < 0
}
return strings.Compare(a.TypeUrl, b.TypeUrl) < 0
})
return out
}),
protocmp.Transform(),
protocmp.IgnoreFields((*v3statuspb.ClientConfig_GenericXdsConfig)(nil), "last_updated"),
protocmp.IgnoreFields((*v3adminpb.UpdateFailureState)(nil), "last_update_attempt", "details"),
}
type s struct {
grpctest.Tester
}
func Test(t *testing.T) {
grpctest.RunSubTests(t, s{})
}
func (s) TestCSDS(t *testing.T) {
// Spin up a xDS management server on a local port.
nodeID := uuid.New().String()
mgmtServer, err := e2e.StartManagementServer(e2e.ManagementServerOptions{})
if err != nil {
t.Fatal(err)
}
defer mgmtServer.Stop()
// Create a bootstrap file in a temporary directory.
bootstrapCleanup, err := bootstrap.CreateFile(bootstrap.Options{
Version: bootstrap.TransportV3,
NodeID: nodeID,
ServerURI: mgmtServer.Address,
})
if err != nil {
t.Fatal(err)
}
defer bootstrapCleanup()
// Create an xDS client. This will end up using the same singleton as used
// by the CSDS service.
xdsC, err := xdsclient.New()
if err != nil {
t.Fatalf("Failed to create xDS client: %v", err)
}
defer xdsC.Close()
// Initialize an gRPC server and register CSDS on it.
server := grpc.NewServer()
csdss, err := csds.NewClientStatusDiscoveryServer()
if err != nil {
t.Fatal(err)
}
v3statuspbgrpc.RegisterClientStatusDiscoveryServiceServer(server, csdss)
defer func() {
server.Stop()
csdss.Close()
}()
// Create a local listener and pass it to Serve().
lis, err := testutils.LocalTCPListener()
if err != nil {
t.Fatalf("testutils.LocalTCPListener() failed: %v", err)
}
go func() {
if err := server.Serve(lis); err != nil {
t.Errorf("Serve() failed: %v", err)
}
}()
// Create a client to the CSDS server.
conn, err := grpc.Dial(lis.Addr().String(), grpc.WithTransportCredentials(insecure.NewCredentials()))
if err != nil {
t.Fatalf("Failed to dial CSDS server %q: %v", lis.Addr().String(), err)
}
c := v3statuspbgrpc.NewClientStatusDiscoveryServiceClient(conn)
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
stream, err := c.StreamClientStatus(ctx, grpc.WaitForReady(true))
if err != nil {
t.Fatalf("Failed to create a stream for CSDS: %v", err)
}
defer conn.Close()
// Verify that the xDS client reports an empty config.
if err := checkClientStatusResponse(stream, nil); err != nil {
t.Fatal(err)
}
// Initialize the xDS resources to be used in this test.
ldsTargets := []string{"lds.target.good:0000", "lds.target.good:1111"}
rdsTargets := []string{"route-config-0", "route-config-1"}
cdsTargets := []string{"cluster-0", "cluster-1"}
edsTargets := []string{"endpoints-0", "endpoints-1"}
listeners := make([]*v3listenerpb.Listener, len(ldsTargets))
listenerAnys := make([]*anypb.Any, len(ldsTargets))
for i := range ldsTargets {
listeners[i] = e2e.DefaultClientListener(ldsTargets[i], rdsTargets[i])
listenerAnys[i] = testutils.MarshalAny(listeners[i])
}
routes := make([]*v3routepb.RouteConfiguration, len(rdsTargets))
routeAnys := make([]*anypb.Any, len(rdsTargets))
for i := range rdsTargets {
routes[i] = e2e.DefaultRouteConfig(rdsTargets[i], ldsTargets[i], cdsTargets[i])
routeAnys[i] = testutils.MarshalAny(routes[i])
}
clusters := make([]*v3clusterpb.Cluster, len(cdsTargets))
clusterAnys := make([]*anypb.Any, len(cdsTargets))
for i := range cdsTargets {
clusters[i] = e2e.DefaultCluster(cdsTargets[i], edsTargets[i], e2e.SecurityLevelNone)
clusterAnys[i] = testutils.MarshalAny(clusters[i])
}
endpoints := make([]*v3endpointpb.ClusterLoadAssignment, len(edsTargets))
endpointAnys := make([]*anypb.Any, len(edsTargets))
ips := []string{"0.0.0.0", "1.1.1.1"}
ports := []uint32{123, 456}
for i := range edsTargets {
endpoints[i] = e2e.DefaultEndpoint(edsTargets[i], ips[i], ports[i:i+1])
endpointAnys[i] = testutils.MarshalAny(endpoints[i])
}
// Register watches on the xDS client for two resources of each type.
for _, target := range ldsTargets {
xdsC.WatchListener(target, func(xdsresource.ListenerUpdate, error) {})
}
for _, target := range rdsTargets {
xdsC.WatchRouteConfig(target, func(xdsresource.RouteConfigUpdate, error) {})
}
for _, target := range cdsTargets {
xdsC.WatchCluster(target, func(xdsresource.ClusterUpdate, error) {})
}
for _, target := range edsTargets {
xdsC.WatchEndpoints(target, func(xdsresource.EndpointsUpdate, error) {})
}
// Verify that the xDS client reports the resources as being in "Requested"
// state.
want := []*v3statuspb.ClientConfig_GenericXdsConfig{}
for i := range ldsTargets {
want = append(want, makeGenericXdsConfig("type.googleapis.com/envoy.config.listener.v3.Listener", ldsTargets[i], "", v3adminpb.ClientResourceStatus_REQUESTED, nil))
}
for i := range rdsTargets {
want = append(want, makeGenericXdsConfig("type.googleapis.com/envoy.config.route.v3.RouteConfiguration", rdsTargets[i], "", v3adminpb.ClientResourceStatus_REQUESTED, nil))
}
for i := range cdsTargets {
want = append(want, makeGenericXdsConfig("type.googleapis.com/envoy.config.cluster.v3.Cluster", cdsTargets[i], "", v3adminpb.ClientResourceStatus_REQUESTED, nil))
}
for i := range edsTargets {
want = append(want, makeGenericXdsConfig("type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment", edsTargets[i], "", v3adminpb.ClientResourceStatus_REQUESTED, nil))
}
for {
if err := ctx.Err(); err != nil {
t.Fatalf("Timeout when waiting for resources in \"Requested\" state: %v", err)
}
if err := checkClientStatusResponse(stream, want); err == nil {
break
}
time.Sleep(time.Millisecond * 100)
}
// Configure the management server with two resources of each type,
// corresponding to the watches registered above.
if err := mgmtServer.Update(ctx, e2e.UpdateOptions{
NodeID: nodeID,
Listeners: listeners,
Routes: routes,
Clusters: clusters,
Endpoints: endpoints,
}); err != nil {
t.Fatal(err)
}
// Verify that the xDS client reports the resources as being in "ACKed"
// state, and in version "1".
want = nil
for i := range ldsTargets {
want = append(want, makeGenericXdsConfig("type.googleapis.com/envoy.config.listener.v3.Listener", ldsTargets[i], "1", v3adminpb.ClientResourceStatus_ACKED, listenerAnys[i]))
}
for i := range rdsTargets {
want = append(want, makeGenericXdsConfig("type.googleapis.com/envoy.config.route.v3.RouteConfiguration", rdsTargets[i], "1", v3adminpb.ClientResourceStatus_ACKED, routeAnys[i]))
}
for i := range cdsTargets {
want = append(want, makeGenericXdsConfig("type.googleapis.com/envoy.config.cluster.v3.Cluster", cdsTargets[i], "1", v3adminpb.ClientResourceStatus_ACKED, clusterAnys[i]))
}
for i := range edsTargets {
want = append(want, makeGenericXdsConfig("type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment", edsTargets[i], "1", v3adminpb.ClientResourceStatus_ACKED, endpointAnys[i]))
}
for {
if err := ctx.Err(); err != nil {
t.Fatalf("Timeout when waiting for resources in \"ACKed\" state: %v", err)
}
err := checkClientStatusResponse(stream, want)
if err == nil {
break
}
time.Sleep(time.Millisecond * 100)
}
// Update the first resource of each type in the management server to a
// value which is expected to be NACK'ed by the xDS client.
const nackResourceIdx = 0
listeners[nackResourceIdx].ApiListener = &v3listenerpb.ApiListener{}
routes[nackResourceIdx].VirtualHosts = []*v3routepb.VirtualHost{{Routes: []*v3routepb.Route{{}}}}
clusters[nackResourceIdx].ClusterDiscoveryType = &v3clusterpb.Cluster_Type{Type: v3clusterpb.Cluster_STATIC}
endpoints[nackResourceIdx].Endpoints = []*v3endpointpb.LocalityLbEndpoints{{}}
if err := mgmtServer.Update(ctx, e2e.UpdateOptions{
NodeID: nodeID,
Listeners: listeners,
Routes: routes,
Clusters: clusters,
Endpoints: endpoints,
SkipValidation: true,
}); err != nil {
t.Fatal(err)
}
// Verify that the xDS client reports the first resource of each type as
// being in "NACKed" state, and the second resource of each type to be in
// "ACKed" state. The version for the ACKed resource would be "2", while
// that for the NACKed resource would be "1". In the NACKed resource, the
// version which is NACKed is stored in the ErrorState field.
want = nil
for i := range ldsTargets {
config := makeGenericXdsConfig("type.googleapis.com/envoy.config.listener.v3.Listener", ldsTargets[i], "2", v3adminpb.ClientResourceStatus_ACKED, listenerAnys[i])
if i == nackResourceIdx {
config.VersionInfo = "1"
config.ClientStatus = v3adminpb.ClientResourceStatus_NACKED
config.ErrorState = &v3adminpb.UpdateFailureState{VersionInfo: "2"}
}
want = append(want, config)
}
for i := range rdsTargets {
config := makeGenericXdsConfig("type.googleapis.com/envoy.config.route.v3.RouteConfiguration", rdsTargets[i], "2", v3adminpb.ClientResourceStatus_ACKED, routeAnys[i])
if i == nackResourceIdx {
config.VersionInfo = "1"
config.ClientStatus = v3adminpb.ClientResourceStatus_NACKED
config.ErrorState = &v3adminpb.UpdateFailureState{VersionInfo: "2"}
}
want = append(want, config)
}
for i := range cdsTargets {
config := makeGenericXdsConfig("type.googleapis.com/envoy.config.cluster.v3.Cluster", cdsTargets[i], "2", v3adminpb.ClientResourceStatus_ACKED, clusterAnys[i])
if i == nackResourceIdx {
config.VersionInfo = "1"
config.ClientStatus = v3adminpb.ClientResourceStatus_NACKED
config.ErrorState = &v3adminpb.UpdateFailureState{VersionInfo: "2"}
}
want = append(want, config)
}
for i := range edsTargets {
config := makeGenericXdsConfig("type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment", edsTargets[i], "2", v3adminpb.ClientResourceStatus_ACKED, endpointAnys[i])
if i == nackResourceIdx {
config.VersionInfo = "1"
config.ClientStatus = v3adminpb.ClientResourceStatus_NACKED
config.ErrorState = &v3adminpb.UpdateFailureState{VersionInfo: "2"}
}
want = append(want, config)
}
for {
if err := ctx.Err(); err != nil {
t.Fatalf("Timeout when waiting for resources in \"NACKed\" state: %v", err)
}
err := checkClientStatusResponse(stream, want)
if err == nil {
break
}
time.Sleep(time.Millisecond * 100)
}
}
func makeGenericXdsConfig(typeURL, name, version string, status v3adminpb.ClientResourceStatus, config *anypb.Any) *v3statuspb.ClientConfig_GenericXdsConfig {
return &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: typeURL,
Name: name,
VersionInfo: version,
ClientStatus: status,
XdsConfig: config,
}
}
func checkClientStatusResponse(stream v3statuspbgrpc.ClientStatusDiscoveryService_StreamClientStatusClient, want []*v3statuspb.ClientConfig_GenericXdsConfig) error {
if err := stream.Send(&v3statuspb.ClientStatusRequest{Node: nil}); err != nil {
if err != io.EOF {
return fmt.Errorf("failed to send ClientStatusRequest: %v", err)
}
// If the stream has closed, we call Recv() until it returns a non-nil
// error to get the actual error on the stream.
for {
if _, err := stream.Recv(); err != nil {
return fmt.Errorf("failed to recv ClientStatusResponse: %v", err)
}
}
}
resp, err := stream.Recv()
if err != nil {
return fmt.Errorf("failed to recv ClientStatusResponse: %v", err)
}
if n := len(resp.Config); n != 1 {
return fmt.Errorf("got %d configs, want 1: %v", n, proto.MarshalTextString(resp))
}
if diff := cmp.Diff(resp.Config[0].GenericXdsConfigs, want, cmpOpts); diff != "" {
return fmt.Errorf(diff)
}
return nil
}
func (s) TestCSDSNoXDSClient(t *testing.T) {
// Create a bootstrap file in a temporary directory. Since we pass empty
// options, it would end up creating a bootstrap file with an empty
// serverURI which will fail xDS client creation.
bootstrapCleanup, err := bootstrap.CreateFile(bootstrap.Options{})
if err != nil {
t.Fatal(err)
}
t.Cleanup(func() { bootstrapCleanup() })
// Initialize an gRPC server and register CSDS on it.
server := grpc.NewServer()
csdss, err := csds.NewClientStatusDiscoveryServer()
if err != nil {
t.Fatal(err)
}
defer csdss.Close()
v3statuspbgrpc.RegisterClientStatusDiscoveryServiceServer(server, csdss)
// Create a local listener and pass it to Serve().
lis, err := testutils.LocalTCPListener()
if err != nil {
t.Fatalf("testutils.LocalTCPListener() failed: %v", err)
}
go func() {
if err := server.Serve(lis); err != nil {
t.Errorf("Serve() failed: %v", err)
}
}()
defer server.Stop()
// Create a client to the CSDS server.
conn, err := grpc.Dial(lis.Addr().String(), grpc.WithTransportCredentials(insecure.NewCredentials()))
if err != nil {
t.Fatalf("Failed to dial CSDS server %q: %v", lis.Addr().String(), err)
}
defer conn.Close()
c := v3statuspbgrpc.NewClientStatusDiscoveryServiceClient(conn)
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
stream, err := c.StreamClientStatus(ctx, grpc.WaitForReady(true))
if err != nil {
t.Fatalf("Failed to create a stream for CSDS: %v", err)
}
if err := stream.Send(&v3statuspb.ClientStatusRequest{Node: nil}); err != nil {
t.Fatalf("Failed to send ClientStatusRequest: %v", err)
}
r, err := stream.Recv()
if err != nil {
// io.EOF is not ok.
t.Fatalf("Failed to recv ClientStatusResponse: %v", err)
}
if n := len(r.Config); n != 0 {
t.Fatalf("got %d configs, want 0: %v", n, proto.MarshalTextString(r))
}
}

View File

@ -19,98 +19,15 @@
package csds
import (
"context"
"fmt"
"sort"
"strings"
"testing"
"time"
"github.com/golang/protobuf/proto"
"github.com/google/go-cmp/cmp"
"github.com/google/uuid"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
"google.golang.org/grpc/internal/grpctest"
"google.golang.org/grpc/internal/testutils"
"google.golang.org/grpc/internal/testutils/xds/bootstrap"
"google.golang.org/grpc/internal/testutils/xds/e2e"
_ "google.golang.org/grpc/xds/internal/httpfilter/router"
"google.golang.org/grpc/xds/internal/xdsclient"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/protobuf/testing/protocmp"
"google.golang.org/protobuf/types/known/anypb"
v3adminpb "github.com/envoyproxy/go-control-plane/envoy/admin/v3"
v2corepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
v3clusterpb "github.com/envoyproxy/go-control-plane/envoy/config/cluster/v3"
v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
v3endpointpb "github.com/envoyproxy/go-control-plane/envoy/config/endpoint/v3"
v3listenerpb "github.com/envoyproxy/go-control-plane/envoy/config/listener/v3"
v3routepb "github.com/envoyproxy/go-control-plane/envoy/config/route/v3"
v3statuspb "github.com/envoyproxy/go-control-plane/envoy/service/status/v3"
v3statuspbgrpc "github.com/envoyproxy/go-control-plane/envoy/service/status/v3"
)
const defaultTestTimeout = 10 * time.Second
var cmpOpts = cmp.Options{
cmp.Transformer("sort", func(in []*v3statuspb.ClientConfig_GenericXdsConfig) []*v3statuspb.ClientConfig_GenericXdsConfig {
out := append([]*v3statuspb.ClientConfig_GenericXdsConfig(nil), in...)
sort.Slice(out, func(i, j int) bool {
a, b := out[i], out[j]
if a == nil {
return true
}
if b == nil {
return false
}
if strings.Compare(a.TypeUrl, b.TypeUrl) == 0 {
return strings.Compare(a.Name, b.Name) < 0
}
return strings.Compare(a.TypeUrl, b.TypeUrl) < 0
})
return out
}),
protocmp.Transform(),
}
// filterFields clears unimportant fields in the proto messages.
//
// protocmp.IgnoreFields() doesn't work on nil messages (it panics).
func filterFields(ms []*v3statuspb.ClientConfig_GenericXdsConfig) []*v3statuspb.ClientConfig_GenericXdsConfig {
out := append([]*v3statuspb.ClientConfig_GenericXdsConfig{}, ms...)
for _, m := range out {
if m == nil {
continue
}
m.LastUpdated = nil
if m.ErrorState != nil {
m.ErrorState.Details = "blahblah"
m.ErrorState.LastUpdateAttempt = nil
}
}
return out
}
var (
ldsTargets = []string{"lds.target.good:0000", "lds.target.good:1111"}
listeners = make([]*v3listenerpb.Listener, len(ldsTargets))
listenerAnys = make([]*anypb.Any, len(ldsTargets))
rdsTargets = []string{"route-config-0", "route-config-1"}
routes = make([]*v3routepb.RouteConfiguration, len(rdsTargets))
routeAnys = make([]*anypb.Any, len(rdsTargets))
cdsTargets = []string{"cluster-0", "cluster-1"}
clusters = make([]*v3clusterpb.Cluster, len(cdsTargets))
clusterAnys = make([]*anypb.Any, len(cdsTargets))
edsTargets = []string{"endpoints-0", "endpoints-1"}
endpoints = make([]*v3endpointpb.ClusterLoadAssignment, len(edsTargets))
endpointAnys = make([]*anypb.Any, len(edsTargets))
ips = []string{"0.0.0.0", "1.1.1.1"}
ports = []uint32{123, 456}
)
type s struct {
@ -121,438 +38,7 @@ func Test(t *testing.T) {
grpctest.RunSubTests(t, s{})
}
func init() {
for i := range ldsTargets {
listeners[i] = e2e.DefaultClientListener(ldsTargets[i], rdsTargets[i])
listenerAnys[i] = testutils.MarshalAny(listeners[i])
}
for i := range rdsTargets {
routes[i] = e2e.DefaultRouteConfig(rdsTargets[i], ldsTargets[i], cdsTargets[i])
routeAnys[i] = testutils.MarshalAny(routes[i])
}
for i := range cdsTargets {
clusters[i] = e2e.DefaultCluster(cdsTargets[i], edsTargets[i], e2e.SecurityLevelNone)
clusterAnys[i] = testutils.MarshalAny(clusters[i])
}
for i := range edsTargets {
endpoints[i] = e2e.DefaultEndpoint(edsTargets[i], ips[i], ports[i:i+1])
endpointAnys[i] = testutils.MarshalAny(endpoints[i])
}
}
func (s) TestCSDS(t *testing.T) {
const retryCount = 10
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
xdsC, mgmServer, nodeID, stream, cleanup := commonSetup(ctx, t)
defer cleanup()
for _, target := range ldsTargets {
xdsC.WatchListener(target, func(xdsresource.ListenerUpdate, error) {})
}
for _, target := range rdsTargets {
xdsC.WatchRouteConfig(target, func(xdsresource.RouteConfigUpdate, error) {})
}
for _, target := range cdsTargets {
xdsC.WatchCluster(target, func(xdsresource.ClusterUpdate, error) {})
}
for _, target := range edsTargets {
xdsC.WatchEndpoints(target, func(xdsresource.EndpointsUpdate, error) {})
}
for i := 0; i < retryCount; i++ {
err := checkForRequested(stream)
if err == nil {
break
}
if i == retryCount-1 {
t.Fatalf("%v", err)
}
time.Sleep(time.Millisecond * 100)
}
if err := mgmServer.Update(ctx, e2e.UpdateOptions{
NodeID: nodeID,
Listeners: listeners,
Routes: routes,
Clusters: clusters,
Endpoints: endpoints,
}); err != nil {
t.Fatal(err)
}
for i := 0; i < retryCount; i++ {
err := checkForACKed(stream)
if err == nil {
break
}
if i == retryCount-1 {
t.Fatalf("%v", err)
}
time.Sleep(time.Millisecond * 100)
}
const nackResourceIdx = 0
var (
nackListeners = append([]*v3listenerpb.Listener{}, listeners...)
nackRoutes = append([]*v3routepb.RouteConfiguration{}, routes...)
nackClusters = append([]*v3clusterpb.Cluster{}, clusters...)
nackEndpoints = append([]*v3endpointpb.ClusterLoadAssignment{}, endpoints...)
)
nackListeners[0] = &v3listenerpb.Listener{Name: ldsTargets[nackResourceIdx], ApiListener: &v3listenerpb.ApiListener{}} // 0 will be nacked. 1 will stay the same.
nackRoutes[0] = &v3routepb.RouteConfiguration{
Name: rdsTargets[nackResourceIdx], VirtualHosts: []*v3routepb.VirtualHost{{Routes: []*v3routepb.Route{{}}}},
}
nackClusters[0] = &v3clusterpb.Cluster{
Name: cdsTargets[nackResourceIdx], ClusterDiscoveryType: &v3clusterpb.Cluster_Type{Type: v3clusterpb.Cluster_STATIC},
}
nackEndpoints[0] = &v3endpointpb.ClusterLoadAssignment{
ClusterName: edsTargets[nackResourceIdx], Endpoints: []*v3endpointpb.LocalityLbEndpoints{{}},
}
if err := mgmServer.Update(ctx, e2e.UpdateOptions{
NodeID: nodeID,
Listeners: nackListeners,
Routes: nackRoutes,
Clusters: nackClusters,
Endpoints: nackEndpoints,
SkipValidation: true,
}); err != nil {
t.Fatal(err)
}
for i := 0; i < retryCount; i++ {
err := checkForNACKed(nackResourceIdx, stream)
if err == nil {
break
}
if i == retryCount-1 {
t.Fatalf("%v", err)
}
time.Sleep(time.Millisecond * 100)
}
}
func commonSetup(ctx context.Context, t *testing.T) (xdsclient.XDSClient, *e2e.ManagementServer, string, v3statuspbgrpc.ClientStatusDiscoveryService_StreamClientStatusClient, func()) {
t.Helper()
// Spin up a xDS management server on a local port.
nodeID := uuid.New().String()
fs, err := e2e.StartManagementServer(e2e.ManagementServerOptions{})
if err != nil {
t.Fatal(err)
}
// Create a bootstrap file in a temporary directory.
bootstrapCleanup, err := bootstrap.CreateFile(bootstrap.Options{
Version: bootstrap.TransportV3,
NodeID: nodeID,
ServerURI: fs.Address,
})
if err != nil {
t.Fatal(err)
}
// Create xds_client.
xdsC, err := xdsclient.New()
if err != nil {
t.Fatalf("failed to create xds client: %v", err)
}
origNewXDSClient := newXDSClient
newXDSClient = func() xdsclient.XDSClient { return xdsC }
// Initialize an gRPC server and register CSDS on it.
server := grpc.NewServer()
csdss, err := NewClientStatusDiscoveryServer()
if err != nil {
t.Fatal(err)
}
v3statuspbgrpc.RegisterClientStatusDiscoveryServiceServer(server, csdss)
// Create a local listener and pass it to Serve().
lis, err := testutils.LocalTCPListener()
if err != nil {
t.Fatalf("testutils.LocalTCPListener() failed: %v", err)
}
go func() {
if err := server.Serve(lis); err != nil {
t.Errorf("Serve() failed: %v", err)
}
}()
// Create CSDS client.
conn, err := grpc.Dial(lis.Addr().String(), grpc.WithTransportCredentials(insecure.NewCredentials()))
if err != nil {
t.Fatalf("cannot connect to server: %v", err)
}
c := v3statuspbgrpc.NewClientStatusDiscoveryServiceClient(conn)
stream, err := c.StreamClientStatus(ctx, grpc.WaitForReady(true))
if err != nil {
t.Fatalf("cannot get ServerReflectionInfo: %v", err)
}
return xdsC, fs, nodeID, stream, func() {
fs.Stop()
conn.Close()
server.Stop()
csdss.Close()
newXDSClient = origNewXDSClient
xdsC.Close()
bootstrapCleanup()
}
}
func checkForRequested(stream v3statuspbgrpc.ClientStatusDiscoveryService_StreamClientStatusClient) error {
if err := stream.Send(&v3statuspb.ClientStatusRequest{Node: nil}); err != nil {
return fmt.Errorf("failed to send request: %v", err)
}
r, err := stream.Recv()
if err != nil {
// io.EOF is not ok.
return fmt.Errorf("failed to recv response: %v", err)
}
if n := len(r.Config); n != 1 {
return fmt.Errorf("got %d configs, want 1: %v", n, proto.MarshalTextString(r))
}
var want []*v3statuspb.ClientConfig_GenericXdsConfig
// Status is Requested, but version and xds config are all unset.
for i := range ldsTargets {
want = append(want, &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: listenerTypeURL, Name: ldsTargets[i], ClientStatus: v3adminpb.ClientResourceStatus_REQUESTED,
})
}
for i := range rdsTargets {
want = append(want, &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: routeConfigTypeURL, Name: rdsTargets[i], ClientStatus: v3adminpb.ClientResourceStatus_REQUESTED,
})
}
for i := range cdsTargets {
want = append(want, &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: clusterTypeURL, Name: cdsTargets[i], ClientStatus: v3adminpb.ClientResourceStatus_REQUESTED,
})
}
for i := range edsTargets {
want = append(want, &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: endpointsTypeURL, Name: edsTargets[i], ClientStatus: v3adminpb.ClientResourceStatus_REQUESTED,
})
}
if diff := cmp.Diff(filterFields(r.Config[0].GenericXdsConfigs), want, cmpOpts); diff != "" {
return fmt.Errorf(diff)
}
return nil
}
func checkForACKed(stream v3statuspbgrpc.ClientStatusDiscoveryService_StreamClientStatusClient) error {
const wantVersion = "1"
if err := stream.Send(&v3statuspb.ClientStatusRequest{Node: nil}); err != nil {
return fmt.Errorf("failed to send: %v", err)
}
r, err := stream.Recv()
if err != nil {
// io.EOF is not ok.
return fmt.Errorf("failed to recv response: %v", err)
}
if n := len(r.Config); n != 1 {
return fmt.Errorf("got %d configs, want 1: %v", n, proto.MarshalTextString(r))
}
var want []*v3statuspb.ClientConfig_GenericXdsConfig
// Status is Acked, config is filled with the prebuilt Anys.
for i := range ldsTargets {
want = append(want, &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: listenerTypeURL,
Name: ldsTargets[i],
VersionInfo: wantVersion,
XdsConfig: listenerAnys[i],
ClientStatus: v3adminpb.ClientResourceStatus_ACKED,
})
}
for i := range rdsTargets {
want = append(want, &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: routeConfigTypeURL,
Name: rdsTargets[i],
VersionInfo: wantVersion,
XdsConfig: routeAnys[i],
ClientStatus: v3adminpb.ClientResourceStatus_ACKED,
})
}
for i := range cdsTargets {
want = append(want, &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: clusterTypeURL,
Name: cdsTargets[i],
VersionInfo: wantVersion,
XdsConfig: clusterAnys[i],
ClientStatus: v3adminpb.ClientResourceStatus_ACKED,
})
}
for i := range edsTargets {
want = append(want, &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: endpointsTypeURL,
Name: edsTargets[i],
VersionInfo: wantVersion,
XdsConfig: endpointAnys[i],
ClientStatus: v3adminpb.ClientResourceStatus_ACKED,
})
}
if diff := cmp.Diff(filterFields(r.Config[0].GenericXdsConfigs), want, cmpOpts); diff != "" {
return fmt.Errorf(diff)
}
return nil
}
func checkForNACKed(nackResourceIdx int, stream v3statuspbgrpc.ClientStatusDiscoveryService_StreamClientStatusClient) error {
const (
ackVersion = "1"
nackVersion = "2"
)
if err := stream.Send(&v3statuspb.ClientStatusRequest{Node: nil}); err != nil {
return fmt.Errorf("failed to send: %v", err)
}
r, err := stream.Recv()
if err != nil {
// io.EOF is not ok.
return fmt.Errorf("failed to recv response: %v", err)
}
if n := len(r.Config); n != 1 {
return fmt.Errorf("got %d configs, want 1: %v", n, proto.MarshalTextString(r))
}
var want []*v3statuspb.ClientConfig_GenericXdsConfig
// Resources with the nackIdx are NACKed.
for i := range ldsTargets {
config := &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: listenerTypeURL,
Name: ldsTargets[i],
VersionInfo: nackVersion,
XdsConfig: listenerAnys[i],
ClientStatus: v3adminpb.ClientResourceStatus_ACKED,
}
if i == nackResourceIdx {
config.VersionInfo = ackVersion
config.ClientStatus = v3adminpb.ClientResourceStatus_NACKED
config.ErrorState = &v3adminpb.UpdateFailureState{
Details: "blahblah",
VersionInfo: nackVersion,
}
}
want = append(want, config)
}
for i := range rdsTargets {
config := &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: routeConfigTypeURL,
Name: rdsTargets[i],
VersionInfo: nackVersion,
XdsConfig: routeAnys[i],
ClientStatus: v3adminpb.ClientResourceStatus_ACKED,
}
if i == nackResourceIdx {
config.VersionInfo = ackVersion
config.ClientStatus = v3adminpb.ClientResourceStatus_NACKED
config.ErrorState = &v3adminpb.UpdateFailureState{
Details: "blahblah",
VersionInfo: nackVersion,
}
}
want = append(want, config)
}
for i := range cdsTargets {
config := &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: clusterTypeURL,
Name: cdsTargets[i],
VersionInfo: nackVersion,
XdsConfig: clusterAnys[i],
ClientStatus: v3adminpb.ClientResourceStatus_ACKED,
}
if i == nackResourceIdx {
config.VersionInfo = ackVersion
config.ClientStatus = v3adminpb.ClientResourceStatus_NACKED
config.ErrorState = &v3adminpb.UpdateFailureState{
Details: "blahblah",
VersionInfo: nackVersion,
}
}
want = append(want, config)
}
for i := range edsTargets {
config := &v3statuspb.ClientConfig_GenericXdsConfig{
TypeUrl: endpointsTypeURL,
Name: edsTargets[i],
VersionInfo: nackVersion,
XdsConfig: endpointAnys[i],
ClientStatus: v3adminpb.ClientResourceStatus_ACKED,
}
if i == nackResourceIdx {
config.VersionInfo = ackVersion
config.ClientStatus = v3adminpb.ClientResourceStatus_NACKED
config.ErrorState = &v3adminpb.UpdateFailureState{
Details: "blahblah",
VersionInfo: nackVersion,
}
}
want = append(want, config)
}
if diff := cmp.Diff(filterFields(r.Config[0].GenericXdsConfigs), want, cmpOpts); diff != "" {
return fmt.Errorf(diff)
}
return nil
}
func (s) TestCSDSNoXDSClient(t *testing.T) {
oldNewXDSClient := newXDSClient
newXDSClient = func() xdsclient.XDSClient { return nil }
defer func() { newXDSClient = oldNewXDSClient }()
// Initialize an gRPC server and register CSDS on it.
server := grpc.NewServer()
csdss, err := NewClientStatusDiscoveryServer()
if err != nil {
t.Fatal(err)
}
defer csdss.Close()
v3statuspbgrpc.RegisterClientStatusDiscoveryServiceServer(server, csdss)
// Create a local listener and pass it to Serve().
lis, err := testutils.LocalTCPListener()
if err != nil {
t.Fatalf("testutils.LocalTCPListener() failed: %v", err)
}
go func() {
if err := server.Serve(lis); err != nil {
t.Errorf("Serve() failed: %v", err)
}
}()
defer server.Stop()
// Create CSDS client.
conn, err := grpc.Dial(lis.Addr().String(), grpc.WithTransportCredentials(insecure.NewCredentials()))
if err != nil {
t.Fatalf("cannot connect to server: %v", err)
}
defer conn.Close()
c := v3statuspbgrpc.NewClientStatusDiscoveryServiceClient(conn)
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
stream, err := c.StreamClientStatus(ctx, grpc.WaitForReady(true))
if err != nil {
t.Fatalf("cannot get ServerReflectionInfo: %v", err)
}
if err := stream.Send(&v3statuspb.ClientStatusRequest{Node: nil}); err != nil {
t.Fatalf("failed to send: %v", err)
}
r, err := stream.Recv()
if err != nil {
// io.EOF is not ok.
t.Fatalf("failed to recv response: %v", err)
}
if n := len(r.Config); n != 0 {
t.Fatalf("got %d configs, want 0: %v", n, proto.MarshalTextString(r))
}
}
func Test_nodeProtoToV3(t *testing.T) {
func (s) Test_nodeProtoToV3(t *testing.T) {
const (
testID = "test-id"
testCluster = "test-cluster"
@ -597,7 +83,7 @@ func Test_nodeProtoToV3(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got := nodeProtoToV3(tt.n)
got := nodeProtoToV3(tt.n, nil)
if diff := cmp.Diff(got, tt.want, protocmp.Transform()); diff != "" {
t.Errorf("nodeProtoToV3() got unexpected result, diff (-got, +want): %v", diff)
}

View File

@ -41,8 +41,6 @@ import (
"google.golang.org/grpc/xds/internal/testutils/fakeclient"
"google.golang.org/grpc/xds/internal/xdsclient"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
_ "google.golang.org/grpc/xds/internal/xdsclient/controller/version/v2" // V2 client registration.
)
const (

View File

@ -48,8 +48,7 @@ import (
testgrpc "google.golang.org/grpc/test/grpc_testing"
testpb "google.golang.org/grpc/test/grpc_testing"
_ "google.golang.org/grpc/xds/internal/balancer/clusterresolver" // Register the "cluster_resolver_experimental" LB policy.
_ "google.golang.org/grpc/xds/internal/xdsclient/controller/version/v3" // Register the v3 xDS API client.
_ "google.golang.org/grpc/xds/internal/balancer/clusterresolver" // Register the "cluster_resolver_experimental" LB policy.
)
const (

View File

@ -52,9 +52,8 @@ import (
tpb "github.com/envoyproxy/go-control-plane/envoy/type/v3"
testpb "google.golang.org/grpc/test/grpc_testing"
_ "google.golang.org/grpc/xds/internal/balancer" // Register the balancers.
_ "google.golang.org/grpc/xds/internal/resolver" // Register the xds_resolver.
_ "google.golang.org/grpc/xds/internal/xdsclient/controller/version/v3" // Register the v3 xDS API client.
_ "google.golang.org/grpc/xds/internal/balancer" // Register the balancers.
_ "google.golang.org/grpc/xds/internal/resolver" // Register the xds_resolver.
)
const defaultTestTimeout = 10 * time.Second

View File

@ -18,102 +18,416 @@
package xdsclient
import (
"context"
"fmt"
"sync"
"time"
"google.golang.org/grpc/internal/grpclog"
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
"google.golang.org/grpc/xds/internal/xdsclient/load"
"google.golang.org/grpc/xds/internal/xdsclient/pubsub"
"google.golang.org/grpc/xds/internal/xdsclient/transport"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/protobuf/types/known/anypb"
)
// authority is a combination of pubsub and the controller for this authority.
//
// Note that it might make sense to use one pubsub for all the resources (for
// all the controllers). One downside is the handling of StoW APIs (LDS/CDS).
// These responses contain all the resources from that control plane, so pubsub
// will need to keep lists of resources from each control plane, to know what
// are removed.
type authority struct {
config *bootstrap.ServerConfig
pubsub *pubsub.Pubsub
controller controllerInterface
refCount int
type watchState int
const (
watchStateStarted watchState = iota
watchStateRespReceived
watchStateTimeout
watchStateCanceled
)
type resourceState struct {
watchers map[xdsresource.ResourceWatcher]bool // Set of watchers for this resource
cache xdsresource.ResourceData // Most recent ACKed update for this resource
md xdsresource.UpdateMetadata // Metadata for the most recent update
// Common watch state for all watchers of this resource.
wTimer *time.Timer // Expiry timer
wState watchState // State of the watch
}
// caller must hold parent's authorityMu.
func (a *authority) ref() {
// authority wraps all state associated with a single management server. It
// contains the transport used to communicate with the management server and a
// cache of resource state for resources requested from the management server.
//
// Bootstrap configuration could contain multiple entries in the authorities map
// that share the same server config (server address and credentials to use). We
// share the same authority instance amongst these entries, and the reference
// counting is taken care of by the `clientImpl` type.
type authority struct {
serverCfg *bootstrap.ServerConfig // Server config for this authority
bootstrapCfg *bootstrap.Config // Full bootstrap configuration
refCount int // Reference count of watches referring to this authority
serializer *callbackSerializer // Callback serializer for invoking watch callbacks
resourceTypeGetter func(string) xdsresource.Type // ResourceType registry lookup
transport *transport.Transport // Underlying xDS transport to the management server
watchExpiryTimeout time.Duration // Resource watch expiry timeout
logger *grpclog.PrefixLogger
// A two level map containing the state of all the resources being watched.
//
// The first level map key is the ResourceType (Listener, Route etc). This
// allows us to have a single map for all resources instead of having per
// resource-type maps.
//
// The second level map key is the resource name, with the value being the
// actual state of the resource.
resourcesMu sync.Mutex
resources map[xdsresource.Type]map[string]*resourceState
}
// authorityArgs is a convenience struct to wrap arguments required to create a
// new authority. All fields here correspond directly to appropriate fields
// stored in the authority struct.
type authorityArgs struct {
// The reason for passing server config and bootstrap config separately
// (although the former is part of the latter) is because authorities in the
// bootstrap config might contain an empty server config, and in this case,
// the top-level server config is to be used.
//
// There are two code paths from where a new authority struct might be
// created. One is when a watch is registered for a resource, and one is
// when load reporting needs to be started. We have the authority name in
// the first case, but do in the second. We only have the server config in
// the second case.
serverCfg *bootstrap.ServerConfig
bootstrapCfg *bootstrap.Config
serializer *callbackSerializer
resourceTypeGetter func(string) xdsresource.Type
watchExpiryTimeout time.Duration
logger *grpclog.PrefixLogger
}
func newAuthority(args authorityArgs) (*authority, error) {
ret := &authority{
serverCfg: args.serverCfg,
bootstrapCfg: args.bootstrapCfg,
serializer: args.serializer,
resourceTypeGetter: args.resourceTypeGetter,
watchExpiryTimeout: args.watchExpiryTimeout,
logger: args.logger,
resources: make(map[xdsresource.Type]map[string]*resourceState),
}
tr, err := transport.New(transport.Options{
ServerCfg: *args.serverCfg,
UpdateHandler: ret.handleResourceUpdate,
StreamErrorHandler: ret.newConnectionError,
Logger: args.logger,
})
if err != nil {
return nil, fmt.Errorf("creating new transport to %q: %v", args.serverCfg, err)
}
ret.transport = tr
return ret, nil
}
func (a *authority) handleResourceUpdate(resourceUpdate transport.ResourceUpdate) error {
rType := a.resourceTypeGetter(resourceUpdate.URL)
if rType == nil {
return xdsresource.NewErrorf(xdsresource.ErrorTypeResourceTypeUnsupported, "Resource URL %v unknown in response from server", resourceUpdate.URL)
}
opts := &xdsresource.DecodeOptions{
BootstrapConfig: a.bootstrapCfg,
Logger: a.logger,
}
updates, md, err := decodeAllResources(opts, rType, resourceUpdate)
a.updateResourceStateAndScheduleCallbacks(rType, updates, md)
return err
}
func (a *authority) updateResourceStateAndScheduleCallbacks(rType xdsresource.Type, updates map[string]resourceDataErrTuple, md xdsresource.UpdateMetadata) {
a.resourcesMu.Lock()
defer a.resourcesMu.Unlock()
resourceStates := a.resources[rType]
for name, uErr := range updates {
if state, ok := resourceStates[name]; ok {
// Cancel the expiry timer associated with the resource once a
// response is received, irrespective of whether the update is a
// good one or not.
state.wTimer.Stop()
state.wState = watchStateRespReceived
if uErr.err != nil {
// On error, keep previous version of the resource. But update
// status and error.
state.md.ErrState = md.ErrState
state.md.Status = md.Status
for watcher := range state.watchers {
watcher := watcher
err := uErr.err
a.serializer.Schedule(func(context.Context) { watcher.OnError(err) })
}
continue
}
// If we get here, it means that the update is a valid one. Notify
// watchers only if this is a first time update or it is different
// from the one currently cached.
if state.cache == nil || !state.cache.Equal(uErr.resource) {
for watcher := range state.watchers {
watcher := watcher
resource := uErr.resource
a.serializer.Schedule(func(context.Context) { watcher.OnUpdate(resource) })
}
}
// Sync cache.
a.logger.Debugf("Resource type %q with name %q, value %s added to cache", rType.TypeEnum().String(), name, uErr.resource.ToJSON())
state.cache = uErr.resource
// Set status to ACK, and clear error state. The metadata might be a
// NACK metadata because some other resources in the same response
// are invalid.
state.md = md
state.md.ErrState = nil
state.md.Status = xdsresource.ServiceStatusACKed
if md.ErrState != nil {
state.md.Version = md.ErrState.Version
}
}
}
// If this resource type requires that all resources be present in every
// SotW response from the server, a response that does not include a
// previously seen resource will be interpreted as a deletion of that
// resource.
if !rType.AllResourcesRequiredInSotW() {
return
}
for name, state := range resourceStates {
if _, ok := updates[name]; !ok {
// The metadata status is set to "ServiceStatusNotExist" if a
// previous update deleted this resource, in which case we do not
// want to repeatedly call the watch callbacks with a
// "resource-not-found" error.
if state.md.Status == xdsresource.ServiceStatusNotExist {
continue
}
// If resource exists in cache, but not in the new update, delete
// the resource from cache, and also send a resource not found error
// to indicate resource removed. Metadata for the resource is still
// maintained, as this is required by CSDS.
state.cache = nil
state.md = xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusNotExist}
for watcher := range state.watchers {
watcher := watcher
a.serializer.Schedule(func(context.Context) { watcher.OnResourceDoesNotExist() })
}
}
}
}
type resourceDataErrTuple struct {
resource xdsresource.ResourceData
err error
}
func decodeAllResources(opts *xdsresource.DecodeOptions, rType xdsresource.Type, update transport.ResourceUpdate) (map[string]resourceDataErrTuple, xdsresource.UpdateMetadata, error) {
timestamp := time.Now()
md := xdsresource.UpdateMetadata{
Version: update.Version,
Timestamp: timestamp,
}
topLevelErrors := make([]error, 0) // Tracks deserialization errors, where we don't have a resource name.
perResourceErrors := make(map[string]error) // Tracks resource validation errors, where we have a resource name.
ret := make(map[string]resourceDataErrTuple) // Return result, a map from resource name to either resource data or error.
for _, r := range update.Resources {
result, err := rType.Decode(opts, r)
// Name field of the result is left unpopulated only when resource
// deserialization fails.
name := ""
if result != nil {
name = xdsresource.ParseName(result.Name).String()
}
if err == nil {
ret[name] = resourceDataErrTuple{resource: result.Resource}
continue
}
if name == "" {
topLevelErrors = append(topLevelErrors, err)
continue
}
perResourceErrors[name] = err
// Add place holder in the map so we know this resource name was in
// the response.
ret[name] = resourceDataErrTuple{err: err}
}
if len(topLevelErrors) == 0 && len(perResourceErrors) == 0 {
md.Status = xdsresource.ServiceStatusACKed
return ret, md, nil
}
typeStr := rType.TypeEnum().String()
md.Status = xdsresource.ServiceStatusNACKed
errRet := xdsresource.CombineErrors(typeStr, topLevelErrors, perResourceErrors)
md.ErrState = &xdsresource.UpdateErrorMetadata{
Version: update.Version,
Err: errRet,
Timestamp: timestamp,
}
return ret, md, errRet
}
// newConnectionError is called by the underlying transport when it receives a
// connection error. The error will be forwarded to all the resource watchers.
func (a *authority) newConnectionError(err error) {
a.resourcesMu.Lock()
defer a.resourcesMu.Unlock()
// For all resource types, for all resources within each resource type, and
// for all the watchers for every resource, propagate the connection error
// from the transport layer.
for _, rType := range a.resources {
for _, state := range rType {
for watcher := range state.watchers {
watcher := watcher
a.serializer.Schedule(func(context.Context) {
watcher.OnError(xdsresource.NewErrorf(xdsresource.ErrorTypeConnection, "xds: error received from xDS stream: %v", err))
})
}
}
}
}
// Increments the reference count. Caller must hold parent's authorityMu.
func (a *authority) refLocked() {
a.refCount++
}
// caller must hold parent's authorityMu.
func (a *authority) unref() int {
// Decrements the reference count. Caller must hold parent's authorityMu.
func (a *authority) unrefLocked() int {
a.refCount--
return a.refCount
}
func (a *authority) close() {
if a.pubsub != nil {
a.pubsub.Close()
a.transport.Close()
}
func (a *authority) watchResource(rType xdsresource.Type, resourceName string, watcher xdsresource.ResourceWatcher) func() {
a.logger.Debugf("New watch for type %q, resource name %q", rType.TypeEnum(), resourceName)
a.resourcesMu.Lock()
defer a.resourcesMu.Unlock()
// Lookup the ResourceType specific resources from the top-level map. If
// there is no entry for this ResourceType, create one.
resources := a.resources[rType]
if resources == nil {
resources = make(map[string]*resourceState)
a.resources[rType] = resources
}
if a.controller != nil {
a.controller.Close()
// Lookup the resourceState for the particular resource that the watch is
// being registered for. If this is the first watch for this resource,
// instruct the transport layer to send a DiscoveryRequest for the same.
state := resources[resourceName]
if state == nil {
a.logger.Debugf("First watch for type %q, resource name %q", rType.TypeEnum(), resourceName)
state = &resourceState{
watchers: make(map[xdsresource.ResourceWatcher]bool),
md: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested},
wState: watchStateStarted,
}
state.wTimer = time.AfterFunc(a.watchExpiryTimeout, func() {
a.handleWatchTimerExpiry(state, fmt.Errorf("watch for resource %q of type %s timed out", resourceName, rType.TypeEnum().String()))
})
resources[resourceName] = state
a.sendDiscoveryRequestLocked(rType, resources)
}
// Always add the new watcher to the set of watchers.
state.watchers[watcher] = true
// If we have a cached copy of the resource, notify the new watcher.
if state.cache != nil {
a.logger.Debugf("Resource type %q with resource name %q found in cache: %s", rType.TypeEnum(), resourceName, state.cache.ToJSON())
resource := state.cache
a.serializer.Schedule(func(context.Context) { watcher.OnUpdate(resource) })
}
return func() {
a.resourcesMu.Lock()
defer a.resourcesMu.Unlock()
// We already have a reference to the resourceState for this particular
// resource. Avoid indexing into the two-level map to figure this out.
// Delete this particular watcher from the list of watchers, so that its
// callback will not be invoked in the future.
state.wState = watchStateCanceled
delete(state.watchers, watcher)
if len(state.watchers) > 0 {
return
}
// There are no more watchers for this resource, delete the state
// associated with it, and instruct the transport to send a request
// which does not include this resource name.
delete(resources, resourceName)
a.sendDiscoveryRequestLocked(rType, resources)
}
}
func (a *authority) watchListener(serviceName string, cb func(xdsresource.ListenerUpdate, error)) (cancel func()) {
first, cancelF := a.pubsub.WatchListener(serviceName, cb)
if first {
a.controller.AddWatch(xdsresource.ListenerResource, serviceName)
func (a *authority) handleWatchTimerExpiry(state *resourceState, err error) {
a.resourcesMu.Lock()
defer a.resourcesMu.Unlock()
if state.wState == watchStateCanceled {
return
}
return func() {
if cancelF() {
a.controller.RemoveWatch(xdsresource.ListenerResource, serviceName)
}
state.wState = watchStateTimeout
for watcher := range state.watchers {
watcher := watcher
a.serializer.Schedule(func(context.Context) { watcher.OnError(err) })
}
}
func (a *authority) watchRouteConfig(routeName string, cb func(xdsresource.RouteConfigUpdate, error)) (cancel func()) {
first, cancelF := a.pubsub.WatchRouteConfig(routeName, cb)
if first {
a.controller.AddWatch(xdsresource.RouteConfigResource, routeName)
}
return func() {
if cancelF() {
a.controller.RemoveWatch(xdsresource.RouteConfigResource, routeName)
}
}
}
func (a *authority) watchCluster(clusterName string, cb func(xdsresource.ClusterUpdate, error)) (cancel func()) {
first, cancelF := a.pubsub.WatchCluster(clusterName, cb)
if first {
a.controller.AddWatch(xdsresource.ClusterResource, clusterName)
}
return func() {
if cancelF() {
a.controller.RemoveWatch(xdsresource.ClusterResource, clusterName)
}
}
}
func (a *authority) watchEndpoints(clusterName string, cb func(xdsresource.EndpointsUpdate, error)) (cancel func()) {
first, cancelF := a.pubsub.WatchEndpoints(clusterName, cb)
if first {
a.controller.AddWatch(xdsresource.EndpointsResource, clusterName)
}
return func() {
if cancelF() {
a.controller.RemoveWatch(xdsresource.EndpointsResource, clusterName)
}
// sendDiscoveryRequestLocked sends a discovery request for the specified
// resource type and resource names. Even though this method does not directly
// access the resource cache, it is important that `resourcesMu` be beld when
// calling this method to ensure that a consistent snapshot of resource names is
// being requested.
func (a *authority) sendDiscoveryRequestLocked(rType xdsresource.Type, resources map[string]*resourceState) {
resourcesToRequest := make([]string, len(resources))
i := 0
for name := range resources {
resourcesToRequest[i] = name
i++
}
a.transport.SendRequest(rType.TypeURL(), resourcesToRequest)
}
func (a *authority) reportLoad() (*load.Store, func()) {
// An empty string means to report load to the same same used for ADS. There
// should never be a need to specify a string other than an empty string. If
// a different server is to be used, a different authority (controller) will
// be created.
return a.controller.ReportLoad("")
return a.transport.ReportLoad()
}
func (a *authority) dump(t xdsresource.ResourceType) map[string]xdsresource.UpdateWithMD {
return a.pubsub.Dump(t)
func (a *authority) dumpResources() map[string]map[string]xdsresource.UpdateWithMD {
a.resourcesMu.Lock()
defer a.resourcesMu.Unlock()
dump := make(map[string]map[string]xdsresource.UpdateWithMD)
for rType, resourceStates := range a.resources {
states := make(map[string]xdsresource.UpdateWithMD)
for name, state := range resourceStates {
var raw *anypb.Any
if state.cache != nil {
raw = state.cache.Raw()
}
states[name] = xdsresource.UpdateWithMD{
MD: state.md,
Raw: raw,
}
}
dump[rType.TypeURL()] = states
}
return dump
}

View File

@ -34,7 +34,6 @@ type XDSClient interface {
WatchRouteConfig(string, func(xdsresource.RouteConfigUpdate, error)) func()
WatchCluster(string, func(xdsresource.ClusterUpdate, error)) func()
WatchEndpoints(string, func(xdsresource.EndpointsUpdate, error)) func()
ReportLoad(*bootstrap.ServerConfig) (*load.Store, func())
// WatchResource uses xDS to discover the resource associated with the
// provided resource name. The resource type implementation determines how
@ -50,11 +49,13 @@ type XDSClient interface {
// delete the resource type specific watch APIs on this interface.
WatchResource(rType xdsresource.Type, resourceName string, watcher xdsresource.ResourceWatcher) (cancel func())
DumpLDS() map[string]xdsresource.UpdateWithMD
DumpRDS() map[string]xdsresource.UpdateWithMD
DumpCDS() map[string]xdsresource.UpdateWithMD
DumpEDS() map[string]xdsresource.UpdateWithMD
// DumpResources returns the status of the xDS resources. Returns a map of
// resource type URLs to a map of resource names to resource state.
DumpResources() map[string]map[string]xdsresource.UpdateWithMD
ReportLoad(*bootstrap.ServerConfig) (*load.Store, func())
BootstrapConfig() *bootstrap.Config
Close()
}

View File

@ -69,8 +69,7 @@ func newWithConfig(config *bootstrap.Config, watchExpiryTimeout time.Duration, i
}
c.logger = prefixLogger(c)
c.logger.Infof("Created ClientConn to xDS management server: %s", config.XDSServer)
c.logger.Infof("Created")
c.logger.Infof("Created client to xDS management server: %s", config.XDSServer)
return c, nil
}

View File

@ -20,30 +20,11 @@ package xdsclient
import (
"errors"
"fmt"
"time"
"google.golang.org/grpc/internal/grpclog"
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
"google.golang.org/grpc/xds/internal/xdsclient/controller"
"google.golang.org/grpc/xds/internal/xdsclient/load"
"google.golang.org/grpc/xds/internal/xdsclient/pubsub"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
v2corepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
)
type controllerInterface interface {
AddWatch(resourceType xdsresource.ResourceType, resourceName string)
RemoveWatch(resourceType xdsresource.ResourceType, resourceName string)
ReportLoad(server string) (*load.Store, func())
Close()
}
var newController = func(config *bootstrap.ServerConfig, pubsub *pubsub.Pubsub, validator xdsresource.UpdateValidatorFunc, logger *grpclog.PrefixLogger, boff func(int) time.Duration) (controllerInterface, error) {
return controller.New(config, pubsub, validator, logger, boff)
}
// findAuthority returns the authority for this name. If it doesn't already
// exist, one will be created.
//
@ -88,12 +69,13 @@ func (c *clientImpl) findAuthority(n *xdsresource.Name) (_ *authority, unref fun
// authority.
//
// unref() will be done when the watch is canceled.
a.ref()
a.refLocked()
return a, func() { c.unrefAuthority(a) }, nil
}
// newAuthorityLocked creates a new authority for the config. But before that, it
// checks the cache to see if an authority for this config already exists.
// newAuthorityLocked creates a new authority for the given config. If an
// authority for the given config exists in the cache, it is returned instead of
// creating a new one.
//
// The caller must take a reference of the returned authority before using, and
// unref afterwards.
@ -121,23 +103,17 @@ func (c *clientImpl) newAuthorityLocked(config *bootstrap.ServerConfig) (_ *auth
}
// Make a new authority since there's no existing authority for this config.
nodeID := ""
if v3, ok := c.config.XDSServer.NodeProto.(*v3corepb.Node); ok {
nodeID = v3.GetId()
} else if v2, ok := c.config.XDSServer.NodeProto.(*v2corepb.Node); ok {
nodeID = v2.GetId()
}
ret := &authority{config: config, pubsub: pubsub.New(c.watchExpiryTimeout, nodeID, c.logger)}
defer func() {
if retErr != nil {
ret.close()
}
}()
ctr, err := newController(config, ret.pubsub, c.updateValidator, c.logger, nil)
ret, err := newAuthority(authorityArgs{
serverCfg: config,
bootstrapCfg: c.config,
serializer: c.serializer,
resourceTypeGetter: c.resourceTypes.get,
watchExpiryTimeout: c.watchExpiryTimeout,
logger: c.logger,
})
if err != nil {
return nil, err
return nil, fmt.Errorf("creating new authority for config %q: %v", config.String(), err)
}
ret.controller = ctr
// Add it to the cache, so it will be reused.
c.authorities[configStr] = ret
return ret, nil
@ -153,10 +129,10 @@ func (c *clientImpl) newAuthorityLocked(config *bootstrap.ServerConfig) (_ *auth
func (c *clientImpl) unrefAuthority(a *authority) {
c.authorityMu.Lock()
defer c.authorityMu.Unlock()
if a.unref() > 0 {
if a.unrefLocked() > 0 {
return
}
configStr := a.config.String()
configStr := a.serverCfg.String()
delete(c.authorities, configStr)
c.idleAuthorities.Add(configStr, a, func() {
a.close()

View File

@ -22,42 +22,32 @@ import (
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
func mergeMaps(maps []map[string]xdsresource.UpdateWithMD) map[string]xdsresource.UpdateWithMD {
ret := make(map[string]xdsresource.UpdateWithMD)
for _, m := range maps {
for k, v := range m {
ret[k] = v
func appendMaps(dst, src map[string]map[string]xdsresource.UpdateWithMD) {
// Iterate through the resource types.
for rType, srcResources := range src {
// Lookup/create the resource type specific map in the destination.
dstResources := dst[rType]
if dstResources == nil {
dstResources = make(map[string]xdsresource.UpdateWithMD)
dst[rType] = dstResources
}
// Iterate through the resources within the resource type in the source,
// and copy them over to the destination.
for name, update := range srcResources {
dstResources[name] = update
}
}
return ret
}
func (c *clientImpl) dump(t xdsresource.ResourceType) map[string]xdsresource.UpdateWithMD {
// DumpResources returns the status and contents of all xDS resources.
func (c *clientImpl) DumpResources() map[string]map[string]xdsresource.UpdateWithMD {
c.authorityMu.Lock()
defer c.authorityMu.Unlock()
maps := make([]map[string]xdsresource.UpdateWithMD, 0, len(c.authorities))
dumps := make(map[string]map[string]xdsresource.UpdateWithMD)
for _, a := range c.authorities {
maps = append(maps, a.dump(t))
dump := a.dumpResources()
appendMaps(dumps, dump)
}
return mergeMaps(maps)
}
// DumpLDS returns the status and contents of LDS.
func (c *clientImpl) DumpLDS() map[string]xdsresource.UpdateWithMD {
return c.dump(xdsresource.ListenerResource)
}
// DumpRDS returns the status and contents of RDS.
func (c *clientImpl) DumpRDS() map[string]xdsresource.UpdateWithMD {
return c.dump(xdsresource.RouteConfigResource)
}
// DumpCDS returns the status and contents of CDS.
func (c *clientImpl) DumpCDS() map[string]xdsresource.UpdateWithMD {
return c.dump(xdsresource.ClusterResource)
}
// DumpEDS returns the status and contents of EDS.
func (c *clientImpl) DumpEDS() map[string]xdsresource.UpdateWithMD {
return c.dump(xdsresource.EndpointsResource)
return dumps
}

View File

@ -36,7 +36,7 @@ func (c *clientImpl) ReportLoad(server *bootstrap.ServerConfig) (*load.Store, fu
return nil, func() {}
}
// Hold the ref before starting load reporting.
a.ref()
a.refLocked()
store, cancelF := a.reportLoad()
return store, func() {
cancelF()

View File

@ -1,67 +0,0 @@
/*
*
* Copyright 2022 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 xdsclient
import (
"fmt"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
func (c *clientImpl) filterChainUpdateValidator(fc *xdsresource.FilterChain) error {
if fc == nil {
return nil
}
return c.securityConfigUpdateValidator(fc.SecurityCfg)
}
func (c *clientImpl) securityConfigUpdateValidator(sc *xdsresource.SecurityConfig) error {
if sc == nil {
return nil
}
if sc.IdentityInstanceName != "" {
if _, ok := c.config.CertProviderConfigs[sc.IdentityInstanceName]; !ok {
return fmt.Errorf("identitiy certificate provider instance name %q missing in bootstrap configuration", sc.IdentityInstanceName)
}
}
if sc.RootInstanceName != "" {
if _, ok := c.config.CertProviderConfigs[sc.RootInstanceName]; !ok {
return fmt.Errorf("root certificate provider instance name %q missing in bootstrap configuration", sc.RootInstanceName)
}
}
return nil
}
func (c *clientImpl) updateValidator(u interface{}) error {
switch update := u.(type) {
case xdsresource.ListenerUpdate:
if update.InboundListenerCfg == nil || update.InboundListenerCfg.FilterChains == nil {
return nil
}
return update.InboundListenerCfg.FilterChains.Validate(c.filterChainUpdateValidator)
case xdsresource.ClusterUpdate:
return c.securityConfigUpdateValidator(update.SecurityCfg)
default:
// We currently invoke this update validation function only for LDS and
// CDS updates. In the future, if we wish to invoke it for other xDS
// updates, corresponding plumbing needs to be added to those unmarshal
// functions.
}
return nil
}

View File

@ -25,46 +25,93 @@ import (
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
// WatchListener uses LDS to discover information about the provided listener.
// This is only required temporarily, while we modify the
// clientImpl.WatchListener API to be implemented via the wrapper
// WatchListener() API which calls the WatchResource() API.
type listenerWatcher struct {
resourceName string
cb func(xdsresource.ListenerUpdate, error)
}
func (l *listenerWatcher) OnUpdate(update *xdsresource.ListenerResourceData) {
l.cb(update.Resource, nil)
}
func (l *listenerWatcher) OnError(err error) {
l.cb(xdsresource.ListenerUpdate{}, err)
}
func (l *listenerWatcher) OnResourceDoesNotExist() {
err := xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "Resource name %q of type Listener not found in received response", l.resourceName)
l.cb(xdsresource.ListenerUpdate{}, err)
}
// WatchListener uses LDS to discover information about the Listener resource
// identified by resourceName.
//
// 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 *clientImpl) WatchListener(serviceName string, cb func(xdsresource.ListenerUpdate, error)) (cancel func()) {
n := xdsresource.ParseName(serviceName)
a, unref, err := c.findAuthority(n)
if err != nil {
cb(xdsresource.ListenerUpdate{}, err)
return func() {}
}
cancelF := a.watchListener(n.String(), cb)
return func() {
cancelF()
unref()
}
func (c *clientImpl) WatchListener(resourceName string, cb func(xdsresource.ListenerUpdate, error)) (cancel func()) {
watcher := &listenerWatcher{resourceName: resourceName, cb: cb}
return xdsresource.WatchListener(c, resourceName, watcher)
}
// WatchRouteConfig starts a listener watcher for the service.
// This is only required temporarily, while we modify the
// clientImpl.WatchRouteConfig API to be implemented via the wrapper
// WatchRouteConfig() API which calls the WatchResource() API.
type routeConfigWatcher struct {
resourceName string
cb func(xdsresource.RouteConfigUpdate, error)
}
func (r *routeConfigWatcher) OnUpdate(update *xdsresource.RouteConfigResourceData) {
r.cb(update.Resource, nil)
}
func (r *routeConfigWatcher) OnError(err error) {
r.cb(xdsresource.RouteConfigUpdate{}, err)
}
func (r *routeConfigWatcher) OnResourceDoesNotExist() {
err := xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "Resource name %q of type RouteConfiguration not found in received response", r.resourceName)
r.cb(xdsresource.RouteConfigUpdate{}, err)
}
// WatchRouteConfig uses RDS to discover information about the
// RouteConfiguration resource identified by resourceName.
//
// 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 *clientImpl) WatchRouteConfig(routeName string, cb func(xdsresource.RouteConfigUpdate, error)) (cancel func()) {
n := xdsresource.ParseName(routeName)
a, unref, err := c.findAuthority(n)
if err != nil {
cb(xdsresource.RouteConfigUpdate{}, err)
return func() {}
}
cancelF := a.watchRouteConfig(n.String(), cb)
return func() {
cancelF()
unref()
}
func (c *clientImpl) WatchRouteConfig(resourceName string, cb func(xdsresource.RouteConfigUpdate, error)) (cancel func()) {
watcher := &routeConfigWatcher{resourceName: resourceName, cb: cb}
return xdsresource.WatchRouteConfig(c, resourceName, watcher)
}
// WatchCluster uses CDS to discover information about the provided
// clusterName.
// This is only required temporarily, while we modify the
// clientImpl.WatchCluster API to be implemented via the wrapper WatchCluster()
// API which calls the WatchResource() API.
type clusterWatcher struct {
resourceName string
cb func(xdsresource.ClusterUpdate, error)
}
func (c *clusterWatcher) OnUpdate(update *xdsresource.ClusterResourceData) {
c.cb(update.Resource, nil)
}
func (c *clusterWatcher) OnError(err error) {
c.cb(xdsresource.ClusterUpdate{}, err)
}
func (c *clusterWatcher) OnResourceDoesNotExist() {
err := xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "Resource name %q of type Cluster not found in received response", c.resourceName)
c.cb(xdsresource.ClusterUpdate{}, err)
}
// WatchCluster uses CDS to discover information about the Cluster resource
// identified by resourceName.
//
// WatchCluster can be called multiple times, with same or different
// clusterNames. Each call will start an independent watcher for the resource.
@ -72,21 +119,34 @@ func (c *clientImpl) WatchRouteConfig(routeName string, cb func(xdsresource.Rout
// 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 *clientImpl) WatchCluster(clusterName string, cb func(xdsresource.ClusterUpdate, error)) (cancel func()) {
n := xdsresource.ParseName(clusterName)
a, unref, err := c.findAuthority(n)
if err != nil {
cb(xdsresource.ClusterUpdate{}, err)
return func() {}
}
cancelF := a.watchCluster(n.String(), cb)
return func() {
cancelF()
unref()
}
func (c *clientImpl) WatchCluster(resourceName string, cb func(xdsresource.ClusterUpdate, error)) (cancel func()) {
watcher := &clusterWatcher{resourceName: resourceName, cb: cb}
return xdsresource.WatchCluster(c, resourceName, watcher)
}
// WatchEndpoints uses EDS to discover endpoints in the provided clusterName.
// This is only required temporarily, while we modify the
// clientImpl.WatchEndpoints API to be implemented via the wrapper
// WatchEndpoints() API which calls the WatchResource() API.
type endpointsWatcher struct {
resourceName string
cb func(xdsresource.EndpointsUpdate, error)
}
func (c *endpointsWatcher) OnUpdate(update *xdsresource.EndpointsResourceData) {
c.cb(update.Resource, nil)
}
func (c *endpointsWatcher) OnError(err error) {
c.cb(xdsresource.EndpointsUpdate{}, err)
}
func (c *endpointsWatcher) OnResourceDoesNotExist() {
err := xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "Resource name %q of type Endpoints not found in received response", c.resourceName)
c.cb(xdsresource.EndpointsUpdate{}, err)
}
// WatchEndpoints uses EDS to discover information about the
// ClusterLoadAssignment resource identified by resourceName.
//
// WatchEndpoints can be called multiple times, with same or different
// clusterNames. Each call will start an independent watcher for the resource.
@ -94,18 +154,9 @@ func (c *clientImpl) WatchCluster(clusterName string, cb func(xdsresource.Cluste
// 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 *clientImpl) WatchEndpoints(clusterName string, cb func(xdsresource.EndpointsUpdate, error)) (cancel func()) {
n := xdsresource.ParseName(clusterName)
a, unref, err := c.findAuthority(n)
if err != nil {
cb(xdsresource.EndpointsUpdate{}, err)
return func() {}
}
cancelF := a.watchEndpoints(n.String(), cb)
return func() {
cancelF()
unref()
}
func (c *clientImpl) WatchEndpoints(resourceName string, cb func(xdsresource.EndpointsUpdate, error)) (cancel func()) {
watcher := &endpointsWatcher{resourceName: resourceName, cb: cb}
return xdsresource.WatchEndpoints(c, resourceName, watcher)
}
// WatchResource uses xDS to discover the resource associated with the provided
@ -126,6 +177,7 @@ func (c *clientImpl) WatchResource(rType xdsresource.Type, resourceName string,
}
if err := c.resourceTypes.maybeRegister(rType); err != nil {
logger.Warningf("Watch registered for name %q of type %q which is already registered", rType.TypeEnum().String(), resourceName)
c.serializer.Schedule(func(context.Context) { watcher.OnError(err) })
return func() {}
}
@ -138,7 +190,21 @@ func (c *clientImpl) WatchResource(rType xdsresource.Type, resourceName string,
// - Call the watchResource() method on the authority.
// - Return a cancel function to cancel the watch on the authority and to
// release the reference.
return func() {}
// TODO: Make ParseName return an error if parsing fails, and
// schedule the OnError callback in that case.
n := xdsresource.ParseName(resourceName)
a, unref, err := c.findAuthority(n)
if err != nil {
logger.Warningf("Watch registered for name %q of type %q, authority %q is not found", rType.TypeEnum().String(), resourceName, n.Authority)
c.serializer.Schedule(func(context.Context) { watcher.OnError(err) })
return func() {}
}
cancelF := a.watchResource(rType, n.String(), watcher)
return func() {
cancelF()
unref()
}
}
// A registry of xdsresource.Type implementations indexed by their corresponding
@ -153,21 +219,21 @@ func newResourceTypeRegistry() *resourceTypeRegistry {
return &resourceTypeRegistry{types: make(map[string]xdsresource.Type)}
}
func (r *resourceTypeRegistry) get(url string) xdsresource.Type {
r.mu.Lock()
defer r.mu.Unlock()
return r.types[url]
}
func (r *resourceTypeRegistry) maybeRegister(rType xdsresource.Type) error {
r.mu.Lock()
defer r.mu.Unlock()
urls := []string{rType.V2TypeURL(), rType.V3TypeURL()}
for _, u := range urls {
if u == "" {
// Silently ignore unsupported versions of the resource.
continue
}
typ, ok := r.types[u]
if ok && typ != rType {
return fmt.Errorf("attempt to re-register a resource type implementation for %v", rType.TypeEnum())
}
r.types[u] = rType
url := rType.TypeURL()
typ, ok := r.types[url]
if ok && typ != rType {
return fmt.Errorf("attempt to re-register a resource type implementation for %v", rType.TypeEnum())
}
r.types[url] = rType
return nil
}

View File

@ -1,201 +0,0 @@
/*
*
* Copyright 2021 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 controller contains implementation to connect to the control plane.
// Including starting the ClientConn, starting the xDS stream, and
// sending/receiving messages.
//
// All the messages are parsed by the resource package (e.g.
// UnmarshalListener()) and sent to the Pubsub watchers.
package controller
import (
"context"
"errors"
"fmt"
"sync"
"time"
"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/keepalive"
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
"google.golang.org/grpc/xds/internal/xdsclient/controller/version"
"google.golang.org/grpc/xds/internal/xdsclient/pubsub"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
// Controller manages the connection and stream to the control plane.
//
// It keeps track of what resources are being watched, and send new requests
// when new watches are added.
//
// It takes a pubsub (as an interface) as input. When a response is received,
// it's parsed, and the updates are sent to the pubsub.
type Controller struct {
config *bootstrap.ServerConfig
updateHandler pubsub.UpdateHandler
updateValidator xdsresource.UpdateValidatorFunc
logger *grpclog.PrefixLogger
cc *grpc.ClientConn // Connection to the management server.
vClient version.VersionedClient
stopRunGoroutine context.CancelFunc
// The run goroutine closes this channel when it exits, and we block on this
// channel in Close(). This ensures that when Close() returns, the
// underlying transport is closed, and we can guarantee that we will not
// process any subsequent responses from the management server.
runDoneCh chan struct{}
backoff func(int) time.Duration
streamCh chan grpc.ClientStream
sendCh *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[xdsresource.ResourceType]map[string]bool
// versionMap contains the version that was acked (the version in the ack
// request that was sent on wire). The key is rType, the value is the
// version string, because the versions for different resource types should
// be independent.
versionMap map[xdsresource.ResourceType]string
// nonceMap contains the nonce from the most recent received response.
nonceMap map[xdsresource.ResourceType]string
closed bool
// Changes to map lrsClients and the lrsClient inside the map need to be
// protected by lrsMu.
//
// TODO: after LRS refactoring, each controller should only manage the LRS
// stream to its server. LRS streams to other servers should be managed by
// other controllers.
lrsMu sync.Mutex
lrsClients map[string]*lrsClient
}
var grpcDial = grpc.Dial
// SetGRPCDial sets the dialer for the controller. The dial can be used to
// manipulate the dial options or change the target if needed.
// The SetGRPCDial must be called before gRPC initialization to make sure it
// affects all the controllers created.
// To reset any dialer set, pass in grpc.Dial as the parameter.
func SetGRPCDial(dialer func(target string, opts ...grpc.DialOption) (*grpc.ClientConn, error)) {
grpcDial = dialer
}
// New creates a new controller.
func New(config *bootstrap.ServerConfig, updateHandler pubsub.UpdateHandler, validator xdsresource.UpdateValidatorFunc, logger *grpclog.PrefixLogger, boff func(int) time.Duration) (_ *Controller, retErr error) {
switch {
case config == nil:
return nil, errors.New("xds: no xds_server provided")
case config.ServerURI == "":
return nil, errors.New("xds: no xds_server name provided in options")
case config.Creds == nil:
return nil, errors.New("xds: no credentials provided in options")
case config.NodeProto == nil:
return nil, errors.New("xds: no node_proto provided in options")
}
dopts := []grpc.DialOption{
config.Creds,
grpc.WithKeepaliveParams(keepalive.ClientParameters{
Time: 5 * time.Minute,
Timeout: 20 * time.Second,
}),
}
if boff == nil {
boff = backoff.DefaultExponential.Backoff
}
ret := &Controller{
config: config,
updateValidator: validator,
updateHandler: updateHandler,
runDoneCh: make(chan struct{}),
backoff: boff,
streamCh: make(chan grpc.ClientStream, 1),
sendCh: buffer.NewUnbounded(),
watchMap: make(map[xdsresource.ResourceType]map[string]bool),
versionMap: make(map[xdsresource.ResourceType]string),
nonceMap: make(map[xdsresource.ResourceType]string),
lrsClients: make(map[string]*lrsClient),
}
defer func() {
if retErr != nil {
ret.Close()
}
}()
cc, err := grpcDial(config.ServerURI, dopts...)
if err != nil {
// An error from a non-blocking dial indicates something serious.
return nil, fmt.Errorf("xds: failed to dial control plane {%s}: %v", config.ServerURI, err)
}
ret.cc = cc
builder := version.GetAPIClientBuilder(config.TransportAPI)
if builder == nil {
return nil, fmt.Errorf("no client builder for xDS API version: %v", config.TransportAPI)
}
apiClient, err := builder(version.BuildOptions{NodeProto: config.NodeProto, Logger: logger})
if err != nil {
return nil, err
}
ret.vClient = apiClient
ctx, cancel := context.WithCancel(context.Background())
ret.stopRunGoroutine = cancel
go ret.run(ctx)
return ret, nil
}
// Close closes the controller.
func (t *Controller) Close() {
t.mu.Lock()
if t.closed {
t.mu.Unlock()
return
}
t.closed = true
t.mu.Unlock()
// Note that Close needs to check for nils even if some of them are always
// set in the constructor. This is because the constructor defers Close() in
// error cases, and the fields might not be set when the error happens.
if t.stopRunGoroutine != nil {
t.stopRunGoroutine()
}
if t.cc != nil {
t.cc.Close()
}
// Wait on the run goroutine to be done only if it was started.
if t.stopRunGoroutine != nil {
<-t.runDoneCh
}
}

View File

@ -1,154 +0,0 @@
/*
*
* Copyright 2021 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 controller
import (
"testing"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
"google.golang.org/grpc/xds/internal/testutils"
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
)
const testXDSServer = "xds-server"
// noopUpdateHandler ignores all updates. It's to be used in tests where the
// updates don't matter. To avoid potential nil panic.
var noopUpdateHandler = &testUpdateReceiver{
f: func(rType xdsresource.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata) {},
}
// TestNew covers that New() returns an error if the input *ServerConfig
// contains invalid content.
func (s) TestNew(t *testing.T) {
tests := []struct {
name string
config *bootstrap.ServerConfig
wantErr bool
}{
{
name: "empty-opts",
config: &bootstrap.ServerConfig{},
wantErr: true,
},
{
name: "empty-balancer-name",
config: &bootstrap.ServerConfig{
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
NodeProto: testutils.EmptyNodeProtoV2,
},
wantErr: true,
},
{
name: "empty-dial-creds",
config: &bootstrap.ServerConfig{
ServerURI: testXDSServer,
NodeProto: testutils.EmptyNodeProtoV2,
},
wantErr: true,
},
{
name: "empty-node-proto",
config: &bootstrap.ServerConfig{
ServerURI: testXDSServer,
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
},
wantErr: true,
},
{
name: "node-proto-version-mismatch",
config: &bootstrap.ServerConfig{
ServerURI: testXDSServer,
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
TransportAPI: version.TransportV2,
NodeProto: testutils.EmptyNodeProtoV3,
},
wantErr: true,
},
{
name: "happy-case",
config: &bootstrap.ServerConfig{
ServerURI: testXDSServer,
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
NodeProto: testutils.EmptyNodeProtoV2,
},
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
c, err := New(test.config, noopUpdateHandler, nil, nil, nil) // Only testing the config, other inputs are left as nil.
defer func() {
if c != nil {
c.Close()
}
}()
if (err != nil) != test.wantErr {
t.Fatalf("New(%+v) = %v, wantErr: %v", test.config, err, test.wantErr)
}
})
}
}
func (s) TestNewWithGRPCDial(t *testing.T) {
config := &bootstrap.ServerConfig{
ServerURI: testXDSServer,
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
NodeProto: testutils.EmptyNodeProtoV2,
}
customDialerCalled := false
customDialer := func(target string, opts ...grpc.DialOption) (*grpc.ClientConn, error) {
customDialerCalled = true
return grpc.Dial(target, opts...)
}
// Set the dialer and make sure it is called.
SetGRPCDial(customDialer)
c, err := New(config, noopUpdateHandler, nil, nil, nil)
if err != nil {
t.Fatalf("New(%+v) = %v, want no error", config, err)
}
if c != nil {
c.Close()
}
if !customDialerCalled {
t.Errorf("New(%+v) custom dialer called = false, want true", config)
}
customDialerCalled = false
// Reset the dialer and make sure it is not called.
SetGRPCDial(grpc.Dial)
c, err = New(config, noopUpdateHandler, nil, nil, nil)
defer func() {
if c != nil {
c.Close()
}
}()
if err != nil {
t.Fatalf("New(%+v) = %v, want no error", config, err)
}
if customDialerCalled {
t.Errorf("New(%+v) interceptor called = true, want false", config)
}
}

View File

@ -1,145 +0,0 @@
/*
*
* Copyright 2021 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 controller
import (
"context"
"google.golang.org/grpc"
"google.golang.org/grpc/xds/internal/xdsclient/controller/version"
"google.golang.org/grpc/xds/internal/xdsclient/load"
)
// ReportLoad starts an load reporting stream to the given server. If the server
// is not an empty string, and is different from the management server, a new
// ClientConn will be created.
//
// The same options used for creating the Client will be used (including
// NodeProto, and dial options if necessary).
//
// It returns a Store for the user to report loads, a function to cancel the
// load reporting stream.
//
// TODO(xdsfed): LRS refactor, delete the parameter of this function, and
// cleanup the multiple LRS ClientConn code. Each controller should have one
// ClientConn to the authority it's created for, all LRS streams (and ADS
// streams) in this controller should all share that ClientConn.
func (c *Controller) ReportLoad(server string) (*load.Store, func()) {
c.lrsMu.Lock()
defer c.lrsMu.Unlock()
// If there's already a client to this server, use it. Otherwise, create
// one.
lrsC, ok := c.lrsClients[server]
if !ok {
lrsC = newLRSClient(c, server)
c.lrsClients[server] = lrsC
}
store := lrsC.ref()
return store, func() {
// This is a callback, need to hold lrsMu.
c.lrsMu.Lock()
defer c.lrsMu.Unlock()
if lrsC.unRef() {
// Delete the lrsClient from map if this is the last reference.
delete(c.lrsClients, server)
}
}
}
// lrsClient maps to one lrsServer. It contains:
// - a ClientConn to this server (only if it's different from the management
// server)
// - a load.Store that contains loads only for this server
type lrsClient struct {
parent *Controller
server string
cc *grpc.ClientConn // nil if the server is same as the management server
refCount int
cancelStream func()
loadStore *load.Store
}
// newLRSClient creates a new LRS stream to the server.
func newLRSClient(parent *Controller, server string) *lrsClient {
return &lrsClient{
parent: parent,
server: server,
refCount: 0,
}
}
// ref increments the refCount. If this is the first ref, it starts the LRS stream.
//
// Not thread-safe, caller needs to synchronize.
func (lrsC *lrsClient) ref() *load.Store {
lrsC.refCount++
if lrsC.refCount == 1 {
lrsC.startStream()
}
return lrsC.loadStore
}
// unRef decrements the refCount, and closes the stream if refCount reaches 0
// (and close the cc if cc is not xDS cc). It returns whether refCount reached 0
// after this call.
//
// Not thread-safe, caller needs to synchronize.
func (lrsC *lrsClient) unRef() (closed bool) {
lrsC.refCount--
if lrsC.refCount != 0 {
return false
}
lrsC.parent.logger.Infof("Stopping load report to server: %s", lrsC.server)
lrsC.cancelStream()
if lrsC.cc != nil {
lrsC.cc.Close()
}
return true
}
// startStream starts the LRS stream to the server. If server is not the same
// management server from the parent, it also creates a ClientConn.
func (lrsC *lrsClient) startStream() {
var cc *grpc.ClientConn
lrsC.parent.logger.Infof("Starting load report to server: %s", lrsC.server)
if lrsC.server == "" || lrsC.server == lrsC.parent.config.ServerURI {
// Reuse the xDS client if server is the same.
cc = lrsC.parent.cc
} else {
lrsC.parent.logger.Infof("LRS server is different from management server, starting a new ClientConn")
ccNew, err := grpc.Dial(lrsC.server, lrsC.parent.config.Creds)
if err != nil {
// An error from a non-blocking dial indicates something serious.
lrsC.parent.logger.Infof("xds: failed to dial load report server {%s}: %v", lrsC.server, err)
return
}
cc = ccNew
lrsC.cc = ccNew
}
var ctx context.Context
ctx, lrsC.cancelStream = context.WithCancel(context.Background())
// Create the store and stream.
lrsC.loadStore = load.NewStore()
go lrsC.parent.reportLoad(ctx, cc, version.LoadReportingOptions{LoadStore: lrsC.loadStore})
}

View File

@ -1,443 +0,0 @@
/*
*
* 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 controller
import (
"context"
"fmt"
"time"
"github.com/golang/protobuf/proto"
"google.golang.org/grpc"
controllerversion "google.golang.org/grpc/xds/internal/xdsclient/controller/version"
xdsresourceversion "google.golang.org/grpc/xds/internal/xdsclient/controller/version"
"google.golang.org/grpc/xds/internal/xdsclient/load"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
// AddWatch adds a watch for an xDS resource given its type and name.
func (t *Controller) AddWatch(rType xdsresource.ResourceType, resourceName string) {
t.sendCh.Put(&watchAction{
rType: rType,
remove: false,
resource: resourceName,
})
}
// RemoveWatch cancels an already registered watch for an xDS resource
// given its type and name.
func (t *Controller) RemoveWatch(rType xdsresource.ResourceType, resourceName string) {
t.sendCh.Put(&watchAction{
rType: rType,
remove: true,
resource: resourceName,
})
}
// 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 (t *Controller) run(ctx context.Context) {
sendDoneCh := make(chan struct{})
defer func() {
<-sendDoneCh
close(t.runDoneCh)
}()
go t.send(ctx, sendDoneCh)
// TODO: start a goroutine monitoring ClientConn's connectivity state, and
// report error (and log) when stats is transient failure.
retries := 0
lastStreamStartTime := time.Time{}
for ctx.Err() == nil {
dur := time.Until(lastStreamStartTime.Add(t.backoff(retries)))
if dur > 0 {
timer := time.NewTimer(dur)
select {
case <-timer.C:
case <-ctx.Done():
timer.Stop()
return
}
}
retries++
lastStreamStartTime = time.Now()
stream, err := t.vClient.NewStream(ctx, t.cc)
if err != nil {
t.updateHandler.NewConnectionError(err)
t.logger.Warningf("xds: ADS stream creation failed: %v", err)
continue
}
t.logger.Infof("ADS stream created")
select {
case <-t.streamCh:
default:
}
t.streamCh <- stream
if t.recv(stream) {
retries = 0
}
}
}
// send is a separate goroutine for sending watch requests on the xds stream.
//
// It watches the stream channel for new streams, and the request channel for
// new requests to send on the stream.
//
// 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
// the current stream is cleared when any send on it fails
//
// For each new stream, all the existing requests will be resent.
//
// Note that this goroutine doesn't do anything to the old stream when there's a
// new one. In fact, there should be only one stream in progress, and new one
// should only be created when the old one fails (recv returns an error).
func (t *Controller) send(ctx context.Context, doneCh chan struct{}) {
defer func() { close(doneCh) }()
var stream grpc.ClientStream
for {
select {
case <-ctx.Done():
return
case stream = <-t.streamCh:
if !t.sendExisting(stream) {
// send failed, clear the current stream.
stream = nil
}
case u := <-t.sendCh.Get():
t.sendCh.Load()
var (
target []string
rType xdsresource.ResourceType
version, nonce, errMsg string
send bool
)
switch update := u.(type) {
case *watchAction:
target, rType, version, nonce = t.processWatchInfo(update)
case *ackAction:
target, rType, version, nonce, send = t.processAckInfo(update, stream)
if !send {
continue
}
errMsg = update.errMsg
}
if stream == nil {
// There's no stream yet. Skip the request. This request
// will be resent to the new streams. If no stream is
// created, the watcher will timeout (same as server not
// sending response back).
continue
}
if err := t.vClient.SendRequest(stream, target, rType, version, nonce, errMsg); err != nil {
t.logger.Warningf("ADS request for {target: %q, type: %v, version: %q, nonce: %q} failed: %v", target, rType, version, nonce, err)
// send failed, clear the current stream.
stream = nil
}
}
}
}
// 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 (t *Controller) sendExisting(stream grpc.ClientStream) bool {
t.mu.Lock()
defer t.mu.Unlock()
// Reset only the nonce when the stream restarts.
//
// xDS spec says the following. See section:
// https://www.envoyproxy.io/docs/envoy/latest/api-docs/xds_protocol#ack-nack-and-resource-type-instance-version
//
// Note that the version for a resource type is not a property of an
// individual xDS stream but rather a property of the resources themselves. If
// the stream becomes broken and the client creates a new stream, the clients
// initial request on the new stream should indicate the most recent version
// seen by the client on the previous stream
t.nonceMap = make(map[xdsresource.ResourceType]string)
for rType, s := range t.watchMap {
if err := t.vClient.SendRequest(stream, mapToSlice(s), rType, t.versionMap[rType], "", ""); err != nil {
t.logger.Warningf("ADS request failed: %v", err)
return false
}
}
return true
}
// recv receives xDS responses on the provided ADS stream and branches out to
// message specific handlers.
func (t *Controller) recv(stream grpc.ClientStream) bool {
msgReceived := false
for {
resp, err := t.vClient.RecvResponse(stream)
if err != nil {
t.updateHandler.NewConnectionError(err)
t.logger.Warningf("ADS stream is closed with error: %v", err)
return msgReceived
}
msgReceived = true
rType, version, nonce, err := t.handleResponse(resp)
if e, ok := err.(xdsresourceversion.ErrResourceTypeUnsupported); ok {
t.logger.Warningf("%s", e.ErrStr)
continue
}
if err != nil {
t.sendCh.Put(&ackAction{
rType: rType,
version: "",
nonce: nonce,
errMsg: err.Error(),
stream: stream,
})
t.logger.Warningf("Sending NACK for response type: %v, version: %v, nonce: %v, reason: %v", rType, version, nonce, err)
continue
}
t.sendCh.Put(&ackAction{
rType: rType,
version: version,
nonce: nonce,
stream: stream,
})
t.logger.Infof("Sending ACK for response type: %v, version: %v, nonce: %v", rType, version, nonce)
}
}
func (t *Controller) handleResponse(resp proto.Message) (xdsresource.ResourceType, string, string, error) {
rType, resource, version, nonce, err := t.vClient.ParseResponse(resp)
if err != nil {
return rType, version, nonce, err
}
opts := &xdsresource.UnmarshalOptions{
Version: version,
Resources: resource,
Logger: t.logger,
UpdateValidator: t.updateValidator,
}
var md xdsresource.UpdateMetadata
switch rType {
case xdsresource.ListenerResource:
var update map[string]xdsresource.ListenerUpdateErrTuple
update, md, err = xdsresource.UnmarshalListener(opts)
t.updateHandler.NewListeners(update, md)
case xdsresource.RouteConfigResource:
var update map[string]xdsresource.RouteConfigUpdateErrTuple
update, md, err = xdsresource.UnmarshalRouteConfig(opts)
t.updateHandler.NewRouteConfigs(update, md)
case xdsresource.ClusterResource:
var update map[string]xdsresource.ClusterUpdateErrTuple
update, md, err = xdsresource.UnmarshalCluster(opts)
t.updateHandler.NewClusters(update, md)
case xdsresource.EndpointsResource:
var update map[string]xdsresource.EndpointsUpdateErrTuple
update, md, err = xdsresource.UnmarshalEndpoints(opts)
t.updateHandler.NewEndpoints(update, md)
default:
return rType, "", "", xdsresourceversion.ErrResourceTypeUnsupported{
ErrStr: fmt.Sprintf("Resource type %v unknown in response from server", rType),
}
}
return rType, version, nonce, err
}
func mapToSlice(m map[string]bool) []string {
ret := make([]string, 0, len(m))
for i := range m {
ret = append(ret, i)
}
return ret
}
type watchAction struct {
rType xdsresource.ResourceType
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 updates the watch map.
func (t *Controller) processWatchInfo(w *watchAction) (target []string, rType xdsresource.ResourceType, ver, nonce string) {
t.mu.Lock()
defer t.mu.Unlock()
var current map[string]bool
current, ok := t.watchMap[w.rType]
if !ok {
current = make(map[string]bool)
t.watchMap[w.rType] = current
}
if w.remove {
delete(current, w.resource)
if len(current) == 0 {
delete(t.watchMap, w.rType)
}
} else {
current[w.resource] = true
}
rType = w.rType
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. Only the nonce
// is reset when the stream is recreated.
ver = t.versionMap[rType]
nonce = t.nonceMap[rType]
return target, rType, ver, nonce
}
type ackAction struct {
rType xdsresource.ResourceType
version string // NACK if version is an empty string.
nonce string
errMsg string // Empty unless it's a NACK.
// 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 grpc.ClientStream
}
// 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 (t *Controller) processAckInfo(ack *ackAction, stream grpc.ClientStream) (target []string, rType xdsresource.ResourceType, version, nonce string, send bool) {
if ack.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 nil, xdsresource.UnknownResource, "", "", false
}
rType = ack.rType
t.mu.Lock()
defer t.mu.Unlock()
// Update the nonce no matter if we are going to send the ACK request on
// wire. We may not send the request if the watch is canceled. But the nonce
// needs to be updated so the next request will have the right nonce.
nonce = ack.nonce
t.nonceMap[rType] = nonce
s, ok := t.watchMap[rType]
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 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, xdsresource.UnknownResource, "", "", false
}
send = true
target = mapToSlice(s)
version = ack.version
if version == "" {
// This is a nack, get the previous acked version.
version = t.versionMap[rType]
// version will still be an empty string if rType isn't
// found in versionMap, this can happen if there wasn't any ack
// before.
} else {
t.versionMap[rType] = version
}
return target, rType, version, nonce, send
}
// reportLoad starts an LRS stream to report load data to the management server.
// It blocks until the context is cancelled.
func (t *Controller) reportLoad(ctx context.Context, cc *grpc.ClientConn, opts controllerversion.LoadReportingOptions) {
retries := 0
lastStreamStartTime := time.Time{}
for ctx.Err() == nil {
dur := time.Until(lastStreamStartTime.Add(t.backoff(retries)))
if dur > 0 {
timer := time.NewTimer(dur)
select {
case <-timer.C:
case <-ctx.Done():
timer.Stop()
return
}
}
retries++
lastStreamStartTime = time.Now()
func() {
// streamCtx is created and canceled in case we terminate the stream
// early for any reason, to avoid gRPC-Go leaking the RPC's monitoring
// goroutine.
streamCtx, cancel := context.WithCancel(ctx)
defer cancel()
stream, err := t.vClient.NewLoadStatsStream(streamCtx, cc)
if err != nil {
t.logger.Warningf("lrs: failed to create stream: %v", err)
return
}
t.logger.Infof("lrs: created LRS stream")
if err := t.vClient.SendFirstLoadStatsRequest(stream); err != nil {
t.logger.Warningf("lrs: failed to send first request: %v", err)
return
}
clusters, interval, err := t.vClient.HandleLoadStatsResponse(stream)
if err != nil {
t.logger.Warningf("lrs: error from stream: %v", err)
return
}
retries = 0
t.sendLoads(streamCtx, stream, opts.LoadStore, clusters, interval)
}()
}
}
func (t *Controller) sendLoads(ctx context.Context, stream grpc.ClientStream, store *load.Store, clusterNames []string, interval time.Duration) {
tick := time.NewTicker(interval)
defer tick.Stop()
for {
select {
case <-tick.C:
case <-ctx.Done():
return
}
if err := t.vClient.SendLoadStatsRequest(stream, store.Stats(clusterNames)); err != nil {
t.logger.Warningf("lrs: error from stream: %v", err)
return
}
}
}

View File

@ -1,483 +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 controller
import (
"context"
"fmt"
"strconv"
"testing"
"time"
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
"github.com/golang/protobuf/proto"
anypb "github.com/golang/protobuf/ptypes/any"
"github.com/google/go-cmp/cmp"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/internal/testutils"
"google.golang.org/grpc/xds/internal/testutils/fakeserver"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
)
const (
defaultTestTimeout = 5 * time.Second
defaultTestShortTimeout = 10 * time.Millisecond
)
func startXDSV2Client(t *testing.T, controlPlaneAddr string) (v2c *Controller, cbLDS, cbRDS, cbCDS, cbEDS *testutils.Channel, cleanup func()) {
cbLDS = testutils.NewChannel()
cbRDS = testutils.NewChannel()
cbCDS = testutils.NewChannel()
cbEDS = testutils.NewChannel()
v2c, err := newTestController(&testUpdateReceiver{
f: func(rType xdsresource.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata) {
t.Logf("Received %v callback with {%+v}", rType, d)
switch rType {
case xdsresource.ListenerResource:
if _, ok := d[goodLDSTarget1]; ok {
cbLDS.Send(struct{}{})
}
case xdsresource.RouteConfigResource:
if _, ok := d[goodRouteName1]; ok {
cbRDS.Send(struct{}{})
}
case xdsresource.ClusterResource:
if _, ok := d[goodClusterName1]; ok {
cbCDS.Send(struct{}{})
}
case xdsresource.EndpointsResource:
if _, ok := d[goodEDSName]; ok {
cbEDS.Send(struct{}{})
}
}
},
}, controlPlaneAddr, goodNodeProto, func(int) time.Duration { return 0 }, nil)
if err != nil {
t.Fatal(err)
}
t.Log("Started xds client...")
return v2c, cbLDS, cbRDS, cbCDS, cbEDS, v2c.Close
}
// compareXDSRequest reads requests from channel, compare it with want.
func compareXDSRequest(ctx context.Context, ch *testutils.Channel, want *xdspb.DiscoveryRequest, ver, nonce string, wantErr bool) error {
val, err := ch.Receive(ctx)
if err != nil {
return err
}
req := val.(*fakeserver.Request)
if req.Err != nil {
return fmt.Errorf("unexpected error from request: %v", req.Err)
}
xdsReq := req.Req.(*xdspb.DiscoveryRequest)
if (xdsReq.ErrorDetail != nil) != wantErr {
return fmt.Errorf("received request with error details: %v, wantErr: %v", xdsReq.ErrorDetail, wantErr)
}
// All NACK request.ErrorDetails have hardcoded status code InvalidArguments.
if xdsReq.ErrorDetail != nil && xdsReq.ErrorDetail.Code != int32(codes.InvalidArgument) {
return fmt.Errorf("received request with error details: %v, want status with code: %v", xdsReq.ErrorDetail, codes.InvalidArgument)
}
xdsReq.ErrorDetail = nil // Clear the error details field before comparing.
wantClone := proto.Clone(want).(*xdspb.DiscoveryRequest)
wantClone.VersionInfo = ver
wantClone.ResponseNonce = nonce
if !cmp.Equal(xdsReq, wantClone, cmp.Comparer(proto.Equal)) {
return fmt.Errorf("received request different from want, diff: %s", cmp.Diff(req.Req, wantClone, cmp.Comparer(proto.Equal)))
}
return nil
}
func sendXDSRespWithVersion(ch chan<- *fakeserver.Response, respWithoutVersion *xdspb.DiscoveryResponse, ver int) (nonce string) {
respToSend := proto.Clone(respWithoutVersion).(*xdspb.DiscoveryResponse)
respToSend.VersionInfo = strconv.Itoa(ver)
nonce = strconv.Itoa(int(time.Now().UnixNano()))
respToSend.Nonce = nonce
ch <- &fakeserver.Response{Resp: respToSend}
return
}
// startXDS calls watch to send the first request. It then sends a good response
// and checks for ack.
func startXDS(ctx context.Context, t *testing.T, rType xdsresource.ResourceType, v2c *Controller, reqChan *testutils.Channel, req *xdspb.DiscoveryRequest, preVersion string, preNonce string) {
nameToWatch := ""
switch rType {
case xdsresource.ListenerResource:
nameToWatch = goodLDSTarget1
case xdsresource.RouteConfigResource:
nameToWatch = goodRouteName1
case xdsresource.ClusterResource:
nameToWatch = goodClusterName1
case xdsresource.EndpointsResource:
nameToWatch = goodEDSName
}
v2c.AddWatch(rType, nameToWatch)
if err := compareXDSRequest(ctx, reqChan, req, preVersion, preNonce, false); err != nil {
t.Fatalf("Failed to receive %v request: %v", rType, err)
}
t.Logf("FakeServer received %v request...", rType)
}
// sendGoodResp sends the good response, with the given version, and a random
// nonce.
//
// It also waits and checks that the ack request contains the given version, and
// the generated nonce.
func sendGoodResp(ctx context.Context, t *testing.T, rType xdsresource.ResourceType, fakeServer *fakeserver.Server, ver int, goodResp *xdspb.DiscoveryResponse, wantReq *xdspb.DiscoveryRequest, callbackCh *testutils.Channel) (string, error) {
nonce := sendXDSRespWithVersion(fakeServer.XDSResponseChan, goodResp, ver)
t.Logf("Good %v response pushed to fakeServer...", rType)
if err := compareXDSRequest(ctx, fakeServer.XDSRequestChan, wantReq, strconv.Itoa(ver), nonce, false); err != nil {
return "", fmt.Errorf("failed to receive %v request: %v", rType, err)
}
t.Logf("Good %v response acked", rType)
if _, err := callbackCh.Receive(ctx); err != nil {
return "", fmt.Errorf("timeout when expecting %v update", rType)
}
t.Logf("Good %v response callback executed", rType)
return nonce, nil
}
// sendBadResp sends a bad response with the given version. This response will
// be nacked, so we expect a request with the previous version (version-1).
//
// But the nonce in request should be the new nonce.
func sendBadResp(ctx context.Context, t *testing.T, rType xdsresource.ResourceType, fakeServer *fakeserver.Server, ver int, wantReq *xdspb.DiscoveryRequest) error {
var typeURL string
switch rType {
case xdsresource.ListenerResource:
typeURL = version.V2ListenerURL
case xdsresource.RouteConfigResource:
typeURL = version.V2RouteConfigURL
case xdsresource.ClusterResource:
typeURL = version.V2ClusterURL
case xdsresource.EndpointsResource:
typeURL = version.V2EndpointsURL
}
nonce := sendXDSRespWithVersion(fakeServer.XDSResponseChan, &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{{}},
TypeUrl: typeURL,
}, ver)
t.Logf("Bad %v response pushed to fakeServer...", rType)
if err := compareXDSRequest(ctx, fakeServer.XDSRequestChan, wantReq, strconv.Itoa(ver-1), nonce, true); err != nil {
return fmt.Errorf("failed to receive %v request: %v", rType, err)
}
t.Logf("Bad %v response nacked", rType)
return nil
}
// TestV2ClientAck verifies that valid responses are acked, and invalid ones
// are nacked.
//
// This test also verifies the version for different types are independent.
func (s) TestV2ClientAck(t *testing.T) {
var (
versionLDS = 1000
versionRDS = 2000
versionCDS = 3000
versionEDS = 4000
)
fakeServer, cleanup := startServer(t)
defer cleanup()
v2c, cbLDS, cbRDS, cbCDS, cbEDS, v2cCleanup := startXDSV2Client(t, fakeServer.Address)
defer v2cCleanup()
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
// Start the watch, send a good response, and check for ack.
startXDS(ctx, t, xdsresource.ListenerResource, v2c, fakeServer.XDSRequestChan, goodLDSRequest, "", "")
if _, err := sendGoodResp(ctx, t, xdsresource.ListenerResource, fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS); err != nil {
t.Fatal(err)
}
versionLDS++
startXDS(ctx, t, xdsresource.RouteConfigResource, v2c, fakeServer.XDSRequestChan, goodRDSRequest, "", "")
if _, err := sendGoodResp(ctx, t, xdsresource.RouteConfigResource, fakeServer, versionRDS, goodRDSResponse1, goodRDSRequest, cbRDS); err != nil {
t.Fatal(err)
}
versionRDS++
startXDS(ctx, t, xdsresource.ClusterResource, v2c, fakeServer.XDSRequestChan, goodCDSRequest, "", "")
if _, err := sendGoodResp(ctx, t, xdsresource.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS); err != nil {
t.Fatal(err)
}
versionCDS++
startXDS(ctx, t, xdsresource.EndpointsResource, v2c, fakeServer.XDSRequestChan, goodEDSRequest, "", "")
if _, err := sendGoodResp(ctx, t, xdsresource.EndpointsResource, fakeServer, versionEDS, goodEDSResponse1, goodEDSRequest, cbEDS); err != nil {
t.Fatal(err)
}
versionEDS++
// Send a bad response, and check for nack.
if err := sendBadResp(ctx, t, xdsresource.ListenerResource, fakeServer, versionLDS, goodLDSRequest); err != nil {
t.Fatal(err)
}
versionLDS++
if err := sendBadResp(ctx, t, xdsresource.RouteConfigResource, fakeServer, versionRDS, goodRDSRequest); err != nil {
t.Fatal(err)
}
versionRDS++
if err := sendBadResp(ctx, t, xdsresource.ClusterResource, fakeServer, versionCDS, goodCDSRequest); err != nil {
t.Fatal(err)
}
versionCDS++
if err := sendBadResp(ctx, t, xdsresource.EndpointsResource, fakeServer, versionEDS, goodEDSRequest); err != nil {
t.Fatal(err)
}
versionEDS++
// send another good response, and check for ack, with the new version.
if _, err := sendGoodResp(ctx, t, xdsresource.ListenerResource, fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS); err != nil {
t.Fatal(err)
}
versionLDS++
if _, err := sendGoodResp(ctx, t, xdsresource.RouteConfigResource, fakeServer, versionRDS, goodRDSResponse1, goodRDSRequest, cbRDS); err != nil {
t.Fatal(err)
}
versionRDS++
if _, err := sendGoodResp(ctx, t, xdsresource.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS); err != nil {
t.Fatal(err)
}
versionCDS++
if _, err := sendGoodResp(ctx, t, xdsresource.EndpointsResource, fakeServer, versionEDS, goodEDSResponse1, goodEDSRequest, cbEDS); err != nil {
t.Fatal(err)
}
versionEDS++
}
// Test when the first response is invalid, and is nacked, the nack requests
// should have an empty version string.
func (s) TestV2ClientAckFirstIsNack(t *testing.T) {
var versionLDS = 1000
fakeServer, cleanup := startServer(t)
defer cleanup()
v2c, cbLDS, _, _, _, v2cCleanup := startXDSV2Client(t, fakeServer.Address)
defer v2cCleanup()
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
// Start the watch, send a good response, and check for ack.
startXDS(ctx, t, xdsresource.ListenerResource, v2c, fakeServer.XDSRequestChan, goodLDSRequest, "", "")
nonce := sendXDSRespWithVersion(fakeServer.XDSResponseChan, &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{{}},
TypeUrl: version.V2ListenerURL,
}, versionLDS)
t.Logf("Bad response pushed to fakeServer...")
// The expected version string is an empty string, because this is the first
// response, and it's nacked (so there's no previous ack version).
if err := compareXDSRequest(ctx, fakeServer.XDSRequestChan, goodLDSRequest, "", nonce, true); err != nil {
t.Errorf("Failed to receive request: %v", err)
}
t.Logf("Bad response nacked")
versionLDS++
sendGoodResp(ctx, t, xdsresource.ListenerResource, fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS)
versionLDS++
}
// Test when a nack is sent after a new watch, we nack with the previous acked
// version (instead of resetting to empty string).
func (s) TestV2ClientAckNackAfterNewWatch(t *testing.T) {
var versionLDS = 1000
fakeServer, cleanup := startServer(t)
defer cleanup()
v2c, cbLDS, _, _, _, v2cCleanup := startXDSV2Client(t, fakeServer.Address)
defer v2cCleanup()
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
// Start the watch, send a good response, and check for ack.
startXDS(ctx, t, xdsresource.ListenerResource, v2c, fakeServer.XDSRequestChan, goodLDSRequest, "", "")
nonce, err := sendGoodResp(ctx, t, xdsresource.ListenerResource, fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS)
if err != nil {
t.Fatal(err)
}
// Start a new watch. The version in the new request should be the version
// from the previous response, thus versionLDS before ++.
startXDS(ctx, t, xdsresource.ListenerResource, v2c, fakeServer.XDSRequestChan, goodLDSRequest, strconv.Itoa(versionLDS), nonce)
versionLDS++
// This is an invalid response after the new watch.
nonce = sendXDSRespWithVersion(fakeServer.XDSResponseChan, &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{{}},
TypeUrl: version.V2ListenerURL,
}, versionLDS)
t.Logf("Bad response pushed to fakeServer...")
// The expected version string is the previous acked version.
if err := compareXDSRequest(ctx, fakeServer.XDSRequestChan, goodLDSRequest, strconv.Itoa(versionLDS-1), nonce, true); err != nil {
t.Errorf("Failed to receive request: %v", err)
}
t.Logf("Bad response nacked")
versionLDS++
if _, err := sendGoodResp(ctx, t, xdsresource.ListenerResource, fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS); err != nil {
t.Fatal(err)
}
versionLDS++
}
// TestV2ClientAckNewWatchAfterCancel verifies the new request for a new watch
// after the previous watch is canceled, has the right version.
func (s) TestV2ClientAckNewWatchAfterCancel(t *testing.T) {
var versionCDS = 3000
fakeServer, cleanup := startServer(t)
defer cleanup()
v2c, _, _, cbCDS, _, v2cCleanup := startXDSV2Client(t, fakeServer.Address)
defer v2cCleanup()
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
// Start a CDS watch.
v2c.AddWatch(xdsresource.ClusterResource, goodClusterName1)
if err := compareXDSRequest(ctx, fakeServer.XDSRequestChan, goodCDSRequest, "", "", false); err != nil {
t.Fatal(err)
}
t.Logf("FakeServer received %v request...", xdsresource.ClusterResource)
// Send a good CDS response, this function waits for the ACK with the right
// version.
nonce, err := sendGoodResp(ctx, t, xdsresource.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS)
if err != nil {
t.Fatal(err)
}
// Cancel the CDS watch, and start a new one. The new watch should have the
// version from the response above.
v2c.RemoveWatch(xdsresource.ClusterResource, goodClusterName1)
// Wait for a request with no resource names, because the only watch was
// removed.
emptyReq := &xdspb.DiscoveryRequest{Node: goodNodeProto, TypeUrl: version.V2ClusterURL}
if err := compareXDSRequest(ctx, fakeServer.XDSRequestChan, emptyReq, strconv.Itoa(versionCDS), nonce, false); err != nil {
t.Fatalf("Failed to receive %v request: %v", xdsresource.ClusterResource, err)
}
v2c.AddWatch(xdsresource.ClusterResource, goodClusterName1)
// Wait for a request with correct resource names and version.
if err := compareXDSRequest(ctx, fakeServer.XDSRequestChan, goodCDSRequest, strconv.Itoa(versionCDS), nonce, false); err != nil {
t.Fatalf("Failed to receive %v request: %v", xdsresource.ClusterResource, err)
}
versionCDS++
// Send a bad response with the next version.
if err := sendBadResp(ctx, t, xdsresource.ClusterResource, fakeServer, versionCDS, goodCDSRequest); err != nil {
t.Fatal(err)
}
versionCDS++
// send another good response, and check for ack, with the new version.
if _, err := sendGoodResp(ctx, t, xdsresource.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS); err != nil {
t.Fatal(err)
}
versionCDS++
}
// TestV2ClientAckCancelResponseRace verifies if the response and ACK request
// race with cancel (which means the ACK request will not be sent on wire,
// because there's no active watch), the nonce will still be updated, and the
// new request with the new watch will have the correct nonce.
func (s) TestV2ClientAckCancelResponseRace(t *testing.T) {
var versionCDS = 3000
fakeServer, cleanup := startServer(t)
defer cleanup()
v2c, _, _, cbCDS, _, v2cCleanup := startXDSV2Client(t, fakeServer.Address)
defer v2cCleanup()
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
// Start a CDS watch.
v2c.AddWatch(xdsresource.ClusterResource, goodClusterName1)
if err := compareXDSRequest(ctx, fakeServer.XDSRequestChan, goodCDSRequest, "", "", false); err != nil {
t.Fatalf("Failed to receive %v request: %v", xdsresource.ClusterResource, err)
}
t.Logf("FakeServer received %v request...", xdsresource.ClusterResource)
// send a good response, and check for ack, with the new version.
nonce, err := sendGoodResp(ctx, t, xdsresource.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS)
if err != nil {
t.Fatal(err)
}
// Cancel the watch before the next response is sent. This mimics the case
// watch is canceled while response is on wire.
v2c.RemoveWatch(xdsresource.ClusterResource, goodClusterName1)
// Wait for a request with no resource names, because the only watch was
// removed.
emptyReq := &xdspb.DiscoveryRequest{Node: goodNodeProto, TypeUrl: version.V2ClusterURL}
if err := compareXDSRequest(ctx, fakeServer.XDSRequestChan, emptyReq, strconv.Itoa(versionCDS), nonce, false); err != nil {
t.Fatalf("Failed to receive %v request: %v", xdsresource.ClusterResource, err)
}
versionCDS++
sCtx, sCancel := context.WithTimeout(context.Background(), defaultTestShortTimeout)
defer sCancel()
if req, err := fakeServer.XDSRequestChan.Receive(sCtx); err != context.DeadlineExceeded {
t.Fatalf("Got unexpected xds request after watch is canceled: %v", req)
}
// Send a good response.
nonce = sendXDSRespWithVersion(fakeServer.XDSResponseChan, goodCDSResponse1, versionCDS)
t.Logf("Good %v response pushed to fakeServer...", xdsresource.ClusterResource)
// Expect no ACK because watch was canceled.
sCtx, sCancel = context.WithTimeout(context.Background(), defaultTestShortTimeout)
defer sCancel()
if req, err := fakeServer.XDSRequestChan.Receive(sCtx); err != context.DeadlineExceeded {
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(ctx); err != nil {
t.Fatalf("Timeout when expecting %v update", xdsresource.ClusterResource)
}
// Start a new watch. The new watch should have the nonce from the response
// above, and version from the first good response.
v2c.AddWatch(xdsresource.ClusterResource, goodClusterName1)
if err := compareXDSRequest(ctx, fakeServer.XDSRequestChan, goodCDSRequest, strconv.Itoa(versionCDS-1), nonce, false); err != nil {
t.Fatalf("Failed to receive %v request: %v", xdsresource.ClusterResource, err)
}
// Send a bad response with the next version.
if err := sendBadResp(ctx, t, xdsresource.ClusterResource, fakeServer, versionCDS, goodCDSRequest); err != nil {
t.Fatal(err)
}
versionCDS++
// send another good response, and check for ack, with the new version.
if _, err := sendGoodResp(ctx, t, xdsresource.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS); err != nil {
t.Fatal(err)
}
versionCDS++
}

View File

@ -1,186 +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 controller
import (
"testing"
"time"
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
corepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
anypb "github.com/golang/protobuf/ptypes/any"
"github.com/google/go-cmp/cmp/cmpopts"
"google.golang.org/grpc/internal/testutils"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
)
const (
serviceName1 = "foo-service"
serviceName2 = "bar-service"
)
var (
badlyMarshaledCDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: version.V2ClusterURL,
Value: []byte{1, 2, 3, 4},
},
},
TypeUrl: version.V2ClusterURL,
}
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 = testutils.MarshalAny(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 = testutils.MarshalAny(goodCluster2)
goodCDSResponse1 = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
marshaledCluster1,
},
TypeUrl: version.V2ClusterURL,
}
goodCDSResponse2 = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
marshaledCluster2,
},
TypeUrl: version.V2ClusterURL,
}
)
// 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 map[string]xdsresource.ClusterUpdateErrTuple
wantUpdateMD xdsresource.UpdateMetadata
wantUpdateErr bool
}{
// Badly marshaled CDS response.
{
name: "badly-marshaled-response",
cdsResponse: badlyMarshaledCDSResponse,
wantErr: true,
wantUpdate: nil,
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
ErrState: &xdsresource.UpdateErrorMetadata{
Err: cmpopts.AnyError,
},
},
wantUpdateErr: false,
},
// Response contains one good cluster we are not interested in.
{
name: "one-uninteresting-cluster",
cdsResponse: goodCDSResponse2,
wantErr: false,
wantUpdate: map[string]xdsresource.ClusterUpdateErrTuple{
goodClusterName2: {Update: xdsresource.ClusterUpdate{ClusterName: goodClusterName2, EDSServiceName: serviceName2, Raw: marshaledCluster2}},
},
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusACKed,
},
wantUpdateErr: false,
},
// Response contains one cluster and it is good.
{
name: "one-good-cluster",
cdsResponse: goodCDSResponse1,
wantErr: false,
wantUpdate: map[string]xdsresource.ClusterUpdateErrTuple{
goodClusterName1: {Update: xdsresource.ClusterUpdate{ClusterName: goodClusterName1, EDSServiceName: serviceName1, LRSServerConfig: xdsresource.ClusterLRSServerSelf, Raw: marshaledCluster1}},
},
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusACKed,
},
wantUpdateErr: false,
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
testWatchHandle(t, &watchHandleTestcase{
rType: xdsresource.ClusterResource,
resourceName: goodClusterName1,
responseToHandle: test.cdsResponse,
wantHandleErr: test.wantErr,
wantUpdate: test.wantUpdate,
wantUpdateMD: test.wantUpdateMD,
wantUpdateErr: test.wantUpdateErr,
})
})
}
}
// TestCDSHandleResponseWithoutWatch tests the case where the v2Client receives
// a CDS response without a registered watcher.
func (s) TestCDSHandleResponseWithoutWatch(t *testing.T) {
fakeServer, cleanup := startServer(t)
defer cleanup()
v2c, err := newTestController(&testUpdateReceiver{
f: func(xdsresource.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) {},
}, fakeServer.Address, goodNodeProto, func(int) time.Duration { return 0 }, nil)
if err != nil {
t.Fatal(err)
}
defer v2c.Close()
if _, _, _, err := v2c.handleResponse(badResourceTypeInLDSResponse); err == nil {
t.Fatal("v2c.handleCDSResponse() succeeded, should have failed")
}
if _, _, _, err := v2c.handleResponse(goodCDSResponse1); err != nil {
t.Fatal("v2c.handleCDSResponse() succeeded, should have failed")
}
}

View File

@ -1,212 +0,0 @@
/*
*
* Copyright 2021 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 controller
import (
"context"
"errors"
"testing"
"time"
"github.com/golang/protobuf/proto"
"google.golang.org/grpc/internal/testutils"
"google.golang.org/grpc/resolver"
"google.golang.org/grpc/resolver/manual"
"google.golang.org/grpc/xds/internal/testutils/fakeserver"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
_ "google.golang.org/grpc/xds/internal/xdsclient/controller/version/v2" // Register the v2 xDS API client.
)
// TestV2ClientBackoffAfterRecvError verifies if the v2Client backs off when it
// encounters a Recv error while receiving an LDS response.
func (s) TestV2ClientBackoffAfterRecvError(t *testing.T) {
fakeServer, cleanup := startServer(t)
defer cleanup()
// Override the v2Client backoff function with this, so that we can verify
// that a backoff actually was triggered.
boCh := make(chan int, 1)
clientBackoff := func(v int) time.Duration {
boCh <- v
return 0
}
callbackCh := make(chan struct{})
v2c, err := newTestController(&testUpdateReceiver{
f: func(xdsresource.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) { close(callbackCh) },
}, fakeServer.Address, goodNodeProto, clientBackoff, nil)
if err != nil {
t.Fatal(err)
}
defer v2c.Close()
t.Log("Started xds v2Client...")
v2c.AddWatch(xdsresource.ListenerResource, goodLDSTarget1)
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
if _, err := fakeServer.XDSRequestChan.Receive(ctx); err != nil {
t.Fatalf("Timeout expired when expecting an LDS request")
}
t.Log("FakeServer received request...")
fakeServer.XDSResponseChan <- &fakeserver.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")
}
if _, err := fakeServer.XDSRequestChan.Receive(ctx); err != nil {
t.Fatalf("Timeout expired when expecting an LDS request")
}
t.Log("FakeServer received request after backoff...")
}
// 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 (s) TestV2ClientRetriesAfterBrokenStream(t *testing.T) {
fakeServer, cleanup := startServer(t)
defer cleanup()
callbackCh := testutils.NewChannel()
v2c, err := newTestController(&testUpdateReceiver{
f: func(rType xdsresource.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata) {
if rType == xdsresource.ListenerResource {
if u, ok := d[goodLDSTarget1]; ok {
t.Logf("Received LDS callback with ldsUpdate {%+v}", u)
callbackCh.Send(struct{}{})
}
}
},
}, fakeServer.Address, goodNodeProto, func(int) time.Duration { return 0 }, nil)
if err != nil {
t.Fatal(err)
}
defer v2c.Close()
t.Log("Started xds v2Client...")
v2c.AddWatch(xdsresource.ListenerResource, goodLDSTarget1)
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
if _, err := fakeServer.XDSRequestChan.Receive(ctx); 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(ctx); err != nil {
t.Fatal("Timeout when expecting LDS update")
}
// Read the ack, so the next request is sent after stream re-creation.
if _, err := fakeServer.XDSRequestChan.Receive(ctx); err != nil {
t.Fatalf("Timeout expired when expecting an LDS ACK")
}
fakeServer.XDSResponseChan <- &fakeserver.Response{Err: errors.New("RPC error")}
t.Log("Bad LDS response pushed to fakeServer...")
val, err := fakeServer.XDSRequestChan.Receive(ctx)
if err != nil {
t.Fatalf("Timeout expired when expecting LDS update")
}
gotRequest := val.(*fakeserver.Request)
if !proto.Equal(gotRequest.Req, goodLDSRequest) {
t.Fatalf("gotRequest: %+v, wantRequest: %+v", gotRequest.Req, goodLDSRequest)
}
}
// 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) {
fakeServer, sCleanup, err := fakeserver.StartServer()
if err != nil {
t.Fatalf("Failed to start fake xDS server: %v", err)
}
defer sCleanup()
const scheme = "xds-client-test-whatever"
rb := manual.NewBuilderWithScheme(scheme)
rb.InitialState(resolver.State{Addresses: []resolver.Address{{Addr: "no.such.server"}}})
resolver.Register(rb)
defer resolver.UnregisterForTesting(scheme)
callbackCh := testutils.NewChannel()
v2c, err := newTestController(&testUpdateReceiver{
f: func(rType xdsresource.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata) {
if rType == xdsresource.ListenerResource {
if u, ok := d[goodLDSTarget1]; ok {
t.Logf("Received LDS callback with ldsUpdate {%+v}", u)
callbackCh.Send(u)
}
}
},
}, scheme+":///whatever", goodNodeProto, func(int) time.Duration { return 0 }, nil)
if err != nil {
t.Fatal(err)
}
defer v2c.Close()
t.Log("Started xds v2Client...")
// This watch is started when the xds-ClientConn is in Transient Failure,
// and no xds stream is created.
v2c.AddWatch(xdsresource.ListenerResource, goodLDSTarget1)
// The watcher should receive an update, with a timeout error in it.
sCtx, sCancel := context.WithTimeout(context.Background(), defaultTestShortTimeout)
defer sCancel()
if v, err := callbackCh.Receive(sCtx); 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
// created, and the previous watch should be sent.
rb.UpdateState(resolver.State{
Addresses: []resolver.Address{{Addr: fakeServer.Address}},
})
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
if _, err := fakeServer.XDSRequestChan.Receive(ctx); 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 v, err := callbackCh.Receive(ctx); err != nil {
t.Fatal("Timeout when expecting LDS update")
} else if _, ok := v.(xdsresource.ListenerUpdateErrTuple); !ok {
t.Fatalf("Expect an LDS update from watcher, got %v", v)
}
}

View File

@ -1,200 +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 controller
import (
"testing"
"time"
v2xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
anypb "github.com/golang/protobuf/ptypes/any"
"github.com/google/go-cmp/cmp/cmpopts"
"google.golang.org/grpc/internal/testutils"
"google.golang.org/grpc/xds/internal"
xtestutils "google.golang.org/grpc/xds/internal/testutils"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
)
var (
badlyMarshaledEDSResponse = &v2xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: version.V2EndpointsURL,
Value: []byte{1, 2, 3, 4},
},
},
TypeUrl: version.V2EndpointsURL,
}
badResourceTypeInEDSResponse = &v2xdspb.DiscoveryResponse{
Resources: []*anypb.Any{marshaledConnMgr1},
TypeUrl: version.V2EndpointsURL,
}
marshaledGoodCLA1 = func() *anypb.Any {
clab0 := xtestutils.NewClusterLoadAssignmentBuilder(goodEDSName, nil)
clab0.AddLocality("locality-1", 1, 1, []string{"addr1:314"}, nil)
clab0.AddLocality("locality-2", 1, 0, []string{"addr2:159"}, nil)
return testutils.MarshalAny(clab0.Build())
}()
goodEDSResponse1 = &v2xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
marshaledGoodCLA1,
},
TypeUrl: version.V2EndpointsURL,
}
marshaledGoodCLA2 = func() *anypb.Any {
clab0 := xtestutils.NewClusterLoadAssignmentBuilder("not-goodEDSName", nil)
clab0.AddLocality("locality-1", 1, 0, []string{"addr1:314"}, nil)
return testutils.MarshalAny(clab0.Build())
}()
goodEDSResponse2 = &v2xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
marshaledGoodCLA2,
},
TypeUrl: version.V2EndpointsURL,
}
)
func (s) TestEDSHandleResponse(t *testing.T) {
tests := []struct {
name string
edsResponse *v2xdspb.DiscoveryResponse
wantErr bool
wantUpdate map[string]xdsresource.EndpointsUpdateErrTuple
wantUpdateMD xdsresource.UpdateMetadata
wantUpdateErr bool
}{
// Any in resource is badly marshaled.
{
name: "badly-marshaled_response",
edsResponse: badlyMarshaledEDSResponse,
wantErr: true,
wantUpdate: nil,
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
ErrState: &xdsresource.UpdateErrorMetadata{
Err: cmpopts.AnyError,
},
},
wantUpdateErr: false,
},
// Response doesn't contain resource with the right type.
{
name: "no-config-in-response",
edsResponse: badResourceTypeInEDSResponse,
wantErr: true,
wantUpdate: nil,
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
ErrState: &xdsresource.UpdateErrorMetadata{
Err: cmpopts.AnyError,
},
},
wantUpdateErr: false,
},
// Response contains one uninteresting ClusterLoadAssignment.
{
name: "one-uninterestring-assignment",
edsResponse: goodEDSResponse2,
wantErr: false,
wantUpdate: map[string]xdsresource.EndpointsUpdateErrTuple{
"not-goodEDSName": {Update: xdsresource.EndpointsUpdate{
Localities: []xdsresource.Locality{
{
Endpoints: []xdsresource.Endpoint{{Address: "addr1:314", Weight: 1}},
ID: internal.LocalityID{SubZone: "locality-1"},
Priority: 0,
Weight: 1,
},
},
Raw: marshaledGoodCLA2,
}},
},
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusACKed,
},
wantUpdateErr: false,
},
// Response contains one good ClusterLoadAssignment.
{
name: "one-good-assignment",
edsResponse: goodEDSResponse1,
wantErr: false,
wantUpdate: map[string]xdsresource.EndpointsUpdateErrTuple{
goodEDSName: {Update: xdsresource.EndpointsUpdate{
Localities: []xdsresource.Locality{
{
Endpoints: []xdsresource.Endpoint{{Address: "addr1:314", Weight: 1}},
ID: internal.LocalityID{SubZone: "locality-1"},
Priority: 1,
Weight: 1,
},
{
Endpoints: []xdsresource.Endpoint{{Address: "addr2:159", Weight: 1}},
ID: internal.LocalityID{SubZone: "locality-2"},
Priority: 0,
Weight: 1,
},
},
Raw: marshaledGoodCLA1,
}},
},
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusACKed,
},
wantUpdateErr: false,
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
testWatchHandle(t, &watchHandleTestcase{
rType: xdsresource.EndpointsResource,
resourceName: goodEDSName,
responseToHandle: test.edsResponse,
wantHandleErr: test.wantErr,
wantUpdate: test.wantUpdate,
wantUpdateMD: test.wantUpdateMD,
wantUpdateErr: test.wantUpdateErr,
})
})
}
}
// TestEDSHandleResponseWithoutWatch tests the case where the v2Client
// receives an EDS response without a registered EDS watcher.
func (s) TestEDSHandleResponseWithoutWatch(t *testing.T) {
fakeServer, cleanup := startServer(t)
defer cleanup()
v2c, err := newTestController(&testUpdateReceiver{
f: func(xdsresource.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) {},
}, fakeServer.Address, goodNodeProto, func(int) time.Duration { return 0 }, nil)
if err != nil {
t.Fatal(err)
}
defer v2c.Close()
if _, _, _, err := v2c.handleResponse(badResourceTypeInEDSResponse); err == nil {
t.Fatal("v2c.handleEDSResponse() succeeded, should have failed")
}
if _, _, _, err := v2c.handleResponse(goodEDSResponse1); err != nil {
t.Fatal("v2c.handleEDSResponse() succeeded, should have failed")
}
}

View File

@ -1,198 +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 controller
import (
"testing"
"time"
v2xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
"github.com/google/go-cmp/cmp/cmpopts"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
// TestLDSHandleResponse starts a fake xDS server, makes a ClientConn to it,
// and creates a client using it. Then, it registers a watchLDS and tests
// different LDS responses.
func (s) TestLDSHandleResponse(t *testing.T) {
tests := []struct {
name string
ldsResponse *v2xdspb.DiscoveryResponse
wantErr bool
wantUpdate map[string]xdsresource.ListenerUpdateErrTuple
wantUpdateMD xdsresource.UpdateMetadata
wantUpdateErr bool
}{
// Badly marshaled LDS response.
{
name: "badly-marshaled-response",
ldsResponse: badlyMarshaledLDSResponse,
wantErr: true,
wantUpdate: nil,
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
ErrState: &xdsresource.UpdateErrorMetadata{
Err: cmpopts.AnyError,
},
},
wantUpdateErr: false,
},
// Response does not contain Listener proto.
{
name: "no-listener-proto-in-response",
ldsResponse: badResourceTypeInLDSResponse,
wantErr: true,
wantUpdate: nil,
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
ErrState: &xdsresource.UpdateErrorMetadata{
Err: cmpopts.AnyError,
},
},
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: map[string]xdsresource.ListenerUpdateErrTuple{
goodLDSTarget1: {Err: cmpopts.AnyError},
},
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
ErrState: &xdsresource.UpdateErrorMetadata{
Err: cmpopts.AnyError,
},
},
wantUpdateErr: false,
},
// Response contains one listener and it is good.
{
name: "one-good-listener",
ldsResponse: goodLDSResponse1,
wantErr: false,
wantUpdate: map[string]xdsresource.ListenerUpdateErrTuple{
goodLDSTarget1: {Update: xdsresource.ListenerUpdate{RouteConfigName: goodRouteName1, Raw: marshaledListener1}},
},
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusACKed,
},
wantUpdateErr: false,
},
// Response contains multiple good listeners, including the one we are
// interested in.
{
name: "multiple-good-listener",
ldsResponse: ldsResponseWithMultipleResources,
wantErr: false,
wantUpdate: map[string]xdsresource.ListenerUpdateErrTuple{
goodLDSTarget1: {Update: xdsresource.ListenerUpdate{RouteConfigName: goodRouteName1, Raw: marshaledListener1}},
goodLDSTarget2: {Update: xdsresource.ListenerUpdate{RouteConfigName: goodRouteName1, Raw: marshaledListener2}},
},
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusACKed,
},
wantUpdateErr: false,
},
// Response contains two good listeners (one interesting and one
// 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: true,
wantUpdate: map[string]xdsresource.ListenerUpdateErrTuple{
goodLDSTarget1: {Update: xdsresource.ListenerUpdate{RouteConfigName: goodRouteName1, Raw: marshaledListener1}},
goodLDSTarget2: {Err: cmpopts.AnyError},
},
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
ErrState: &xdsresource.UpdateErrorMetadata{
Err: cmpopts.AnyError,
},
},
wantUpdateErr: false,
},
// Response contains one listener, but we are not interested in it.
{
name: "one-uninteresting-listener",
ldsResponse: goodLDSResponse2,
wantErr: false,
wantUpdate: map[string]xdsresource.ListenerUpdateErrTuple{
goodLDSTarget2: {Update: xdsresource.ListenerUpdate{RouteConfigName: goodRouteName1, Raw: marshaledListener2}},
},
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusACKed,
},
wantUpdateErr: false,
},
// 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: nil,
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusACKed,
},
wantUpdateErr: false,
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
testWatchHandle(t, &watchHandleTestcase{
rType: xdsresource.ListenerResource,
resourceName: goodLDSTarget1,
responseToHandle: test.ldsResponse,
wantHandleErr: test.wantErr,
wantUpdate: test.wantUpdate,
wantUpdateMD: test.wantUpdateMD,
wantUpdateErr: test.wantUpdateErr,
})
})
}
}
// TestLDSHandleResponseWithoutWatch tests the case where the client receives
// an LDS response without a registered watcher.
func (s) TestLDSHandleResponseWithoutWatch(t *testing.T) {
fakeServer, cleanup := startServer(t)
defer cleanup()
v2c, err := newTestController(&testUpdateReceiver{
f: func(xdsresource.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) {},
}, fakeServer.Address, goodNodeProto, func(int) time.Duration { return 0 }, nil)
if err != nil {
t.Fatal(err)
}
defer v2c.Close()
if _, _, _, err := v2c.handleResponse(badResourceTypeInLDSResponse); err == nil {
t.Fatal("v2c.handleLDSResponse() succeeded, should have failed")
}
if _, _, _, err := v2c.handleResponse(goodLDSResponse1); err != nil {
t.Fatal("v2c.handleLDSResponse() succeeded, should have failed")
}
}

View File

@ -1,203 +0,0 @@
/*
*
* 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 controller
import (
"context"
"testing"
"time"
xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
"github.com/google/go-cmp/cmp/cmpopts"
"google.golang.org/grpc/xds/internal/testutils/fakeserver"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
// doLDS makes a LDS watch, and waits for the response and ack to finish.
//
// This is called by RDS tests to start LDS first, because LDS is a
// pre-requirement for RDS, and RDS handle would fail without an existing LDS
// watch.
func doLDS(ctx context.Context, t *testing.T, v2c *Controller, fakeServer *fakeserver.Server) {
v2c.AddWatch(xdsresource.ListenerResource, goodLDSTarget1)
if _, err := fakeServer.XDSRequestChan.Receive(ctx); err != nil {
t.Fatalf("Timeout waiting for LDS request: %v", err)
}
}
// TestRDSHandleResponseWithRouting 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) TestRDSHandleResponseWithRouting(t *testing.T) {
tests := []struct {
name string
rdsResponse *xdspb.DiscoveryResponse
wantErr bool
wantUpdate map[string]xdsresource.RouteConfigUpdateErrTuple
wantUpdateMD xdsresource.UpdateMetadata
wantUpdateErr bool
}{
// Badly marshaled RDS response.
{
name: "badly-marshaled-response",
rdsResponse: badlyMarshaledRDSResponse,
wantErr: true,
wantUpdate: nil,
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
ErrState: &xdsresource.UpdateErrorMetadata{
Err: cmpopts.AnyError,
},
},
wantUpdateErr: false,
},
// Response does not contain RouteConfiguration proto.
{
name: "no-route-config-in-response",
rdsResponse: badResourceTypeInRDSResponse,
wantErr: true,
wantUpdate: nil,
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
ErrState: &xdsresource.UpdateErrorMetadata{
Err: cmpopts.AnyError,
},
},
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: false,
wantUpdate: map[string]xdsresource.RouteConfigUpdateErrTuple{
goodRouteName1: {Update: xdsresource.RouteConfigUpdate{
VirtualHosts: nil,
Raw: marshaledNoVirtualHostsRouteConfig,
}},
},
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusACKed,
},
wantUpdateErr: false,
},
// Response contains one good RouteConfiguration, uninteresting though.
{
name: "one-uninteresting-route-config",
rdsResponse: goodRDSResponse2,
wantErr: false,
wantUpdate: map[string]xdsresource.RouteConfigUpdateErrTuple{
goodRouteName2: {Update: xdsresource.RouteConfigUpdate{
VirtualHosts: []*xdsresource.VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*xdsresource.Route{{Prefix: newStringP(""),
WeightedClusters: map[string]xdsresource.WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: xdsresource.RouteActionRoute}},
},
{
Domains: []string{goodLDSTarget1},
Routes: []*xdsresource.Route{{
Prefix: newStringP(""),
WeightedClusters: map[string]xdsresource.WeightedCluster{goodClusterName2: {Weight: 1}},
ActionType: xdsresource.RouteActionRoute}},
},
},
Raw: marshaledGoodRouteConfig2,
}},
},
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusACKed,
},
wantUpdateErr: false,
},
// Response contains one good interesting RouteConfiguration.
{
name: "one-good-route-config",
rdsResponse: goodRDSResponse1,
wantErr: false,
wantUpdate: map[string]xdsresource.RouteConfigUpdateErrTuple{
goodRouteName1: {Update: xdsresource.RouteConfigUpdate{
VirtualHosts: []*xdsresource.VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*xdsresource.Route{{
Prefix: newStringP(""),
WeightedClusters: map[string]xdsresource.WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: xdsresource.RouteActionRoute}},
},
{
Domains: []string{goodLDSTarget1},
Routes: []*xdsresource.Route{{Prefix: newStringP(""),
WeightedClusters: map[string]xdsresource.WeightedCluster{goodClusterName1: {Weight: 1}},
ActionType: xdsresource.RouteActionRoute}},
},
},
Raw: marshaledGoodRouteConfig1,
}},
},
wantUpdateMD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusACKed,
},
wantUpdateErr: false,
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
testWatchHandle(t, &watchHandleTestcase{
rType: xdsresource.RouteConfigResource,
resourceName: goodRouteName1,
responseToHandle: test.rdsResponse,
wantHandleErr: test.wantErr,
wantUpdate: test.wantUpdate,
wantUpdateMD: test.wantUpdateMD,
wantUpdateErr: test.wantUpdateErr,
})
})
}
}
// TestRDSHandleResponseWithoutRDSWatch tests the case where the v2Client
// receives an RDS response without a registered RDS watcher.
func (s) TestRDSHandleResponseWithoutRDSWatch(t *testing.T) {
fakeServer, cleanup := startServer(t)
defer cleanup()
v2c, err := newTestController(&testUpdateReceiver{
f: func(xdsresource.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) {},
}, fakeServer.Address, goodNodeProto, func(int) time.Duration { return 0 }, nil)
if err != nil {
t.Fatal(err)
}
defer v2c.Close()
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
doLDS(ctx, t, v2c, fakeServer)
if _, _, _, err := v2c.handleResponse(badResourceTypeInRDSResponse); err == nil {
t.Fatal("v2c.handleRDSResponse() succeeded, should have failed")
}
if _, _, _, err := v2c.handleResponse(goodRDSResponse1); err != nil {
t.Fatal("v2c.handleRDSResponse() succeeded, should have failed")
}
}

View File

@ -1,470 +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 controller
import (
"context"
"testing"
"time"
"github.com/golang/protobuf/proto"
"github.com/google/go-cmp/cmp"
"github.com/google/go-cmp/cmp/cmpopts"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
"google.golang.org/grpc/internal/grpclog"
"google.golang.org/grpc/internal/grpctest"
"google.golang.org/grpc/internal/testutils"
"google.golang.org/grpc/xds/internal/testutils/fakeserver"
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
"google.golang.org/grpc/xds/internal/xdsclient/pubsub"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
"google.golang.org/protobuf/testing/protocmp"
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"
)
type s struct {
grpctest.Tester
}
func Test(t *testing.T) {
grpctest.RunSubTests(t, s{})
}
const (
goodLDSTarget1 = "lds.target.good:1111"
goodLDSTarget2 = "lds.target.good:2222"
goodRouteName1 = "GoodRouteConfig1"
goodRouteName2 = "GoodRouteConfig2"
goodEDSName = "GoodClusterAssignment1"
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: version.V2ListenerURL,
ResourceNames: []string{goodLDSTarget1},
}
goodRDSRequest = &xdspb.DiscoveryRequest{
Node: goodNodeProto,
TypeUrl: version.V2RouteConfigURL,
ResourceNames: []string{goodRouteName1},
}
goodCDSRequest = &xdspb.DiscoveryRequest{
Node: goodNodeProto,
TypeUrl: version.V2ClusterURL,
ResourceNames: []string{goodClusterName1},
}
goodEDSRequest = &xdspb.DiscoveryRequest{
Node: goodNodeProto,
TypeUrl: version.V2EndpointsURL,
ResourceNames: []string{goodEDSName},
}
goodHTTPConnManager1 = &httppb.HttpConnectionManager{
RouteSpecifier: &httppb.HttpConnectionManager_Rds{
Rds: &httppb.Rds{
ConfigSource: &basepb.ConfigSource{
ConfigSourceSpecifier: &basepb.ConfigSource_Ads{Ads: &basepb.AggregatedConfigSource{}},
},
RouteConfigName: goodRouteName1,
},
},
}
marshaledConnMgr1 = testutils.MarshalAny(goodHTTPConnManager1)
goodListener1 = &xdspb.Listener{
Name: goodLDSTarget1,
ApiListener: &listenerpb.ApiListener{
ApiListener: marshaledConnMgr1,
},
}
marshaledListener1 = testutils.MarshalAny(goodListener1)
goodListener2 = &xdspb.Listener{
Name: goodLDSTarget2,
ApiListener: &listenerpb.ApiListener{
ApiListener: marshaledConnMgr1,
},
}
marshaledListener2 = testutils.MarshalAny(goodListener2)
noAPIListener = &xdspb.Listener{Name: goodLDSTarget1}
marshaledNoAPIListener = testutils.MarshalAny(noAPIListener)
badAPIListener2 = &xdspb.Listener{
Name: goodLDSTarget2,
ApiListener: &listenerpb.ApiListener{
ApiListener: &anypb.Any{
TypeUrl: httpConnManagerURL,
Value: []byte{1, 2, 3, 4},
},
},
}
badlyMarshaledAPIListener2, _ = proto.Marshal(badAPIListener2)
goodLDSResponse1 = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
marshaledListener1,
},
TypeUrl: version.V2ListenerURL,
}
goodLDSResponse2 = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
marshaledListener2,
},
TypeUrl: version.V2ListenerURL,
}
emptyLDSResponse = &xdspb.DiscoveryResponse{TypeUrl: version.V2ListenerURL}
badlyMarshaledLDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: version.V2ListenerURL,
Value: []byte{1, 2, 3, 4},
},
},
TypeUrl: version.V2ListenerURL,
}
badResourceTypeInLDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{marshaledConnMgr1},
TypeUrl: version.V2ListenerURL,
}
ldsResponseWithMultipleResources = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
marshaledListener2,
marshaledListener1,
},
TypeUrl: version.V2ListenerURL,
}
noAPIListenerLDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{marshaledNoAPIListener},
TypeUrl: version.V2ListenerURL,
}
goodBadUglyLDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
marshaledListener2,
marshaledListener1,
{
TypeUrl: version.V2ListenerURL,
Value: badlyMarshaledAPIListener2,
},
},
TypeUrl: version.V2ListenerURL,
}
badlyMarshaledRDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
{
TypeUrl: version.V2RouteConfigURL,
Value: []byte{1, 2, 3, 4},
},
},
TypeUrl: version.V2RouteConfigURL,
}
badResourceTypeInRDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{marshaledConnMgr1},
TypeUrl: version.V2RouteConfigURL,
}
noVirtualHostsRouteConfig = &xdspb.RouteConfiguration{
Name: goodRouteName1,
}
marshaledNoVirtualHostsRouteConfig = testutils.MarshalAny(noVirtualHostsRouteConfig)
noVirtualHostsInRDSResponse = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
marshaledNoVirtualHostsRouteConfig,
},
TypeUrl: version.V2RouteConfigURL,
}
goodRouteConfig1 = &xdspb.RouteConfiguration{
Name: goodRouteName1,
VirtualHosts: []*routepb.VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*routepb.Route{
{
Match: &routepb.RouteMatch{PathSpecifier: &routepb.RouteMatch_Prefix{Prefix: ""}},
Action: &routepb.Route_Route{
Route: &routepb.RouteAction{
ClusterSpecifier: &routepb.RouteAction_Cluster{Cluster: uninterestingClusterName},
},
},
},
},
},
{
Domains: []string{goodLDSTarget1},
Routes: []*routepb.Route{
{
Match: &routepb.RouteMatch{PathSpecifier: &routepb.RouteMatch_Prefix{Prefix: ""}},
Action: &routepb.Route_Route{
Route: &routepb.RouteAction{
ClusterSpecifier: &routepb.RouteAction_Cluster{Cluster: goodClusterName1},
},
},
},
},
},
},
}
marshaledGoodRouteConfig1 = testutils.MarshalAny(goodRouteConfig1)
goodRouteConfig2 = &xdspb.RouteConfiguration{
Name: goodRouteName2,
VirtualHosts: []*routepb.VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*routepb.Route{
{
Match: &routepb.RouteMatch{PathSpecifier: &routepb.RouteMatch_Prefix{Prefix: ""}},
Action: &routepb.Route_Route{
Route: &routepb.RouteAction{
ClusterSpecifier: &routepb.RouteAction_Cluster{Cluster: uninterestingClusterName},
},
},
},
},
},
{
Domains: []string{goodLDSTarget1},
Routes: []*routepb.Route{
{
Match: &routepb.RouteMatch{PathSpecifier: &routepb.RouteMatch_Prefix{Prefix: ""}},
Action: &routepb.Route_Route{
Route: &routepb.RouteAction{
ClusterSpecifier: &routepb.RouteAction_Cluster{Cluster: goodClusterName2},
},
},
},
},
},
},
}
marshaledGoodRouteConfig2 = testutils.MarshalAny(goodRouteConfig2)
goodRDSResponse1 = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
marshaledGoodRouteConfig1,
},
TypeUrl: version.V2RouteConfigURL,
}
goodRDSResponse2 = &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{
marshaledGoodRouteConfig2,
},
TypeUrl: version.V2RouteConfigURL,
}
)
type watchHandleTestcase struct {
rType xdsresource.ResourceType
resourceName string
responseToHandle *xdspb.DiscoveryResponse
wantHandleErr bool
wantUpdate interface{}
wantUpdateMD xdsresource.UpdateMetadata
wantUpdateErr bool
}
type testUpdateReceiver struct {
f func(rType xdsresource.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata)
}
func (t *testUpdateReceiver) NewListeners(d map[string]xdsresource.ListenerUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
dd := make(map[string]interface{})
for k, v := range d {
dd[k] = v
}
t.newUpdate(xdsresource.ListenerResource, dd, metadata)
}
func (t *testUpdateReceiver) NewRouteConfigs(d map[string]xdsresource.RouteConfigUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
dd := make(map[string]interface{})
for k, v := range d {
dd[k] = v
}
t.newUpdate(xdsresource.RouteConfigResource, dd, metadata)
}
func (t *testUpdateReceiver) NewClusters(d map[string]xdsresource.ClusterUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
dd := make(map[string]interface{})
for k, v := range d {
dd[k] = v
}
t.newUpdate(xdsresource.ClusterResource, dd, metadata)
}
func (t *testUpdateReceiver) NewEndpoints(d map[string]xdsresource.EndpointsUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
dd := make(map[string]interface{})
for k, v := range d {
dd[k] = v
}
t.newUpdate(xdsresource.EndpointsResource, dd, metadata)
}
func (t *testUpdateReceiver) NewConnectionError(error) {}
func (t *testUpdateReceiver) newUpdate(rType xdsresource.ResourceType, d map[string]interface{}, metadata xdsresource.UpdateMetadata) {
t.f(rType, d, metadata)
}
// testWatchHandle is called to test response handling for each xDS.
//
// It starts the xDS watch as configured in test, waits for the fake xds server
// to receive the request (so watch callback is installed), and calls
// 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) {
t.Helper()
fakeServer, cleanup := startServer(t)
defer cleanup()
type updateErr struct {
u interface{}
md xdsresource.UpdateMetadata
err error
}
gotUpdateCh := testutils.NewChannel()
v2c, err := newTestController(&testUpdateReceiver{
f: func(rType xdsresource.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata) {
if rType == test.rType {
switch test.rType {
case xdsresource.ListenerResource:
dd := make(map[string]xdsresource.ListenerUpdateErrTuple)
for n, u := range d {
dd[n] = u.(xdsresource.ListenerUpdateErrTuple)
}
gotUpdateCh.Send(updateErr{dd, md, nil})
case xdsresource.RouteConfigResource:
dd := make(map[string]xdsresource.RouteConfigUpdateErrTuple)
for n, u := range d {
dd[n] = u.(xdsresource.RouteConfigUpdateErrTuple)
}
gotUpdateCh.Send(updateErr{dd, md, nil})
case xdsresource.ClusterResource:
dd := make(map[string]xdsresource.ClusterUpdateErrTuple)
for n, u := range d {
dd[n] = u.(xdsresource.ClusterUpdateErrTuple)
}
gotUpdateCh.Send(updateErr{dd, md, nil})
case xdsresource.EndpointsResource:
dd := make(map[string]xdsresource.EndpointsUpdateErrTuple)
for n, u := range d {
dd[n] = u.(xdsresource.EndpointsUpdateErrTuple)
}
gotUpdateCh.Send(updateErr{dd, md, nil})
}
}
},
}, fakeServer.Address, goodNodeProto, func(int) time.Duration { return 0 }, nil)
if err != nil {
t.Fatal(err)
}
defer v2c.Close()
// Register the watcher, this will also trigger the v2Client to send the xDS
// request.
v2c.AddWatch(test.rType, test.resourceName)
// Wait till the request makes it to the fakeServer. This ensures that
// the watch request has been processed by the v2Client.
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
if _, err := fakeServer.XDSRequestChan.Receive(ctx); err != nil {
t.Fatalf("Timeout waiting for an xDS request: %v", err)
}
// Directly push the response through a call to handleXDSResp. This bypasses
// the fakeServer, so it's only testing the handle logic. Client response
// processing is covered elsewhere.
//
// Also note that this won't trigger ACK, so there's no need to clear the
// request channel afterwards.
if _, _, _, err := v2c.handleResponse(test.responseToHandle); (err != nil) != test.wantHandleErr {
t.Fatalf("v2c.handleRDSResponse() returned err: %v, wantErr: %v", err, test.wantHandleErr)
}
wantUpdate := test.wantUpdate
cmpOpts := cmp.Options{
cmpopts.EquateEmpty(), protocmp.Transform(),
cmpopts.IgnoreFields(xdsresource.UpdateMetadata{}, "Timestamp"),
cmpopts.IgnoreFields(xdsresource.UpdateErrorMetadata{}, "Timestamp"),
cmp.FilterValues(func(x, y error) bool { return true }, cmpopts.EquateErrors()),
}
uErr, err := gotUpdateCh.Receive(ctx)
if err == context.DeadlineExceeded {
t.Fatal("Timeout expecting xDS update")
}
gotUpdate := uErr.(updateErr).u
if diff := cmp.Diff(gotUpdate, wantUpdate, cmpOpts); diff != "" {
t.Fatalf("got update : %+v, want %+v, diff: %s", gotUpdate, wantUpdate, diff)
}
gotUpdateMD := uErr.(updateErr).md
if diff := cmp.Diff(gotUpdateMD, test.wantUpdateMD, cmpOpts); diff != "" {
t.Fatalf("got update : %+v, want %+v, diff: %s", gotUpdateMD, test.wantUpdateMD, diff)
}
gotUpdateErr := uErr.(updateErr).err
if (gotUpdateErr != nil) != test.wantUpdateErr {
t.Fatalf("got xDS update error {%v}, wantErr: %v", gotUpdateErr, test.wantUpdateErr)
}
}
// startServer starts a fake XDS server and also returns a ClientConn
// connected to it.
func startServer(t *testing.T) (*fakeserver.Server, func()) {
t.Helper()
fs, sCleanup, err := fakeserver.StartServer()
if err != nil {
t.Fatalf("Failed to start fake xDS server: %v", err)
}
return fs, sCleanup
}
func newTestController(p pubsub.UpdateHandler, controlPlanAddr string, n *basepb.Node, b func(int) time.Duration, l *grpclog.PrefixLogger) (*Controller, error) {
c, err := New(&bootstrap.ServerConfig{
ServerURI: controlPlanAddr,
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
TransportAPI: version.TransportV2,
NodeProto: n,
}, p, nil, l, b)
if err != nil {
return nil, err
}
return c, nil
}
func newStringP(s string) *string {
return &s
}

View File

@ -1,155 +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 v2 provides xDS v2 transport protocol specific functionality.
package v2
import (
"context"
"fmt"
"github.com/golang/protobuf/proto"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/internal/grpclog"
"google.golang.org/grpc/internal/pretty"
controllerversion "google.golang.org/grpc/xds/internal/xdsclient/controller/version"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
xdsresourceversion "google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
"google.golang.org/protobuf/types/known/anypb"
v2xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
v2corepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
v2adsgrpc "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v2"
statuspb "google.golang.org/genproto/googleapis/rpc/status"
)
func init() {
controllerversion.RegisterAPIClientBuilder(xdsresourceversion.TransportV2, newClient)
}
var (
resourceTypeToURL = map[xdsresource.ResourceType]string{
xdsresource.ListenerResource: xdsresourceversion.V2ListenerURL,
xdsresource.RouteConfigResource: xdsresourceversion.V2RouteConfigURL,
xdsresource.ClusterResource: xdsresourceversion.V2ClusterURL,
xdsresource.EndpointsResource: xdsresourceversion.V2EndpointsURL,
}
)
func newClient(opts controllerversion.BuildOptions) (controllerversion.VersionedClient, error) {
nodeProto, ok := opts.NodeProto.(*v2corepb.Node)
if !ok {
return nil, fmt.Errorf("xds: unsupported Node proto type: %T, want %T", opts.NodeProto, (*v2corepb.Node)(nil))
}
v2c := &client{nodeProto: nodeProto, logger: opts.Logger}
return v2c, nil
}
type adsStream v2adsgrpc.AggregatedDiscoveryService_StreamAggregatedResourcesClient
// client 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.
type client struct {
nodeProto *v2corepb.Node
logger *grpclog.PrefixLogger
}
func (v2c *client) NewStream(ctx context.Context, cc *grpc.ClientConn) (grpc.ClientStream, error) {
return v2adsgrpc.NewAggregatedDiscoveryServiceClient(cc).StreamAggregatedResources(ctx, grpc.WaitForReady(true))
}
// SendRequest sends out a DiscoveryRequest for the given resourceNames, of type
// rType, on the provided stream.
//
// version is the ack version to be sent with the request
// - If this is the new request (not an ack/nack), version will be empty.
// - If this is an ack, version will be the version from the response.
// - If this is a nack, version will be the previous acked version (from
// versionMap). If there was no ack before, it will be empty.
func (v2c *client) SendRequest(s grpc.ClientStream, resourceNames []string, rType xdsresource.ResourceType, version, nonce, errMsg string) error {
stream, ok := s.(adsStream)
if !ok {
return fmt.Errorf("xds: Attempt to send request on unsupported stream type: %T", s)
}
req := &v2xdspb.DiscoveryRequest{
Node: v2c.nodeProto,
TypeUrl: resourceTypeToURL[rType],
ResourceNames: resourceNames,
VersionInfo: version,
ResponseNonce: nonce,
}
if errMsg != "" {
req.ErrorDetail = &statuspb.Status{
Code: int32(codes.InvalidArgument), Message: errMsg,
}
}
if err := stream.Send(req); err != nil {
return fmt.Errorf("xds: stream.Send(%+v) failed: %v", req, err)
}
v2c.logger.Debugf("ADS request sent: %v", pretty.ToJSON(req))
return nil
}
// RecvResponse blocks on the receipt of one response message on the provided
// stream.
func (v2c *client) RecvResponse(s grpc.ClientStream) (proto.Message, error) {
stream, ok := s.(adsStream)
if !ok {
return nil, fmt.Errorf("xds: Attempt to receive response on unsupported stream type: %T", s)
}
resp, err := stream.Recv()
if err != nil {
return nil, fmt.Errorf("xds: stream.Recv() failed: %v", err)
}
v2c.logger.Infof("ADS response received, type: %v", resp.GetTypeUrl())
v2c.logger.Debugf("ADS response received: %v", pretty.ToJSON(resp))
return resp, nil
}
func (v2c *client) ParseResponse(r proto.Message) (xdsresource.ResourceType, []*anypb.Any, string, string, error) {
rType := xdsresource.UnknownResource
resp, ok := r.(*v2xdspb.DiscoveryResponse)
if !ok {
return rType, nil, "", "", fmt.Errorf("xds: unsupported message type: %T", resp)
}
// Note that the xDS transport protocol is versioned independently of
// the resource types, and it is supported to transfer older versions
// of resource types using new versions of the transport protocol, or
// vice-versa. Hence we need to handle v3 type_urls as well here.
var err error
url := resp.GetTypeUrl()
switch {
case xdsresource.IsListenerResource(url):
rType = xdsresource.ListenerResource
case xdsresource.IsRouteConfigResource(url):
rType = xdsresource.RouteConfigResource
case xdsresource.IsClusterResource(url):
rType = xdsresource.ClusterResource
case xdsresource.IsEndpointsResource(url):
rType = xdsresource.EndpointsResource
default:
return rType, nil, "", "", controllerversion.ErrResourceTypeUnsupported{
ErrStr: fmt.Sprintf("Resource type %v unknown in response from server", resp.GetTypeUrl()),
}
}
return rType, resp.GetResources(), resp.GetVersionInfo(), resp.GetNonce(), err
}

View File

@ -1,170 +0,0 @@
/*
*
* 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 v2
import (
"context"
"errors"
"fmt"
"io"
"time"
"github.com/golang/protobuf/proto"
"github.com/golang/protobuf/ptypes"
"google.golang.org/grpc/internal/pretty"
"google.golang.org/grpc/xds/internal/xdsclient/load"
v2corepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
v2endpointpb "github.com/envoyproxy/go-control-plane/envoy/api/v2/endpoint"
lrsgrpc "github.com/envoyproxy/go-control-plane/envoy/service/load_stats/v2"
lrspb "github.com/envoyproxy/go-control-plane/envoy/service/load_stats/v2"
"google.golang.org/grpc"
"google.golang.org/grpc/xds/internal"
)
const clientFeatureLRSSendAllClusters = "envoy.lrs.supports_send_all_clusters"
type lrsStream lrsgrpc.LoadReportingService_StreamLoadStatsClient
func (v2c *client) NewLoadStatsStream(ctx context.Context, cc *grpc.ClientConn) (grpc.ClientStream, error) {
c := lrsgrpc.NewLoadReportingServiceClient(cc)
return c.StreamLoadStats(ctx)
}
func (v2c *client) SendFirstLoadStatsRequest(s grpc.ClientStream) error {
stream, ok := s.(lrsStream)
if !ok {
return fmt.Errorf("lrs: Attempt to send request on unsupported stream type: %T", s)
}
node := proto.Clone(v2c.nodeProto).(*v2corepb.Node)
if node == nil {
node = &v2corepb.Node{}
}
node.ClientFeatures = append(node.ClientFeatures, clientFeatureLRSSendAllClusters)
req := &lrspb.LoadStatsRequest{Node: node}
v2c.logger.Infof("lrs: sending init LoadStatsRequest: %v", pretty.ToJSON(req))
err := stream.Send(req)
if err == io.EOF {
return getStreamError(stream)
}
return err
}
func (v2c *client) HandleLoadStatsResponse(s grpc.ClientStream) ([]string, time.Duration, error) {
stream, ok := s.(lrsStream)
if !ok {
return nil, 0, fmt.Errorf("lrs: Attempt to receive response on unsupported stream type: %T", s)
}
resp, err := stream.Recv()
if err != nil {
return nil, 0, fmt.Errorf("lrs: failed to receive first response: %v", err)
}
v2c.logger.Infof("lrs: received first LoadStatsResponse: %+v", pretty.ToJSON(resp))
interval, err := ptypes.Duration(resp.GetLoadReportingInterval())
if err != nil {
return nil, 0, fmt.Errorf("lrs: failed to convert report interval: %v", err)
}
if resp.ReportEndpointGranularity {
// TODO: fixme to support per endpoint loads.
return nil, 0, errors.New("lrs: endpoint loads requested, but not supported by current implementation")
}
clusters := resp.Clusters
if resp.SendAllClusters {
// Return nil to send stats for all clusters.
clusters = nil
}
return clusters, interval, nil
}
func (v2c *client) SendLoadStatsRequest(s grpc.ClientStream, loads []*load.Data) error {
stream, ok := s.(lrsStream)
if !ok {
return fmt.Errorf("lrs: Attempt to send request on unsupported stream type: %T", s)
}
clusterStats := make([]*v2endpointpb.ClusterStats, 0, len(loads))
for _, sd := range loads {
droppedReqs := make([]*v2endpointpb.ClusterStats_DroppedRequests, 0, len(sd.Drops))
for category, count := range sd.Drops {
droppedReqs = append(droppedReqs, &v2endpointpb.ClusterStats_DroppedRequests{
Category: category,
DroppedCount: count,
})
}
localityStats := make([]*v2endpointpb.UpstreamLocalityStats, 0, len(sd.LocalityStats))
for l, localityData := range sd.LocalityStats {
lid, err := internal.LocalityIDFromString(l)
if err != nil {
return err
}
loadMetricStats := make([]*v2endpointpb.EndpointLoadMetricStats, 0, len(localityData.LoadStats))
for name, loadData := range localityData.LoadStats {
loadMetricStats = append(loadMetricStats, &v2endpointpb.EndpointLoadMetricStats{
MetricName: name,
NumRequestsFinishedWithMetric: loadData.Count,
TotalMetricValue: loadData.Sum,
})
}
localityStats = append(localityStats, &v2endpointpb.UpstreamLocalityStats{
Locality: &v2corepb.Locality{
Region: lid.Region,
Zone: lid.Zone,
SubZone: lid.SubZone,
},
TotalSuccessfulRequests: localityData.RequestStats.Succeeded,
TotalRequestsInProgress: localityData.RequestStats.InProgress,
TotalErrorRequests: localityData.RequestStats.Errored,
LoadMetricStats: loadMetricStats,
UpstreamEndpointStats: nil, // TODO: populate for per endpoint loads.
})
}
clusterStats = append(clusterStats, &v2endpointpb.ClusterStats{
ClusterName: sd.Cluster,
ClusterServiceName: sd.Service,
UpstreamLocalityStats: localityStats,
TotalDroppedRequests: sd.TotalDrops,
DroppedRequests: droppedReqs,
LoadReportInterval: ptypes.DurationProto(sd.ReportInterval),
})
}
req := &lrspb.LoadStatsRequest{ClusterStats: clusterStats}
v2c.logger.Infof("lrs: sending LRS loads: %+v", pretty.ToJSON(req))
err := stream.Send(req)
if err == io.EOF {
return getStreamError(stream)
}
return err
}
func getStreamError(stream lrsStream) error {
for {
if _, err := stream.Recv(); err != nil {
return err
}
}
}

View File

@ -1,157 +0,0 @@
/*
*
* 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 v3 provides xDS v3 transport protocol specific functionality.
package v3
import (
"context"
"fmt"
"github.com/golang/protobuf/proto"
statuspb "google.golang.org/genproto/googleapis/rpc/status"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/internal/grpclog"
"google.golang.org/grpc/internal/pretty"
controllerversion "google.golang.org/grpc/xds/internal/xdsclient/controller/version"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
xdsresourceversion "google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
"google.golang.org/protobuf/types/known/anypb"
v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
v3adsgrpc "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
v3discoverypb "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
)
func init() {
controllerversion.RegisterAPIClientBuilder(xdsresourceversion.TransportV3, newClient)
}
var (
resourceTypeToURL = map[xdsresource.ResourceType]string{
xdsresource.ListenerResource: xdsresourceversion.V3ListenerURL,
xdsresource.RouteConfigResource: xdsresourceversion.V3RouteConfigURL,
xdsresource.ClusterResource: xdsresourceversion.V3ClusterURL,
xdsresource.EndpointsResource: xdsresourceversion.V3EndpointsURL,
}
)
func newClient(opts controllerversion.BuildOptions) (controllerversion.VersionedClient, error) {
nodeProto, ok := opts.NodeProto.(*v3corepb.Node)
if !ok {
return nil, fmt.Errorf("xds: unsupported Node proto type: %T, want %T", opts.NodeProto, v3corepb.Node{})
}
v3c := &client{
nodeProto: nodeProto, logger: opts.Logger,
}
return v3c, nil
}
type adsStream v3adsgrpc.AggregatedDiscoveryService_StreamAggregatedResourcesClient
// client performs the actual xDS RPCs using the xDS v3 API. It creates a
// single ADS stream on which the different types of xDS requests and responses
// are multiplexed.
type client struct {
nodeProto *v3corepb.Node
logger *grpclog.PrefixLogger
}
func (v3c *client) NewStream(ctx context.Context, cc *grpc.ClientConn) (grpc.ClientStream, error) {
return v3adsgrpc.NewAggregatedDiscoveryServiceClient(cc).StreamAggregatedResources(ctx, grpc.WaitForReady(true))
}
// SendRequest sends out a DiscoveryRequest for the given resourceNames, of type
// rType, on the provided stream.
//
// version is the ack version to be sent with the request
// - If this is the new request (not an ack/nack), version will be empty.
// - If this is an ack, version will be the version from the response.
// - If this is a nack, version will be the previous acked version (from
// versionMap). If there was no ack before, it will be empty.
func (v3c *client) SendRequest(s grpc.ClientStream, resourceNames []string, rType xdsresource.ResourceType, version, nonce, errMsg string) error {
stream, ok := s.(adsStream)
if !ok {
return fmt.Errorf("xds: Attempt to send request on unsupported stream type: %T", s)
}
req := &v3discoverypb.DiscoveryRequest{
Node: v3c.nodeProto,
TypeUrl: resourceTypeToURL[rType],
ResourceNames: resourceNames,
VersionInfo: version,
ResponseNonce: nonce,
}
if errMsg != "" {
req.ErrorDetail = &statuspb.Status{
Code: int32(codes.InvalidArgument), Message: errMsg,
}
}
if err := stream.Send(req); err != nil {
return fmt.Errorf("xds: stream.Send(%+v) failed: %v", req, err)
}
v3c.logger.Debugf("ADS request sent: %v", pretty.ToJSON(req))
return nil
}
// RecvResponse blocks on the receipt of one response message on the provided
// stream.
func (v3c *client) RecvResponse(s grpc.ClientStream) (proto.Message, error) {
stream, ok := s.(adsStream)
if !ok {
return nil, fmt.Errorf("xds: Attempt to receive response on unsupported stream type: %T", s)
}
resp, err := stream.Recv()
if err != nil {
return nil, fmt.Errorf("xds: stream.Recv() failed: %v", err)
}
v3c.logger.Infof("ADS response received, type: %v", resp.GetTypeUrl())
v3c.logger.Debugf("ADS response received: %+v", pretty.ToJSON(resp))
return resp, nil
}
func (v3c *client) ParseResponse(r proto.Message) (xdsresource.ResourceType, []*anypb.Any, string, string, error) {
rType := xdsresource.UnknownResource
resp, ok := r.(*v3discoverypb.DiscoveryResponse)
if !ok {
return rType, nil, "", "", fmt.Errorf("xds: unsupported message type: %T", resp)
}
// Note that the xDS transport protocol is versioned independently of
// the resource types, and it is supported to transfer older versions
// of resource types using new versions of the transport protocol, or
// vice-versa. Hence we need to handle v3 type_urls as well here.
var err error
url := resp.GetTypeUrl()
switch {
case xdsresource.IsListenerResource(url):
rType = xdsresource.ListenerResource
case xdsresource.IsRouteConfigResource(url):
rType = xdsresource.RouteConfigResource
case xdsresource.IsClusterResource(url):
rType = xdsresource.ClusterResource
case xdsresource.IsEndpointsResource(url):
rType = xdsresource.EndpointsResource
default:
return rType, nil, "", "", controllerversion.ErrResourceTypeUnsupported{
ErrStr: fmt.Sprintf("Resource type %v unknown in response from server", resp.GetTypeUrl()),
}
}
return rType, resp.GetResources(), resp.GetVersionInfo(), resp.GetNonce(), err
}

View File

@ -1,169 +0,0 @@
/*
*
* 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 v3
import (
"context"
"errors"
"fmt"
"io"
"time"
"github.com/golang/protobuf/proto"
"github.com/golang/protobuf/ptypes"
"google.golang.org/grpc/internal/pretty"
"google.golang.org/grpc/xds/internal/xdsclient/load"
v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
v3endpointpb "github.com/envoyproxy/go-control-plane/envoy/config/endpoint/v3"
lrsgrpc "github.com/envoyproxy/go-control-plane/envoy/service/load_stats/v3"
lrspb "github.com/envoyproxy/go-control-plane/envoy/service/load_stats/v3"
"google.golang.org/grpc"
"google.golang.org/grpc/xds/internal"
)
const clientFeatureLRSSendAllClusters = "envoy.lrs.supports_send_all_clusters"
type lrsStream lrsgrpc.LoadReportingService_StreamLoadStatsClient
func (v3c *client) NewLoadStatsStream(ctx context.Context, cc *grpc.ClientConn) (grpc.ClientStream, error) {
c := lrsgrpc.NewLoadReportingServiceClient(cc)
return c.StreamLoadStats(ctx)
}
func (v3c *client) SendFirstLoadStatsRequest(s grpc.ClientStream) error {
stream, ok := s.(lrsStream)
if !ok {
return fmt.Errorf("lrs: Attempt to send request on unsupported stream type: %T", s)
}
node := proto.Clone(v3c.nodeProto).(*v3corepb.Node)
if node == nil {
node = &v3corepb.Node{}
}
node.ClientFeatures = append(node.ClientFeatures, clientFeatureLRSSendAllClusters)
req := &lrspb.LoadStatsRequest{Node: node}
v3c.logger.Infof("lrs: sending init LoadStatsRequest: %v", pretty.ToJSON(req))
err := stream.Send(req)
if err == io.EOF {
return getStreamError(stream)
}
return err
}
func (v3c *client) HandleLoadStatsResponse(s grpc.ClientStream) ([]string, time.Duration, error) {
stream, ok := s.(lrsStream)
if !ok {
return nil, 0, fmt.Errorf("lrs: Attempt to receive response on unsupported stream type: %T", s)
}
resp, err := stream.Recv()
if err != nil {
return nil, 0, fmt.Errorf("lrs: failed to receive first response: %v", err)
}
v3c.logger.Infof("lrs: received first LoadStatsResponse: %+v", pretty.ToJSON(resp))
interval, err := ptypes.Duration(resp.GetLoadReportingInterval())
if err != nil {
return nil, 0, fmt.Errorf("lrs: failed to convert report interval: %v", err)
}
if resp.ReportEndpointGranularity {
// TODO: fixme to support per endpoint loads.
return nil, 0, errors.New("lrs: endpoint loads requested, but not supported by current implementation")
}
clusters := resp.Clusters
if resp.SendAllClusters {
// Return nil to send stats for all clusters.
clusters = nil
}
return clusters, interval, nil
}
func (v3c *client) SendLoadStatsRequest(s grpc.ClientStream, loads []*load.Data) error {
stream, ok := s.(lrsStream)
if !ok {
return fmt.Errorf("lrs: Attempt to send request on unsupported stream type: %T", s)
}
clusterStats := make([]*v3endpointpb.ClusterStats, 0, len(loads))
for _, sd := range loads {
droppedReqs := make([]*v3endpointpb.ClusterStats_DroppedRequests, 0, len(sd.Drops))
for category, count := range sd.Drops {
droppedReqs = append(droppedReqs, &v3endpointpb.ClusterStats_DroppedRequests{
Category: category,
DroppedCount: count,
})
}
localityStats := make([]*v3endpointpb.UpstreamLocalityStats, 0, len(sd.LocalityStats))
for l, localityData := range sd.LocalityStats {
lid, err := internal.LocalityIDFromString(l)
if err != nil {
return err
}
loadMetricStats := make([]*v3endpointpb.EndpointLoadMetricStats, 0, len(localityData.LoadStats))
for name, loadData := range localityData.LoadStats {
loadMetricStats = append(loadMetricStats, &v3endpointpb.EndpointLoadMetricStats{
MetricName: name,
NumRequestsFinishedWithMetric: loadData.Count,
TotalMetricValue: loadData.Sum,
})
}
localityStats = append(localityStats, &v3endpointpb.UpstreamLocalityStats{
Locality: &v3corepb.Locality{
Region: lid.Region,
Zone: lid.Zone,
SubZone: lid.SubZone,
},
TotalSuccessfulRequests: localityData.RequestStats.Succeeded,
TotalRequestsInProgress: localityData.RequestStats.InProgress,
TotalErrorRequests: localityData.RequestStats.Errored,
LoadMetricStats: loadMetricStats,
UpstreamEndpointStats: nil, // TODO: populate for per endpoint loads.
})
}
clusterStats = append(clusterStats, &v3endpointpb.ClusterStats{
ClusterName: sd.Cluster,
ClusterServiceName: sd.Service,
UpstreamLocalityStats: localityStats,
TotalDroppedRequests: sd.TotalDrops,
DroppedRequests: droppedReqs,
LoadReportInterval: ptypes.DurationProto(sd.ReportInterval),
})
}
req := &lrspb.LoadStatsRequest{ClusterStats: clusterStats}
v3c.logger.Infof("lrs: sending LRS loads: %+v", pretty.ToJSON(req))
err := stream.Send(req)
if err == io.EOF {
return getStreamError(stream)
}
return err
}
func getStreamError(stream lrsStream) error {
for {
if _, err := stream.Recv(); err != nil {
return err
}
}
}

View File

@ -1,123 +0,0 @@
/*
*
* Copyright 2021 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 version defines APIs to deal with different versions of xDS.
package version
import (
"context"
"time"
"github.com/golang/protobuf/proto"
"google.golang.org/grpc"
"google.golang.org/grpc/internal/grpclog"
"google.golang.org/grpc/xds/internal/xdsclient/load"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
"google.golang.org/protobuf/types/known/anypb"
)
var (
m = make(map[version.TransportAPI]func(opts BuildOptions) (VersionedClient, error))
)
// RegisterAPIClientBuilder registers a client builder for xDS transport protocol
// version specified by b.Version().
//
// NOTE: this function must only be called during initialization time (i.e. in
// an init() function), and is not thread-safe. If multiple builders are
// registered for the same version, the one registered last will take effect.
func RegisterAPIClientBuilder(v version.TransportAPI, f func(opts BuildOptions) (VersionedClient, error)) {
m[v] = f
}
// GetAPIClientBuilder returns the client builder registered for the provided
// xDS transport API version.
func GetAPIClientBuilder(version version.TransportAPI) func(opts BuildOptions) (VersionedClient, error) {
if f, ok := m[version]; ok {
return f
}
return nil
}
// BuildOptions contains options to be passed to client builders.
type BuildOptions struct {
// NodeProto contains the Node proto to be used in xDS requests. The actual
// type depends on the transport protocol version used.
NodeProto proto.Message
// // Backoff returns the amount of time to backoff before retrying broken
// // streams.
// Backoff func(int) time.Duration
// Logger provides enhanced logging capabilities.
Logger *grpclog.PrefixLogger
}
// LoadReportingOptions contains configuration knobs for reporting load data.
type LoadReportingOptions struct {
LoadStore *load.Store
}
// ErrResourceTypeUnsupported is an error used to indicate an unsupported xDS
// resource type. The wrapped ErrStr contains the details.
type ErrResourceTypeUnsupported struct {
ErrStr string
}
// Error helps implements the error interface.
func (e ErrResourceTypeUnsupported) Error() string {
return e.ErrStr
}
// VersionedClient is the interface to version specific operations of the
// client.
//
// It mainly deals with the type assertion from proto.Message to the real v2/v3
// types, and grpc.Stream to the versioned stream types.
type VersionedClient interface {
// NewStream returns a new xDS client stream specific to the underlying
// transport protocol version.
NewStream(ctx context.Context, cc *grpc.ClientConn) (grpc.ClientStream, error)
// SendRequest constructs and sends out a DiscoveryRequest message specific
// to the underlying transport protocol version.
SendRequest(s grpc.ClientStream, resourceNames []string, rType xdsresource.ResourceType, version, nonce, errMsg string) error
// RecvResponse uses the provided stream to receive a response specific to
// the underlying transport protocol version.
RecvResponse(s grpc.ClientStream) (proto.Message, error)
// ParseResponse type asserts message to the versioned response, and
// retrieves the fields.
ParseResponse(r proto.Message) (xdsresource.ResourceType, []*anypb.Any, string, string, error)
// The following are LRS methods.
// NewLoadStatsStream returns a new LRS client stream specific to the
// underlying transport protocol version.
NewLoadStatsStream(ctx context.Context, cc *grpc.ClientConn) (grpc.ClientStream, error)
// SendFirstLoadStatsRequest constructs and sends the first request on the
// LRS stream.
SendFirstLoadStatsRequest(s grpc.ClientStream) error
// HandleLoadStatsResponse receives the first response from the server which
// contains the load reporting interval and the clusters for which the
// server asks the client to report load for.
//
// If the response sets SendAllClusters to true, the returned clusters is
// nil.
HandleLoadStatsResponse(s grpc.ClientStream) (clusters []string, _ time.Duration, _ error)
// SendLoadStatsRequest will be invoked at regular intervals to send load
// report with load data reported since the last time this method was
// invoked.
SendLoadStatsRequest(s grpc.ClientStream, loads []*load.Data) error
}

View File

@ -1,511 +0,0 @@
/*
*
* Copyright 2021 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 xdsclient
import (
"fmt"
"testing"
"time"
v3clusterpb "github.com/envoyproxy/go-control-plane/envoy/config/cluster/v3"
v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
v3listenerpb "github.com/envoyproxy/go-control-plane/envoy/config/listener/v3"
v3routepb "github.com/envoyproxy/go-control-plane/envoy/config/route/v3"
v3httppb "github.com/envoyproxy/go-control-plane/envoy/extensions/filters/network/http_connection_manager/v3"
"github.com/google/go-cmp/cmp"
"github.com/google/go-cmp/cmp/cmpopts"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/protobuf/testing/protocmp"
"google.golang.org/protobuf/types/known/anypb"
"google.golang.org/protobuf/types/known/durationpb"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
"google.golang.org/grpc/internal/testutils"
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
)
func (s) TestLDSConfigDump(t *testing.T) {
const testVersion = "test-version-lds"
var (
ldsTargets = []string{"lds.target.good:0000", "lds.target.good:1111"}
routeConfigNames = []string{"route-config-0", "route-config-1"}
listenerRaws = make(map[string]*anypb.Any, len(ldsTargets))
)
for i := range ldsTargets {
listenersT := &v3listenerpb.Listener{
Name: ldsTargets[i],
ApiListener: &v3listenerpb.ApiListener{
ApiListener: testutils.MarshalAny(&v3httppb.HttpConnectionManager{
RouteSpecifier: &v3httppb.HttpConnectionManager_Rds{
Rds: &v3httppb.Rds{
ConfigSource: &v3corepb.ConfigSource{
ConfigSourceSpecifier: &v3corepb.ConfigSource_Ads{Ads: &v3corepb.AggregatedConfigSource{}},
},
RouteConfigName: routeConfigNames[i],
},
},
CommonHttpProtocolOptions: &v3corepb.HttpProtocolOptions{
MaxStreamDuration: durationpb.New(time.Second),
},
}),
},
}
listenerRaws[ldsTargets[i]] = testutils.MarshalAny(listenersT)
}
client, err := NewWithConfigForTesting(&bootstrap.Config{
XDSServer: &bootstrap.ServerConfig{
ServerURI: testXDSServer,
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
NodeProto: xdstestutils.EmptyNodeProtoV2,
},
}, defaultTestWatchExpiryTimeout, time.Duration(0))
if err != nil {
t.Fatalf("failed to create client: %v", err)
}
defer client.Close()
// Expected unknown.
if err := compareDump(client.DumpLDS, map[string]xdsresource.UpdateWithMD{}); err != nil {
t.Fatalf(err.Error())
}
wantRequested := make(map[string]xdsresource.UpdateWithMD)
for _, n := range ldsTargets {
cancel := client.WatchListener(n, func(update xdsresource.ListenerUpdate, err error) {})
defer cancel()
wantRequested[n] = xdsresource.UpdateWithMD{MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}
}
// Expected requested.
if err := compareDump(client.DumpLDS, wantRequested); err != nil {
t.Fatalf(err.Error())
}
update0 := make(map[string]xdsresource.ListenerUpdateErrTuple)
want0 := make(map[string]xdsresource.UpdateWithMD)
for n, r := range listenerRaws {
update0[n] = xdsresource.ListenerUpdateErrTuple{Update: xdsresource.ListenerUpdate{Raw: r}}
want0[n] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: testVersion},
Raw: r,
}
}
updateHandler := findPubsubForTest(t, client.(*clientRefCounted).clientImpl, "")
updateHandler.NewListeners(update0, xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: testVersion})
// Expect ACK.
if err := compareDump(client.DumpLDS, want0); err != nil {
t.Fatalf(err.Error())
}
const nackVersion = "lds-version-nack"
var nackErr = fmt.Errorf("lds nack error")
updateHandler.NewListeners(
map[string]xdsresource.ListenerUpdateErrTuple{
ldsTargets[0]: {Err: nackErr},
ldsTargets[1]: {Update: xdsresource.ListenerUpdate{Raw: listenerRaws[ldsTargets[1]]}},
},
xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
ErrState: &xdsresource.UpdateErrorMetadata{
Version: nackVersion,
Err: nackErr,
},
},
)
// Expect NACK for [0], but old ACK for [1].
wantDump := make(map[string]xdsresource.UpdateWithMD)
// Though resource 0 was NACKed, the dump should show the previous ACKed raw
// message, as well as the NACK error.
wantDump[ldsTargets[0]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
Version: testVersion,
ErrState: &xdsresource.UpdateErrorMetadata{
Version: nackVersion,
Err: nackErr,
},
},
Raw: listenerRaws[ldsTargets[0]],
}
wantDump[ldsTargets[1]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: nackVersion},
Raw: listenerRaws[ldsTargets[1]],
}
if err := compareDump(client.DumpLDS, wantDump); err != nil {
t.Fatalf(err.Error())
}
}
func (s) TestRDSConfigDump(t *testing.T) {
const testVersion = "test-version-rds"
var (
listenerNames = []string{"lds.target.good:0000", "lds.target.good:1111"}
rdsTargets = []string{"route-config-0", "route-config-1"}
clusterNames = []string{"cluster-0", "cluster-1"}
routeRaws = make(map[string]*anypb.Any, len(rdsTargets))
)
for i := range rdsTargets {
routeConfigT := &v3routepb.RouteConfiguration{
Name: rdsTargets[i],
VirtualHosts: []*v3routepb.VirtualHost{
{
Domains: []string{listenerNames[i]},
Routes: []*v3routepb.Route{{
Match: &v3routepb.RouteMatch{PathSpecifier: &v3routepb.RouteMatch_Prefix{Prefix: ""}},
Action: &v3routepb.Route_Route{
Route: &v3routepb.RouteAction{
ClusterSpecifier: &v3routepb.RouteAction_Cluster{Cluster: clusterNames[i]},
},
},
}},
},
},
}
routeRaws[rdsTargets[i]] = testutils.MarshalAny(routeConfigT)
}
client, err := NewWithConfigForTesting(&bootstrap.Config{
XDSServer: &bootstrap.ServerConfig{
ServerURI: testXDSServer,
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
NodeProto: xdstestutils.EmptyNodeProtoV2,
},
}, defaultTestWatchExpiryTimeout, time.Duration(0))
if err != nil {
t.Fatalf("failed to create client: %v", err)
}
defer client.Close()
// Expected unknown.
if err := compareDump(client.DumpRDS, map[string]xdsresource.UpdateWithMD{}); err != nil {
t.Fatalf(err.Error())
}
wantRequested := make(map[string]xdsresource.UpdateWithMD)
for _, n := range rdsTargets {
cancel := client.WatchRouteConfig(n, func(update xdsresource.RouteConfigUpdate, err error) {})
defer cancel()
wantRequested[n] = xdsresource.UpdateWithMD{MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}
}
// Expected requested.
if err := compareDump(client.DumpRDS, wantRequested); err != nil {
t.Fatalf(err.Error())
}
update0 := make(map[string]xdsresource.RouteConfigUpdateErrTuple)
want0 := make(map[string]xdsresource.UpdateWithMD)
for n, r := range routeRaws {
update0[n] = xdsresource.RouteConfigUpdateErrTuple{Update: xdsresource.RouteConfigUpdate{Raw: r}}
want0[n] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: testVersion},
Raw: r,
}
}
updateHandler := findPubsubForTest(t, client.(*clientRefCounted).clientImpl, "")
updateHandler.NewRouteConfigs(update0, xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: testVersion})
// Expect ACK.
if err := compareDump(client.DumpRDS, want0); err != nil {
t.Fatalf(err.Error())
}
const nackVersion = "rds-version-nack"
var nackErr = fmt.Errorf("rds nack error")
updateHandler.NewRouteConfigs(
map[string]xdsresource.RouteConfigUpdateErrTuple{
rdsTargets[0]: {Err: nackErr},
rdsTargets[1]: {Update: xdsresource.RouteConfigUpdate{Raw: routeRaws[rdsTargets[1]]}},
},
xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
ErrState: &xdsresource.UpdateErrorMetadata{
Version: nackVersion,
Err: nackErr,
},
},
)
// Expect NACK for [0], but old ACK for [1].
wantDump := make(map[string]xdsresource.UpdateWithMD)
// Though resource 0 was NACKed, the dump should show the previous ACKed raw
// message, as well as the NACK error.
wantDump[rdsTargets[0]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
Version: testVersion,
ErrState: &xdsresource.UpdateErrorMetadata{
Version: nackVersion,
Err: nackErr,
},
},
Raw: routeRaws[rdsTargets[0]],
}
wantDump[rdsTargets[1]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: nackVersion},
Raw: routeRaws[rdsTargets[1]],
}
if err := compareDump(client.DumpRDS, wantDump); err != nil {
t.Fatalf(err.Error())
}
}
func (s) TestCDSConfigDump(t *testing.T) {
const testVersion = "test-version-cds"
var (
cdsTargets = []string{"cluster-0", "cluster-1"}
serviceNames = []string{"service-0", "service-1"}
clusterRaws = make(map[string]*anypb.Any, len(cdsTargets))
)
for i := range cdsTargets {
clusterT := &v3clusterpb.Cluster{
Name: cdsTargets[i],
ClusterDiscoveryType: &v3clusterpb.Cluster_Type{Type: v3clusterpb.Cluster_EDS},
EdsClusterConfig: &v3clusterpb.Cluster_EdsClusterConfig{
EdsConfig: &v3corepb.ConfigSource{
ConfigSourceSpecifier: &v3corepb.ConfigSource_Ads{
Ads: &v3corepb.AggregatedConfigSource{},
},
},
ServiceName: serviceNames[i],
},
LbPolicy: v3clusterpb.Cluster_ROUND_ROBIN,
LrsServer: &v3corepb.ConfigSource{
ConfigSourceSpecifier: &v3corepb.ConfigSource_Self{
Self: &v3corepb.SelfConfigSource{},
},
},
}
clusterRaws[cdsTargets[i]] = testutils.MarshalAny(clusterT)
}
client, err := NewWithConfigForTesting(&bootstrap.Config{
XDSServer: &bootstrap.ServerConfig{
ServerURI: testXDSServer,
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
NodeProto: xdstestutils.EmptyNodeProtoV2,
},
}, defaultTestWatchExpiryTimeout, time.Duration(0))
if err != nil {
t.Fatalf("failed to create client: %v", err)
}
defer client.Close()
// Expected unknown.
if err := compareDump(client.DumpCDS, map[string]xdsresource.UpdateWithMD{}); err != nil {
t.Fatalf(err.Error())
}
wantRequested := make(map[string]xdsresource.UpdateWithMD)
for _, n := range cdsTargets {
cancel := client.WatchCluster(n, func(update xdsresource.ClusterUpdate, err error) {})
defer cancel()
wantRequested[n] = xdsresource.UpdateWithMD{MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}
}
// Expected requested.
if err := compareDump(client.DumpCDS, wantRequested); err != nil {
t.Fatalf(err.Error())
}
update0 := make(map[string]xdsresource.ClusterUpdateErrTuple)
want0 := make(map[string]xdsresource.UpdateWithMD)
for n, r := range clusterRaws {
update0[n] = xdsresource.ClusterUpdateErrTuple{Update: xdsresource.ClusterUpdate{Raw: r}}
want0[n] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: testVersion},
Raw: r,
}
}
updateHandler := findPubsubForTest(t, client.(*clientRefCounted).clientImpl, "")
updateHandler.NewClusters(update0, xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: testVersion})
// Expect ACK.
if err := compareDump(client.DumpCDS, want0); err != nil {
t.Fatalf(err.Error())
}
const nackVersion = "cds-version-nack"
var nackErr = fmt.Errorf("cds nack error")
updateHandler.NewClusters(
map[string]xdsresource.ClusterUpdateErrTuple{
cdsTargets[0]: {Err: nackErr},
cdsTargets[1]: {Update: xdsresource.ClusterUpdate{Raw: clusterRaws[cdsTargets[1]]}},
},
xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
ErrState: &xdsresource.UpdateErrorMetadata{
Version: nackVersion,
Err: nackErr,
},
},
)
// Expect NACK for [0], but old ACK for [1].
wantDump := make(map[string]xdsresource.UpdateWithMD)
// Though resource 0 was NACKed, the dump should show the previous ACKed raw
// message, as well as the NACK error.
wantDump[cdsTargets[0]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
Version: testVersion,
ErrState: &xdsresource.UpdateErrorMetadata{
Version: nackVersion,
Err: nackErr,
},
},
Raw: clusterRaws[cdsTargets[0]],
}
wantDump[cdsTargets[1]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: nackVersion},
Raw: clusterRaws[cdsTargets[1]],
}
if err := compareDump(client.DumpCDS, wantDump); err != nil {
t.Fatalf(err.Error())
}
}
func (s) TestEDSConfigDump(t *testing.T) {
const testVersion = "test-version-cds"
var (
edsTargets = []string{"cluster-0", "cluster-1"}
localityNames = []string{"locality-0", "locality-1"}
addrs = []string{"addr0:123", "addr1:456"}
endpointRaws = make(map[string]*anypb.Any, len(edsTargets))
)
for i := range edsTargets {
clab0 := xdstestutils.NewClusterLoadAssignmentBuilder(edsTargets[i], nil)
clab0.AddLocality(localityNames[i], 1, 1, []string{addrs[i]}, nil)
claT := clab0.Build()
endpointRaws[edsTargets[i]] = testutils.MarshalAny(claT)
}
client, err := NewWithConfigForTesting(&bootstrap.Config{
XDSServer: &bootstrap.ServerConfig{
ServerURI: testXDSServer,
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
NodeProto: xdstestutils.EmptyNodeProtoV2,
},
}, defaultTestWatchExpiryTimeout, time.Duration(0))
if err != nil {
t.Fatalf("failed to create client: %v", err)
}
defer client.Close()
// Expected unknown.
if err := compareDump(client.DumpEDS, map[string]xdsresource.UpdateWithMD{}); err != nil {
t.Fatalf(err.Error())
}
wantRequested := make(map[string]xdsresource.UpdateWithMD)
for _, n := range edsTargets {
cancel := client.WatchEndpoints(n, func(update xdsresource.EndpointsUpdate, err error) {})
defer cancel()
wantRequested[n] = xdsresource.UpdateWithMD{MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}
}
// Expected requested.
if err := compareDump(client.DumpEDS, wantRequested); err != nil {
t.Fatalf(err.Error())
}
update0 := make(map[string]xdsresource.EndpointsUpdateErrTuple)
want0 := make(map[string]xdsresource.UpdateWithMD)
for n, r := range endpointRaws {
update0[n] = xdsresource.EndpointsUpdateErrTuple{Update: xdsresource.EndpointsUpdate{Raw: r}}
want0[n] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: testVersion},
Raw: r,
}
}
updateHandler := findPubsubForTest(t, client.(*clientRefCounted).clientImpl, "")
updateHandler.NewEndpoints(update0, xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: testVersion})
// Expect ACK.
if err := compareDump(client.DumpEDS, want0); err != nil {
t.Fatalf(err.Error())
}
const nackVersion = "eds-version-nack"
var nackErr = fmt.Errorf("eds nack error")
updateHandler.NewEndpoints(
map[string]xdsresource.EndpointsUpdateErrTuple{
edsTargets[0]: {Err: nackErr},
edsTargets[1]: {Update: xdsresource.EndpointsUpdate{Raw: endpointRaws[edsTargets[1]]}},
},
xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
ErrState: &xdsresource.UpdateErrorMetadata{
Version: nackVersion,
Err: nackErr,
},
},
)
// Expect NACK for [0], but old ACK for [1].
wantDump := make(map[string]xdsresource.UpdateWithMD)
// Though resource 0 was NACKed, the dump should show the previous ACKed raw
// message, as well as the NACK error.
wantDump[edsTargets[0]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
Version: testVersion,
ErrState: &xdsresource.UpdateErrorMetadata{
Version: nackVersion,
Err: nackErr,
},
},
Raw: endpointRaws[edsTargets[0]],
}
wantDump[edsTargets[1]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: nackVersion},
Raw: endpointRaws[edsTargets[1]],
}
if err := compareDump(client.DumpEDS, wantDump); err != nil {
t.Fatalf(err.Error())
}
}
func compareDump(dumpFunc func() map[string]xdsresource.UpdateWithMD, wantDump interface{}) error {
dump := dumpFunc()
cmpOpts := cmp.Options{
cmpopts.EquateEmpty(),
cmp.Comparer(func(a, b time.Time) bool { return true }),
cmp.Comparer(func(x, y error) bool {
if x == nil || y == nil {
return x == nil && y == nil
}
return x.Error() == y.Error()
}),
protocmp.Transform(),
}
if diff := cmp.Diff(dump, wantDump, cmpOpts); diff != "" {
return fmt.Errorf("Dump() returned unexpected dump, diff (-got +want): %s", diff)
}
return nil
}

View File

@ -0,0 +1,255 @@
/*
*
* Copyright 2022 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 e2e_test
import (
"context"
"fmt"
"testing"
"time"
"github.com/google/go-cmp/cmp"
"github.com/google/go-cmp/cmp/cmpopts"
"google.golang.org/grpc/internal/testutils"
"google.golang.org/grpc/internal/testutils/xds/e2e"
"google.golang.org/grpc/xds/internal/xdsclient"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/protobuf/testing/protocmp"
"google.golang.org/protobuf/types/known/anypb"
v3clusterpb "github.com/envoyproxy/go-control-plane/envoy/config/cluster/v3"
v3endpointpb "github.com/envoyproxy/go-control-plane/envoy/config/endpoint/v3"
v3listenerpb "github.com/envoyproxy/go-control-plane/envoy/config/listener/v3"
v3routepb "github.com/envoyproxy/go-control-plane/envoy/config/route/v3"
)
func compareDump(ctx context.Context, client xdsclient.XDSClient, want map[string]map[string]xdsresource.UpdateWithMD) error {
var lastErr error
for {
if err := ctx.Err(); err != nil {
return fmt.Errorf("Timeout when waiting for expected dump: %v", lastErr)
}
cmpOpts := cmp.Options{
cmpopts.EquateEmpty(),
cmp.Comparer(func(a, b time.Time) bool { return true }),
cmpopts.EquateErrors(),
protocmp.Transform(),
}
diff := cmp.Diff(want, client.DumpResources(), cmpOpts)
if diff == "" {
return nil
}
lastErr = fmt.Errorf("DumpResources() returned unexpected dump, diff (-want +got):\n%s", diff)
time.Sleep(100 * time.Millisecond)
}
}
func (s) TestDumpResources(t *testing.T) {
// Initialize the xDS resources to be used in this test.
ldsTargets := []string{"lds.target.good:0000", "lds.target.good:1111"}
rdsTargets := []string{"route-config-0", "route-config-1"}
cdsTargets := []string{"cluster-0", "cluster-1"}
edsTargets := []string{"endpoints-0", "endpoints-1"}
listeners := make([]*v3listenerpb.Listener, len(ldsTargets))
listenerAnys := make([]*anypb.Any, len(ldsTargets))
for i := range ldsTargets {
listeners[i] = e2e.DefaultClientListener(ldsTargets[i], rdsTargets[i])
listenerAnys[i] = testutils.MarshalAny(listeners[i])
}
routes := make([]*v3routepb.RouteConfiguration, len(rdsTargets))
routeAnys := make([]*anypb.Any, len(rdsTargets))
for i := range rdsTargets {
routes[i] = e2e.DefaultRouteConfig(rdsTargets[i], ldsTargets[i], cdsTargets[i])
routeAnys[i] = testutils.MarshalAny(routes[i])
}
clusters := make([]*v3clusterpb.Cluster, len(cdsTargets))
clusterAnys := make([]*anypb.Any, len(cdsTargets))
for i := range cdsTargets {
clusters[i] = e2e.DefaultCluster(cdsTargets[i], edsTargets[i], e2e.SecurityLevelNone)
clusterAnys[i] = testutils.MarshalAny(clusters[i])
}
endpoints := make([]*v3endpointpb.ClusterLoadAssignment, len(edsTargets))
endpointAnys := make([]*anypb.Any, len(edsTargets))
ips := []string{"0.0.0.0", "1.1.1.1"}
ports := []uint32{123, 456}
for i := range edsTargets {
endpoints[i] = e2e.DefaultEndpoint(edsTargets[i], ips[i], ports[i:i+1])
endpointAnys[i] = testutils.MarshalAny(endpoints[i])
}
// Spin up an xDS management server on a local port.
mgmtServer, nodeID, bootstrapContents, _, cleanup := e2e.SetupManagementServer(t, e2e.ManagementServerOptions{})
defer cleanup()
// Create an xDS client with the above bootstrap contents.
client, err := xdsclient.NewWithBootstrapContentsForTesting(bootstrapContents)
if err != nil {
t.Fatalf("Failed to create xDS client: %v", err)
}
defer client.Close()
// Dump resources and expect empty configs.
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
if err := compareDump(ctx, client, nil); err != nil {
t.Fatal(err)
}
// Register watches, dump resources and expect configs in requested state.
for _, target := range ldsTargets {
client.WatchListener(target, func(xdsresource.ListenerUpdate, error) {})
}
for _, target := range rdsTargets {
client.WatchRouteConfig(target, func(xdsresource.RouteConfigUpdate, error) {})
}
for _, target := range cdsTargets {
client.WatchCluster(target, func(xdsresource.ClusterUpdate, error) {})
}
for _, target := range edsTargets {
client.WatchEndpoints(target, func(xdsresource.EndpointsUpdate, error) {})
}
want := map[string]map[string]xdsresource.UpdateWithMD{
"type.googleapis.com/envoy.config.listener.v3.Listener": {
ldsTargets[0]: {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}},
ldsTargets[1]: {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}},
},
"type.googleapis.com/envoy.config.route.v3.RouteConfiguration": {
rdsTargets[0]: {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}},
rdsTargets[1]: {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}},
},
"type.googleapis.com/envoy.config.cluster.v3.Cluster": {
cdsTargets[0]: {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}},
cdsTargets[1]: {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}},
},
"type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment": {
edsTargets[0]: {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}},
edsTargets[1]: {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}},
},
}
if err := compareDump(ctx, client, want); err != nil {
t.Fatal(err)
}
// Configure the resources on the management server.
if err := mgmtServer.Update(ctx, e2e.UpdateOptions{
NodeID: nodeID,
Listeners: listeners,
Routes: routes,
Clusters: clusters,
Endpoints: endpoints,
}); err != nil {
t.Fatal(err)
}
// Dump resources and expect ACK configs.
want = map[string]map[string]xdsresource.UpdateWithMD{
"type.googleapis.com/envoy.config.listener.v3.Listener": {
ldsTargets[0]: {Raw: listenerAnys[0], MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}},
ldsTargets[1]: {Raw: listenerAnys[1], MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}},
},
"type.googleapis.com/envoy.config.route.v3.RouteConfiguration": {
rdsTargets[0]: {Raw: routeAnys[0], MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}},
rdsTargets[1]: {Raw: routeAnys[1], MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}},
},
"type.googleapis.com/envoy.config.cluster.v3.Cluster": {
cdsTargets[0]: {Raw: clusterAnys[0], MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}},
cdsTargets[1]: {Raw: clusterAnys[1], MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}},
},
"type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment": {
edsTargets[0]: {Raw: endpointAnys[0], MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}},
edsTargets[1]: {Raw: endpointAnys[1], MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}},
},
}
if err := compareDump(ctx, client, want); err != nil {
t.Fatal(err)
}
// Update the first resource of each type in the management server to a
// value which is expected to be NACK'ed by the xDS client.
const nackResourceIdx = 0
listeners[nackResourceIdx].ApiListener = &v3listenerpb.ApiListener{}
routes[nackResourceIdx].VirtualHosts = []*v3routepb.VirtualHost{{Routes: []*v3routepb.Route{{}}}}
clusters[nackResourceIdx].ClusterDiscoveryType = &v3clusterpb.Cluster_Type{Type: v3clusterpb.Cluster_STATIC}
endpoints[nackResourceIdx].Endpoints = []*v3endpointpb.LocalityLbEndpoints{{}}
if err := mgmtServer.Update(ctx, e2e.UpdateOptions{
NodeID: nodeID,
Listeners: listeners,
Routes: routes,
Clusters: clusters,
Endpoints: endpoints,
SkipValidation: true,
}); err != nil {
t.Fatal(err)
}
// Verify that the xDS client reports the first resource of each type as
// being in "NACKed" state, and the second resource of each type to be in
// "ACKed" state. The version for the ACKed resource would be "2", while
// that for the NACKed resource would be "1". In the NACKed resource, the
// version which is NACKed is stored in the ErrorState field.
want = map[string]map[string]xdsresource.UpdateWithMD{
"type.googleapis.com/envoy.config.listener.v3.Listener": {
ldsTargets[0]: {
Raw: listenerAnys[0],
MD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
Version: "1",
ErrState: &xdsresource.UpdateErrorMetadata{Version: "2", Err: cmpopts.AnyError},
},
},
ldsTargets[1]: {Raw: listenerAnys[1], MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "2"}},
},
"type.googleapis.com/envoy.config.route.v3.RouteConfiguration": {
rdsTargets[0]: {
Raw: routeAnys[0],
MD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
Version: "1",
ErrState: &xdsresource.UpdateErrorMetadata{Version: "2", Err: cmpopts.AnyError},
},
},
rdsTargets[1]: {Raw: routeAnys[1], MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "2"}},
},
"type.googleapis.com/envoy.config.cluster.v3.Cluster": {
cdsTargets[0]: {
Raw: clusterAnys[0],
MD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
Version: "1",
ErrState: &xdsresource.UpdateErrorMetadata{Version: "2", Err: cmpopts.AnyError},
},
},
cdsTargets[1]: {Raw: clusterAnys[1], MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "2"}},
},
"type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment": {
edsTargets[0]: {
Raw: endpointAnys[0],
MD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
Version: "1",
ErrState: &xdsresource.UpdateErrorMetadata{Version: "2", Err: cmpopts.AnyError},
},
},
edsTargets[1]: {Raw: endpointAnys[1], MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "2"}},
},
}
if err := compareDump(ctx, client, want); err != nil {
t.Fatal(err)
}
}

View File

@ -70,18 +70,17 @@ func (s) TestWatchCallAnotherWatch(t *testing.T) {
updateCh1 := testutils.NewChannel()
updateCh2 := testutils.NewChannel()
updateCh3 := testutils.NewChannel()
var rdsCancel2, rdsCancel3 func()
rdsCancel1 := client.WatchRouteConfig(rdsName, func(u xdsresource.RouteConfigUpdate, err error) {
updateCh1.Send(xdsresource.RouteConfigUpdateErrTuple{Update: u, Err: err})
// Watch for the same resource name.
rdsCancel2 = client.WatchRouteConfig(rdsName, func(u xdsresource.RouteConfigUpdate, err error) {
rdsCancel2 := client.WatchRouteConfig(rdsName, func(u xdsresource.RouteConfigUpdate, err error) {
updateCh2.Send(xdsresource.RouteConfigUpdateErrTuple{Update: u, Err: err})
})
t.Cleanup(rdsCancel2)
// Watch for a different resource name.
rdsCancel3 = client.WatchRouteConfig(rdsNameNewStyle, func(u xdsresource.RouteConfigUpdate, err error) {
rdsCancel3 := client.WatchRouteConfig(rdsNameNewStyle, func(u xdsresource.RouteConfigUpdate, err error) {
updateCh3.Send(xdsresource.RouteConfigUpdateErrTuple{Update: u, Err: err})
rdsCancel3()
})
t.Cleanup(rdsCancel3)
})

File diff suppressed because it is too large Load Diff

View File

@ -23,10 +23,6 @@ import (
"testing"
"time"
v2corepb "github.com/envoyproxy/go-control-plane/envoy/api/v2/core"
endpointpb "github.com/envoyproxy/go-control-plane/envoy/api/v2/endpoint"
lrspb "github.com/envoyproxy/go-control-plane/envoy/service/load_stats/v2"
durationpb "github.com/golang/protobuf/ptypes/duration"
"github.com/google/go-cmp/cmp"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
@ -37,7 +33,10 @@ import (
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
"google.golang.org/protobuf/testing/protocmp"
_ "google.golang.org/grpc/xds/internal/xdsclient/controller/version/v2" // Register the v2 xDS API client.
v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
v3endpointpb "github.com/envoyproxy/go-control-plane/envoy/config/endpoint/v3"
v3lrspb "github.com/envoyproxy/go-control-plane/envoy/service/load_stats/v3"
durationpb "github.com/golang/protobuf/ptypes/duration"
)
const (
@ -55,8 +54,8 @@ func (s) TestLRSClient(t *testing.T) {
XDSServer: &bootstrap.ServerConfig{
ServerURI: fs.Address,
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
TransportAPI: version.TransportV2,
NodeProto: &v2corepb.Node{},
TransportAPI: version.TransportV3,
NodeProto: &v3corepb.Node{},
},
}, defaultClientWatchExpiryTimeout, time.Duration(0))
if err != nil {
@ -72,8 +71,8 @@ func (s) TestLRSClient(t *testing.T) {
ServerURI: fs.Address,
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
CredsType: "insecure",
TransportAPI: version.TransportV2,
NodeProto: &v2corepb.Node{},
TransportAPI: version.TransportV3,
NodeProto: &v3corepb.Node{},
},
)
defer lrsCancel1()
@ -101,7 +100,7 @@ func (s) TestLRSClient(t *testing.T) {
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
CredsType: "insecure",
TransportAPI: version.TransportV2,
NodeProto: &v2corepb.Node{},
NodeProto: &v3corepb.Node{},
},
)
defer lrsCancel2()
@ -120,7 +119,7 @@ func (s) TestLRSClient(t *testing.T) {
// Send one resp to the client.
fs2.LRSResponseChan <- &fakeserver.Response{
Resp: &lrspb.LoadStatsResponse{
Resp: &v3lrspb.LoadStatsResponse{
SendAllClusters: true,
LoadReportingInterval: &durationpb.Duration{Nanos: 50000000},
},
@ -131,16 +130,16 @@ func (s) TestLRSClient(t *testing.T) {
if err != nil {
t.Fatalf("unexpected LRS request: %v, %v, want error canceled", u, err)
}
receivedLoad := u.(*fakeserver.Request).Req.(*lrspb.LoadStatsRequest).ClusterStats
receivedLoad := u.(*fakeserver.Request).Req.(*v3lrspb.LoadStatsRequest).ClusterStats
if len(receivedLoad) <= 0 {
t.Fatalf("unexpected load received, want load for cluster, eds, dropped for test")
}
receivedLoad[0].LoadReportInterval = nil
want := &endpointpb.ClusterStats{
want := &v3endpointpb.ClusterStats{
ClusterName: "cluster",
ClusterServiceName: "eds",
TotalDroppedRequests: 1,
DroppedRequests: []*endpointpb.ClusterStats_DroppedRequests{{Category: "test", DroppedCount: 1}},
DroppedRequests: []*v3endpointpb.ClusterStats_DroppedRequests{{Category: "test", DroppedCount: 1}},
}
if d := cmp.Diff(want, receivedLoad[0], protocmp.Transform()); d != "" {
t.Fatalf("unexpected load received, want load for cluster, eds, dropped for test, diff (-want +got):\n%s", d)

View File

@ -1,87 +0,0 @@
/*
*
* Copyright 2021 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 pubsub
import (
anypb "github.com/golang/protobuf/ptypes/any"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
func rawFromCache(s string, cache interface{}) *anypb.Any {
switch c := cache.(type) {
case map[string]xdsresource.ListenerUpdate:
if v, ok := c[s]; ok {
return v.Raw
}
return nil
case map[string]xdsresource.RouteConfigUpdate:
if v, ok := c[s]; ok {
return v.Raw
}
return nil
case map[string]xdsresource.ClusterUpdate:
if v, ok := c[s]; ok {
return v.Raw
}
return nil
case map[string]xdsresource.EndpointsUpdate:
if v, ok := c[s]; ok {
return v.Raw
}
return nil
default:
return nil
}
}
// Dump dumps the resource for the given type.
func (pb *Pubsub) Dump(t xdsresource.ResourceType) map[string]xdsresource.UpdateWithMD {
pb.mu.Lock()
defer pb.mu.Unlock()
var (
md map[string]xdsresource.UpdateMetadata
cache interface{}
)
switch t {
case xdsresource.ListenerResource:
md = pb.ldsMD
cache = pb.ldsCache
case xdsresource.RouteConfigResource:
md = pb.rdsMD
cache = pb.rdsCache
case xdsresource.ClusterResource:
md = pb.cdsMD
cache = pb.cdsCache
case xdsresource.EndpointsResource:
md = pb.edsMD
cache = pb.edsCache
default:
pb.logger.Errorf("dumping resource of unknown type: %v", t)
return nil
}
ret := make(map[string]xdsresource.UpdateWithMD, len(md))
for s, md := range md {
ret[s] = xdsresource.UpdateWithMD{
MD: md,
Raw: rawFromCache(s, cache),
}
}
return ret
}

View File

@ -1,39 +0,0 @@
/*
*
* Copyright 2021 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 pubsub
import "google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
// UpdateHandler receives and processes (by taking appropriate actions) xDS
// resource updates from an APIClient for a specific version.
//
// It's a subset of the APIs of a *Pubsub.
type UpdateHandler interface {
// NewListeners handles updates to xDS listener resources.
NewListeners(map[string]xdsresource.ListenerUpdateErrTuple, xdsresource.UpdateMetadata)
// NewRouteConfigs handles updates to xDS RouteConfiguration resources.
NewRouteConfigs(map[string]xdsresource.RouteConfigUpdateErrTuple, xdsresource.UpdateMetadata)
// NewClusters handles updates to xDS Cluster resources.
NewClusters(map[string]xdsresource.ClusterUpdateErrTuple, xdsresource.UpdateMetadata)
// NewEndpoints handles updates to xDS ClusterLoadAssignment (or tersely
// referred to as Endpoints) resources.
NewEndpoints(map[string]xdsresource.EndpointsUpdateErrTuple, xdsresource.UpdateMetadata)
// NewConnectionError handles connection errors from the xDS stream. The
// error will be reported to all the resource watchers.
NewConnectionError(err error)
}

View File

@ -1,186 +0,0 @@
/*
*
* Copyright 2021 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 pubsub implements a utility type to maintain resource watchers and
// the updates.
//
// This package is designed to work with the xds resources. It could be made a
// general system that works with all types.
package pubsub
import (
"sync"
"time"
"google.golang.org/grpc/internal/buffer"
"google.golang.org/grpc/internal/grpclog"
"google.golang.org/grpc/internal/grpcsync"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
// Pubsub maintains resource watchers and resource updates.
//
// There can be multiple watchers for the same resource. An update to a resource
// triggers updates to all the existing watchers. Watchers can be canceled at
// any time.
type Pubsub struct {
done *grpcsync.Event
logger *grpclog.PrefixLogger
watchExpiryTimeout time.Duration
nodeID string
updateCh *buffer.Unbounded // chan *watcherInfoWithUpdate
// All the following maps are to keep the updates/metadata in a cache.
mu sync.Mutex
ldsWatchers map[string]map[*watchInfo]bool
ldsCache map[string]xdsresource.ListenerUpdate
ldsMD map[string]xdsresource.UpdateMetadata
rdsWatchers map[string]map[*watchInfo]bool
rdsCache map[string]xdsresource.RouteConfigUpdate
rdsMD map[string]xdsresource.UpdateMetadata
cdsWatchers map[string]map[*watchInfo]bool
cdsCache map[string]xdsresource.ClusterUpdate
cdsMD map[string]xdsresource.UpdateMetadata
edsWatchers map[string]map[*watchInfo]bool
edsCache map[string]xdsresource.EndpointsUpdate
edsMD map[string]xdsresource.UpdateMetadata
}
// New creates a new Pubsub.
//
// The passed in nodeID will be attached to all errors sent to the watchers.
func New(watchExpiryTimeout time.Duration, nodeID string, logger *grpclog.PrefixLogger) *Pubsub {
pb := &Pubsub{
done: grpcsync.NewEvent(),
logger: logger,
watchExpiryTimeout: watchExpiryTimeout,
nodeID: nodeID,
updateCh: buffer.NewUnbounded(),
ldsWatchers: make(map[string]map[*watchInfo]bool),
ldsCache: make(map[string]xdsresource.ListenerUpdate),
ldsMD: make(map[string]xdsresource.UpdateMetadata),
rdsWatchers: make(map[string]map[*watchInfo]bool),
rdsCache: make(map[string]xdsresource.RouteConfigUpdate),
rdsMD: make(map[string]xdsresource.UpdateMetadata),
cdsWatchers: make(map[string]map[*watchInfo]bool),
cdsCache: make(map[string]xdsresource.ClusterUpdate),
cdsMD: make(map[string]xdsresource.UpdateMetadata),
edsWatchers: make(map[string]map[*watchInfo]bool),
edsCache: make(map[string]xdsresource.EndpointsUpdate),
edsMD: make(map[string]xdsresource.UpdateMetadata),
}
go pb.run()
return pb
}
// WatchListener registers a watcher for the LDS resource.
//
// It also returns whether this is the first watch for this resource.
func (pb *Pubsub) WatchListener(serviceName string, cb func(xdsresource.ListenerUpdate, error)) (first bool, cancel func() bool) {
wi := &watchInfo{
c: pb,
rType: xdsresource.ListenerResource,
target: serviceName,
ldsCallback: cb,
}
wi.expiryTimer = time.AfterFunc(pb.watchExpiryTimeout, func() {
wi.timeout()
})
return pb.watch(wi)
}
// WatchRouteConfig register a watcher for the RDS resource.
//
// It also returns whether this is the first watch for this resource.
func (pb *Pubsub) WatchRouteConfig(routeName string, cb func(xdsresource.RouteConfigUpdate, error)) (first bool, cancel func() bool) {
wi := &watchInfo{
c: pb,
rType: xdsresource.RouteConfigResource,
target: routeName,
rdsCallback: cb,
}
wi.expiryTimer = time.AfterFunc(pb.watchExpiryTimeout, func() {
wi.timeout()
})
return pb.watch(wi)
}
// WatchCluster register a watcher for the CDS resource.
//
// It also returns whether this is the first watch for this resource.
func (pb *Pubsub) WatchCluster(clusterName string, cb func(xdsresource.ClusterUpdate, error)) (first bool, cancel func() bool) {
wi := &watchInfo{
c: pb,
rType: xdsresource.ClusterResource,
target: clusterName,
cdsCallback: cb,
}
wi.expiryTimer = time.AfterFunc(pb.watchExpiryTimeout, func() {
wi.timeout()
})
return pb.watch(wi)
}
// WatchEndpoints registers a watcher for the EDS resource.
//
// It also returns whether this is the first watch for this resource.
func (pb *Pubsub) WatchEndpoints(clusterName string, cb func(xdsresource.EndpointsUpdate, error)) (first bool, cancel func() bool) {
wi := &watchInfo{
c: pb,
rType: xdsresource.EndpointsResource,
target: clusterName,
edsCallback: cb,
}
wi.expiryTimer = time.AfterFunc(pb.watchExpiryTimeout, func() {
wi.timeout()
})
return pb.watch(wi)
}
// Close closes the pubsub.
func (pb *Pubsub) Close() {
if pb.done.HasFired() {
return
}
pb.done.Fire()
}
// 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 (pb *Pubsub) run() {
for {
select {
case t := <-pb.updateCh.Get():
pb.updateCh.Load()
if pb.done.HasFired() {
return
}
pb.callCallback(t.(*watcherInfoWithUpdate))
case <-pb.done.Done():
return
}
}
}

View File

@ -1,318 +0,0 @@
/*
*
* Copyright 2021 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 pubsub
import (
"google.golang.org/grpc/internal/pretty"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/protobuf/proto"
)
type watcherInfoWithUpdate struct {
wi *watchInfo
update interface{}
err error
}
// scheduleCallback should only be called by methods of watchInfo, which checks
// for watcher states and maintain consistency.
func (pb *Pubsub) scheduleCallback(wi *watchInfo, update interface{}, err error) {
pb.updateCh.Put(&watcherInfoWithUpdate{
wi: wi,
update: update,
err: err,
})
}
func (pb *Pubsub) callCallback(wiu *watcherInfoWithUpdate) {
pb.mu.Lock()
// Use a closure to capture the callback and type assertion, to save one
// more switch case.
//
// The callback must be called without pb.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.rType {
case xdsresource.ListenerResource:
if s, ok := pb.ldsWatchers[wiu.wi.target]; ok && s[wiu.wi] {
ccb = func() { wiu.wi.ldsCallback(wiu.update.(xdsresource.ListenerUpdate), wiu.err) }
}
case xdsresource.RouteConfigResource:
if s, ok := pb.rdsWatchers[wiu.wi.target]; ok && s[wiu.wi] {
ccb = func() { wiu.wi.rdsCallback(wiu.update.(xdsresource.RouteConfigUpdate), wiu.err) }
}
case xdsresource.ClusterResource:
if s, ok := pb.cdsWatchers[wiu.wi.target]; ok && s[wiu.wi] {
ccb = func() { wiu.wi.cdsCallback(wiu.update.(xdsresource.ClusterUpdate), wiu.err) }
}
case xdsresource.EndpointsResource:
if s, ok := pb.edsWatchers[wiu.wi.target]; ok && s[wiu.wi] {
ccb = func() { wiu.wi.edsCallback(wiu.update.(xdsresource.EndpointsUpdate), wiu.err) }
}
}
pb.mu.Unlock()
if ccb != nil {
ccb()
}
}
// NewListeners is called when there's a new LDS update.
func (pb *Pubsub) NewListeners(updates map[string]xdsresource.ListenerUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
pb.mu.Lock()
defer pb.mu.Unlock()
for name, uErr := range updates {
if s, ok := pb.ldsWatchers[name]; ok {
if uErr.Err != nil {
// On error, keep previous version for each resource. But update
// status and error.
mdCopy := pb.ldsMD[name]
mdCopy.ErrState = metadata.ErrState
mdCopy.Status = metadata.Status
pb.ldsMD[name] = mdCopy
for wi := range s {
wi.newError(uErr.Err)
}
continue
}
// If we get here, it means that the update is a valid one. Notify
// watchers only if this is a first time update or it is different
// from the one currently cached.
if cur, ok := pb.ldsCache[name]; !ok || !proto.Equal(cur.Raw, uErr.Update.Raw) {
for wi := range s {
wi.newUpdate(uErr.Update)
}
}
// Sync cache.
pb.logger.Debugf("LDS resource with name %v, value %+v added to cache", name, pretty.ToJSON(uErr))
pb.ldsCache[name] = uErr.Update
// Set status to ACK, and clear error state. The metadata might be a
// NACK metadata because some other resources in the same response
// are invalid.
mdCopy := metadata
mdCopy.Status = xdsresource.ServiceStatusACKed
mdCopy.ErrState = nil
if metadata.ErrState != nil {
mdCopy.Version = metadata.ErrState.Version
}
pb.ldsMD[name] = mdCopy
}
}
// Resources not in the new update were removed by the server, so delete
// them.
for name := range pb.ldsCache {
if _, ok := updates[name]; !ok {
// If resource exists in cache, but not in the new update, delete
// the resource from cache, and also send an resource not found
// error to indicate resource removed.
delete(pb.ldsCache, name)
pb.ldsMD[name] = xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusNotExist}
for wi := range pb.ldsWatchers[name] {
wi.resourceNotFound()
}
}
}
// When LDS resource is removed, we don't delete corresponding RDS cached
// data. The RDS watch will be canceled, and cache entry is removed when the
// last watch is canceled.
}
// NewRouteConfigs is called when there's a new RDS update.
func (pb *Pubsub) NewRouteConfigs(updates map[string]xdsresource.RouteConfigUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
pb.mu.Lock()
defer pb.mu.Unlock()
// If no error received, the status is ACK.
for name, uErr := range updates {
if s, ok := pb.rdsWatchers[name]; ok {
if uErr.Err != nil {
// On error, keep previous version for each resource. But update
// status and error.
mdCopy := pb.rdsMD[name]
mdCopy.ErrState = metadata.ErrState
mdCopy.Status = metadata.Status
pb.rdsMD[name] = mdCopy
for wi := range s {
wi.newError(uErr.Err)
}
continue
}
// If we get here, it means that the update is a valid one. Notify
// watchers only if this is a first time update or it is different
// from the one currently cached.
if cur, ok := pb.rdsCache[name]; !ok || !proto.Equal(cur.Raw, uErr.Update.Raw) {
for wi := range s {
wi.newUpdate(uErr.Update)
}
}
// Sync cache.
pb.logger.Debugf("RDS resource with name %v, value %+v added to cache", name, pretty.ToJSON(uErr))
pb.rdsCache[name] = uErr.Update
// Set status to ACK, and clear error state. The metadata might be a
// NACK metadata because some other resources in the same response
// are invalid.
mdCopy := metadata
mdCopy.Status = xdsresource.ServiceStatusACKed
mdCopy.ErrState = nil
if metadata.ErrState != nil {
mdCopy.Version = metadata.ErrState.Version
}
pb.rdsMD[name] = mdCopy
}
}
}
// NewClusters is called when there's a new CDS update.
func (pb *Pubsub) NewClusters(updates map[string]xdsresource.ClusterUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
pb.mu.Lock()
defer pb.mu.Unlock()
for name, uErr := range updates {
if s, ok := pb.cdsWatchers[name]; ok {
if uErr.Err != nil {
// On error, keep previous version for each resource. But update
// status and error.
mdCopy := pb.cdsMD[name]
mdCopy.ErrState = metadata.ErrState
mdCopy.Status = metadata.Status
pb.cdsMD[name] = mdCopy
for wi := range s {
// Send the watcher the individual error, instead of the
// overall combined error from the metadata.ErrState.
wi.newError(uErr.Err)
}
continue
}
// If we get here, it means that the update is a valid one. Notify
// watchers only if this is a first time update or it is different
// from the one currently cached.
if cur, ok := pb.cdsCache[name]; !ok || !proto.Equal(cur.Raw, uErr.Update.Raw) {
for wi := range s {
wi.newUpdate(uErr.Update)
}
}
// Sync cache.
pb.logger.Debugf("CDS resource with name %v, value %+v added to cache", name, pretty.ToJSON(uErr))
pb.cdsCache[name] = uErr.Update
// Set status to ACK, and clear error state. The metadata might be a
// NACK metadata because some other resources in the same response
// are invalid.
mdCopy := metadata
mdCopy.Status = xdsresource.ServiceStatusACKed
mdCopy.ErrState = nil
if metadata.ErrState != nil {
mdCopy.Version = metadata.ErrState.Version
}
pb.cdsMD[name] = mdCopy
}
}
// Resources not in the new update were removed by the server, so delete
// them.
for name := range pb.cdsCache {
if _, ok := updates[name]; !ok {
// If resource exists in cache, but not in the new update, delete it
// from cache, and also send an resource not found error to indicate
// resource removed.
delete(pb.cdsCache, name)
pb.cdsMD[name] = xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusNotExist}
for wi := range pb.cdsWatchers[name] {
wi.resourceNotFound()
}
}
}
// When CDS resource is removed, we don't delete corresponding EDS cached
// data. The EDS watch will be canceled, and cache entry is removed when the
// last watch is canceled.
}
// NewEndpoints is called when there's anew EDS update.
func (pb *Pubsub) NewEndpoints(updates map[string]xdsresource.EndpointsUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
pb.mu.Lock()
defer pb.mu.Unlock()
for name, uErr := range updates {
if s, ok := pb.edsWatchers[name]; ok {
if uErr.Err != nil {
// On error, keep previous version for each resource. But update
// status and error.
mdCopy := pb.edsMD[name]
mdCopy.ErrState = metadata.ErrState
mdCopy.Status = metadata.Status
pb.edsMD[name] = mdCopy
for wi := range s {
// Send the watcher the individual error, instead of the
// overall combined error from the metadata.ErrState.
wi.newError(uErr.Err)
}
continue
}
// If we get here, it means that the update is a valid one. Notify
// watchers only if this is a first time update or it is different
// from the one currently cached.
if cur, ok := pb.edsCache[name]; !ok || !proto.Equal(cur.Raw, uErr.Update.Raw) {
for wi := range s {
wi.newUpdate(uErr.Update)
}
}
// Sync cache.
pb.logger.Debugf("EDS resource with name %v, value %+v added to cache", name, pretty.ToJSON(uErr))
pb.edsCache[name] = uErr.Update
// Set status to ACK, and clear error state. The metadata might be a
// NACK metadata because some other resources in the same response
// are invalid.
mdCopy := metadata
mdCopy.Status = xdsresource.ServiceStatusACKed
mdCopy.ErrState = nil
if metadata.ErrState != nil {
mdCopy.Version = metadata.ErrState.Version
}
pb.edsMD[name] = mdCopy
}
}
}
// NewConnectionError is called by the underlying xdsAPIClient when it receives
// a connection error. The error will be forwarded to all the resource watchers.
func (pb *Pubsub) NewConnectionError(err error) {
pb.mu.Lock()
defer pb.mu.Unlock()
for _, s := range pb.ldsWatchers {
for wi := range s {
wi.newError(xdsresource.NewErrorf(xdsresource.ErrorTypeConnection, "xds: error received from xDS stream: %v", err))
}
}
for _, s := range pb.rdsWatchers {
for wi := range s {
wi.newError(xdsresource.NewErrorf(xdsresource.ErrorTypeConnection, "xds: error received from xDS stream: %v", err))
}
}
for _, s := range pb.cdsWatchers {
for wi := range s {
wi.newError(xdsresource.NewErrorf(xdsresource.ErrorTypeConnection, "xds: error received from xDS stream: %v", err))
}
}
for _, s := range pb.edsWatchers {
for wi := range s {
wi.newError(xdsresource.NewErrorf(xdsresource.ErrorTypeConnection, "xds: error received from xDS stream: %v", err))
}
}
}

View File

@ -1,239 +0,0 @@
/*
*
* Copyright 2021 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 pubsub
import (
"fmt"
"sync"
"time"
"google.golang.org/grpc/internal/pretty"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
type watchInfoState int
const (
watchInfoStateStarted watchInfoState = iota
watchInfoStateRespReceived
watchInfoStateTimeout
watchInfoStateCanceled
)
// watchInfo holds all the information from a watch() call.
type watchInfo struct {
c *Pubsub
rType xdsresource.ResourceType
target string
ldsCallback func(xdsresource.ListenerUpdate, error)
rdsCallback func(xdsresource.RouteConfigUpdate, error)
cdsCallback func(xdsresource.ClusterUpdate, error)
edsCallback func(xdsresource.EndpointsUpdate, error)
expiryTimer *time.Timer
// mu protects state, and c.scheduleCallback().
// - No callback should be scheduled after watchInfo is canceled.
// - No timeout error should be scheduled after watchInfo is resp received.
mu sync.Mutex
state watchInfoState
}
func (wi *watchInfo) newUpdate(update interface{}) {
wi.mu.Lock()
defer wi.mu.Unlock()
if wi.state == watchInfoStateCanceled {
return
}
wi.state = watchInfoStateRespReceived
wi.expiryTimer.Stop()
wi.c.scheduleCallback(wi, update, nil)
}
func (wi *watchInfo) newError(err error) {
wi.mu.Lock()
defer wi.mu.Unlock()
if wi.state == watchInfoStateCanceled {
return
}
wi.state = watchInfoStateRespReceived
wi.expiryTimer.Stop()
wi.sendErrorLocked(err)
}
func (wi *watchInfo) resourceNotFound() {
wi.mu.Lock()
defer wi.mu.Unlock()
if wi.state == watchInfoStateCanceled {
return
}
wi.state = watchInfoStateRespReceived
wi.expiryTimer.Stop()
wi.sendErrorLocked(xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "xds: %v target %s not found in received response", wi.rType, wi.target))
}
func (wi *watchInfo) timeout() {
wi.mu.Lock()
defer wi.mu.Unlock()
if wi.state == watchInfoStateCanceled || wi.state == watchInfoStateRespReceived {
return
}
wi.state = watchInfoStateTimeout
wi.sendErrorLocked(fmt.Errorf("xds: %v target %s not found, watcher timeout", wi.rType, wi.target))
}
// Caller must hold wi.mu.
func (wi *watchInfo) sendErrorLocked(err error) {
var u interface{}
switch wi.rType {
case xdsresource.ListenerResource:
u = xdsresource.ListenerUpdate{}
case xdsresource.RouteConfigResource:
u = xdsresource.RouteConfigUpdate{}
case xdsresource.ClusterResource:
u = xdsresource.ClusterUpdate{}
case xdsresource.EndpointsResource:
u = xdsresource.EndpointsUpdate{}
}
errMsg := err.Error()
errTyp := xdsresource.ErrType(err)
if errTyp == xdsresource.ErrorTypeUnknown {
err = fmt.Errorf("%v, xDS client nodeID: %s", errMsg, wi.c.nodeID)
} else {
err = xdsresource.NewErrorf(errTyp, "%v, xDS client nodeID: %s", errMsg, wi.c.nodeID)
}
wi.c.scheduleCallback(wi, u, err)
}
func (wi *watchInfo) cancel() {
wi.mu.Lock()
defer wi.mu.Unlock()
if wi.state == watchInfoStateCanceled {
return
}
wi.expiryTimer.Stop()
wi.state = watchInfoStateCanceled
}
func (pb *Pubsub) watch(wi *watchInfo) (first bool, cancel func() bool) {
pb.mu.Lock()
defer pb.mu.Unlock()
pb.logger.Debugf("new watch for type %v, resource name %v", wi.rType, wi.target)
var (
watchers map[string]map[*watchInfo]bool
mds map[string]xdsresource.UpdateMetadata
)
switch wi.rType {
case xdsresource.ListenerResource:
watchers = pb.ldsWatchers
mds = pb.ldsMD
case xdsresource.RouteConfigResource:
watchers = pb.rdsWatchers
mds = pb.rdsMD
case xdsresource.ClusterResource:
watchers = pb.cdsWatchers
mds = pb.cdsMD
case xdsresource.EndpointsResource:
watchers = pb.edsWatchers
mds = pb.edsMD
default:
pb.logger.Errorf("unknown watch type: %v", wi.rType)
return false, nil
}
var firstWatcher bool
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 versioned apiClient.
pb.logger.Debugf("first watch for type %v, resource name %v, will send a new xDS request", wi.rType, wi.target)
s = make(map[*watchInfo]bool)
watchers[resourceName] = s
mds[resourceName] = xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}
firstWatcher = true
}
// 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.rType {
case xdsresource.ListenerResource:
if v, ok := pb.ldsCache[resourceName]; ok {
pb.logger.Debugf("LDS resource with name %v found in cache: %+v", wi.target, pretty.ToJSON(v))
wi.newUpdate(v)
}
case xdsresource.RouteConfigResource:
if v, ok := pb.rdsCache[resourceName]; ok {
pb.logger.Debugf("RDS resource with name %v found in cache: %+v", wi.target, pretty.ToJSON(v))
wi.newUpdate(v)
}
case xdsresource.ClusterResource:
if v, ok := pb.cdsCache[resourceName]; ok {
pb.logger.Debugf("CDS resource with name %v found in cache: %+v", wi.target, pretty.ToJSON(v))
wi.newUpdate(v)
}
case xdsresource.EndpointsResource:
if v, ok := pb.edsCache[resourceName]; ok {
pb.logger.Debugf("EDS resource with name %v found in cache: %+v", wi.target, pretty.ToJSON(v))
wi.newUpdate(v)
}
}
return firstWatcher, func() bool {
pb.logger.Debugf("watch for type %v, resource name %v canceled", wi.rType, wi.target)
wi.cancel()
pb.mu.Lock()
defer pb.mu.Unlock()
var lastWatcher bool
if s := watchers[resourceName]; s != nil {
// Remove this watcher, so it's callback will not be called in the
// future.
delete(s, wi)
if len(s) == 0 {
pb.logger.Debugf("last watch for type %v, resource name %v canceled, will send a new xDS request", wi.rType, wi.target)
// If this was the last watcher, also tell xdsv2Client to stop
// watching this resource.
delete(watchers, resourceName)
delete(mds, resourceName)
lastWatcher = true
// Remove the resource from cache. When a watch for this
// resource is added later, it will trigger a xDS request with
// resource names, and client will receive new xDS responses.
switch wi.rType {
case xdsresource.ListenerResource:
delete(pb.ldsCache, resourceName)
case xdsresource.RouteConfigResource:
delete(pb.rdsCache, resourceName)
case xdsresource.ClusterResource:
delete(pb.cdsCache, resourceName)
case xdsresource.EndpointsResource:
delete(pb.edsCache, resourceName)
}
}
}
return lastWatcher
}
}

View File

@ -1,47 +0,0 @@
/*
*
* Copyright 2021 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 xdsclient
import (
"testing"
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
"google.golang.org/grpc/xds/internal/xdsclient/pubsub"
)
// findPubsubForTest returns the pubsub for the given authority, to send updates
// to. If authority is "", the default is returned. If the authority is not
// found, the test will fail.
func findPubsubForTest(t *testing.T, c *clientImpl, authority string) pubsub.UpdateHandler {
t.Helper()
var config *bootstrap.ServerConfig
if authority == "" {
config = c.config.XDSServer
} else {
authConfig, ok := c.config.Authorities[authority]
if !ok {
t.Fatalf("failed to find authority %q", authority)
}
config = authConfig.XDSServer
}
a := c.authorities[config.String()]
if a == nil {
t.Fatalf("authority for %q is not created", authority)
}
return a.pubsub
}

View File

@ -22,7 +22,6 @@ import (
"testing"
"google.golang.org/grpc/internal/grpctest"
_ "google.golang.org/grpc/xds/internal/xdsclient/controller/version/v2" // Register the v2 API client.
)
type s struct {

View File

@ -31,8 +31,7 @@ var (
// Singleton instantiation of the resource type implementation.
clusterType = clusterResourceType{
resourceTypeState: resourceTypeState{
v2TypeURL: "type.googleapis.com/envoy.api.v2.Cluster",
v3TypeURL: "type.googleapis.com/envoy.config.cluster.v3.Cluster",
typeURL: "type.googleapis.com/envoy.config.cluster.v3.Cluster",
typeEnum: ClusterResource,
allResourcesRequiredInSotW: true,
},
@ -50,7 +49,7 @@ type clusterResourceType struct {
// Decode deserializes and validates an xDS resource serialized inside the
// provided `Any` proto, as received from the xDS management server.
func (clusterResourceType) Decode(opts *DecodeOptions, resource *anypb.Any) (*DecodeResult, error) {
name, cluster, err := unmarshalClusterResource(resource, nil, opts.Logger)
name, cluster, err := unmarshalClusterResource(resource, opts.Logger)
switch {
case name == "":
// Name is unset only when protobuf deserialization fails.
@ -90,7 +89,6 @@ func (c *ClusterResourceData) Equal(other ResourceData) bool {
return false
}
return proto.Equal(c.Resource.Raw, other.Raw())
}
// ToJSON returns a JSON string representation of the resource data.

View File

@ -31,8 +31,7 @@ var (
// Singleton instantiation of the resource type implementation.
endpointsType = endpointsResourceType{
resourceTypeState: resourceTypeState{
v2TypeURL: "type.googleapis.com/envoy.api.v2.ClusterLoadAssignment",
v3TypeURL: "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment",
typeURL: "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment",
typeEnum: EndpointsResource,
allResourcesRequiredInSotW: false,
},

View File

@ -34,8 +34,7 @@ var (
// Singleton instantiation of the resource type implementation.
listenerType = listenerResourceType{
resourceTypeState: resourceTypeState{
v2TypeURL: "type.googleapis.com/envoy.api.v2.Listener",
v3TypeURL: "type.googleapis.com/envoy.config.listener.v3.Listener",
typeURL: "type.googleapis.com/envoy.config.listener.v3.Listener",
typeEnum: ListenerResource,
allResourcesRequiredInSotW: true,
},
@ -82,7 +81,7 @@ func listenerValidator(bc *bootstrap.Config, lis ListenerUpdate) error {
// Decode deserializes and validates an xDS resource serialized inside the
// provided `Any` proto, as received from the xDS management server.
func (listenerResourceType) Decode(opts *DecodeOptions, resource *anypb.Any) (*DecodeResult, error) {
name, listener, err := unmarshalListenerResource(resource, nil, opts.Logger)
name, listener, err := unmarshalListenerResource(resource, opts.Logger)
switch {
case name == "":
// Name is unset only when protobuf deserialization fails.

View File

@ -65,11 +65,8 @@ type ResourceWatcher interface {
// Type wraps all resource-type specific functionality. Each supported resource
// type will provide an implementation of this interface.
type Type interface {
// V2TypeURL is the xDS type URL of this resource type for v2 transport.
V2TypeURL() string
// V3TypeURL is the xDS type URL of this resource type for v3 transport.
V3TypeURL() string
// TypeURL is the xDS type URL of this resource type for v3 transport.
TypeURL() string
// TypeEnum is an enumerated value for this resource type. This can be used
// for logging/debugging purposes, as well in cases where the resource type
@ -135,18 +132,13 @@ type DecodeResult struct {
// type implementations, which can then embed this struct and get the methods
// implemented here for free.
type resourceTypeState struct {
v2TypeURL string
v3TypeURL string
typeURL string
typeEnum ResourceType
allResourcesRequiredInSotW bool
}
func (r resourceTypeState) V2TypeURL() string {
return r.v2TypeURL
}
func (r resourceTypeState) V3TypeURL() string {
return r.v3TypeURL
func (r resourceTypeState) TypeURL() string {
return r.typeURL
}
func (r resourceTypeState) TypeEnum() ResourceType {

View File

@ -31,8 +31,7 @@ var (
// Singleton instantiation of the resource type implementation.
routeConfigType = routeConfigResourceType{
resourceTypeState: resourceTypeState{
v2TypeURL: "type.googleapis.com/envoy.api.v2.RouteConfiguration",
v3TypeURL: "type.googleapis.com/envoy.config.route.v3.RouteConfiguration",
typeURL: "type.googleapis.com/envoy.config.route.v3.RouteConfiguration",
typeEnum: RouteConfigResource,
allResourcesRequiredInSotW: false,
},

View File

@ -42,11 +42,4 @@ var (
cmp.Comparer(func(a, b time.Time) bool { return true }),
protocmp.Transform(),
}
cmpOptsIgnoreDetails = cmp.Options{
cmp.Comparer(func(a, b time.Time) bool { return true }),
cmp.Comparer(func(x, y error) bool {
return (x == nil) == (y == nil)
}),
}
)

View File

@ -165,3 +165,57 @@ func (r ResourceType) String() string {
return "UnknownResource"
}
}
var v2ResourceTypeToURL = map[ResourceType]string{
ListenerResource: version.V2ListenerURL,
HTTPConnManagerResource: version.V2HTTPConnManagerURL,
RouteConfigResource: version.V2RouteConfigURL,
ClusterResource: version.V2ClusterURL,
EndpointsResource: version.V2EndpointsURL,
}
var v3ResourceTypeToURL = map[ResourceType]string{
ListenerResource: version.V3ListenerURL,
HTTPConnManagerResource: version.V3HTTPConnManagerURL,
RouteConfigResource: version.V3RouteConfigURL,
ClusterResource: version.V3ClusterURL,
EndpointsResource: version.V3EndpointsURL,
}
// URL returns the transport protocol specific resource type URL.
func (r ResourceType) URL(v version.TransportAPI) string {
var mapping map[ResourceType]string
switch v {
case version.TransportV2:
mapping = v2ResourceTypeToURL
case version.TransportV3:
mapping = v3ResourceTypeToURL
default:
return "UnknownResource"
}
if url, ok := mapping[r]; ok {
return url
}
return "UnknownResource"
}
var urlToResourceType = map[string]ResourceType{
version.V2ListenerURL: ListenerResource,
version.V2RouteConfigURL: RouteConfigResource,
version.V2ClusterURL: ClusterResource,
version.V2EndpointsURL: EndpointsResource,
version.V2HTTPConnManagerURL: HTTPConnManagerResource,
version.V3ListenerURL: ListenerResource,
version.V3RouteConfigURL: RouteConfigResource,
version.V3ClusterURL: ClusterResource,
version.V3EndpointsURL: EndpointsResource,
version.V3HTTPConnManagerURL: HTTPConnManagerResource,
}
// ResourceTypeFromURL returns the xDS resource type associated with the given
// resource type URL.
func ResourceTypeFromURL(url string) ResourceType {
if typ, ok := urlToResourceType[url]; ok {
return typ
}
return UnknownResource
}

View File

@ -23,7 +23,6 @@ import (
"errors"
"fmt"
"strings"
"time"
"google.golang.org/grpc/internal/grpclog"
"google.golang.org/protobuf/types/known/anypb"
@ -42,117 +41,8 @@ type UnmarshalOptions struct {
UpdateValidator UpdateValidatorFunc
}
// processAllResources unmarshals and validates the resources, populates the
// provided ret (a map), and returns metadata and error.
//
// After this function, the ret map will be populated with both valid and
// invalid updates. Invalid resources will have an entry with the key as the
// resource name, value as an empty update.
//
// The type of the resource is determined by the type of ret. E.g.
// map[string]ListenerUpdate means this is for LDS.
func processAllResources(opts *UnmarshalOptions, ret interface{}) (UpdateMetadata, error) {
timestamp := time.Now()
md := UpdateMetadata{
Version: opts.Version,
Timestamp: timestamp,
}
var topLevelErrors []error
perResourceErrors := make(map[string]error)
for _, r := range opts.Resources {
switch ret2 := ret.(type) {
case map[string]ListenerUpdateErrTuple:
name, update, err := unmarshalListenerResource(r, opts.UpdateValidator, opts.Logger)
name = ParseName(name).String()
if err == nil {
ret2[name] = ListenerUpdateErrTuple{Update: update}
continue
}
if name == "" {
topLevelErrors = append(topLevelErrors, err)
continue
}
perResourceErrors[name] = err
// Add place holder in the map so we know this resource name was in
// the response.
ret2[name] = ListenerUpdateErrTuple{Err: err}
case map[string]RouteConfigUpdateErrTuple:
name, update, err := unmarshalRouteConfigResource(r, opts.Logger)
name = ParseName(name).String()
if err == nil {
ret2[name] = RouteConfigUpdateErrTuple{Update: update}
continue
}
if name == "" {
topLevelErrors = append(topLevelErrors, err)
continue
}
perResourceErrors[name] = err
// Add place holder in the map so we know this resource name was in
// the response.
ret2[name] = RouteConfigUpdateErrTuple{Err: err}
case map[string]ClusterUpdateErrTuple:
name, update, err := unmarshalClusterResource(r, opts.UpdateValidator, opts.Logger)
name = ParseName(name).String()
if err == nil {
ret2[name] = ClusterUpdateErrTuple{Update: update}
continue
}
if name == "" {
topLevelErrors = append(topLevelErrors, err)
continue
}
perResourceErrors[name] = err
// Add place holder in the map so we know this resource name was in
// the response.
ret2[name] = ClusterUpdateErrTuple{Err: err}
case map[string]EndpointsUpdateErrTuple:
name, update, err := unmarshalEndpointsResource(r, opts.Logger)
name = ParseName(name).String()
if err == nil {
ret2[name] = EndpointsUpdateErrTuple{Update: update}
continue
}
if name == "" {
topLevelErrors = append(topLevelErrors, err)
continue
}
perResourceErrors[name] = err
// Add place holder in the map so we know this resource name was in
// the response.
ret2[name] = EndpointsUpdateErrTuple{Err: err}
}
}
if len(topLevelErrors) == 0 && len(perResourceErrors) == 0 {
md.Status = ServiceStatusACKed
return md, nil
}
var typeStr string
switch ret.(type) {
case map[string]ListenerUpdate:
typeStr = "LDS"
case map[string]RouteConfigUpdate:
typeStr = "RDS"
case map[string]ClusterUpdate:
typeStr = "CDS"
case map[string]EndpointsUpdate:
typeStr = "EDS"
}
md.Status = ServiceStatusNACKed
errRet := combineErrors(typeStr, topLevelErrors, perResourceErrors)
md.ErrState = &UpdateErrorMetadata{
Version: opts.Version,
Err: errRet,
Timestamp: timestamp,
}
return md, errRet
}
func combineErrors(rType string, topLevelErrors []error, perResourceErrors map[string]error) error {
// CombineErrors TBD.
func CombineErrors(rType string, topLevelErrors []error, perResourceErrors map[string]error) error {
var errStrB strings.Builder
errStrB.WriteString(fmt.Sprintf("error parsing %q response: ", rType))
if len(topLevelErrors) > 0 {

View File

@ -41,16 +41,7 @@ import (
// to this value by the management server.
const transportSocketName = "envoy.transport_sockets.tls"
// UnmarshalCluster processes resources received in an CDS response, validates
// them, and transforms them into a native struct which contains only fields we
// are interested in.
func UnmarshalCluster(opts *UnmarshalOptions) (map[string]ClusterUpdateErrTuple, UpdateMetadata, error) {
update := make(map[string]ClusterUpdateErrTuple)
md, err := processAllResources(opts, update)
return update, md, err
}
func unmarshalClusterResource(r *anypb.Any, f UpdateValidatorFunc, logger *grpclog.PrefixLogger) (string, ClusterUpdate, error) {
func unmarshalClusterResource(r *anypb.Any, logger *grpclog.PrefixLogger) (string, ClusterUpdate, error) {
r, err := unwrapResource(r)
if err != nil {
return "", ClusterUpdate{}, fmt.Errorf("failed to unwrap resource: %v", err)
@ -70,11 +61,6 @@ func unmarshalClusterResource(r *anypb.Any, f UpdateValidatorFunc, logger *grpcl
return cluster.GetName(), ClusterUpdate{}, err
}
cu.Raw = r
if f != nil {
if err := f(cu); err != nil {
return "", ClusterUpdate{}, err
}
}
return cluster.GetName(), cu, nil
}

View File

@ -27,6 +27,7 @@ import (
"github.com/google/go-cmp/cmp"
"github.com/google/go-cmp/cmp/cmpopts"
"google.golang.org/grpc/internal/envconfig"
"google.golang.org/grpc/internal/pretty"
"google.golang.org/grpc/internal/testutils"
"google.golang.org/grpc/internal/xds/matcher"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
@ -1442,248 +1443,121 @@ func (s) TestUnmarshalCluster(t *testing.T) {
},
})
)
const testVersion = "test-version-cds"
tests := []struct {
name string
resources []*anypb.Any
wantUpdate map[string]ClusterUpdateErrTuple
wantMD UpdateMetadata
resource *anypb.Any
wantName string
wantUpdate ClusterUpdate
wantErr bool
}{
{
name: "non-cluster resource type",
resources: []*anypb.Any{{TypeUrl: version.V3HTTPConnManagerURL}},
wantMD: UpdateMetadata{
Status: ServiceStatusNACKed,
Version: testVersion,
ErrState: &UpdateErrorMetadata{
Version: testVersion,
Err: cmpopts.AnyError,
},
},
wantErr: true,
name: "non-cluster resource type",
resource: &anypb.Any{TypeUrl: version.V3HTTPConnManagerURL},
wantErr: true,
},
{
name: "badly marshaled cluster resource",
resources: []*anypb.Any{
{
TypeUrl: version.V3ClusterURL,
Value: []byte{1, 2, 3, 4},
},
},
wantMD: UpdateMetadata{
Status: ServiceStatusNACKed,
Version: testVersion,
ErrState: &UpdateErrorMetadata{
Version: testVersion,
Err: cmpopts.AnyError,
},
resource: &anypb.Any{
TypeUrl: version.V3ClusterURL,
Value: []byte{1, 2, 3, 4},
},
wantErr: true,
},
{
name: "bad cluster resource",
resources: []*anypb.Any{
testutils.MarshalAny(&v3clusterpb.Cluster{
Name: "test",
ClusterDiscoveryType: &v3clusterpb.Cluster_Type{Type: v3clusterpb.Cluster_STATIC},
}),
},
wantUpdate: map[string]ClusterUpdateErrTuple{
"test": {Err: cmpopts.AnyError},
},
wantMD: UpdateMetadata{
Status: ServiceStatusNACKed,
Version: testVersion,
ErrState: &UpdateErrorMetadata{
Version: testVersion,
Err: cmpopts.AnyError,
},
},
wantErr: true,
resource: testutils.MarshalAny(&v3clusterpb.Cluster{
Name: "test",
ClusterDiscoveryType: &v3clusterpb.Cluster_Type{Type: v3clusterpb.Cluster_STATIC},
}),
wantName: "test",
wantErr: true,
},
{
name: "cluster resource with non-self lrs_server field",
resources: []*anypb.Any{
testutils.MarshalAny(&v3clusterpb.Cluster{
Name: "test",
ClusterDiscoveryType: &v3clusterpb.Cluster_Type{Type: v3clusterpb.Cluster_EDS},
EdsClusterConfig: &v3clusterpb.Cluster_EdsClusterConfig{
EdsConfig: &v3corepb.ConfigSource{
ConfigSourceSpecifier: &v3corepb.ConfigSource_Ads{
Ads: &v3corepb.AggregatedConfigSource{},
},
},
ServiceName: v3Service,
},
LbPolicy: v3clusterpb.Cluster_ROUND_ROBIN,
LrsServer: &v3corepb.ConfigSource{
resource: testutils.MarshalAny(&v3clusterpb.Cluster{
Name: "test",
ClusterDiscoveryType: &v3clusterpb.Cluster_Type{Type: v3clusterpb.Cluster_EDS},
EdsClusterConfig: &v3clusterpb.Cluster_EdsClusterConfig{
EdsConfig: &v3corepb.ConfigSource{
ConfigSourceSpecifier: &v3corepb.ConfigSource_Ads{
Ads: &v3corepb.AggregatedConfigSource{},
},
},
}),
},
wantUpdate: map[string]ClusterUpdateErrTuple{
"test": {Err: cmpopts.AnyError},
},
wantMD: UpdateMetadata{
Status: ServiceStatusNACKed,
Version: testVersion,
ErrState: &UpdateErrorMetadata{
Version: testVersion,
Err: cmpopts.AnyError,
ServiceName: v3Service,
},
},
wantErr: true,
},
{
name: "v2 cluster",
resources: []*anypb.Any{v2ClusterAny},
wantUpdate: map[string]ClusterUpdateErrTuple{
v2ClusterName: {Update: ClusterUpdate{
ClusterName: v2ClusterName,
EDSServiceName: v2Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v2ClusterAny,
}},
},
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
},
},
{
name: "v2 cluster wrapped",
resources: []*anypb.Any{testutils.MarshalAny(&v2xdspb.Resource{Resource: v2ClusterAny})},
wantUpdate: map[string]ClusterUpdateErrTuple{
v2ClusterName: {Update: ClusterUpdate{
ClusterName: v2ClusterName,
EDSServiceName: v2Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v2ClusterAny,
}},
},
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
},
},
{
name: "v3 cluster",
resources: []*anypb.Any{v3ClusterAny},
wantUpdate: map[string]ClusterUpdateErrTuple{
v3ClusterName: {Update: ClusterUpdate{
ClusterName: v3ClusterName,
EDSServiceName: v3Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v3ClusterAny,
}},
},
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
},
},
{
name: "v3 cluster wrapped",
resources: []*anypb.Any{testutils.MarshalAny(&v3discoverypb.Resource{Resource: v3ClusterAny})},
wantUpdate: map[string]ClusterUpdateErrTuple{
v3ClusterName: {Update: ClusterUpdate{
ClusterName: v3ClusterName,
EDSServiceName: v3Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v3ClusterAny,
}},
},
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
},
},
{
name: "v3 cluster with EDS config source self",
resources: []*anypb.Any{v3ClusterAnyWithEDSConfigSourceSelf},
wantUpdate: map[string]ClusterUpdateErrTuple{
v3ClusterName: {Update: ClusterUpdate{
ClusterName: v3ClusterName,
EDSServiceName: v3Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v3ClusterAnyWithEDSConfigSourceSelf,
}},
},
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
},
},
{
name: "multiple clusters",
resources: []*anypb.Any{v2ClusterAny, v3ClusterAny},
wantUpdate: map[string]ClusterUpdateErrTuple{
v2ClusterName: {Update: ClusterUpdate{
ClusterName: v2ClusterName,
EDSServiceName: v2Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v2ClusterAny,
}},
v3ClusterName: {Update: ClusterUpdate{
ClusterName: v3ClusterName,
EDSServiceName: v3Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v3ClusterAny,
}},
},
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
},
},
{
// To test that unmarshal keeps processing on errors.
name: "good and bad clusters",
resources: []*anypb.Any{
v2ClusterAny,
// bad cluster resource
testutils.MarshalAny(&v3clusterpb.Cluster{
Name: "bad",
ClusterDiscoveryType: &v3clusterpb.Cluster_Type{Type: v3clusterpb.Cluster_STATIC},
}),
v3ClusterAny,
},
wantUpdate: map[string]ClusterUpdateErrTuple{
v2ClusterName: {Update: ClusterUpdate{
ClusterName: v2ClusterName,
EDSServiceName: v2Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v2ClusterAny,
}},
v3ClusterName: {Update: ClusterUpdate{
ClusterName: v3ClusterName,
EDSServiceName: v3Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v3ClusterAny,
}},
"bad": {Err: cmpopts.AnyError},
},
wantMD: UpdateMetadata{
Status: ServiceStatusNACKed,
Version: testVersion,
ErrState: &UpdateErrorMetadata{
Version: testVersion,
Err: cmpopts.AnyError,
LbPolicy: v3clusterpb.Cluster_ROUND_ROBIN,
LrsServer: &v3corepb.ConfigSource{
ConfigSourceSpecifier: &v3corepb.ConfigSource_Ads{
Ads: &v3corepb.AggregatedConfigSource{},
},
},
}),
wantName: "test",
wantErr: true,
},
{
name: "v2 cluster",
resource: v2ClusterAny,
wantName: v2ClusterName,
wantUpdate: ClusterUpdate{
ClusterName: v2ClusterName,
EDSServiceName: v2Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v2ClusterAny,
},
},
{
name: "v2 cluster wrapped",
resource: testutils.MarshalAny(&v2xdspb.Resource{Resource: v2ClusterAny}),
wantName: v2ClusterName,
wantUpdate: ClusterUpdate{
ClusterName: v2ClusterName,
EDSServiceName: v2Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v2ClusterAny,
},
},
{
name: "v3 cluster",
resource: v3ClusterAny,
wantName: v3ClusterName,
wantUpdate: ClusterUpdate{
ClusterName: v3ClusterName,
EDSServiceName: v3Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v3ClusterAny,
},
},
{
name: "v3 cluster wrapped",
resource: testutils.MarshalAny(&v3discoverypb.Resource{Resource: v3ClusterAny}),
wantName: v3ClusterName,
wantUpdate: ClusterUpdate{
ClusterName: v3ClusterName,
EDSServiceName: v3Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v3ClusterAny,
},
},
{
name: "v3 cluster with EDS config source self",
resource: v3ClusterAnyWithEDSConfigSourceSelf,
wantName: v3ClusterName,
wantUpdate: ClusterUpdate{
ClusterName: v3ClusterName,
EDSServiceName: v3Service, LRSServerConfig: ClusterLRSServerSelf,
Raw: v3ClusterAnyWithEDSConfigSourceSelf,
},
wantErr: true,
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
opts := &UnmarshalOptions{
Version: testVersion,
Resources: test.resources,
}
update, md, err := UnmarshalCluster(opts)
name, update, err := unmarshalClusterResource(test.resource, nil)
if (err != nil) != test.wantErr {
t.Fatalf("UnmarshalCluster(%+v), got err: %v, wantErr: %v", opts, err, test.wantErr)
t.Fatalf("unmarshalClusterResource(%s), got err: %v, wantErr: %v", pretty.ToJSON(test.resource), err, test.wantErr)
}
if name != test.wantName {
t.Errorf("unmarshalClusterResource(%s), got name: %s, want: %s", pretty.ToJSON(test.resource), name, test.wantName)
}
if diff := cmp.Diff(update, test.wantUpdate, cmpOpts); diff != "" {
t.Errorf("got unexpected update, diff (-got +want): %v", diff)
}
if diff := cmp.Diff(md, test.wantMD, cmpOptsIgnoreDetails); diff != "" {
t.Errorf("got unexpected metadata, diff (-got +want): %v", diff)
t.Errorf("unmarshalClusterResource(%s), got unexpected update, diff (-got +want): %v", pretty.ToJSON(test.resource), diff)
}
})
}

View File

@ -33,15 +33,6 @@ import (
"google.golang.org/protobuf/types/known/anypb"
)
// UnmarshalEndpoints processes resources received in an EDS response,
// validates them, and transforms them into a native struct which contains only
// fields we are interested in.
func UnmarshalEndpoints(opts *UnmarshalOptions) (map[string]EndpointsUpdateErrTuple, UpdateMetadata, error) {
update := make(map[string]EndpointsUpdateErrTuple)
md, err := processAllResources(opts, update)
return update, md, err
}
func unmarshalEndpointsResource(r *anypb.Any, logger *grpclog.PrefixLogger) (string, EndpointsUpdate, error) {
r, err := unwrapResource(r)
if err != nil {

View File

@ -30,7 +30,7 @@ import (
anypb "github.com/golang/protobuf/ptypes/any"
wrapperspb "github.com/golang/protobuf/ptypes/wrappers"
"github.com/google/go-cmp/cmp"
"github.com/google/go-cmp/cmp/cmpopts"
"google.golang.org/grpc/internal/pretty"
"google.golang.org/grpc/internal/testutils"
"google.golang.org/grpc/xds/internal"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
@ -233,204 +233,112 @@ func (s) TestUnmarshalEndpoints(t *testing.T) {
})
return clab0.Build()
}())
const testVersion = "test-version-eds"
tests := []struct {
name string
resources []*anypb.Any
wantUpdate map[string]EndpointsUpdateErrTuple
wantMD UpdateMetadata
resource *anypb.Any
wantName string
wantUpdate EndpointsUpdate
wantErr bool
}{
{
name: "non-clusterLoadAssignment resource type",
resources: []*anypb.Any{{TypeUrl: version.V3HTTPConnManagerURL}},
wantMD: UpdateMetadata{
Status: ServiceStatusNACKed,
Version: testVersion,
ErrState: &UpdateErrorMetadata{
Version: testVersion,
Err: cmpopts.AnyError,
},
},
wantErr: true,
name: "non-clusterLoadAssignment resource type",
resource: &anypb.Any{TypeUrl: version.V3HTTPConnManagerURL},
wantErr: true,
},
{
name: "badly marshaled clusterLoadAssignment resource",
resources: []*anypb.Any{
{
TypeUrl: version.V3EndpointsURL,
Value: []byte{1, 2, 3, 4},
},
},
wantMD: UpdateMetadata{
Status: ServiceStatusNACKed,
Version: testVersion,
ErrState: &UpdateErrorMetadata{
Version: testVersion,
Err: cmpopts.AnyError,
},
resource: &anypb.Any{
TypeUrl: version.V3EndpointsURL,
Value: []byte{1, 2, 3, 4},
},
wantErr: true,
},
{
name: "bad endpoints resource",
resources: []*anypb.Any{testutils.MarshalAny(func() *v3endpointpb.ClusterLoadAssignment {
resource: testutils.MarshalAny(func() *v3endpointpb.ClusterLoadAssignment {
clab0 := newClaBuilder("test", nil)
clab0.addLocality("locality-1", 1, 0, []string{"addr1:314"}, nil)
clab0.addLocality("locality-2", 1, 2, []string{"addr2:159"}, nil)
return clab0.Build()
}())},
wantUpdate: map[string]EndpointsUpdateErrTuple{"test": {Err: cmpopts.AnyError}},
wantMD: UpdateMetadata{
Status: ServiceStatusNACKed,
Version: testVersion,
ErrState: &UpdateErrorMetadata{
Version: testVersion,
Err: cmpopts.AnyError,
}()),
wantName: "test",
wantErr: true,
},
{
name: "v3 endpoints",
resource: v3EndpointsAny,
wantName: "test",
wantUpdate: EndpointsUpdate{
Drops: nil,
Localities: []Locality{
{
Endpoints: []Endpoint{{
Address: "addr1:314",
HealthStatus: EndpointHealthStatusUnhealthy,
Weight: 271,
}},
ID: internal.LocalityID{SubZone: "locality-1"},
Priority: 1,
Weight: 1,
},
{
Endpoints: []Endpoint{{
Address: "addr2:159",
HealthStatus: EndpointHealthStatusDraining,
Weight: 828,
}},
ID: internal.LocalityID{SubZone: "locality-2"},
Priority: 0,
Weight: 1,
},
},
},
wantErr: true,
},
{
name: "v3 endpoints",
resources: []*anypb.Any{v3EndpointsAny},
wantUpdate: map[string]EndpointsUpdateErrTuple{
"test": {Update: EndpointsUpdate{
Drops: nil,
Localities: []Locality{
{
Endpoints: []Endpoint{{
Address: "addr1:314",
HealthStatus: EndpointHealthStatusUnhealthy,
Weight: 271,
}},
ID: internal.LocalityID{SubZone: "locality-1"},
Priority: 1,
Weight: 1,
},
{
Endpoints: []Endpoint{{
Address: "addr2:159",
HealthStatus: EndpointHealthStatusDraining,
Weight: 828,
}},
ID: internal.LocalityID{SubZone: "locality-2"},
Priority: 0,
Weight: 1,
},
},
Raw: v3EndpointsAny,
}},
},
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
Raw: v3EndpointsAny,
},
},
{
name: "v3 endpoints wrapped",
resources: []*anypb.Any{testutils.MarshalAny(&v3discoverypb.Resource{Resource: v3EndpointsAny})},
wantUpdate: map[string]EndpointsUpdateErrTuple{
"test": {Update: EndpointsUpdate{
Drops: nil,
Localities: []Locality{
{
Endpoints: []Endpoint{{
Address: "addr1:314",
HealthStatus: EndpointHealthStatusUnhealthy,
Weight: 271,
}},
ID: internal.LocalityID{SubZone: "locality-1"},
Priority: 1,
Weight: 1,
},
{
Endpoints: []Endpoint{{
Address: "addr2:159",
HealthStatus: EndpointHealthStatusDraining,
Weight: 828,
}},
ID: internal.LocalityID{SubZone: "locality-2"},
Priority: 0,
Weight: 1,
},
name: "v3 endpoints wrapped",
resource: testutils.MarshalAny(&v3discoverypb.Resource{Resource: v3EndpointsAny}),
wantName: "test",
wantUpdate: EndpointsUpdate{
Drops: nil,
Localities: []Locality{
{
Endpoints: []Endpoint{{
Address: "addr1:314",
HealthStatus: EndpointHealthStatusUnhealthy,
Weight: 271,
}},
ID: internal.LocalityID{SubZone: "locality-1"},
Priority: 1,
Weight: 1,
},
Raw: v3EndpointsAny,
}},
},
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
},
},
{
// To test that unmarshal keeps processing on errors.
name: "good and bad endpoints",
resources: []*anypb.Any{
v3EndpointsAny,
testutils.MarshalAny(func() *v3endpointpb.ClusterLoadAssignment {
clab0 := newClaBuilder("bad", nil)
clab0.addLocality("locality-1", 1, 0, []string{"addr1:314"}, nil)
clab0.addLocality("locality-2", 1, 2, []string{"addr2:159"}, nil)
return clab0.Build()
}()),
},
wantUpdate: map[string]EndpointsUpdateErrTuple{
"test": {Update: EndpointsUpdate{
Drops: nil,
Localities: []Locality{
{
Endpoints: []Endpoint{{
Address: "addr1:314",
HealthStatus: EndpointHealthStatusUnhealthy,
Weight: 271,
}},
ID: internal.LocalityID{SubZone: "locality-1"},
Priority: 1,
Weight: 1,
},
{
Endpoints: []Endpoint{{
Address: "addr2:159",
HealthStatus: EndpointHealthStatusDraining,
Weight: 828,
}},
ID: internal.LocalityID{SubZone: "locality-2"},
Priority: 0,
Weight: 1,
},
{
Endpoints: []Endpoint{{
Address: "addr2:159",
HealthStatus: EndpointHealthStatusDraining,
Weight: 828,
}},
ID: internal.LocalityID{SubZone: "locality-2"},
Priority: 0,
Weight: 1,
},
Raw: v3EndpointsAny,
}},
"bad": {Err: cmpopts.AnyError},
},
wantMD: UpdateMetadata{
Status: ServiceStatusNACKed,
Version: testVersion,
ErrState: &UpdateErrorMetadata{
Version: testVersion,
Err: cmpopts.AnyError,
},
Raw: v3EndpointsAny,
},
wantErr: true,
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
opts := &UnmarshalOptions{
Version: testVersion,
Resources: test.resources,
}
update, md, err := UnmarshalEndpoints(opts)
name, update, err := unmarshalEndpointsResource(test.resource, nil)
if (err != nil) != test.wantErr {
t.Fatalf("UnmarshalEndpoints(%+v), got err: %v, wantErr: %v", opts, err, test.wantErr)
t.Fatalf("unmarshalEndpointsResource(%s), got err: %v, wantErr: %v", pretty.ToJSON(test.resource), err, test.wantErr)
}
if name != test.wantName {
t.Errorf("unmarshalEndpointsResource(%s), got name: %s, want: %s", pretty.ToJSON(test.resource), name, test.wantName)
}
if diff := cmp.Diff(update, test.wantUpdate, cmpOpts); diff != "" {
t.Errorf("got unexpected update, diff (-got +want): %v", diff)
}
if diff := cmp.Diff(md, test.wantMD, cmpOptsIgnoreDetails); diff != "" {
t.Errorf("got unexpected metadata, diff (-got +want): %v", diff)
t.Errorf("unmarshalEndpointsResource(%s), got unexpected update, diff (-got +want): %v", pretty.ToJSON(test.resource), diff)
}
})
}

View File

@ -36,16 +36,7 @@ import (
"google.golang.org/protobuf/types/known/anypb"
)
// UnmarshalListener processes resources received in an LDS response, validates
// them, and transforms them into a native struct which contains only fields we
// are interested in.
func UnmarshalListener(opts *UnmarshalOptions) (map[string]ListenerUpdateErrTuple, UpdateMetadata, error) {
update := make(map[string]ListenerUpdateErrTuple)
md, err := processAllResources(opts, update)
return update, md, err
}
func unmarshalListenerResource(r *anypb.Any, f UpdateValidatorFunc, logger *grpclog.PrefixLogger) (string, ListenerUpdate, error) {
func unmarshalListenerResource(r *anypb.Any, logger *grpclog.PrefixLogger) (string, ListenerUpdate, error) {
r, err := unwrapResource(r)
if err != nil {
return "", ListenerUpdate{}, fmt.Errorf("failed to unwrap resource: %v", err)
@ -66,11 +57,6 @@ func unmarshalListenerResource(r *anypb.Any, f UpdateValidatorFunc, logger *grpc
if err != nil {
return lis.GetName(), ListenerUpdate{}, err
}
if f != nil {
if err := f(*lu); err != nil {
return lis.GetName(), ListenerUpdate{}, err
}
}
lu.Raw = r
return lis.GetName(), *lu, nil
}

File diff suppressed because it is too large Load Diff

View File

@ -35,16 +35,6 @@ import (
"google.golang.org/protobuf/types/known/anypb"
)
// UnmarshalRouteConfig processes resources received in an RDS response,
// validates them, and transforms them into a native struct which contains only
// fields we are interested in. The provided hostname determines the route
// configuration resources of interest.
func UnmarshalRouteConfig(opts *UnmarshalOptions) (map[string]RouteConfigUpdateErrTuple, UpdateMetadata, error) {
update := make(map[string]RouteConfigUpdateErrTuple)
md, err := processAllResources(opts, update)
return update, md, err
}
func unmarshalRouteConfigResource(r *anypb.Any, logger *grpclog.PrefixLogger) (string, RouteConfigUpdate, error) {
r, err := unwrapResource(r)
if err != nil {

View File

@ -30,6 +30,7 @@ import (
"github.com/google/go-cmp/cmp/cmpopts"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/internal/envconfig"
"google.golang.org/grpc/internal/pretty"
"google.golang.org/grpc/internal/testutils"
"google.golang.org/grpc/xds/internal/clusterspecifier"
"google.golang.org/grpc/xds/internal/httpfilter"
@ -907,282 +908,127 @@ func (s) TestUnmarshalRouteConfig(t *testing.T) {
VirtualHosts: v3VirtualHost,
})
)
const testVersion = "test-version-rds"
tests := []struct {
name string
resources []*anypb.Any
wantUpdate map[string]RouteConfigUpdateErrTuple
wantMD UpdateMetadata
resource *anypb.Any
wantName string
wantUpdate RouteConfigUpdate
wantErr bool
}{
{
name: "non-routeConfig resource type",
resources: []*anypb.Any{{TypeUrl: version.V3HTTPConnManagerURL}},
wantMD: UpdateMetadata{
Status: ServiceStatusNACKed,
Version: testVersion,
ErrState: &UpdateErrorMetadata{
Version: testVersion,
Err: cmpopts.AnyError,
},
},
wantErr: true,
name: "non-routeConfig resource type",
resource: &anypb.Any{TypeUrl: version.V3HTTPConnManagerURL},
wantErr: true,
},
{
name: "badly marshaled routeconfig resource",
resources: []*anypb.Any{
{
TypeUrl: version.V3RouteConfigURL,
Value: []byte{1, 2, 3, 4},
},
},
wantMD: UpdateMetadata{
Status: ServiceStatusNACKed,
Version: testVersion,
ErrState: &UpdateErrorMetadata{
Version: testVersion,
Err: cmpopts.AnyError,
},
resource: &anypb.Any{
TypeUrl: version.V3RouteConfigURL,
Value: []byte{1, 2, 3, 4},
},
wantErr: true,
},
{
name: "empty resource list",
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
},
},
{
name: "v2 routeConfig resource",
resources: []*anypb.Any{v2RouteConfig},
wantUpdate: map[string]RouteConfigUpdateErrTuple{
v2RouteConfigName: {Update: RouteConfigUpdate{
VirtualHosts: []*VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
{
Domains: []string{ldsTarget},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{v2ClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
name: "v2 routeConfig resource",
resource: v2RouteConfig,
wantName: v2RouteConfigName,
wantUpdate: RouteConfigUpdate{
VirtualHosts: []*VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
Raw: v2RouteConfig,
}},
},
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
},
},
{
name: "v2 routeConfig resource wrapped",
resources: []*anypb.Any{testutils.MarshalAny(&v2xdspb.Resource{Resource: v2RouteConfig})},
wantUpdate: map[string]RouteConfigUpdateErrTuple{
v2RouteConfigName: {Update: RouteConfigUpdate{
VirtualHosts: []*VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
{
Domains: []string{ldsTarget},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{v2ClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
{
Domains: []string{ldsTarget},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{v2ClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
Raw: v2RouteConfig,
}},
},
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
},
},
{
name: "v3 routeConfig resource",
resources: []*anypb.Any{v3RouteConfig},
wantUpdate: map[string]RouteConfigUpdateErrTuple{
v3RouteConfigName: {Update: RouteConfigUpdate{
VirtualHosts: []*VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
{
Domains: []string{ldsTarget},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{v3ClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
},
Raw: v3RouteConfig,
}},
},
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
},
},
{
name: "v3 routeConfig resource wrapped",
resources: []*anypb.Any{testutils.MarshalAny(&v3discoverypb.Resource{Resource: v3RouteConfig})},
wantUpdate: map[string]RouteConfigUpdateErrTuple{
v3RouteConfigName: {Update: RouteConfigUpdate{
VirtualHosts: []*VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
{
Domains: []string{ldsTarget},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{v3ClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
},
Raw: v3RouteConfig,
}},
},
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
},
},
{
name: "multiple routeConfig resources",
resources: []*anypb.Any{v2RouteConfig, v3RouteConfig},
wantUpdate: map[string]RouteConfigUpdateErrTuple{
v3RouteConfigName: {Update: RouteConfigUpdate{
VirtualHosts: []*VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
{
Domains: []string{ldsTarget},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{v3ClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
},
Raw: v3RouteConfig,
}},
v2RouteConfigName: {Update: RouteConfigUpdate{
VirtualHosts: []*VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
{
Domains: []string{ldsTarget},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{v2ClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
},
Raw: v2RouteConfig,
}},
},
wantMD: UpdateMetadata{
Status: ServiceStatusACKed,
Version: testVersion,
},
},
{
// To test that unmarshal keeps processing on errors.
name: "good and bad routeConfig resources",
resources: []*anypb.Any{
v2RouteConfig,
testutils.MarshalAny(&v3routepb.RouteConfiguration{
Name: "bad",
VirtualHosts: []*v3routepb.VirtualHost{
{Domains: []string{ldsTarget},
Routes: []*v3routepb.Route{{
Match: &v3routepb.RouteMatch{PathSpecifier: &v3routepb.RouteMatch_ConnectMatcher_{}},
}}}}}),
v3RouteConfig,
},
wantUpdate: map[string]RouteConfigUpdateErrTuple{
v3RouteConfigName: {Update: RouteConfigUpdate{
VirtualHosts: []*VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
{
Domains: []string{ldsTarget},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{v3ClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
},
Raw: v3RouteConfig,
}},
v2RouteConfigName: {Update: RouteConfigUpdate{
VirtualHosts: []*VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
{
Domains: []string{ldsTarget},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{v2ClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
},
Raw: v2RouteConfig,
}},
"bad": {Err: cmpopts.AnyError},
},
wantMD: UpdateMetadata{
Status: ServiceStatusNACKed,
Version: testVersion,
ErrState: &UpdateErrorMetadata{
Version: testVersion,
Err: cmpopts.AnyError,
},
Raw: v2RouteConfig,
},
},
{
name: "v2 routeConfig resource wrapped",
resource: testutils.MarshalAny(&v2xdspb.Resource{Resource: v2RouteConfig}),
wantName: v2RouteConfigName,
wantUpdate: RouteConfigUpdate{
VirtualHosts: []*VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
{
Domains: []string{ldsTarget},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{v2ClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
},
Raw: v2RouteConfig,
},
},
{
name: "v3 routeConfig resource",
resource: v3RouteConfig,
wantName: v3RouteConfigName,
wantUpdate: RouteConfigUpdate{
VirtualHosts: []*VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
{
Domains: []string{ldsTarget},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{v3ClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
},
Raw: v3RouteConfig,
},
},
{
name: "v3 routeConfig resource wrapped",
resource: testutils.MarshalAny(&v3discoverypb.Resource{Resource: v3RouteConfig}),
wantName: v3RouteConfigName,
wantUpdate: RouteConfigUpdate{
VirtualHosts: []*VirtualHost{
{
Domains: []string{uninterestingDomain},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{uninterestingClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
{
Domains: []string{ldsTarget},
Routes: []*Route{{Prefix: newStringP(""),
WeightedClusters: map[string]WeightedCluster{v3ClusterName: {Weight: 1}},
ActionType: RouteActionRoute}},
},
},
Raw: v3RouteConfig,
},
wantErr: true,
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
opts := &UnmarshalOptions{
Version: testVersion,
Resources: test.resources,
}
update, md, err := UnmarshalRouteConfig(opts)
name, update, err := unmarshalRouteConfigResource(test.resource, nil)
if (err != nil) != test.wantErr {
t.Fatalf("UnmarshalRouteConfig(%+v), got err: %v, wantErr: %v", opts, err, test.wantErr)
t.Errorf("unmarshalRouteConfigResource(%s), got err: %v, wantErr: %v", pretty.ToJSON(test.resource), err, test.wantErr)
}
if name != test.wantName {
t.Errorf("unmarshalRouteConfigResource(%s), got name: %s, want: %s", pretty.ToJSON(test.resource), name, test.wantName)
}
if diff := cmp.Diff(update, test.wantUpdate, cmpOpts); diff != "" {
t.Errorf("got unexpected update, diff (-got +want): %v", diff)
}
if diff := cmp.Diff(md, test.wantMD, cmpOptsIgnoreDetails); diff != "" {
t.Errorf("got unexpected metadata, diff (-got +want): %v", diff)
t.Errorf("unmarshalRouteConfigResource(%s), got unexpected update, diff (-got +want): %v", pretty.ToJSON(test.resource), diff)
}
})
}

View File

@ -30,21 +30,21 @@ package xds
import (
"fmt"
v3statusgrpc "github.com/envoyproxy/go-control-plane/envoy/service/status/v3"
"google.golang.org/grpc"
_ "google.golang.org/grpc/credentials/tls/certprovider/pemfile" // Register the file watcher certificate provider plugin.
"google.golang.org/grpc/internal"
internaladmin "google.golang.org/grpc/internal/admin"
"google.golang.org/grpc/resolver"
"google.golang.org/grpc/xds/csds"
_ "google.golang.org/grpc/xds/internal/balancer" // Register the balancers.
_ "google.golang.org/grpc/xds/internal/clusterspecifier/rls" // Register the RLS cluster specifier plugin. Note that this does not register the RLS LB policy.
_ "google.golang.org/grpc/xds/internal/httpfilter/fault" // Register the fault injection filter.
_ "google.golang.org/grpc/xds/internal/httpfilter/rbac" // Register the RBAC filter.
_ "google.golang.org/grpc/xds/internal/httpfilter/router" // Register the router filter.
_ "google.golang.org/grpc/xds/internal/resolver" // Register the xds_resolver
_ "google.golang.org/grpc/xds/internal/xdsclient/controller/version/v2" // Register the v2 xDS API client.
_ "google.golang.org/grpc/xds/internal/xdsclient/controller/version/v3" // Register the v3 xDS API client.
_ "google.golang.org/grpc/credentials/tls/certprovider/pemfile" // Register the file watcher certificate provider plugin.
_ "google.golang.org/grpc/xds/internal/balancer" // Register the balancers.
_ "google.golang.org/grpc/xds/internal/clusterspecifier/rls" // Register the RLS cluster specifier plugin. Note that this does not register the RLS LB policy.
_ "google.golang.org/grpc/xds/internal/httpfilter/fault" // Register the fault injection filter.
_ "google.golang.org/grpc/xds/internal/httpfilter/rbac" // Register the RBAC filter.
_ "google.golang.org/grpc/xds/internal/httpfilter/router" // Register the router filter.
_ "google.golang.org/grpc/xds/internal/resolver" // Register the xds_resolver
v3statusgrpc "github.com/envoyproxy/go-control-plane/envoy/service/status/v3"
)
func init() {