From 08479c5e2ecb7d30f5210fc74127179745ffca46 Mon Sep 17 00:00:00 2001 From: Easwar Swaminathan Date: Wed, 21 Dec 2022 13:53:03 -0800 Subject: [PATCH] xdsclient: resource agnostic API implementation (#5776) --- xds/csds/csds.go | 128 +- xds/csds/csds_e2e_test.go | 434 ++++++ xds/csds/csds_test.go | 518 +------ .../clusterresolver/clusterresolver_test.go | 2 - .../clusterresolver/e2e_test/eds_impl_test.go | 3 +- xds/internal/httpfilter/fault/fault_test.go | 5 +- xds/internal/xdsclient/authority.go | 452 +++++- xds/internal/xdsclient/client.go | 11 +- xds/internal/xdsclient/client_new.go | 3 +- .../xdsclient/clientimpl_authority.go | 54 +- xds/internal/xdsclient/clientimpl_dump.go | 50 +- .../xdsclient/clientimpl_loadreport.go | 2 +- .../xdsclient/clientimpl_validator.go | 67 - xds/internal/xdsclient/clientimpl_watchers.go | 196 ++- .../xdsclient/controller/controller.go | 201 --- .../xdsclient/controller/controller_test.go | 154 -- .../xdsclient/controller/loadreport.go | 145 -- .../xdsclient/controller/transport.go | 443 ------ .../xdsclient/controller/v2_ack_test.go | 483 ------ .../xdsclient/controller/v2_cds_test.go | 186 --- .../xdsclient/controller/v2_client_test.go | 212 --- .../xdsclient/controller/v2_eds_test.go | 200 --- .../xdsclient/controller/v2_lds_test.go | 198 --- .../xdsclient/controller/v2_rds_test.go | 203 --- .../xdsclient/controller/v2_testutils_test.go | 470 ------ .../xdsclient/controller/version/v2/client.go | 155 -- .../controller/version/v2/loadreport.go | 170 --- .../xdsclient/controller/version/v3/client.go | 157 -- .../controller/version/v3/loadreport.go | 169 --- .../xdsclient/controller/version/version.go | 123 -- xds/internal/xdsclient/dump_test.go | 511 ------- xds/internal/xdsclient/e2e_test/dump_test.go | 255 ++++ .../xdsclient/e2e_test/misc_watchers_test.go | 7 +- .../e2e_test/resource_update_test.go | 1161 +++++++++++++++ xds/internal/xdsclient/loadreport_test.go | 27 +- xds/internal/xdsclient/pubsub/dump.go | 87 -- xds/internal/xdsclient/pubsub/interface.go | 39 - xds/internal/xdsclient/pubsub/pubsub.go | 186 --- xds/internal/xdsclient/pubsub/update.go | 318 ---- xds/internal/xdsclient/pubsub/watch.go | 239 --- xds/internal/xdsclient/watchers_test.go | 47 - xds/internal/xdsclient/xdsclient_test.go | 1 - .../xdsresource/cluster_resource_type.go | 6 +- .../xdsresource/endpoints_resource_type.go | 3 +- .../xdsresource/listener_resource_type.go | 5 +- .../xdsclient/xdsresource/resource_type.go | 18 +- .../xdsresource/route_config_resource_type.go | 3 +- .../xdsclient/xdsresource/test_utils_test.go | 7 - xds/internal/xdsclient/xdsresource/type.go | 54 + .../xdsclient/xdsresource/unmarshal.go | 114 +- .../xdsclient/xdsresource/unmarshal_cds.go | 16 +- .../xdsresource/unmarshal_cds_test.go | 296 ++-- .../xdsclient/xdsresource/unmarshal_eds.go | 9 - .../xdsresource/unmarshal_eds_test.go | 240 +-- .../xdsclient/xdsresource/unmarshal_lds.go | 16 +- .../xdsresource/unmarshal_lds_test.go | 1303 +++++++---------- .../xdsclient/xdsresource/unmarshal_rds.go | 10 - .../xdsresource/unmarshal_rds_test.go | 350 ++--- xds/xds.go | 20 +- 59 files changed, 3379 insertions(+), 7563 deletions(-) create mode 100644 xds/csds/csds_e2e_test.go delete mode 100644 xds/internal/xdsclient/clientimpl_validator.go delete mode 100644 xds/internal/xdsclient/controller/controller.go delete mode 100644 xds/internal/xdsclient/controller/controller_test.go delete mode 100644 xds/internal/xdsclient/controller/loadreport.go delete mode 100644 xds/internal/xdsclient/controller/transport.go delete mode 100644 xds/internal/xdsclient/controller/v2_ack_test.go delete mode 100644 xds/internal/xdsclient/controller/v2_cds_test.go delete mode 100644 xds/internal/xdsclient/controller/v2_client_test.go delete mode 100644 xds/internal/xdsclient/controller/v2_eds_test.go delete mode 100644 xds/internal/xdsclient/controller/v2_lds_test.go delete mode 100644 xds/internal/xdsclient/controller/v2_rds_test.go delete mode 100644 xds/internal/xdsclient/controller/v2_testutils_test.go delete mode 100644 xds/internal/xdsclient/controller/version/v2/client.go delete mode 100644 xds/internal/xdsclient/controller/version/v2/loadreport.go delete mode 100644 xds/internal/xdsclient/controller/version/v3/client.go delete mode 100644 xds/internal/xdsclient/controller/version/v3/loadreport.go delete mode 100644 xds/internal/xdsclient/controller/version/version.go delete mode 100644 xds/internal/xdsclient/dump_test.go create mode 100644 xds/internal/xdsclient/e2e_test/dump_test.go create mode 100644 xds/internal/xdsclient/e2e_test/resource_update_test.go delete mode 100644 xds/internal/xdsclient/pubsub/dump.go delete mode 100644 xds/internal/xdsclient/pubsub/interface.go delete mode 100644 xds/internal/xdsclient/pubsub/pubsub.go delete mode 100644 xds/internal/xdsclient/pubsub/update.go delete mode 100644 xds/internal/xdsclient/pubsub/watch.go delete mode 100644 xds/internal/xdsclient/watchers_test.go diff --git a/xds/csds/csds.go b/xds/csds/csds.go index 0d71f8f85..150397932 100644 --- a/xds/csds/csds.go +++ b/xds/csds/csds.go @@ -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 } diff --git a/xds/csds/csds_e2e_test.go b/xds/csds/csds_e2e_test.go new file mode 100644 index 000000000..55ae77274 --- /dev/null +++ b/xds/csds/csds_e2e_test.go @@ -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)) + } +} diff --git a/xds/csds/csds_test.go b/xds/csds/csds_test.go index 6772389c6..48002b850 100644 --- a/xds/csds/csds_test.go +++ b/xds/csds/csds_test.go @@ -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) } diff --git a/xds/internal/balancer/clusterresolver/clusterresolver_test.go b/xds/internal/balancer/clusterresolver/clusterresolver_test.go index c2a5729e3..7a3ced47b 100644 --- a/xds/internal/balancer/clusterresolver/clusterresolver_test.go +++ b/xds/internal/balancer/clusterresolver/clusterresolver_test.go @@ -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 ( diff --git a/xds/internal/balancer/clusterresolver/e2e_test/eds_impl_test.go b/xds/internal/balancer/clusterresolver/e2e_test/eds_impl_test.go index ee6b029d8..75d37e71e 100644 --- a/xds/internal/balancer/clusterresolver/e2e_test/eds_impl_test.go +++ b/xds/internal/balancer/clusterresolver/e2e_test/eds_impl_test.go @@ -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 ( diff --git a/xds/internal/httpfilter/fault/fault_test.go b/xds/internal/httpfilter/fault/fault_test.go index 6ae345552..6bf00771f 100644 --- a/xds/internal/httpfilter/fault/fault_test.go +++ b/xds/internal/httpfilter/fault/fault_test.go @@ -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 diff --git a/xds/internal/xdsclient/authority.go b/xds/internal/xdsclient/authority.go index 817cb7338..7a533e662 100644 --- a/xds/internal/xdsclient/authority.go +++ b/xds/internal/xdsclient/authority.go @@ -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 } diff --git a/xds/internal/xdsclient/client.go b/xds/internal/xdsclient/client.go index 976b86fe3..332b31409 100644 --- a/xds/internal/xdsclient/client.go +++ b/xds/internal/xdsclient/client.go @@ -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() } diff --git a/xds/internal/xdsclient/client_new.go b/xds/internal/xdsclient/client_new.go index 4b2b8cfd3..b3aecb7fa 100644 --- a/xds/internal/xdsclient/client_new.go +++ b/xds/internal/xdsclient/client_new.go @@ -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 } diff --git a/xds/internal/xdsclient/clientimpl_authority.go b/xds/internal/xdsclient/clientimpl_authority.go index 623420ccc..2531b3947 100644 --- a/xds/internal/xdsclient/clientimpl_authority.go +++ b/xds/internal/xdsclient/clientimpl_authority.go @@ -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() diff --git a/xds/internal/xdsclient/clientimpl_dump.go b/xds/internal/xdsclient/clientimpl_dump.go index 69407d20c..b9d049930 100644 --- a/xds/internal/xdsclient/clientimpl_dump.go +++ b/xds/internal/xdsclient/clientimpl_dump.go @@ -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 } diff --git a/xds/internal/xdsclient/clientimpl_loadreport.go b/xds/internal/xdsclient/clientimpl_loadreport.go index cba5afd45..dd0ae225e 100644 --- a/xds/internal/xdsclient/clientimpl_loadreport.go +++ b/xds/internal/xdsclient/clientimpl_loadreport.go @@ -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() diff --git a/xds/internal/xdsclient/clientimpl_validator.go b/xds/internal/xdsclient/clientimpl_validator.go deleted file mode 100644 index 50bdbe4e2..000000000 --- a/xds/internal/xdsclient/clientimpl_validator.go +++ /dev/null @@ -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 -} diff --git a/xds/internal/xdsclient/clientimpl_watchers.go b/xds/internal/xdsclient/clientimpl_watchers.go index 7095a8394..05eea38f4 100644 --- a/xds/internal/xdsclient/clientimpl_watchers.go +++ b/xds/internal/xdsclient/clientimpl_watchers.go @@ -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 } diff --git a/xds/internal/xdsclient/controller/controller.go b/xds/internal/xdsclient/controller/controller.go deleted file mode 100644 index 520da06a1..000000000 --- a/xds/internal/xdsclient/controller/controller.go +++ /dev/null @@ -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 - } -} diff --git a/xds/internal/xdsclient/controller/controller_test.go b/xds/internal/xdsclient/controller/controller_test.go deleted file mode 100644 index 599afb3a3..000000000 --- a/xds/internal/xdsclient/controller/controller_test.go +++ /dev/null @@ -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) - } -} diff --git a/xds/internal/xdsclient/controller/loadreport.go b/xds/internal/xdsclient/controller/loadreport.go deleted file mode 100644 index a28cc95dc..000000000 --- a/xds/internal/xdsclient/controller/loadreport.go +++ /dev/null @@ -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}) -} diff --git a/xds/internal/xdsclient/controller/transport.go b/xds/internal/xdsclient/controller/transport.go deleted file mode 100644 index 526aefae2..000000000 --- a/xds/internal/xdsclient/controller/transport.go +++ /dev/null @@ -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 client’s - // 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 - } - } -} diff --git a/xds/internal/xdsclient/controller/v2_ack_test.go b/xds/internal/xdsclient/controller/v2_ack_test.go deleted file mode 100644 index 6680de791..000000000 --- a/xds/internal/xdsclient/controller/v2_ack_test.go +++ /dev/null @@ -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++ -} diff --git a/xds/internal/xdsclient/controller/v2_cds_test.go b/xds/internal/xdsclient/controller/v2_cds_test.go deleted file mode 100644 index d262b53a4..000000000 --- a/xds/internal/xdsclient/controller/v2_cds_test.go +++ /dev/null @@ -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") - } -} diff --git a/xds/internal/xdsclient/controller/v2_client_test.go b/xds/internal/xdsclient/controller/v2_client_test.go deleted file mode 100644 index 942f18649..000000000 --- a/xds/internal/xdsclient/controller/v2_client_test.go +++ /dev/null @@ -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) - } -} diff --git a/xds/internal/xdsclient/controller/v2_eds_test.go b/xds/internal/xdsclient/controller/v2_eds_test.go deleted file mode 100644 index 665fee92a..000000000 --- a/xds/internal/xdsclient/controller/v2_eds_test.go +++ /dev/null @@ -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") - } -} diff --git a/xds/internal/xdsclient/controller/v2_lds_test.go b/xds/internal/xdsclient/controller/v2_lds_test.go deleted file mode 100644 index 56b292988..000000000 --- a/xds/internal/xdsclient/controller/v2_lds_test.go +++ /dev/null @@ -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") - } -} diff --git a/xds/internal/xdsclient/controller/v2_rds_test.go b/xds/internal/xdsclient/controller/v2_rds_test.go deleted file mode 100644 index 476df7109..000000000 --- a/xds/internal/xdsclient/controller/v2_rds_test.go +++ /dev/null @@ -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") - } -} diff --git a/xds/internal/xdsclient/controller/v2_testutils_test.go b/xds/internal/xdsclient/controller/v2_testutils_test.go deleted file mode 100644 index de147d480..000000000 --- a/xds/internal/xdsclient/controller/v2_testutils_test.go +++ /dev/null @@ -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 -} diff --git a/xds/internal/xdsclient/controller/version/v2/client.go b/xds/internal/xdsclient/controller/version/v2/client.go deleted file mode 100644 index 968947b06..000000000 --- a/xds/internal/xdsclient/controller/version/v2/client.go +++ /dev/null @@ -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 -} diff --git a/xds/internal/xdsclient/controller/version/v2/loadreport.go b/xds/internal/xdsclient/controller/version/v2/loadreport.go deleted file mode 100644 index da5128ac4..000000000 --- a/xds/internal/xdsclient/controller/version/v2/loadreport.go +++ /dev/null @@ -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 - } - } -} diff --git a/xds/internal/xdsclient/controller/version/v3/client.go b/xds/internal/xdsclient/controller/version/v3/client.go deleted file mode 100644 index 4cacd94dd..000000000 --- a/xds/internal/xdsclient/controller/version/v3/client.go +++ /dev/null @@ -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 -} diff --git a/xds/internal/xdsclient/controller/version/v3/loadreport.go b/xds/internal/xdsclient/controller/version/v3/loadreport.go deleted file mode 100644 index f8d866bb1..000000000 --- a/xds/internal/xdsclient/controller/version/v3/loadreport.go +++ /dev/null @@ -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 - } - } -} diff --git a/xds/internal/xdsclient/controller/version/version.go b/xds/internal/xdsclient/controller/version/version.go deleted file mode 100644 index f79a21e29..000000000 --- a/xds/internal/xdsclient/controller/version/version.go +++ /dev/null @@ -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 -} diff --git a/xds/internal/xdsclient/dump_test.go b/xds/internal/xdsclient/dump_test.go deleted file mode 100644 index 165d608d3..000000000 --- a/xds/internal/xdsclient/dump_test.go +++ /dev/null @@ -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 -} diff --git a/xds/internal/xdsclient/e2e_test/dump_test.go b/xds/internal/xdsclient/e2e_test/dump_test.go new file mode 100644 index 000000000..1a2765cc2 --- /dev/null +++ b/xds/internal/xdsclient/e2e_test/dump_test.go @@ -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) + } +} diff --git a/xds/internal/xdsclient/e2e_test/misc_watchers_test.go b/xds/internal/xdsclient/e2e_test/misc_watchers_test.go index 05e7083cd..42544aa53 100644 --- a/xds/internal/xdsclient/e2e_test/misc_watchers_test.go +++ b/xds/internal/xdsclient/e2e_test/misc_watchers_test.go @@ -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) }) diff --git a/xds/internal/xdsclient/e2e_test/resource_update_test.go b/xds/internal/xdsclient/e2e_test/resource_update_test.go new file mode 100644 index 000000000..7294b40f9 --- /dev/null +++ b/xds/internal/xdsclient/e2e_test/resource_update_test.go @@ -0,0 +1,1161 @@ +/* + * + * 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" + "strings" + "testing" + "time" + + "github.com/google/go-cmp/cmp" + "github.com/google/go-cmp/cmp/cmpopts" + "github.com/google/uuid" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/internal/testutils" + "google.golang.org/grpc/internal/testutils/xds/e2e" + "google.golang.org/grpc/xds/internal" + "google.golang.org/grpc/xds/internal/testutils/fakeserver" + "google.golang.org/grpc/xds/internal/xdsclient" + "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" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/testing/protocmp" + "google.golang.org/protobuf/types/known/anypb" + "google.golang.org/protobuf/types/known/wrapperspb" + + 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" + v3httppb "github.com/envoyproxy/go-control-plane/envoy/extensions/filters/network/http_connection_manager/v3" + v3discoverypb "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3" + + _ "google.golang.org/grpc/xds/internal/httpfilter/router" // Register the router filter. +) + +// startFakeManagementServer starts a fake xDS management server and returns a +// cleanup function to close the fake server. +func startFakeManagementServer(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 compareUpdateMetadata(ctx context.Context, dumpFunc func() map[string]xdsresource.UpdateWithMD, want map[string]xdsresource.UpdateWithMD) error { + var lastErr error + for ; ctx.Err() == nil; <-time.After(100 * time.Millisecond) { + cmpOpts := cmp.Options{ + cmpopts.EquateEmpty(), + cmp.Comparer(func(a, b time.Time) bool { return true }), + cmpopts.EquateErrors(), + protocmp.Transform(), + } + gotUpdateMetadata := dumpFunc() + diff := cmp.Diff(want, gotUpdateMetadata, cmpOpts) + if diff == "" { + return nil + } + lastErr = fmt.Errorf("unexpected diff in metadata, diff (-want +got):\n%s\n want: %+v\n got: %+v", diff, want, gotUpdateMetadata) + } + return fmt.Errorf("timeout when waiting for expected update metadata: %v", lastErr) +} + +// TestHandleListenerResponseFromManagementServer covers different scenarios +// involving receipt of an LDS response from the management server. The test +// verifies that the internal state of the xDS client (parsed resource and +// metadata) matches expectations. +func (s) TestHandleListenerResponseFromManagementServer(t *testing.T) { + const ( + resourceName1 = "resource-name-1" + resourceName2 = "resource-name-2" + ) + var ( + emptyRouterFilter = e2e.RouterHTTPFilter + apiListener = &v3listenerpb.ApiListener{ + ApiListener: func() *anypb.Any { + return testutils.MarshalAny(&v3httppb.HttpConnectionManager{ + RouteSpecifier: &v3httppb.HttpConnectionManager_Rds{ + Rds: &v3httppb.Rds{ + ConfigSource: &v3corepb.ConfigSource{ + ConfigSourceSpecifier: &v3corepb.ConfigSource_Ads{Ads: &v3corepb.AggregatedConfigSource{}}, + }, + RouteConfigName: "route-configuration-name", + }, + }, + HttpFilters: []*v3httppb.HttpFilter{emptyRouterFilter}, + }) + }(), + } + resource1 = &v3listenerpb.Listener{ + Name: resourceName1, + ApiListener: apiListener, + } + resource2 = &v3listenerpb.Listener{ + Name: resourceName2, + ApiListener: apiListener, + } + ) + + tests := []struct { + desc string + resourceName string + managementServerResponse *v3discoverypb.DiscoveryResponse + wantUpdate xdsresource.ListenerUpdate + wantErr string + wantUpdateMetadata map[string]xdsresource.UpdateWithMD + }{ + { + desc: "badly-marshaled-response", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.listener.v3.Listener", + VersionInfo: "1", + Resources: []*anypb.Any{{ + TypeUrl: "type.googleapis.com/envoy.config.listener.v3.Listener", + Value: []byte{1, 2, 3, 4}, + }}, + }, + wantErr: "Listener not found in received response", + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusNotExist}}, + }, + }, + { + desc: "empty-response", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.listener.v3.Listener", + VersionInfo: "1", + }, + wantErr: "Listener not found in received response", + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusNotExist}}, + }, + }, + { + desc: "unexpected-type-in-response", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.listener.v3.Listener", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(&v3routepb.RouteConfiguration{})}, + }, + wantErr: "Listener not found in received response", + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusNotExist}}, + }, + }, + { + desc: "one-bad-resource", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.listener.v3.Listener", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + Name: resourceName1, + ApiListener: &v3listenerpb.ApiListener{ + ApiListener: testutils.MarshalAny(&v3httppb.HttpConnectionManager{}), + }}), + }, + }, + wantErr: "no RouteSpecifier", + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{ + Status: xdsresource.ServiceStatusNACKed, + ErrState: &xdsresource.UpdateErrorMetadata{ + Version: "1", + Err: cmpopts.AnyError, + }, + }}, + }, + }, + { + desc: "one-good-resource", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.listener.v3.Listener", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(resource1)}, + }, + wantUpdate: xdsresource.ListenerUpdate{ + RouteConfigName: "route-configuration-name", + HTTPFilters: []xdsresource.HTTPFilter{{Name: "router"}}, + }, + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": { + MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}, + Raw: testutils.MarshalAny(resource1), + }, + }, + }, + { + desc: "two-resources-when-we-requested-one", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.listener.v3.Listener", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(resource1), testutils.MarshalAny(resource2)}, + }, + wantUpdate: xdsresource.ListenerUpdate{ + RouteConfigName: "route-configuration-name", + HTTPFilters: []xdsresource.HTTPFilter{{Name: "router"}}, + }, + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": { + MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}, + Raw: testutils.MarshalAny(resource1), + }, + }, + }, + } + + for _, test := range tests { + t.Run(test.desc, func(t *testing.T) { + // Create a fake xDS management server listening on a local port, + // and set it up with the response to send. + mgmtServer, cleanup := startFakeManagementServer(t) + defer cleanup() + t.Logf("Started xDS management server on %s", mgmtServer.Address) + + // Create an xDS client talking to the above management server. + nodeID := uuid.New().String() + client, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{ + XDSServer: &bootstrap.ServerConfig{ + ServerURI: mgmtServer.Address, + Creds: grpc.WithTransportCredentials(insecure.NewCredentials()), + CredsType: "insecure", + TransportAPI: version.TransportV3, + NodeProto: &v3corepb.Node{Id: nodeID}, + }, + }, defaultTestWatchExpiryTimeout, time.Duration(0)) + if err != nil { + t.Fatalf("failed to create xds client: %v", err) + } + defer client.Close() + t.Logf("Created xDS client to %s", mgmtServer.Address) + + // A wrapper struct to wrap the update and the associated error, as + // received by the resource watch callback. + type updateAndErr struct { + update xdsresource.ListenerUpdate + err error + } + updateAndErrCh := testutils.NewChannel() + + // Register a watch, and push the results on to a channel. + client.WatchListener(test.resourceName, func(update xdsresource.ListenerUpdate, err error) { + updateAndErrCh.Send(updateAndErr{update: update, err: err}) + }) + t.Logf("Registered a watch for Listener %q", test.resourceName) + + // Wait for the discovery request to be sent out. + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + val, err := mgmtServer.XDSRequestChan.Receive(ctx) + if err != nil { + t.Fatalf("Timeout when waiting for discovery request at the management server: %v", ctx) + } + wantReq := &fakeserver.Request{Req: &v3discoverypb.DiscoveryRequest{ + Node: &v3corepb.Node{Id: nodeID}, + ResourceNames: []string{test.resourceName}, + TypeUrl: "type.googleapis.com/envoy.config.listener.v3.Listener", + }} + gotReq := val.(*fakeserver.Request) + if diff := cmp.Diff(gotReq, wantReq, protocmp.Transform()); diff != "" { + t.Fatalf("Discovery request received at management server is %+v, want %+v", gotReq, wantReq) + } + t.Logf("Discovery request received at management server") + + // Configure the fake management server with a response. + mgmtServer.XDSResponseChan <- &fakeserver.Response{Resp: test.managementServerResponse} + + // Wait for an update from the xDS client and compare with expected + // update. + val, err = updateAndErrCh.Receive(ctx) + if err != nil { + t.Fatalf("Timeout when waiting for watch callback to invoked after response from management server: %v", err) + } + gotUpdate := val.(updateAndErr).update + gotErr := val.(updateAndErr).err + if (gotErr != nil) != (test.wantErr != "") { + t.Fatalf("Got error from handling update: %v, want %v", gotErr, test.wantErr) + } + if gotErr != nil && !strings.Contains(gotErr.Error(), test.wantErr) { + t.Fatalf("Got error from handling update: %v, want %v", gotErr, test.wantErr) + } + cmpOpts := []cmp.Option{ + cmpopts.EquateEmpty(), + cmpopts.IgnoreFields(xdsresource.HTTPFilter{}, "Filter", "Config"), + cmpopts.IgnoreFields(xdsresource.ListenerUpdate{}, "Raw"), + } + if diff := cmp.Diff(test.wantUpdate, gotUpdate, cmpOpts...); diff != "" { + t.Fatalf("Unexpected diff in metadata, diff (-want +got):\n%s", diff) + } + if err := compareUpdateMetadata(ctx, func() map[string]xdsresource.UpdateWithMD { + dump := client.DumpResources() + return dump["type.googleapis.com/envoy.config.listener.v3.Listener"] + }, test.wantUpdateMetadata); err != nil { + t.Fatal(err) + } + }) + } +} + +// TestHandleRouteConfigResponseFromManagementServer covers different scenarios +// involving receipt of an RDS response from the management server. The test +// verifies that the internal state of the xDS client (parsed resource and +// metadata) matches expectations. +func (s) TestHandleRouteConfigResponseFromManagementServer(t *testing.T) { + const ( + resourceName1 = "resource-name-1" + resourceName2 = "resource-name-2" + ) + var ( + virtualHosts = []*v3routepb.VirtualHost{ + { + Domains: []string{"lds-target-name"}, + Routes: []*v3routepb.Route{ + { + Match: &v3routepb.RouteMatch{PathSpecifier: &v3routepb.RouteMatch_Prefix{Prefix: ""}}, + Action: &v3routepb.Route_Route{ + Route: &v3routepb.RouteAction{ + ClusterSpecifier: &v3routepb.RouteAction_Cluster{Cluster: "cluster-name"}, + }, + }, + }, + }, + }, + } + resource1 = &v3routepb.RouteConfiguration{ + Name: resourceName1, + VirtualHosts: virtualHosts, + } + resource2 = &v3routepb.RouteConfiguration{ + Name: resourceName2, + VirtualHosts: virtualHosts, + } + ) + + tests := []struct { + desc string + resourceName string + managementServerResponse *v3discoverypb.DiscoveryResponse + wantUpdate xdsresource.RouteConfigUpdate + wantErr string + wantUpdateMetadata map[string]xdsresource.UpdateWithMD + }{ + // The first three tests involve scenarios where the response fails + // protobuf deserialization (because it contains an invalid data or type + // in the anypb.Any) or the requested resource is not present in the + // response. In either case, no resource update makes its way to the + // top-level xDS client. An RDS response without a requested resource + // does not mean that the resource does not exist in the server. It + // could be part of a future update. Therefore, the only failure mode + // for this resource is for the watch to timeout. + { + desc: "badly-marshaled-response", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.route.v3.RouteConfiguration", + VersionInfo: "1", + Resources: []*anypb.Any{{ + TypeUrl: "type.googleapis.com/envoy.config.route.v3.RouteConfiguration", + Value: []byte{1, 2, 3, 4}, + }}, + }, + wantErr: fmt.Sprintf("watch for resource %q of type RouteConfigResource timed out", resourceName1), + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}, + }, + }, + { + desc: "empty-response", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.route.v3.RouteConfiguration", + VersionInfo: "1", + }, + wantErr: fmt.Sprintf("watch for resource %q of type RouteConfigResource timed out", resourceName1), + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}, + }, + }, + { + desc: "unexpected-type-in-response", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.route.v3.RouteConfiguration", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(&v3clusterpb.Cluster{})}, + }, + wantErr: fmt.Sprintf("watch for resource %q of type RouteConfigResource timed out", resourceName1), + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}, + }, + }, + { + desc: "one-bad-resource", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.route.v3.RouteConfiguration", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(&v3routepb.RouteConfiguration{ + Name: resourceName1, + VirtualHosts: []*v3routepb.VirtualHost{{ + Domains: []string{"lds-resource-name"}, + Routes: []*v3routepb.Route{{ + Match: &v3routepb.RouteMatch{PathSpecifier: &v3routepb.RouteMatch_Prefix{Prefix: "/"}}, + Action: &v3routepb.Route_Route{Route: &v3routepb.RouteAction{ + ClusterSpecifier: &v3routepb.RouteAction_Cluster{Cluster: "cluster-resource-name"}, + }}}}, + RetryPolicy: &v3routepb.RetryPolicy{ + NumRetries: &wrapperspb.UInt32Value{Value: 0}, + }, + }}, + })}, + }, + wantErr: "received route is invalid: retry_policy.num_retries = 0; must be >= 1", + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{ + Status: xdsresource.ServiceStatusNACKed, + ErrState: &xdsresource.UpdateErrorMetadata{ + Version: "1", + Err: cmpopts.AnyError, + }, + }}, + }, + }, + { + desc: "one-good-resource", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.route.v3.RouteConfiguration", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(resource1)}, + }, + wantUpdate: xdsresource.RouteConfigUpdate{ + VirtualHosts: []*xdsresource.VirtualHost{ + { + Domains: []string{"lds-target-name"}, + Routes: []*xdsresource.Route{{Prefix: newStringP(""), + WeightedClusters: map[string]xdsresource.WeightedCluster{"cluster-name": {Weight: 1}}, + ActionType: xdsresource.RouteActionRoute}}, + }, + }, + }, + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": { + MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}, + Raw: testutils.MarshalAny(resource1), + }, + }, + }, + { + desc: "two-resources-when-we-requested-one", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.route.v3.RouteConfiguration", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(resource1), testutils.MarshalAny(resource2)}, + }, + wantUpdate: xdsresource.RouteConfigUpdate{ + VirtualHosts: []*xdsresource.VirtualHost{ + { + Domains: []string{"lds-target-name"}, + Routes: []*xdsresource.Route{{Prefix: newStringP(""), + WeightedClusters: map[string]xdsresource.WeightedCluster{"cluster-name": {Weight: 1}}, + ActionType: xdsresource.RouteActionRoute}}, + }, + }, + }, + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": { + MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}, + Raw: testutils.MarshalAny(resource1), + }, + }, + }, + } + for _, test := range tests { + t.Run(test.desc, func(t *testing.T) { + // Create a fake xDS management server listening on a local port, + // and set it up with the response to send. + mgmtServer, cleanup := startFakeManagementServer(t) + defer cleanup() + t.Logf("Started xDS management server on %s", mgmtServer.Address) + + // Create an xDS client talking to the above management server. + nodeID := uuid.New().String() + client, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{ + XDSServer: &bootstrap.ServerConfig{ + ServerURI: mgmtServer.Address, + Creds: grpc.WithTransportCredentials(insecure.NewCredentials()), + CredsType: "insecure", + TransportAPI: version.TransportV3, + NodeProto: &v3corepb.Node{Id: nodeID}, + }, + }, defaultTestWatchExpiryTimeout, time.Duration(0)) + if err != nil { + t.Fatalf("failed to create xds client: %v", err) + } + defer client.Close() + t.Logf("Created xDS client to %s", mgmtServer.Address) + + // A wrapper struct to wrap the update and the associated error, as + // received by the resource watch callback. + type updateAndErr struct { + update xdsresource.RouteConfigUpdate + err error + } + updateAndErrCh := testutils.NewChannel() + + // Register a watch, and push the results on to a channel. + client.WatchRouteConfig(test.resourceName, func(update xdsresource.RouteConfigUpdate, err error) { + updateAndErrCh.Send(updateAndErr{update: update, err: err}) + }) + t.Logf("Registered a watch for Route Configuration %q", test.resourceName) + + // Wait for the discovery request to be sent out. + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + val, err := mgmtServer.XDSRequestChan.Receive(ctx) + if err != nil { + t.Fatalf("Timeout when waiting for discovery request at the management server: %v", ctx) + } + wantReq := &fakeserver.Request{Req: &v3discoverypb.DiscoveryRequest{ + Node: &v3corepb.Node{Id: nodeID}, + ResourceNames: []string{test.resourceName}, + TypeUrl: "type.googleapis.com/envoy.config.route.v3.RouteConfiguration", + }} + gotReq := val.(*fakeserver.Request) + if diff := cmp.Diff(gotReq, wantReq, protocmp.Transform()); diff != "" { + t.Fatalf("Discovery request received at management server is %+v, want %+v", gotReq, wantReq) + } + t.Logf("Discovery request received at management server") + + // Configure the fake management server with a response. + mgmtServer.XDSResponseChan <- &fakeserver.Response{Resp: test.managementServerResponse} + + // Wait for an update from the xDS client and compare with expected + // update. + val, err = updateAndErrCh.Receive(ctx) + if err != nil { + t.Fatalf("Timeout when waiting for watch callback to invoked after response from management server: %v", err) + } + gotUpdate := val.(updateAndErr).update + gotErr := val.(updateAndErr).err + if (gotErr != nil) != (test.wantErr != "") { + t.Fatalf("Got error from handling update: %v, want %v", gotErr, test.wantErr) + } + if gotErr != nil && !strings.Contains(gotErr.Error(), test.wantErr) { + t.Fatalf("Got error from handling update: %v, want %v", gotErr, test.wantErr) + } + cmpOpts := []cmp.Option{ + cmpopts.EquateEmpty(), + cmpopts.IgnoreFields(xdsresource.RouteConfigUpdate{}, "Raw"), + } + if diff := cmp.Diff(test.wantUpdate, gotUpdate, cmpOpts...); diff != "" { + t.Fatalf("Unexpected diff in metadata, diff (-want +got):\n%s", diff) + } + if err := compareUpdateMetadata(ctx, func() map[string]xdsresource.UpdateWithMD { + dump := client.DumpResources() + return dump["type.googleapis.com/envoy.config.route.v3.RouteConfiguration"] + }, test.wantUpdateMetadata); err != nil { + t.Fatal(err) + } + }) + } +} + +// TestHandleClusterResponseFromManagementServer covers different scenarios +// involving receipt of a CDS response from the management server. The test +// verifies that the internal state of the xDS client (parsed resource and +// metadata) matches expectations. +func (s) TestHandleClusterResponseFromManagementServer(t *testing.T) { + const ( + resourceName1 = "resource-name-1" + resourceName2 = "resource-name-2" + ) + resource1 := &v3clusterpb.Cluster{ + Name: resourceName1, + ClusterDiscoveryType: &v3clusterpb.Cluster_Type{Type: v3clusterpb.Cluster_EDS}, + EdsClusterConfig: &v3clusterpb.Cluster_EdsClusterConfig{ + EdsConfig: &v3corepb.ConfigSource{ + ConfigSourceSpecifier: &v3corepb.ConfigSource_Ads{ + Ads: &v3corepb.AggregatedConfigSource{}, + }, + }, + ServiceName: "eds-service-name", + }, + LbPolicy: v3clusterpb.Cluster_ROUND_ROBIN, + LrsServer: &v3corepb.ConfigSource{ + ConfigSourceSpecifier: &v3corepb.ConfigSource_Self{ + Self: &v3corepb.SelfConfigSource{}, + }, + }, + } + resource2 := proto.Clone(resource1).(*v3clusterpb.Cluster) + resource2.Name = resourceName2 + + tests := []struct { + desc string + resourceName string + managementServerResponse *v3discoverypb.DiscoveryResponse + wantUpdate xdsresource.ClusterUpdate + wantErr string + wantUpdateMetadata map[string]xdsresource.UpdateWithMD + }{ + { + desc: "badly-marshaled-response", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.cluster.v3.Cluster", + VersionInfo: "1", + Resources: []*anypb.Any{{ + TypeUrl: "type.googleapis.com/envoy.config.cluster.v3.Cluster", + Value: []byte{1, 2, 3, 4}, + }}, + }, + wantErr: "Cluster not found in received response", + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusNotExist}}, + }, + }, + { + desc: "empty-response", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.cluster.v3.Cluster", + VersionInfo: "1", + }, + wantErr: "Cluster not found in received response", + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusNotExist}}, + }, + }, + { + desc: "unexpected-type-in-response", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.cluster.v3.Cluster", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(&v3endpointpb.ClusterLoadAssignment{})}, + }, + wantErr: "Cluster not found in received response", + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusNotExist}}, + }, + }, + { + desc: "one-bad-resource", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.cluster.v3.Cluster", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(&v3clusterpb.Cluster{ + Name: resourceName1, + ClusterDiscoveryType: &v3clusterpb.Cluster_Type{Type: v3clusterpb.Cluster_EDS}, + EdsClusterConfig: &v3clusterpb.Cluster_EdsClusterConfig{ + EdsConfig: &v3corepb.ConfigSource{ + ConfigSourceSpecifier: &v3corepb.ConfigSource_Ads{ + Ads: &v3corepb.AggregatedConfigSource{}, + }, + }, + ServiceName: "eds-service-name", + }, + LbPolicy: v3clusterpb.Cluster_MAGLEV, + })}, + }, + wantErr: "unexpected lbPolicy MAGLEV", + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{ + Status: xdsresource.ServiceStatusNACKed, + ErrState: &xdsresource.UpdateErrorMetadata{ + Version: "1", + Err: cmpopts.AnyError, + }, + }}, + }, + }, + { + desc: "one-good-resource", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.cluster.v3.Cluster", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(resource1)}, + }, + wantUpdate: xdsresource.ClusterUpdate{ + ClusterName: "resource-name-1", + EDSServiceName: "eds-service-name", + LRSServerConfig: xdsresource.ClusterLRSServerSelf, + }, + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": { + MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}, + Raw: testutils.MarshalAny(resource1), + }, + }, + }, + { + desc: "two-resources-when-we-requested-one", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.cluster.v3.Cluster", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(resource1), testutils.MarshalAny(resource2)}, + }, + wantUpdate: xdsresource.ClusterUpdate{ + ClusterName: "resource-name-1", + EDSServiceName: "eds-service-name", + LRSServerConfig: xdsresource.ClusterLRSServerSelf, + }, + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": { + MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}, + Raw: testutils.MarshalAny(resource1), + }, + }, + }, + } + + for _, test := range tests { + t.Run(test.desc, func(t *testing.T) { + // Create a fake xDS management server listening on a local port, + // and set it up with the response to send. + mgmtServer, cleanup := startFakeManagementServer(t) + defer cleanup() + t.Logf("Started xDS management server on %s", mgmtServer.Address) + + // Create an xDS client talking to the above management server. + nodeID := uuid.New().String() + client, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{ + XDSServer: &bootstrap.ServerConfig{ + ServerURI: mgmtServer.Address, + Creds: grpc.WithTransportCredentials(insecure.NewCredentials()), + CredsType: "insecure", + TransportAPI: version.TransportV3, + NodeProto: &v3corepb.Node{Id: nodeID}, + }, + }, defaultTestWatchExpiryTimeout, time.Duration(0)) + if err != nil { + t.Fatalf("failed to create xds client: %v", err) + } + defer client.Close() + t.Logf("Created xDS client to %s", mgmtServer.Address) + + // A wrapper struct to wrap the update and the associated error, as + // received by the resource watch callback. + type updateAndErr struct { + update xdsresource.ClusterUpdate + err error + } + updateAndErrCh := testutils.NewChannel() + + // Register a watch, and push the results on to a channel. + client.WatchCluster(test.resourceName, func(update xdsresource.ClusterUpdate, err error) { + updateAndErrCh.Send(updateAndErr{update: update, err: err}) + }) + t.Logf("Registered a watch for Cluster %q", test.resourceName) + + // Wait for the discovery request to be sent out. + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + val, err := mgmtServer.XDSRequestChan.Receive(ctx) + if err != nil { + t.Fatalf("Timeout when waiting for discovery request at the management server: %v", ctx) + } + wantReq := &fakeserver.Request{Req: &v3discoverypb.DiscoveryRequest{ + Node: &v3corepb.Node{Id: nodeID}, + ResourceNames: []string{test.resourceName}, + TypeUrl: "type.googleapis.com/envoy.config.cluster.v3.Cluster", + }} + gotReq := val.(*fakeserver.Request) + if diff := cmp.Diff(gotReq, wantReq, protocmp.Transform()); diff != "" { + t.Fatalf("Discovery request received at management server is %+v, want %+v", gotReq, wantReq) + } + t.Logf("Discovery request received at management server") + + // Configure the fake management server with a response. + mgmtServer.XDSResponseChan <- &fakeserver.Response{Resp: test.managementServerResponse} + + // Wait for an update from the xDS client and compare with expected + // update. + val, err = updateAndErrCh.Receive(ctx) + if err != nil { + t.Fatalf("Timeout when waiting for watch callback to invoked after response from management server: %v", err) + } + gotUpdate := val.(updateAndErr).update + gotErr := val.(updateAndErr).err + if (gotErr != nil) != (test.wantErr != "") { + t.Fatalf("Got error from handling update: %v, want %v", gotErr, test.wantErr) + } + if gotErr != nil && !strings.Contains(gotErr.Error(), test.wantErr) { + t.Fatalf("Got error from handling update: %v, want %v", gotErr, test.wantErr) + } + cmpOpts := []cmp.Option{ + cmpopts.EquateEmpty(), + cmpopts.IgnoreFields(xdsresource.ClusterUpdate{}, "Raw"), + } + if diff := cmp.Diff(test.wantUpdate, gotUpdate, cmpOpts...); diff != "" { + t.Fatalf("Unexpected diff in metadata, diff (-want +got):\n%s", diff) + } + if err := compareUpdateMetadata(ctx, func() map[string]xdsresource.UpdateWithMD { + dump := client.DumpResources() + return dump["type.googleapis.com/envoy.config.cluster.v3.Cluster"] + }, test.wantUpdateMetadata); err != nil { + t.Fatal(err) + } + }) + } +} + +// TestHandleEndpointsResponseFromManagementServer covers different scenarios +// involving receipt of a CDS response from the management server. The test +// verifies that the internal state of the xDS client (parsed resource and +// metadata) matches expectations. +func (s) TestHandleEndpointsResponseFromManagementServer(t *testing.T) { + const ( + resourceName1 = "resource-name-1" + resourceName2 = "resource-name-2" + ) + resource1 := &v3endpointpb.ClusterLoadAssignment{ + ClusterName: resourceName1, + Endpoints: []*v3endpointpb.LocalityLbEndpoints{ + { + Locality: &v3corepb.Locality{SubZone: "locality-1"}, + LbEndpoints: []*v3endpointpb.LbEndpoint{ + { + HostIdentifier: &v3endpointpb.LbEndpoint_Endpoint{ + Endpoint: &v3endpointpb.Endpoint{ + Address: &v3corepb.Address{ + Address: &v3corepb.Address_SocketAddress{ + SocketAddress: &v3corepb.SocketAddress{ + Protocol: v3corepb.SocketAddress_TCP, + Address: "addr1", + PortSpecifier: &v3corepb.SocketAddress_PortValue{ + PortValue: uint32(314), + }, + }, + }, + }, + }, + }, + }, + }, + LoadBalancingWeight: &wrapperspb.UInt32Value{Value: 1}, + Priority: 1, + }, + { + Locality: &v3corepb.Locality{SubZone: "locality-2"}, + LbEndpoints: []*v3endpointpb.LbEndpoint{ + { + HostIdentifier: &v3endpointpb.LbEndpoint_Endpoint{ + Endpoint: &v3endpointpb.Endpoint{ + Address: &v3corepb.Address{ + Address: &v3corepb.Address_SocketAddress{ + SocketAddress: &v3corepb.SocketAddress{ + Protocol: v3corepb.SocketAddress_TCP, + Address: "addr2", + PortSpecifier: &v3corepb.SocketAddress_PortValue{ + PortValue: uint32(159), + }, + }, + }, + }, + }, + }, + }, + }, + LoadBalancingWeight: &wrapperspb.UInt32Value{Value: 1}, + Priority: 0, + }, + }, + } + resource2 := proto.Clone(resource1).(*v3endpointpb.ClusterLoadAssignment) + resource2.ClusterName = resourceName2 + + tests := []struct { + desc string + resourceName string + managementServerResponse *v3discoverypb.DiscoveryResponse + wantUpdate xdsresource.EndpointsUpdate + wantErr string + wantUpdateMetadata map[string]xdsresource.UpdateWithMD + }{ + // The first three tests involve scenarios where the response fails + // protobuf deserialization (because it contains an invalid data or type + // in the anypb.Any) or the requested resource is not present in the + // response. In either case, no resource update makes its way to the + // top-level xDS client. An EDS response without a requested resource + // does not mean that the resource does not exist in the server. It + // could be part of a future update. Therefore, the only failure mode + // for this resource is for the watch to timeout. + { + desc: "badly-marshaled-response", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment", + VersionInfo: "1", + Resources: []*anypb.Any{{ + TypeUrl: "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment", + Value: []byte{1, 2, 3, 4}, + }}, + }, + wantErr: fmt.Sprintf("watch for resource %q of type EndpointsResource timed out", resourceName1), + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}, + }, + }, + { + desc: "empty-response", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment", + VersionInfo: "1", + }, + wantErr: fmt.Sprintf("watch for resource %q of type EndpointsResource timed out", resourceName1), + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}, + }, + }, + { + desc: "unexpected-type-in-response", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.route.v3.RouteConfiguration", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{})}, + }, + wantErr: fmt.Sprintf("watch for resource %q of type EndpointsResource timed out", resourceName1), + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}, + }, + }, + { + desc: "one-bad-resource", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(&v3endpointpb.ClusterLoadAssignment{ + ClusterName: resourceName1, + Endpoints: []*v3endpointpb.LocalityLbEndpoints{ + { + Locality: &v3corepb.Locality{SubZone: "locality-1"}, + LbEndpoints: []*v3endpointpb.LbEndpoint{ + { + HostIdentifier: &v3endpointpb.LbEndpoint_Endpoint{ + Endpoint: &v3endpointpb.Endpoint{ + Address: &v3corepb.Address{ + Address: &v3corepb.Address_SocketAddress{ + SocketAddress: &v3corepb.SocketAddress{ + Protocol: v3corepb.SocketAddress_TCP, + Address: "addr1", + PortSpecifier: &v3corepb.SocketAddress_PortValue{ + PortValue: uint32(314), + }, + }, + }, + }, + }, + }, + LoadBalancingWeight: &wrapperspb.UInt32Value{Value: 0}, + }, + }, + LoadBalancingWeight: &wrapperspb.UInt32Value{Value: 1}, + Priority: 1, + }, + }, + }), + }, + }, + wantErr: "EDS response contains an endpoint with zero weight", + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": {MD: xdsresource.UpdateMetadata{ + Status: xdsresource.ServiceStatusNACKed, + ErrState: &xdsresource.UpdateErrorMetadata{ + Version: "1", + Err: cmpopts.AnyError, + }, + }}, + }, + }, + { + desc: "one-good-resource", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(resource1)}, + }, + wantUpdate: 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, + }, + }, + }, + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": { + MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}, + Raw: testutils.MarshalAny(resource1), + }, + }, + }, + { + desc: "two-resources-when-we-requested-one", + resourceName: resourceName1, + managementServerResponse: &v3discoverypb.DiscoveryResponse{ + TypeUrl: "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment", + VersionInfo: "1", + Resources: []*anypb.Any{testutils.MarshalAny(resource1), testutils.MarshalAny(resource2)}, + }, + wantUpdate: 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, + }, + }, + }, + wantUpdateMetadata: map[string]xdsresource.UpdateWithMD{ + "resource-name-1": { + MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: "1"}, + Raw: testutils.MarshalAny(resource1), + }, + }, + }, + } + + for _, test := range tests { + t.Run(test.desc, func(t *testing.T) { + // Create a fake xDS management server listening on a local port, + // and set it up with the response to send. + mgmtServer, cleanup := startFakeManagementServer(t) + defer cleanup() + t.Logf("Started xDS management server on %s", mgmtServer.Address) + + // Create an xDS client talking to the above management server. + nodeID := uuid.New().String() + client, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{ + XDSServer: &bootstrap.ServerConfig{ + ServerURI: mgmtServer.Address, + Creds: grpc.WithTransportCredentials(insecure.NewCredentials()), + CredsType: "insecure", + TransportAPI: version.TransportV3, + NodeProto: &v3corepb.Node{Id: nodeID}, + }, + }, defaultTestWatchExpiryTimeout, time.Duration(0)) + if err != nil { + t.Fatalf("failed to create xds client: %v", err) + } + defer client.Close() + t.Logf("Created xDS client to %s", mgmtServer.Address) + + // A wrapper struct to wrap the update and the associated error, as + // received by the resource watch callback. + type updateAndErr struct { + update xdsresource.EndpointsUpdate + err error + } + updateAndErrCh := testutils.NewChannel() + + // Register a watch, and push the results on to a channel. + client.WatchEndpoints(test.resourceName, func(update xdsresource.EndpointsUpdate, err error) { + updateAndErrCh.Send(updateAndErr{update: update, err: err}) + }) + t.Logf("Registered a watch for Endpoint %q", test.resourceName) + + // Wait for the discovery request to be sent out. + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + val, err := mgmtServer.XDSRequestChan.Receive(ctx) + if err != nil { + t.Fatalf("Timeout when waiting for discovery request at the management server: %v", ctx) + } + wantReq := &fakeserver.Request{Req: &v3discoverypb.DiscoveryRequest{ + Node: &v3corepb.Node{Id: nodeID}, + ResourceNames: []string{test.resourceName}, + TypeUrl: "type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment", + }} + gotReq := val.(*fakeserver.Request) + if diff := cmp.Diff(gotReq, wantReq, protocmp.Transform()); diff != "" { + t.Fatalf("Discovery request received at management server is %+v, want %+v", gotReq, wantReq) + } + t.Logf("Discovery request received at management server") + + // Configure the fake management server with a response. + mgmtServer.XDSResponseChan <- &fakeserver.Response{Resp: test.managementServerResponse} + + // Wait for an update from the xDS client and compare with expected + // update. + val, err = updateAndErrCh.Receive(ctx) + if err != nil { + t.Fatalf("Timeout when waiting for watch callback to invoked after response from management server: %v", err) + } + gotUpdate := val.(updateAndErr).update + gotErr := val.(updateAndErr).err + if (gotErr != nil) != (test.wantErr != "") { + t.Fatalf("Got error from handling update: %v, want %v", gotErr, test.wantErr) + } + if gotErr != nil && !strings.Contains(gotErr.Error(), test.wantErr) { + t.Fatalf("Got error from handling update: %v, want %v", gotErr, test.wantErr) + } + cmpOpts := []cmp.Option{ + cmpopts.EquateEmpty(), + cmpopts.IgnoreFields(xdsresource.EndpointsUpdate{}, "Raw"), + } + if diff := cmp.Diff(test.wantUpdate, gotUpdate, cmpOpts...); diff != "" { + t.Fatalf("Unexpected diff in metadata, diff (-want +got):\n%s", diff) + } + if err := compareUpdateMetadata(ctx, func() map[string]xdsresource.UpdateWithMD { + dump := client.DumpResources() + return dump["type.googleapis.com/envoy.config.endpoint.v3.ClusterLoadAssignment"] + }, test.wantUpdateMetadata); err != nil { + t.Fatal(err) + } + }) + } +} diff --git a/xds/internal/xdsclient/loadreport_test.go b/xds/internal/xdsclient/loadreport_test.go index 8a0703190..c199ae767 100644 --- a/xds/internal/xdsclient/loadreport_test.go +++ b/xds/internal/xdsclient/loadreport_test.go @@ -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) diff --git a/xds/internal/xdsclient/pubsub/dump.go b/xds/internal/xdsclient/pubsub/dump.go deleted file mode 100644 index 2ff19a901..000000000 --- a/xds/internal/xdsclient/pubsub/dump.go +++ /dev/null @@ -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 -} diff --git a/xds/internal/xdsclient/pubsub/interface.go b/xds/internal/xdsclient/pubsub/interface.go deleted file mode 100644 index 334ec101e..000000000 --- a/xds/internal/xdsclient/pubsub/interface.go +++ /dev/null @@ -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) -} diff --git a/xds/internal/xdsclient/pubsub/pubsub.go b/xds/internal/xdsclient/pubsub/pubsub.go deleted file mode 100644 index 95e8ac773..000000000 --- a/xds/internal/xdsclient/pubsub/pubsub.go +++ /dev/null @@ -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 - } - } -} diff --git a/xds/internal/xdsclient/pubsub/update.go b/xds/internal/xdsclient/pubsub/update.go deleted file mode 100644 index 9ae6ae976..000000000 --- a/xds/internal/xdsclient/pubsub/update.go +++ /dev/null @@ -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)) - } - } -} diff --git a/xds/internal/xdsclient/pubsub/watch.go b/xds/internal/xdsclient/pubsub/watch.go deleted file mode 100644 index bef179936..000000000 --- a/xds/internal/xdsclient/pubsub/watch.go +++ /dev/null @@ -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 - } -} diff --git a/xds/internal/xdsclient/watchers_test.go b/xds/internal/xdsclient/watchers_test.go deleted file mode 100644 index 36409821a..000000000 --- a/xds/internal/xdsclient/watchers_test.go +++ /dev/null @@ -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 -} diff --git a/xds/internal/xdsclient/xdsclient_test.go b/xds/internal/xdsclient/xdsclient_test.go index 74da4de7c..d7bb92665 100644 --- a/xds/internal/xdsclient/xdsclient_test.go +++ b/xds/internal/xdsclient/xdsclient_test.go @@ -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 { diff --git a/xds/internal/xdsclient/xdsresource/cluster_resource_type.go b/xds/internal/xdsclient/xdsresource/cluster_resource_type.go index a25297933..87e6dbd11 100644 --- a/xds/internal/xdsclient/xdsresource/cluster_resource_type.go +++ b/xds/internal/xdsclient/xdsresource/cluster_resource_type.go @@ -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. diff --git a/xds/internal/xdsclient/xdsresource/endpoints_resource_type.go b/xds/internal/xdsclient/xdsresource/endpoints_resource_type.go index 2ba7e494a..dc1c09da0 100644 --- a/xds/internal/xdsclient/xdsresource/endpoints_resource_type.go +++ b/xds/internal/xdsclient/xdsresource/endpoints_resource_type.go @@ -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, }, diff --git a/xds/internal/xdsclient/xdsresource/listener_resource_type.go b/xds/internal/xdsclient/xdsresource/listener_resource_type.go index 442389f1c..6b2fff9f6 100644 --- a/xds/internal/xdsclient/xdsresource/listener_resource_type.go +++ b/xds/internal/xdsclient/xdsresource/listener_resource_type.go @@ -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. diff --git a/xds/internal/xdsclient/xdsresource/resource_type.go b/xds/internal/xdsclient/xdsresource/resource_type.go index 6946c5647..6fced7784 100644 --- a/xds/internal/xdsclient/xdsresource/resource_type.go +++ b/xds/internal/xdsclient/xdsresource/resource_type.go @@ -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 { diff --git a/xds/internal/xdsclient/xdsresource/route_config_resource_type.go b/xds/internal/xdsclient/xdsresource/route_config_resource_type.go index 9c7465fd7..31be4d6ae 100644 --- a/xds/internal/xdsclient/xdsresource/route_config_resource_type.go +++ b/xds/internal/xdsclient/xdsresource/route_config_resource_type.go @@ -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, }, diff --git a/xds/internal/xdsclient/xdsresource/test_utils_test.go b/xds/internal/xdsclient/xdsresource/test_utils_test.go index b352caa23..04a15f96c 100644 --- a/xds/internal/xdsclient/xdsresource/test_utils_test.go +++ b/xds/internal/xdsclient/xdsresource/test_utils_test.go @@ -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) - }), - } ) diff --git a/xds/internal/xdsclient/xdsresource/type.go b/xds/internal/xdsclient/xdsresource/type.go index faf34f98e..d9c78997c 100644 --- a/xds/internal/xdsclient/xdsresource/type.go +++ b/xds/internal/xdsclient/xdsresource/type.go @@ -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 +} diff --git a/xds/internal/xdsclient/xdsresource/unmarshal.go b/xds/internal/xdsclient/xdsresource/unmarshal.go index eda110887..28ae41e43 100644 --- a/xds/internal/xdsclient/xdsresource/unmarshal.go +++ b/xds/internal/xdsclient/xdsresource/unmarshal.go @@ -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 { diff --git a/xds/internal/xdsclient/xdsresource/unmarshal_cds.go b/xds/internal/xdsclient/xdsresource/unmarshal_cds.go index 3621d6120..fce59f65c 100644 --- a/xds/internal/xdsclient/xdsresource/unmarshal_cds.go +++ b/xds/internal/xdsclient/xdsresource/unmarshal_cds.go @@ -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 } diff --git a/xds/internal/xdsclient/xdsresource/unmarshal_cds_test.go b/xds/internal/xdsclient/xdsresource/unmarshal_cds_test.go index fa8f4d4bb..3705c02be 100644 --- a/xds/internal/xdsclient/xdsresource/unmarshal_cds_test.go +++ b/xds/internal/xdsclient/xdsresource/unmarshal_cds_test.go @@ -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) } }) } diff --git a/xds/internal/xdsclient/xdsresource/unmarshal_eds.go b/xds/internal/xdsclient/xdsresource/unmarshal_eds.go index e091d0dde..a1809a62f 100644 --- a/xds/internal/xdsclient/xdsresource/unmarshal_eds.go +++ b/xds/internal/xdsclient/xdsresource/unmarshal_eds.go @@ -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 { diff --git a/xds/internal/xdsclient/xdsresource/unmarshal_eds_test.go b/xds/internal/xdsclient/xdsresource/unmarshal_eds_test.go index f89333c76..3fd3f417e 100644 --- a/xds/internal/xdsclient/xdsresource/unmarshal_eds_test.go +++ b/xds/internal/xdsclient/xdsresource/unmarshal_eds_test.go @@ -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) } }) } diff --git a/xds/internal/xdsclient/xdsresource/unmarshal_lds.go b/xds/internal/xdsclient/xdsresource/unmarshal_lds.go index 2e59c0605..6b273e82f 100644 --- a/xds/internal/xdsclient/xdsresource/unmarshal_lds.go +++ b/xds/internal/xdsclient/xdsresource/unmarshal_lds.go @@ -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 } diff --git a/xds/internal/xdsclient/xdsresource/unmarshal_lds_test.go b/xds/internal/xdsclient/xdsresource/unmarshal_lds_test.go index f46cf3801..62089bc31 100644 --- a/xds/internal/xdsclient/xdsresource/unmarshal_lds_test.go +++ b/xds/internal/xdsclient/xdsresource/unmarshal_lds_test.go @@ -23,11 +23,10 @@ import ( "testing" "time" - v3discoverypb "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3" "github.com/golang/protobuf/proto" "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/testutils/xds/e2e" "google.golang.org/grpc/xds/internal/httpfilter" @@ -47,6 +46,7 @@ import ( v3rbacpb "github.com/envoyproxy/go-control-plane/envoy/extensions/filters/http/rbac/v3" v3httppb "github.com/envoyproxy/go-control-plane/envoy/extensions/filters/network/http_connection_manager/v3" v3tlspb "github.com/envoyproxy/go-control-plane/envoy/extensions/transport_sockets/tls/v3" + v3discoverypb "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3" v3matcherpb "github.com/envoyproxy/go-control-plane/envoy/type/matcher/v3" anypb "github.com/golang/protobuf/ptypes/any" spb "github.com/golang/protobuf/ptypes/struct" @@ -224,69 +224,55 @@ func (s) TestUnmarshalListener_ClientSide(t *testing.T) { }), }, }) - - errMD = UpdateMetadata{ - Status: ServiceStatusNACKed, - Version: testVersion, - ErrState: &UpdateErrorMetadata{ - Version: testVersion, - Err: cmpopts.AnyError, - }, - } ) tests := []struct { name string - resources []*anypb.Any - wantUpdate map[string]ListenerUpdateErrTuple - wantMD UpdateMetadata + resource *anypb.Any + wantName string + wantUpdate ListenerUpdate wantErr bool }{ { - name: "non-listener resource", - resources: []*anypb.Any{{TypeUrl: version.V3HTTPConnManagerURL}}, - wantMD: errMD, - wantErr: true, + name: "non-listener resource", + resource: &anypb.Any{TypeUrl: version.V3HTTPConnManagerURL}, + wantErr: true, }, { name: "badly marshaled listener resource", - resources: []*anypb.Any{ - { - TypeUrl: version.V3ListenerURL, - Value: func() []byte { - lis := &v3listenerpb.Listener{ - Name: v3LDSTarget, - ApiListener: &v3listenerpb.ApiListener{ - ApiListener: &anypb.Any{ - TypeUrl: version.V3HTTPConnManagerURL, - Value: []byte{1, 2, 3, 4}, - }, + resource: &anypb.Any{ + TypeUrl: version.V3ListenerURL, + Value: func() []byte { + lis := &v3listenerpb.Listener{ + Name: v3LDSTarget, + ApiListener: &v3listenerpb.ApiListener{ + ApiListener: &anypb.Any{ + TypeUrl: version.V3HTTPConnManagerURL, + Value: []byte{1, 2, 3, 4}, }, - } - mLis, _ := proto.Marshal(lis) - return mLis - }(), - }, + }, + } + mLis, _ := proto.Marshal(lis) + return mLis + }(), }, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: true, + wantName: v3LDSTarget, + wantErr: true, }, { name: "wrong type in apiListener", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, ApiListener: &v3listenerpb.ApiListener{ ApiListener: testutils.MarshalAny(&v2xdspb.Listener{}), }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: true, + }), + wantName: v3LDSTarget, + wantErr: true, }, { name: "empty httpConnMgr in apiListener", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, ApiListener: &v3listenerpb.ApiListener{ ApiListener: testutils.MarshalAny(&v3httppb.HttpConnectionManager{ @@ -295,43 +281,36 @@ func (s) TestUnmarshalListener_ClientSide(t *testing.T) { }, }), }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: true, + }), + wantName: v3LDSTarget, + wantErr: true, }, { name: "scopedRoutes routeConfig in apiListener", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, ApiListener: &v3listenerpb.ApiListener{ ApiListener: testutils.MarshalAny(&v3httppb.HttpConnectionManager{ RouteSpecifier: &v3httppb.HttpConnectionManager_ScopedRoutes{}, }), }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: true, + }), + wantName: v3LDSTarget, + wantErr: true, }, { - name: "rds.ConfigSource in apiListener is Self", - resources: []*anypb.Any{v3ListenerWithCDSConfigSourceSelf}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - RouteConfigName: v3RouteConfigName, - HTTPFilters: []HTTPFilter{routerFilter}, - Raw: v3ListenerWithCDSConfigSourceSelf, - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + name: "rds.ConfigSource in apiListener is Self", + resource: v3ListenerWithCDSConfigSourceSelf, + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v3RouteConfigName, + HTTPFilters: []HTTPFilter{routerFilter}, + Raw: v3ListenerWithCDSConfigSourceSelf, }, }, { name: "rds.ConfigSource in apiListener is not ADS or Self", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, ApiListener: &v3listenerpb.ApiListener{ ApiListener: testutils.MarshalAny(&v3httppb.HttpConnectionManager{ @@ -347,32 +326,24 @@ func (s) TestUnmarshalListener_ClientSide(t *testing.T) { }, }), }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: true, + }), + wantName: v3LDSTarget, + wantErr: true, }, { - name: "empty resource list", - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, - }, - }, - { - name: "v3 with no filters", - resources: []*anypb.Any{v3LisWithFilters()}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, HTTPFilters: routerFilterList, Raw: v3LisWithFilters()}}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + name: "v3 with no filters", + resource: v3LisWithFilters(), + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v3RouteConfigName, + MaxStreamDuration: time.Second, + HTTPFilters: routerFilterList, + Raw: v3LisWithFilters(), }, }, { name: "v3 no terminal filter", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, ApiListener: &v3listenerpb.ApiListener{ ApiListener: testutils.MarshalAny( @@ -390,254 +361,204 @@ func (s) TestUnmarshalListener_ClientSide(t *testing.T) { }, }), }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: true, + }), + wantName: v3LDSTarget, + wantErr: true, }, { - name: "v3 with custom filter", - resources: []*anypb.Any{v3LisWithFilters(customFilter)}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, - HTTPFilters: []HTTPFilter{ - { - Name: "customFilter", - Filter: httpFilter{}, - Config: filterConfig{Cfg: customFilterConfig}, - }, - routerFilter, - }, - Raw: v3LisWithFilters(customFilter), - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, - }, - }, - { - name: "v3 with custom filter in old typed struct", - resources: []*anypb.Any{v3LisWithFilters(oldTypedStructFilter)}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, - HTTPFilters: []HTTPFilter{ - { - Name: "customFilter", - Filter: httpFilter{}, - Config: filterConfig{Cfg: customFilterOldTypedStructConfig}, - }, - routerFilter, - }, - Raw: v3LisWithFilters(oldTypedStructFilter), - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, - }, - }, - { - name: "v3 with custom filter in new typed struct", - resources: []*anypb.Any{v3LisWithFilters(newTypedStructFilter)}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, - HTTPFilters: []HTTPFilter{ - { - Name: "customFilter", - Filter: httpFilter{}, - Config: filterConfig{Cfg: customFilterNewTypedStructConfig}, - }, - routerFilter, - }, - Raw: v3LisWithFilters(newTypedStructFilter), - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, - }, - }, - { - name: "v3 with optional custom filter", - resources: []*anypb.Any{v3LisWithFilters(customOptionalFilter)}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, - HTTPFilters: []HTTPFilter{ - { - Name: "customFilter", - Filter: httpFilter{}, - Config: filterConfig{Cfg: customFilterConfig}, - }, - routerFilter, - }, - Raw: v3LisWithFilters(customOptionalFilter), - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, - }, - }, - { - name: "v3 with two filters with same name", - resources: []*anypb.Any{v3LisWithFilters(customFilter, customFilter)}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: true, - }, - { - name: "v3 with two filters - same type different name", - resources: []*anypb.Any{v3LisWithFilters(customFilter, customFilter2)}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, - HTTPFilters: []HTTPFilter{{ + name: "v3 with custom filter", + resource: v3LisWithFilters(customFilter), + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, + HTTPFilters: []HTTPFilter{ + { Name: "customFilter", Filter: httpFilter{}, Config: filterConfig{Cfg: customFilterConfig}, - }, { - Name: "customFilter2", + }, + routerFilter, + }, + Raw: v3LisWithFilters(customFilter), + }, + }, + { + name: "v3 with custom filter in old typed struct", + resource: v3LisWithFilters(oldTypedStructFilter), + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, + HTTPFilters: []HTTPFilter{ + { + Name: "customFilter", + Filter: httpFilter{}, + Config: filterConfig{Cfg: customFilterOldTypedStructConfig}, + }, + routerFilter, + }, + Raw: v3LisWithFilters(oldTypedStructFilter), + }, + }, + { + name: "v3 with custom filter in new typed struct", + resource: v3LisWithFilters(newTypedStructFilter), + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, + HTTPFilters: []HTTPFilter{ + { + Name: "customFilter", + Filter: httpFilter{}, + Config: filterConfig{Cfg: customFilterNewTypedStructConfig}, + }, + routerFilter, + }, + Raw: v3LisWithFilters(newTypedStructFilter), + }, + }, + { + name: "v3 with optional custom filter", + resource: v3LisWithFilters(customOptionalFilter), + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, + HTTPFilters: []HTTPFilter{ + { + Name: "customFilter", Filter: httpFilter{}, Config: filterConfig{Cfg: customFilterConfig}, }, - routerFilter, + routerFilter, + }, + Raw: v3LisWithFilters(customOptionalFilter), + }, + }, + { + name: "v3 with two filters with same name", + resource: v3LisWithFilters(customFilter, customFilter), + wantName: v3LDSTarget, + wantErr: true, + }, + { + name: "v3 with two filters - same type different name", + resource: v3LisWithFilters(customFilter, customFilter2), + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, + HTTPFilters: []HTTPFilter{{ + Name: "customFilter", + Filter: httpFilter{}, + Config: filterConfig{Cfg: customFilterConfig}, + }, { + Name: "customFilter2", + Filter: httpFilter{}, + Config: filterConfig{Cfg: customFilterConfig}, + }, + routerFilter, + }, + Raw: v3LisWithFilters(customFilter, customFilter2), + }, + }, + { + name: "v3 with server-only filter", + resource: v3LisWithFilters(serverOnlyCustomFilter), + wantName: v3LDSTarget, + wantErr: true, + }, + { + name: "v3 with optional server-only filter", + resource: v3LisWithFilters(serverOnlyOptionalCustomFilter), + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v3RouteConfigName, + MaxStreamDuration: time.Second, + Raw: v3LisWithFilters(serverOnlyOptionalCustomFilter), + HTTPFilters: routerFilterList, + }, + }, + { + name: "v3 with client-only filter", + resource: v3LisWithFilters(clientOnlyCustomFilter), + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, + HTTPFilters: []HTTPFilter{ + { + Name: "clientOnlyCustomFilter", + Filter: clientOnlyHTTPFilter{}, + Config: filterConfig{Cfg: clientOnlyCustomFilterConfig}, }, - Raw: v3LisWithFilters(customFilter, customFilter2), - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + routerFilter}, + Raw: v3LisWithFilters(clientOnlyCustomFilter), }, }, { - name: "v3 with server-only filter", - resources: []*anypb.Any{v3LisWithFilters(serverOnlyCustomFilter)}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: true, + name: "v3 with err filter", + resource: v3LisWithFilters(errFilter), + wantName: v3LDSTarget, + wantErr: true, }, { - name: "v3 with optional server-only filter", - resources: []*anypb.Any{v3LisWithFilters(serverOnlyOptionalCustomFilter)}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - RouteConfigName: v3RouteConfigName, - MaxStreamDuration: time.Second, - Raw: v3LisWithFilters(serverOnlyOptionalCustomFilter), - HTTPFilters: routerFilterList, - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + name: "v3 with optional err filter", + resource: v3LisWithFilters(errOptionalFilter), + wantName: v3LDSTarget, + wantErr: true, + }, + { + name: "v3 with unknown filter", + resource: v3LisWithFilters(unknownFilter), + wantName: v3LDSTarget, + wantErr: true, + }, + { + name: "v3 with unknown filter (optional)", + resource: v3LisWithFilters(unknownOptionalFilter), + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v3RouteConfigName, + MaxStreamDuration: time.Second, + HTTPFilters: routerFilterList, + Raw: v3LisWithFilters(unknownOptionalFilter), }, }, { - name: "v3 with client-only filter", - resources: []*anypb.Any{v3LisWithFilters(clientOnlyCustomFilter)}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, - HTTPFilters: []HTTPFilter{ - { - Name: "clientOnlyCustomFilter", - Filter: clientOnlyHTTPFilter{}, - Config: filterConfig{Cfg: clientOnlyCustomFilterConfig}, - }, - routerFilter}, - Raw: v3LisWithFilters(clientOnlyCustomFilter), - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + name: "v2 listener resource", + resource: v2Lis, + wantName: v2LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v2RouteConfigName, + Raw: v2Lis, }, }, { - name: "v3 with err filter", - resources: []*anypb.Any{v3LisWithFilters(errFilter)}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: true, - }, - { - name: "v3 with optional err filter", - resources: []*anypb.Any{v3LisWithFilters(errOptionalFilter)}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: true, - }, - { - name: "v3 with unknown filter", - resources: []*anypb.Any{v3LisWithFilters(unknownFilter)}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: true, - }, - { - name: "v3 with unknown filter (optional)", - resources: []*anypb.Any{v3LisWithFilters(unknownOptionalFilter)}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - RouteConfigName: v3RouteConfigName, - MaxStreamDuration: time.Second, - HTTPFilters: routerFilterList, - Raw: v3LisWithFilters(unknownOptionalFilter), - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + name: "v2 listener resource wrapped", + resource: testutils.MarshalAny(&v2xdspb.Resource{Resource: v2Lis}), + wantName: v2LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v2RouteConfigName, + Raw: v2Lis, }, }, { - name: "v2 listener resource", - resources: []*anypb.Any{v2Lis}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v2LDSTarget: {Update: ListenerUpdate{RouteConfigName: v2RouteConfigName, Raw: v2Lis}}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + name: "v3 listener resource", + resource: v3LisWithFilters(), + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v3RouteConfigName, + MaxStreamDuration: time.Second, + HTTPFilters: routerFilterList, + Raw: v3LisWithFilters(), }, }, { - name: "v2 listener resource wrapped", - resources: []*anypb.Any{testutils.MarshalAny(&v2xdspb.Resource{Resource: v2Lis})}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v2LDSTarget: {Update: ListenerUpdate{RouteConfigName: v2RouteConfigName, Raw: v2Lis}}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, - }, - }, - { - name: "v3 listener resource", - resources: []*anypb.Any{v3LisWithFilters()}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, HTTPFilters: routerFilterList, Raw: v3LisWithFilters()}}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, - }, - }, - { - name: "v3 listener resource wrapped", - resources: []*anypb.Any{testutils.MarshalAny(&v3discoverypb.Resource{Resource: v3LisWithFilters()})}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, HTTPFilters: routerFilterList, Raw: v3LisWithFilters()}}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + name: "v3 listener resource wrapped", + resource: testutils.MarshalAny(&v3discoverypb.Resource{Resource: v3LisWithFilters()}), + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v3RouteConfigName, + MaxStreamDuration: time.Second, + HTTPFilters: routerFilterList, + Raw: v3LisWithFilters(), }, }, // "To allow equating RBAC's direct_remote_ip and @@ -645,19 +566,14 @@ func (s) TestUnmarshalListener_ClientSide(t *testing.T) { // or zero and HttpConnectionManager.original_ip_detection_extensions // must be empty." - A41 { - name: "rbac-allow-equating-direct-remote-ip-and-remote-ip-valid", - resources: []*anypb.Any{v3LisToTestRBAC(0, nil)}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - RouteConfigName: v3RouteConfigName, - MaxStreamDuration: time.Second, - HTTPFilters: []HTTPFilter{routerFilter}, - Raw: v3LisToTestRBAC(0, nil), - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + name: "rbac-allow-equating-direct-remote-ip-and-remote-ip-valid", + resource: v3LisToTestRBAC(0, nil), + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + RouteConfigName: v3RouteConfigName, + MaxStreamDuration: time.Second, + HTTPFilters: []HTTPFilter{routerFilter}, + Raw: v3LisToTestRBAC(0, nil), }, }, // In order to support xDS Configured RBAC HTTPFilter equating direct @@ -666,94 +582,49 @@ func (s) TestUnmarshalListener_ClientSide(t *testing.T) { // determining an origin clients ip address, direct remote ip != remote // ip. { - name: "rbac-allow-equating-direct-remote-ip-and-remote-ip-invalid-num-untrusted-hops", - resources: []*anypb.Any{v3LisToTestRBAC(1, nil)}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: true, + name: "rbac-allow-equating-direct-remote-ip-and-remote-ip-invalid-num-untrusted-hops", + resource: v3LisToTestRBAC(1, nil), + wantName: v3LDSTarget, + wantErr: true, }, // In order to support xDS Configured RBAC HTTPFilter equating direct // remote ip and remote ip, originalIpDetectionExtensions must be empty. // This is because if you have to ask ip-detection-extension for the // original ip, direct remote ip might not equal remote ip. { - name: "rbac-allow-equating-direct-remote-ip-and-remote-ip-invalid-original-ip-detection-extension", - resources: []*anypb.Any{v3LisToTestRBAC(0, []*v3corepb.TypedExtensionConfig{{Name: "something"}})}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: true, + name: "rbac-allow-equating-direct-remote-ip-and-remote-ip-invalid-original-ip-detection-extension", + resource: v3LisToTestRBAC(0, []*v3corepb.TypedExtensionConfig{{Name: "something"}}), + wantName: v3LDSTarget, + wantErr: true, }, { - name: "v3 listener with inline route configuration", - resources: []*anypb.Any{v3LisWithInlineRoute}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - InlineRouteConfig: &RouteConfigUpdate{ - VirtualHosts: []*VirtualHost{{ - Domains: []string{v3LDSTarget}, - Routes: []*Route{{Prefix: newStringP("/"), WeightedClusters: map[string]WeightedCluster{clusterName: {Weight: 1}}, ActionType: RouteActionRoute}}, - }}}, - MaxStreamDuration: time.Second, - Raw: v3LisWithInlineRoute, - HTTPFilters: routerFilterList, - }}, + name: "v3 listener with inline route configuration", + resource: v3LisWithInlineRoute, + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + InlineRouteConfig: &RouteConfigUpdate{ + VirtualHosts: []*VirtualHost{{ + Domains: []string{v3LDSTarget}, + Routes: []*Route{{Prefix: newStringP("/"), WeightedClusters: map[string]WeightedCluster{clusterName: {Weight: 1}}, ActionType: RouteActionRoute}}, + }}}, + MaxStreamDuration: time.Second, + Raw: v3LisWithInlineRoute, + HTTPFilters: routerFilterList, }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, - }, - }, - { - name: "multiple listener resources", - resources: []*anypb.Any{v2Lis, v3LisWithFilters()}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v2LDSTarget: {Update: ListenerUpdate{RouteConfigName: v2RouteConfigName, Raw: v2Lis}}, - v3LDSTarget: {Update: ListenerUpdate{RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, Raw: v3LisWithFilters(), HTTPFilters: routerFilterList}}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, - }, - }, - { - // To test that unmarshal keeps processing on errors. - name: "good and bad listener resources", - resources: []*anypb.Any{ - v2Lis, - testutils.MarshalAny(&v3listenerpb.Listener{ - Name: "bad", - ApiListener: &v3listenerpb.ApiListener{ - ApiListener: testutils.MarshalAny(&v3httppb.HttpConnectionManager{ - RouteSpecifier: &v3httppb.HttpConnectionManager_ScopedRoutes{}, - }), - }}), - v3LisWithFilters(), - }, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v2LDSTarget: {Update: ListenerUpdate{RouteConfigName: v2RouteConfigName, Raw: v2Lis}}, - v3LDSTarget: {Update: ListenerUpdate{RouteConfigName: v3RouteConfigName, MaxStreamDuration: time.Second, Raw: v3LisWithFilters(), HTTPFilters: routerFilterList}}, - "bad": {Err: cmpopts.AnyError}, - }, - wantMD: errMD, - wantErr: true, }, } for _, test := range tests { t.Run(test.name, func(t *testing.T) { - opts := &UnmarshalOptions{ - Version: testVersion, - Resources: test.resources, - } - update, md, err := UnmarshalListener(opts) + name, update, err := unmarshalListenerResource(test.resource, nil) if (err != nil) != test.wantErr { - t.Fatalf("UnmarshalListener(%+v), got err: %v, wantErr: %v", opts, err, test.wantErr) + t.Errorf("unmarshalListenerResource(%s), got err: %v, wantErr: %v", pretty.ToJSON(test.resource), err, test.wantErr) + } + if name != test.wantName { + t.Errorf("unmarshalListenerResource(%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("unmarshalListenerResource(%s), got unexpected update, diff (-got +want): %v", pretty.ToJSON(test.resource), diff) } }) } @@ -1017,14 +888,6 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }) - errMD = UpdateMetadata{ - Status: ServiceStatusNACKed, - Version: testVersion, - ErrState: &UpdateErrorMetadata{ - Version: testVersion, - Err: cmpopts.AnyError, - }, - } ) v3LisToTestRBAC := func(xffNumTrustedHops uint32, originalIpDetectionExtensions []*v3corepb.TypedExtensionConfig) *anypb.Any { return testutils.MarshalAny(&v3listenerpb.Listener{ @@ -1109,53 +972,49 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { tests := []struct { name string - resources []*anypb.Any - wantUpdate map[string]ListenerUpdateErrTuple - wantMD UpdateMetadata + resource *anypb.Any + wantName string + wantUpdate ListenerUpdate wantErr string }{ { name: "non-empty listener filters", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, ListenerFilters: []*v3listenerpb.ListenerFilter{ {Name: "listener-filter-1"}, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "unsupported field 'listener_filters'", + }), + wantName: v3LDSTarget, + wantErr: "unsupported field 'listener_filters'", }, { name: "use_original_dst is set", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, UseOriginalDst: &wrapperspb.BoolValue{Value: true}, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "unsupported field 'use_original_dst'", + }), + wantName: v3LDSTarget, + wantErr: "unsupported field 'use_original_dst'", }, { - name: "no address field", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{Name: v3LDSTarget})}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "no address field in LDS response", + name: "no address field", + resource: testutils.MarshalAny(&v3listenerpb.Listener{Name: v3LDSTarget}), + wantName: v3LDSTarget, + wantErr: "no address field in LDS response", }, { name: "no socket address field", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: &v3corepb.Address{}, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "no socket_address field in LDS response", + }), + wantName: v3LDSTarget, + wantErr: "no socket_address field in LDS response", }, { name: "no filter chains and no default filter chain", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1164,14 +1023,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { Filters: emptyValidNetworkFilters, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "no supported filter chains and no default filter chain", + }), + wantName: v3LDSTarget, + wantErr: "no supported filter chains and no default filter chain", }, { name: "missing http connection manager network filter", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1179,14 +1037,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { Name: "filter-chain-1", }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "missing HttpConnectionManager filter", + }), + wantName: v3LDSTarget, + wantErr: "missing HttpConnectionManager filter", }, { name: "missing filter name in http filter", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1201,14 +1058,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "missing name field in filter", + }), + wantName: v3LDSTarget, + wantErr: "missing name field in filter", }, { name: "duplicate filter names in http filter", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1240,14 +1096,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "duplicate filter name", + }), + wantName: v3LDSTarget, + wantErr: "duplicate filter name", }, { name: "no terminal filter", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1267,14 +1122,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "http filters list is empty", + }), + wantName: v3LDSTarget, + wantErr: "http filters list is empty", }, { name: "terminal filter not last", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1295,14 +1149,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "is a terminal filter but it is not last in the filter chain", + }), + wantName: v3LDSTarget, + wantErr: "is a terminal filter but it is not last in the filter chain", }, { name: "last not terminal filter", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1323,14 +1176,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "is not a terminal filter", + }), + wantName: v3LDSTarget, + wantErr: "is not a terminal filter", }, { name: "unsupported oneof in typed config of http filter", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1344,14 +1196,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "unsupported config_type", + }), + wantName: v3LDSTarget, + wantErr: "unsupported config_type", }, { name: "overlapping filter chain match criteria", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1368,14 +1219,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { Filters: emptyValidNetworkFilters, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "multiple filter chains with overlapping matching rules are defined", + }), + wantName: v3LDSTarget, + wantErr: "multiple filter chains with overlapping matching rules are defined", }, { name: "unsupported network filter", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1391,14 +1241,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "unsupported network filter", + }), + wantName: v3LDSTarget, + wantErr: "unsupported network filter", }, { name: "badly marshaled network filter", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1417,14 +1266,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "failed unmarshaling of network filter", + }), + wantName: v3LDSTarget, + wantErr: "failed unmarshaling of network filter", }, { name: "unexpected transport socket name", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1436,14 +1284,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "transport_socket field has unexpected name", + }), + wantName: v3LDSTarget, + wantErr: "transport_socket field has unexpected name", }, { name: "unexpected transport socket typedConfig URL", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1458,14 +1305,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "transport_socket field has unexpected typeURL", + }), + wantName: v3LDSTarget, + wantErr: "transport_socket field has unexpected typeURL", }, { name: "badly marshaled transport socket", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1483,14 +1329,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "failed to unmarshal DownstreamTlsContext in LDS response", + }), + wantName: v3LDSTarget, + wantErr: "failed to unmarshal DownstreamTlsContext in LDS response", }, { name: "missing CommonTlsContext", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1505,31 +1350,29 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "DownstreamTlsContext in LDS response does not contain a CommonTlsContext", + }), + wantName: v3LDSTarget, + wantErr: "DownstreamTlsContext in LDS response does not contain a CommonTlsContext", }, { - name: "rbac-allow-equating-direct-remote-ip-and-remote-ip-valid", - resources: []*anypb.Any{v3LisToTestRBAC(0, nil)}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - InboundListenerCfg: &InboundListenerConfig{ - Address: "0.0.0.0", - Port: "9999", - FilterChains: &FilterChainManager{ - dstPrefixMap: map[string]*destPrefixEntry{ - unspecifiedPrefixMapKey: { - srcTypeArr: [3]*sourcePrefixes{ - { - srcPrefixMap: map[string]*sourcePrefixEntry{ - unspecifiedPrefixMapKey: { - srcPortMap: map[int]*FilterChain{ - 0: { - InlineRouteConfig: inlineRouteConfig, - HTTPFilters: routerFilterList, - }, + name: "rbac-allow-equating-direct-remote-ip-and-remote-ip-valid", + resource: v3LisToTestRBAC(0, nil), + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + InboundListenerCfg: &InboundListenerConfig{ + Address: "0.0.0.0", + Port: "9999", + FilterChains: &FilterChainManager{ + dstPrefixMap: map[string]*destPrefixEntry{ + unspecifiedPrefixMapKey: { + srcTypeArr: [3]*sourcePrefixes{ + { + srcPrefixMap: map[string]*sourcePrefixEntry{ + unspecifiedPrefixMapKey: { + srcPortMap: map[int]*FilterChain{ + 0: { + InlineRouteConfig: inlineRouteConfig, + HTTPFilters: routerFilterList, }, }, }, @@ -1539,45 +1382,37 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - Raw: listenerEmptyTransportSocket, - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + }, + Raw: listenerEmptyTransportSocket, }, }, { - name: "rbac-allow-equating-direct-remote-ip-and-remote-ip-invalid-num-untrusted-hops", - resources: []*anypb.Any{v3LisToTestRBAC(1, nil)}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "xff_num_trusted_hops must be unset or zero", + name: "rbac-allow-equating-direct-remote-ip-and-remote-ip-invalid-num-untrusted-hops", + resource: v3LisToTestRBAC(1, nil), + wantName: v3LDSTarget, + wantErr: "xff_num_trusted_hops must be unset or zero", }, { - name: "rbac-allow-equating-direct-remote-ip-and-remote-ip-invalid-original-ip-detection-extension", - resources: []*anypb.Any{v3LisToTestRBAC(0, []*v3corepb.TypedExtensionConfig{{Name: "something"}})}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "original_ip_detection_extensions must be empty", + name: "rbac-allow-equating-direct-remote-ip-and-remote-ip-invalid-original-ip-detection-extension", + resource: v3LisToTestRBAC(0, []*v3corepb.TypedExtensionConfig{{Name: "something"}}), + wantName: v3LDSTarget, + wantErr: "original_ip_detection_extensions must be empty", }, { - name: "rbac-with-invalid-regex", - resources: []*anypb.Any{v3LisWithBadRBACConfiguration(badRBACCfgRegex)}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "error parsing config for filter", + name: "rbac-with-invalid-regex", + resource: v3LisWithBadRBACConfiguration(badRBACCfgRegex), + wantName: v3LDSTarget, + wantErr: "error parsing config for filter", }, { - name: "rbac-with-invalid-destination-ip-matcher", - resources: []*anypb.Any{v3LisWithBadRBACConfiguration(badRBACCfgDestIP)}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "error parsing config for filter", + name: "rbac-with-invalid-destination-ip-matcher", + resource: v3LisWithBadRBACConfiguration(badRBACCfgDestIP), + wantName: v3LDSTarget, + wantErr: "error parsing config for filter", }, { name: "unsupported validation context in transport socket", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1600,31 +1435,29 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "validation context contains unexpected type", + }), + wantName: v3LDSTarget, + wantErr: "validation context contains unexpected type", }, { - name: "empty transport socket", - resources: []*anypb.Any{listenerEmptyTransportSocket}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - InboundListenerCfg: &InboundListenerConfig{ - Address: "0.0.0.0", - Port: "9999", - FilterChains: &FilterChainManager{ - dstPrefixMap: map[string]*destPrefixEntry{ - unspecifiedPrefixMapKey: { - srcTypeArr: [3]*sourcePrefixes{ - { - srcPrefixMap: map[string]*sourcePrefixEntry{ - unspecifiedPrefixMapKey: { - srcPortMap: map[int]*FilterChain{ - 0: { - InlineRouteConfig: inlineRouteConfig, - HTTPFilters: routerFilterList, - }, + name: "empty transport socket", + resource: listenerEmptyTransportSocket, + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + InboundListenerCfg: &InboundListenerConfig{ + Address: "0.0.0.0", + Port: "9999", + FilterChains: &FilterChainManager{ + dstPrefixMap: map[string]*destPrefixEntry{ + unspecifiedPrefixMapKey: { + srcTypeArr: [3]*sourcePrefixes{ + { + srcPrefixMap: map[string]*sourcePrefixEntry{ + unspecifiedPrefixMapKey: { + srcPortMap: map[int]*FilterChain{ + 0: { + InlineRouteConfig: inlineRouteConfig, + HTTPFilters: routerFilterList, }, }, }, @@ -1634,17 +1467,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - Raw: listenerEmptyTransportSocket, - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + }, + Raw: listenerEmptyTransportSocket, }, }, { name: "no identity and root certificate providers using deprecated fields", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1667,14 +1496,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "security configuration on the server-side does not contain root certificate provider instance name, but require_client_cert field is set", + }), + wantName: v3LDSTarget, + wantErr: "security configuration on the server-side does not contain root certificate provider instance name, but require_client_cert field is set", }, { name: "no identity and root certificate providers using new fields", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1697,14 +1525,13 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "security configuration on the server-side does not contain root certificate provider instance name, but require_client_cert field is set", + }), + wantName: v3LDSTarget, + wantErr: "security configuration on the server-side does not contain root certificate provider instance name, but require_client_cert field is set", }, { name: "no identity certificate provider with require_client_cert", - resources: []*anypb.Any{testutils.MarshalAny(&v3listenerpb.Listener{ + resource: testutils.MarshalAny(&v3listenerpb.Listener{ Name: v3LDSTarget, Address: localSocketAddress, FilterChains: []*v3listenerpb.FilterChain{ @@ -1721,35 +1548,33 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - })}, - wantUpdate: map[string]ListenerUpdateErrTuple{v3LDSTarget: {Err: cmpopts.AnyError}}, - wantMD: errMD, - wantErr: "security configuration on the server-side does not contain identity certificate provider instance name", + }), + wantName: v3LDSTarget, + wantErr: "security configuration on the server-side does not contain identity certificate provider instance name", }, { - name: "happy case with no validation context using deprecated fields", - resources: []*anypb.Any{listenerNoValidationContextDeprecatedFields}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - InboundListenerCfg: &InboundListenerConfig{ - Address: "0.0.0.0", - Port: "9999", - FilterChains: &FilterChainManager{ - dstPrefixMap: map[string]*destPrefixEntry{ - unspecifiedPrefixMapKey: { - srcTypeArr: [3]*sourcePrefixes{ - { - srcPrefixMap: map[string]*sourcePrefixEntry{ - unspecifiedPrefixMapKey: { - srcPortMap: map[int]*FilterChain{ - 0: { - SecurityCfg: &SecurityConfig{ - IdentityInstanceName: "identityPluginInstance", - IdentityCertName: "identityCertName", - }, - InlineRouteConfig: inlineRouteConfig, - HTTPFilters: routerFilterList, + name: "happy case with no validation context using deprecated fields", + resource: listenerNoValidationContextDeprecatedFields, + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + InboundListenerCfg: &InboundListenerConfig{ + Address: "0.0.0.0", + Port: "9999", + FilterChains: &FilterChainManager{ + dstPrefixMap: map[string]*destPrefixEntry{ + unspecifiedPrefixMapKey: { + srcTypeArr: [3]*sourcePrefixes{ + { + srcPrefixMap: map[string]*sourcePrefixEntry{ + unspecifiedPrefixMapKey: { + srcPortMap: map[int]*FilterChain{ + 0: { + SecurityCfg: &SecurityConfig{ + IdentityInstanceName: "identityPluginInstance", + IdentityCertName: "identityCertName", }, + InlineRouteConfig: inlineRouteConfig, + HTTPFilters: routerFilterList, }, }, }, @@ -1757,48 +1582,43 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - def: &FilterChain{ - SecurityCfg: &SecurityConfig{ - IdentityInstanceName: "defaultIdentityPluginInstance", - IdentityCertName: "defaultIdentityCertName", - }, - InlineRouteConfig: inlineRouteConfig, - HTTPFilters: routerFilterList, + }, + def: &FilterChain{ + SecurityCfg: &SecurityConfig{ + IdentityInstanceName: "defaultIdentityPluginInstance", + IdentityCertName: "defaultIdentityCertName", }, + InlineRouteConfig: inlineRouteConfig, + HTTPFilters: routerFilterList, }, }, - Raw: listenerNoValidationContextDeprecatedFields, - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + }, + Raw: listenerNoValidationContextDeprecatedFields, }, }, { - name: "happy case with no validation context using new fields", - resources: []*anypb.Any{listenerNoValidationContextNewFields}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - InboundListenerCfg: &InboundListenerConfig{ - Address: "0.0.0.0", - Port: "9999", - FilterChains: &FilterChainManager{ - dstPrefixMap: map[string]*destPrefixEntry{ - unspecifiedPrefixMapKey: { - srcTypeArr: [3]*sourcePrefixes{ - { - srcPrefixMap: map[string]*sourcePrefixEntry{ - unspecifiedPrefixMapKey: { - srcPortMap: map[int]*FilterChain{ - 0: { - SecurityCfg: &SecurityConfig{ - IdentityInstanceName: "identityPluginInstance", - IdentityCertName: "identityCertName", - }, - InlineRouteConfig: inlineRouteConfig, - HTTPFilters: routerFilterList, + name: "happy case with no validation context using new fields", + resource: listenerNoValidationContextNewFields, + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + InboundListenerCfg: &InboundListenerConfig{ + Address: "0.0.0.0", + Port: "9999", + FilterChains: &FilterChainManager{ + dstPrefixMap: map[string]*destPrefixEntry{ + unspecifiedPrefixMapKey: { + srcTypeArr: [3]*sourcePrefixes{ + { + srcPrefixMap: map[string]*sourcePrefixEntry{ + unspecifiedPrefixMapKey: { + srcPortMap: map[int]*FilterChain{ + 0: { + SecurityCfg: &SecurityConfig{ + IdentityInstanceName: "identityPluginInstance", + IdentityCertName: "identityCertName", }, + InlineRouteConfig: inlineRouteConfig, + HTTPFilters: routerFilterList, }, }, }, @@ -1806,51 +1626,46 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - def: &FilterChain{ - SecurityCfg: &SecurityConfig{ - IdentityInstanceName: "defaultIdentityPluginInstance", - IdentityCertName: "defaultIdentityCertName", - }, - InlineRouteConfig: inlineRouteConfig, - HTTPFilters: routerFilterList, + }, + def: &FilterChain{ + SecurityCfg: &SecurityConfig{ + IdentityInstanceName: "defaultIdentityPluginInstance", + IdentityCertName: "defaultIdentityCertName", }, + InlineRouteConfig: inlineRouteConfig, + HTTPFilters: routerFilterList, }, }, - Raw: listenerNoValidationContextNewFields, - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + }, + Raw: listenerNoValidationContextNewFields, }, }, { - name: "happy case with validation context provider instance with deprecated fields", - resources: []*anypb.Any{listenerWithValidationContextDeprecatedFields}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - InboundListenerCfg: &InboundListenerConfig{ - Address: "0.0.0.0", - Port: "9999", - FilterChains: &FilterChainManager{ - dstPrefixMap: map[string]*destPrefixEntry{ - unspecifiedPrefixMapKey: { - srcTypeArr: [3]*sourcePrefixes{ - { - srcPrefixMap: map[string]*sourcePrefixEntry{ - unspecifiedPrefixMapKey: { - srcPortMap: map[int]*FilterChain{ - 0: { - SecurityCfg: &SecurityConfig{ - RootInstanceName: "rootPluginInstance", - RootCertName: "rootCertName", - IdentityInstanceName: "identityPluginInstance", - IdentityCertName: "identityCertName", - RequireClientCert: true, - }, - InlineRouteConfig: inlineRouteConfig, - HTTPFilters: routerFilterList, + name: "happy case with validation context provider instance with deprecated fields", + resource: listenerWithValidationContextDeprecatedFields, + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + InboundListenerCfg: &InboundListenerConfig{ + Address: "0.0.0.0", + Port: "9999", + FilterChains: &FilterChainManager{ + dstPrefixMap: map[string]*destPrefixEntry{ + unspecifiedPrefixMapKey: { + srcTypeArr: [3]*sourcePrefixes{ + { + srcPrefixMap: map[string]*sourcePrefixEntry{ + unspecifiedPrefixMapKey: { + srcPortMap: map[int]*FilterChain{ + 0: { + SecurityCfg: &SecurityConfig{ + RootInstanceName: "rootPluginInstance", + RootCertName: "rootCertName", + IdentityInstanceName: "identityPluginInstance", + IdentityCertName: "identityCertName", + RequireClientCert: true, }, + InlineRouteConfig: inlineRouteConfig, + HTTPFilters: routerFilterList, }, }, }, @@ -1858,54 +1673,49 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - def: &FilterChain{ - SecurityCfg: &SecurityConfig{ - RootInstanceName: "defaultRootPluginInstance", - RootCertName: "defaultRootCertName", - IdentityInstanceName: "defaultIdentityPluginInstance", - IdentityCertName: "defaultIdentityCertName", - RequireClientCert: true, - }, - InlineRouteConfig: inlineRouteConfig, - HTTPFilters: routerFilterList, + }, + def: &FilterChain{ + SecurityCfg: &SecurityConfig{ + RootInstanceName: "defaultRootPluginInstance", + RootCertName: "defaultRootCertName", + IdentityInstanceName: "defaultIdentityPluginInstance", + IdentityCertName: "defaultIdentityCertName", + RequireClientCert: true, }, + InlineRouteConfig: inlineRouteConfig, + HTTPFilters: routerFilterList, }, }, - Raw: listenerWithValidationContextDeprecatedFields, - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + }, + Raw: listenerWithValidationContextDeprecatedFields, }, }, { - name: "happy case with validation context provider instance with new fields", - resources: []*anypb.Any{listenerWithValidationContextNewFields}, - wantUpdate: map[string]ListenerUpdateErrTuple{ - v3LDSTarget: {Update: ListenerUpdate{ - InboundListenerCfg: &InboundListenerConfig{ - Address: "0.0.0.0", - Port: "9999", - FilterChains: &FilterChainManager{ - dstPrefixMap: map[string]*destPrefixEntry{ - unspecifiedPrefixMapKey: { - srcTypeArr: [3]*sourcePrefixes{ - { - srcPrefixMap: map[string]*sourcePrefixEntry{ - unspecifiedPrefixMapKey: { - srcPortMap: map[int]*FilterChain{ - 0: { - SecurityCfg: &SecurityConfig{ - RootInstanceName: "rootPluginInstance", - RootCertName: "rootCertName", - IdentityInstanceName: "identityPluginInstance", - IdentityCertName: "identityCertName", - RequireClientCert: true, - }, - InlineRouteConfig: inlineRouteConfig, - HTTPFilters: routerFilterList, + name: "happy case with validation context provider instance with new fields", + resource: listenerWithValidationContextNewFields, + wantName: v3LDSTarget, + wantUpdate: ListenerUpdate{ + InboundListenerCfg: &InboundListenerConfig{ + Address: "0.0.0.0", + Port: "9999", + FilterChains: &FilterChainManager{ + dstPrefixMap: map[string]*destPrefixEntry{ + unspecifiedPrefixMapKey: { + srcTypeArr: [3]*sourcePrefixes{ + { + srcPrefixMap: map[string]*sourcePrefixEntry{ + unspecifiedPrefixMapKey: { + srcPortMap: map[int]*FilterChain{ + 0: { + SecurityCfg: &SecurityConfig{ + RootInstanceName: "rootPluginInstance", + RootCertName: "rootCertName", + IdentityInstanceName: "identityPluginInstance", + IdentityCertName: "identityCertName", + RequireClientCert: true, }, + InlineRouteConfig: inlineRouteConfig, + HTTPFilters: routerFilterList, }, }, }, @@ -1913,47 +1723,36 @@ func (s) TestUnmarshalListener_ServerSide(t *testing.T) { }, }, }, - def: &FilterChain{ - SecurityCfg: &SecurityConfig{ - RootInstanceName: "defaultRootPluginInstance", - RootCertName: "defaultRootCertName", - IdentityInstanceName: "defaultIdentityPluginInstance", - IdentityCertName: "defaultIdentityCertName", - RequireClientCert: true, - }, - InlineRouteConfig: inlineRouteConfig, - HTTPFilters: routerFilterList, + }, + def: &FilterChain{ + SecurityCfg: &SecurityConfig{ + RootInstanceName: "defaultRootPluginInstance", + RootCertName: "defaultRootCertName", + IdentityInstanceName: "defaultIdentityPluginInstance", + IdentityCertName: "defaultIdentityCertName", + RequireClientCert: true, }, + InlineRouteConfig: inlineRouteConfig, + HTTPFilters: routerFilterList, }, }, - Raw: listenerWithValidationContextNewFields, - }}, - }, - wantMD: UpdateMetadata{ - Status: ServiceStatusACKed, - Version: testVersion, + }, + Raw: listenerWithValidationContextNewFields, }, }, } for _, test := range tests { t.Run(test.name, func(t *testing.T) { - opts := &UnmarshalOptions{ - Version: testVersion, - Resources: test.resources, - } - gotUpdate, md, err := UnmarshalListener(opts) - if (err != nil) != (test.wantErr != "") { - t.Fatalf("UnmarshalListener(%+v), got err: %v, wantErr: %v", opts, err, test.wantErr) - } + name, update, err := unmarshalListenerResource(test.resource, nil) if err != nil && !strings.Contains(err.Error(), test.wantErr) { - t.Fatalf("UnmarshalListener(%+v) = %v wantErr: %q", opts, err, test.wantErr) + t.Errorf("unmarshalListenerResource(%s) = %v wantErr: %q", pretty.ToJSON(test.resource), err, test.wantErr) } - if diff := cmp.Diff(gotUpdate, test.wantUpdate, cmpOpts); diff != "" { - t.Errorf("got unexpected update, diff (-got +want): %v", diff) + if name != test.wantName { + t.Errorf("unmarshalListenerResource(%s), got name: %s, want: %s", pretty.ToJSON(test.resource), name, test.wantName) } - if diff := cmp.Diff(md, test.wantMD, cmpOptsIgnoreDetails); diff != "" { - t.Errorf("got unexpected metadata, diff (-got +want): %v", diff) + if diff := cmp.Diff(update, test.wantUpdate, cmpOpts); diff != "" { + t.Errorf("unmarshalListenerResource(%s), got unexpected update, diff (-got +want): %v", pretty.ToJSON(test.resource), diff) } }) } diff --git a/xds/internal/xdsclient/xdsresource/unmarshal_rds.go b/xds/internal/xdsclient/xdsresource/unmarshal_rds.go index 32c48d46b..8f434d11d 100644 --- a/xds/internal/xdsclient/xdsresource/unmarshal_rds.go +++ b/xds/internal/xdsclient/xdsresource/unmarshal_rds.go @@ -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 { diff --git a/xds/internal/xdsclient/xdsresource/unmarshal_rds_test.go b/xds/internal/xdsclient/xdsresource/unmarshal_rds_test.go index b6d02f30b..ea308825c 100644 --- a/xds/internal/xdsclient/xdsresource/unmarshal_rds_test.go +++ b/xds/internal/xdsclient/xdsresource/unmarshal_rds_test.go @@ -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) } }) } diff --git a/xds/xds.go b/xds/xds.go index 7c479f5f8..706e11c49 100644 --- a/xds/xds.go +++ b/xds/xds.go @@ -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() {