mirror of https://github.com/grpc/grpc-go.git
				
				
				
			xdsclient: resource agnostic API implementation (#5776)
This commit is contained in:
		
							parent
							
								
									07ac97c355
								
							
						
					
					
						commit
						08479c5e2e
					
				
							
								
								
									
										110
									
								
								xds/csds/csds.go
								
								
								
								
							
							
						
						
									
										110
									
								
								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,19 +173,19 @@ 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 { | ||||
| 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:  d.MD.Version, | ||||
| 			XdsConfig:    d.Raw, | ||||
| 			LastUpdated:  timestamppb.New(d.MD.Timestamp), | ||||
| 			ClientStatus: serviceStatusToProto(d.MD.Status), | ||||
| 				VersionInfo:  update.MD.Version, | ||||
| 				XdsConfig:    update.Raw, | ||||
| 				LastUpdated:  timestamppb.New(update.MD.Timestamp), | ||||
| 				ClientStatus: serviceStatusToProto(update.MD.Status), | ||||
| 			} | ||||
| 		if errState := d.MD.ErrState; errState != nil { | ||||
| 			if errState := update.MD.ErrState; errState != nil { | ||||
| 				config.ErrorState = &v3adminpb.UpdateFailureState{ | ||||
| 					LastUpdateAttempt: timestamppb.New(errState.Timestamp), | ||||
| 					Details:           errState.Err.Error(), | ||||
|  | @ -195,6 +194,7 @@ func dumpToGenericXdsConfig(typeURL string, dumpF func() map[string]xdsresource. | |||
| 			} | ||||
| 			ret = append(ret, config) | ||||
| 		} | ||||
| 	} | ||||
| 	return ret | ||||
| } | ||||
| 
 | ||||
|  |  | |||
|  | @ -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)) | ||||
| 	} | ||||
| } | ||||
|  | @ -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) | ||||
| 			} | ||||
|  |  | |||
|  | @ -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 ( | ||||
|  |  | |||
|  | @ -49,7 +49,6 @@ import ( | |||
| 	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.
 | ||||
| ) | ||||
| 
 | ||||
| const ( | ||||
|  |  | |||
|  | @ -54,7 +54,6 @@ import ( | |||
| 
 | ||||
| 	_ "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.
 | ||||
| ) | ||||
| 
 | ||||
| const defaultTestTimeout = 10 * time.Second | ||||
|  |  | |||
|  | @ -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) | ||||
| 	} | ||||
| 	return func() { | ||||
| 		if cancelF() { | ||||
| 			a.controller.RemoveWatch(xdsresource.ListenerResource, serviceName) | ||||
| func (a *authority) handleWatchTimerExpiry(state *resourceState, err error) { | ||||
| 	a.resourcesMu.Lock() | ||||
| 	defer a.resourcesMu.Unlock() | ||||
| 
 | ||||
| 	if state.wState == watchStateCanceled { | ||||
| 		return | ||||
| 	} | ||||
| 
 | ||||
| 	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 | ||||
| } | ||||
|  |  | |||
|  | @ -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() | ||||
| } | ||||
|  |  | |||
|  | @ -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 | ||||
| } | ||||
| 
 | ||||
|  |  | |||
|  | @ -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() | ||||
|  |  | |||
|  | @ -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 | ||||
| } | ||||
|  |  | |||
|  | @ -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() | ||||
|  |  | |||
|  | @ -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 | ||||
| } | ||||
|  | @ -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.
 | ||||
| 
 | ||||
| 	// 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] | ||||
| 	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[u] = rType | ||||
| 	} | ||||
| 	r.types[url] = rType | ||||
| 	return nil | ||||
| } | ||||
|  |  | |||
|  | @ -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 | ||||
| 	} | ||||
| } | ||||
|  | @ -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) | ||||
| 	} | ||||
| } | ||||
|  | @ -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}) | ||||
| } | ||||
|  | @ -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 | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
|  | @ -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++ | ||||
| } | ||||
|  | @ -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") | ||||
| 	} | ||||
| } | ||||
|  | @ -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) | ||||
| 	} | ||||
| } | ||||
|  | @ -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") | ||||
| 	} | ||||
| } | ||||
|  | @ -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") | ||||
| 	} | ||||
| } | ||||
|  | @ -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") | ||||
| 	} | ||||
| } | ||||
|  | @ -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 | ||||
| } | ||||
|  | @ -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 | ||||
| } | ||||
|  | @ -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 | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
|  | @ -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 | ||||
| } | ||||
|  | @ -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 | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
|  | @ -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 | ||||
| } | ||||
|  | @ -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 | ||||
| } | ||||
|  | @ -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) | ||||
| 	} | ||||
| } | ||||
|  | @ -70,18 +70,17 @@ func (s) TestWatchCallAnotherWatch(t *testing.T) { | |||
| 	updateCh1 := testutils.NewChannel() | ||||
| 	updateCh2 := testutils.NewChannel() | ||||
| 	updateCh3 := testutils.NewChannel() | ||||
| 	var rdsCancel2, rdsCancel3 func() | ||||
| 	rdsCancel1 := client.WatchRouteConfig(rdsName, func(u xdsresource.RouteConfigUpdate, err error) { | ||||
| 		updateCh1.Send(xdsresource.RouteConfigUpdateErrTuple{Update: u, Err: err}) | ||||
| 
 | ||||
| 		// Watch for the same resource name.
 | ||||
| 		rdsCancel2 = client.WatchRouteConfig(rdsName, func(u xdsresource.RouteConfigUpdate, err error) { | ||||
| 		rdsCancel2 := client.WatchRouteConfig(rdsName, func(u xdsresource.RouteConfigUpdate, err error) { | ||||
| 			updateCh2.Send(xdsresource.RouteConfigUpdateErrTuple{Update: u, Err: err}) | ||||
| 		}) | ||||
| 		t.Cleanup(rdsCancel2) | ||||
| 		// Watch for a different resource name.
 | ||||
| 		rdsCancel3 = client.WatchRouteConfig(rdsNameNewStyle, func(u xdsresource.RouteConfigUpdate, err error) { | ||||
| 		rdsCancel3 := client.WatchRouteConfig(rdsNameNewStyle, func(u xdsresource.RouteConfigUpdate, err error) { | ||||
| 			updateCh3.Send(xdsresource.RouteConfigUpdateErrTuple{Update: u, Err: err}) | ||||
| 			rdsCancel3() | ||||
| 		}) | ||||
| 		t.Cleanup(rdsCancel3) | ||||
| 	}) | ||||
|  |  | |||
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							|  | @ -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) | ||||
|  |  | |||
|  | @ -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 | ||||
| } | ||||
|  | @ -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) | ||||
| } | ||||
|  | @ -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 | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
|  | @ -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)) | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
|  | @ -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 | ||||
| 	} | ||||
| } | ||||
|  | @ -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 | ||||
| } | ||||
|  | @ -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 { | ||||
|  |  | |||
|  | @ -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.
 | ||||
|  |  | |||
|  | @ -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, | ||||
| 		}, | ||||
|  |  | |||
|  | @ -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.
 | ||||
|  |  | |||
|  | @ -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 { | ||||
|  |  | |||
|  | @ -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, | ||||
| 		}, | ||||
|  |  | |||
|  | @ -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) | ||||
| 		}), | ||||
| 	} | ||||
| ) | ||||
|  |  | |||
|  | @ -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 | ||||
| } | ||||
|  |  | |||
|  | @ -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 { | ||||
|  |  | |||
|  | @ -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 | ||||
| } | ||||
|  |  | |||
|  | @ -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,71 +1443,39 @@ 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, | ||||
| 				}, | ||||
| 			}, | ||||
| 			resource: &anypb.Any{TypeUrl: version.V3HTTPConnManagerURL}, | ||||
| 			wantErr:  true, | ||||
| 		}, | ||||
| 		{ | ||||
| 			name: "badly marshaled cluster resource", | ||||
| 			resources: []*anypb.Any{ | ||||
| 				{ | ||||
| 			resource: &anypb.Any{ | ||||
| 				TypeUrl: version.V3ClusterURL, | ||||
| 				Value:   []byte{1, 2, 3, 4}, | ||||
| 			}, | ||||
| 			}, | ||||
| 			wantMD: UpdateMetadata{ | ||||
| 				Status:  ServiceStatusNACKed, | ||||
| 				Version: testVersion, | ||||
| 				ErrState: &UpdateErrorMetadata{ | ||||
| 					Version: testVersion, | ||||
| 					Err:     cmpopts.AnyError, | ||||
| 				}, | ||||
| 			}, | ||||
| 			wantErr: true, | ||||
| 		}, | ||||
| 		{ | ||||
| 			name: "bad cluster resource", | ||||
| 			resources: []*anypb.Any{ | ||||
| 				testutils.MarshalAny(&v3clusterpb.Cluster{ | ||||
| 			resource: 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, | ||||
| 				}, | ||||
| 			}, | ||||
| 			wantName: "test", | ||||
| 			wantErr:  true, | ||||
| 		}, | ||||
| 		{ | ||||
| 			name: "cluster resource with non-self lrs_server field", | ||||
| 			resources: []*anypb.Any{ | ||||
| 				testutils.MarshalAny(&v3clusterpb.Cluster{ | ||||
| 			resource: testutils.MarshalAny(&v3clusterpb.Cluster{ | ||||
| 				Name:                 "test", | ||||
| 				ClusterDiscoveryType: &v3clusterpb.Cluster_Type{Type: v3clusterpb.Cluster_EDS}, | ||||
| 				EdsClusterConfig: &v3clusterpb.Cluster_EdsClusterConfig{ | ||||
|  | @ -1524,166 +1493,71 @@ func (s) TestUnmarshalCluster(t *testing.T) { | |||
| 					}, | ||||
| 				}, | ||||
| 			}), | ||||
| 			}, | ||||
| 			wantUpdate: map[string]ClusterUpdateErrTuple{ | ||||
| 				"test": {Err: cmpopts.AnyError}, | ||||
| 			}, | ||||
| 			wantMD: UpdateMetadata{ | ||||
| 				Status:  ServiceStatusNACKed, | ||||
| 				Version: testVersion, | ||||
| 				ErrState: &UpdateErrorMetadata{ | ||||
| 					Version: testVersion, | ||||
| 					Err:     cmpopts.AnyError, | ||||
| 				}, | ||||
| 			}, | ||||
| 			wantName: "test", | ||||
| 			wantErr:  true, | ||||
| 		}, | ||||
| 		{ | ||||
| 			name:     "v2 cluster", | ||||
| 			resources: []*anypb.Any{v2ClusterAny}, | ||||
| 			wantUpdate: map[string]ClusterUpdateErrTuple{ | ||||
| 				v2ClusterName: {Update: ClusterUpdate{ | ||||
| 			resource: v2ClusterAny, | ||||
| 			wantName: v2ClusterName, | ||||
| 			wantUpdate: 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{ | ||||
| 			resource: testutils.MarshalAny(&v2xdspb.Resource{Resource: v2ClusterAny}), | ||||
| 			wantName: v2ClusterName, | ||||
| 			wantUpdate: 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{ | ||||
| 			resource: v3ClusterAny, | ||||
| 			wantName: v3ClusterName, | ||||
| 			wantUpdate: 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{ | ||||
| 			resource: testutils.MarshalAny(&v3discoverypb.Resource{Resource: v3ClusterAny}), | ||||
| 			wantName: v3ClusterName, | ||||
| 			wantUpdate: 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{ | ||||
| 			resource: v3ClusterAnyWithEDSConfigSourceSelf, | ||||
| 			wantName: v3ClusterName, | ||||
| 			wantUpdate: 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, | ||||
| 				}, | ||||
| 			}, | ||||
| 			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) | ||||
| 			} | ||||
| 		}) | ||||
| 	} | ||||
|  |  | |||
|  | @ -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 { | ||||
|  |  | |||
|  | @ -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,70 +233,43 @@ 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, | ||||
| 				}, | ||||
| 			}, | ||||
| 			resource: &anypb.Any{TypeUrl: version.V3HTTPConnManagerURL}, | ||||
| 			wantErr:  true, | ||||
| 		}, | ||||
| 		{ | ||||
| 			name: "badly marshaled clusterLoadAssignment resource", | ||||
| 			resources: []*anypb.Any{ | ||||
| 				{ | ||||
| 			resource: &anypb.Any{ | ||||
| 				TypeUrl: version.V3EndpointsURL, | ||||
| 				Value:   []byte{1, 2, 3, 4}, | ||||
| 			}, | ||||
| 			}, | ||||
| 			wantMD: UpdateMetadata{ | ||||
| 				Status:  ServiceStatusNACKed, | ||||
| 				Version: testVersion, | ||||
| 				ErrState: &UpdateErrorMetadata{ | ||||
| 					Version: testVersion, | ||||
| 					Err:     cmpopts.AnyError, | ||||
| 				}, | ||||
| 			}, | ||||
| 			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", | ||||
| 			resources: []*anypb.Any{v3EndpointsAny}, | ||||
| 			wantUpdate: map[string]EndpointsUpdateErrTuple{ | ||||
| 				"test": {Update: EndpointsUpdate{ | ||||
| 			resource: v3EndpointsAny, | ||||
| 			wantName: "test", | ||||
| 			wantUpdate: EndpointsUpdate{ | ||||
| 				Drops: nil, | ||||
| 				Localities: []Locality{ | ||||
| 					{ | ||||
|  | @ -321,18 +294,13 @@ func (s) TestUnmarshalEndpoints(t *testing.T) { | |||
| 					}, | ||||
| 				}, | ||||
| 				Raw: v3EndpointsAny, | ||||
| 				}}, | ||||
| 			}, | ||||
| 			wantMD: UpdateMetadata{ | ||||
| 				Status:  ServiceStatusACKed, | ||||
| 				Version: testVersion, | ||||
| 			}, | ||||
| 		}, | ||||
| 		{ | ||||
| 			name:     "v3 endpoints wrapped", | ||||
| 			resources: []*anypb.Any{testutils.MarshalAny(&v3discoverypb.Resource{Resource: v3EndpointsAny})}, | ||||
| 			wantUpdate: map[string]EndpointsUpdateErrTuple{ | ||||
| 				"test": {Update: EndpointsUpdate{ | ||||
| 			resource: testutils.MarshalAny(&v3discoverypb.Resource{Resource: v3EndpointsAny}), | ||||
| 			wantName: "test", | ||||
| 			wantUpdate: EndpointsUpdate{ | ||||
| 				Drops: nil, | ||||
| 				Localities: []Locality{ | ||||
| 					{ | ||||
|  | @ -357,80 +325,20 @@ func (s) TestUnmarshalEndpoints(t *testing.T) { | |||
| 					}, | ||||
| 				}, | ||||
| 				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, | ||||
| 						}, | ||||
| 					}, | ||||
| 					Raw: v3EndpointsAny, | ||||
| 				}}, | ||||
| 				"bad": {Err: cmpopts.AnyError}, | ||||
| 			}, | ||||
| 			wantMD: UpdateMetadata{ | ||||
| 				Status:  ServiceStatusNACKed, | ||||
| 				Version: testVersion, | ||||
| 				ErrState: &UpdateErrorMetadata{ | ||||
| 					Version: testVersion, | ||||
| 					Err:     cmpopts.AnyError, | ||||
| 				}, | ||||
| 			}, | ||||
| 			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) | ||||
| 			} | ||||
| 		}) | ||||
| 	} | ||||
|  |  | |||
|  | @ -36,16 +36,7 @@ import ( | |||
| 	"google.golang.org/protobuf/types/known/anypb" | ||||
| ) | ||||
| 
 | ||||
| // UnmarshalListener processes resources received in an LDS response, validates
 | ||||
| // them, and transforms them into a native struct which contains only fields we
 | ||||
| // are interested in.
 | ||||
| func UnmarshalListener(opts *UnmarshalOptions) (map[string]ListenerUpdateErrTuple, UpdateMetadata, error) { | ||||
| 	update := make(map[string]ListenerUpdateErrTuple) | ||||
| 	md, err := processAllResources(opts, update) | ||||
| 	return update, md, err | ||||
| } | ||||
| 
 | ||||
| func unmarshalListenerResource(r *anypb.Any, f UpdateValidatorFunc, logger *grpclog.PrefixLogger) (string, ListenerUpdate, error) { | ||||
| func unmarshalListenerResource(r *anypb.Any, logger *grpclog.PrefixLogger) (string, ListenerUpdate, error) { | ||||
| 	r, err := unwrapResource(r) | ||||
| 	if err != nil { | ||||
| 		return "", ListenerUpdate{}, fmt.Errorf("failed to unwrap resource: %v", err) | ||||
|  | @ -66,11 +57,6 @@ func unmarshalListenerResource(r *anypb.Any, f UpdateValidatorFunc, logger *grpc | |||
| 	if err != nil { | ||||
| 		return lis.GetName(), ListenerUpdate{}, err | ||||
| 	} | ||||
| 	if f != nil { | ||||
| 		if err := f(*lu); err != nil { | ||||
| 			return lis.GetName(), ListenerUpdate{}, err | ||||
| 		} | ||||
| 	} | ||||
| 	lu.Raw = r | ||||
| 	return lis.GetName(), *lu, nil | ||||
| } | ||||
|  |  | |||
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							|  | @ -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 { | ||||
|  |  | |||
|  | @ -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,58 +908,32 @@ 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, | ||||
| 				}, | ||||
| 			}, | ||||
| 			resource: &anypb.Any{TypeUrl: version.V3HTTPConnManagerURL}, | ||||
| 			wantErr:  true, | ||||
| 		}, | ||||
| 		{ | ||||
| 			name: "badly marshaled routeconfig resource", | ||||
| 			resources: []*anypb.Any{ | ||||
| 				{ | ||||
| 			resource: &anypb.Any{ | ||||
| 				TypeUrl: version.V3RouteConfigURL, | ||||
| 				Value:   []byte{1, 2, 3, 4}, | ||||
| 			}, | ||||
| 			}, | ||||
| 			wantMD: UpdateMetadata{ | ||||
| 				Status:  ServiceStatusNACKed, | ||||
| 				Version: testVersion, | ||||
| 				ErrState: &UpdateErrorMetadata{ | ||||
| 					Version: testVersion, | ||||
| 					Err:     cmpopts.AnyError, | ||||
| 				}, | ||||
| 			}, | ||||
| 			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{ | ||||
| 			resource: v2RouteConfig, | ||||
| 			wantName: v2RouteConfigName, | ||||
| 			wantUpdate: RouteConfigUpdate{ | ||||
| 				VirtualHosts: []*VirtualHost{ | ||||
| 					{ | ||||
| 						Domains: []string{uninterestingDomain}, | ||||
|  | @ -974,18 +949,13 @@ func (s) TestUnmarshalRouteConfig(t *testing.T) { | |||
| 					}, | ||||
| 				}, | ||||
| 				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{ | ||||
| 			resource: testutils.MarshalAny(&v2xdspb.Resource{Resource: v2RouteConfig}), | ||||
| 			wantName: v2RouteConfigName, | ||||
| 			wantUpdate: RouteConfigUpdate{ | ||||
| 				VirtualHosts: []*VirtualHost{ | ||||
| 					{ | ||||
| 						Domains: []string{uninterestingDomain}, | ||||
|  | @ -1001,18 +971,13 @@ func (s) TestUnmarshalRouteConfig(t *testing.T) { | |||
| 					}, | ||||
| 				}, | ||||
| 				Raw: v2RouteConfig, | ||||
| 				}}, | ||||
| 			}, | ||||
| 			wantMD: UpdateMetadata{ | ||||
| 				Status:  ServiceStatusACKed, | ||||
| 				Version: testVersion, | ||||
| 			}, | ||||
| 		}, | ||||
| 		{ | ||||
| 			name:     "v3 routeConfig resource", | ||||
| 			resources: []*anypb.Any{v3RouteConfig}, | ||||
| 			wantUpdate: map[string]RouteConfigUpdateErrTuple{ | ||||
| 				v3RouteConfigName: {Update: RouteConfigUpdate{ | ||||
| 			resource: v3RouteConfig, | ||||
| 			wantName: v3RouteConfigName, | ||||
| 			wantUpdate: RouteConfigUpdate{ | ||||
| 				VirtualHosts: []*VirtualHost{ | ||||
| 					{ | ||||
| 						Domains: []string{uninterestingDomain}, | ||||
|  | @ -1028,18 +993,13 @@ func (s) TestUnmarshalRouteConfig(t *testing.T) { | |||
| 					}, | ||||
| 				}, | ||||
| 				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{ | ||||
| 			resource: testutils.MarshalAny(&v3discoverypb.Resource{Resource: v3RouteConfig}), | ||||
| 			wantName: v3RouteConfigName, | ||||
| 			wantUpdate: RouteConfigUpdate{ | ||||
| 				VirtualHosts: []*VirtualHost{ | ||||
| 					{ | ||||
| 						Domains: []string{uninterestingDomain}, | ||||
|  | @ -1055,134 +1015,20 @@ func (s) TestUnmarshalRouteConfig(t *testing.T) { | |||
| 					}, | ||||
| 				}, | ||||
| 				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, | ||||
| 				}, | ||||
| 			}, | ||||
| 			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) | ||||
| 			} | ||||
| 		}) | ||||
| 	} | ||||
|  |  | |||
|  | @ -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/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
 | ||||
| 	_ "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.
 | ||||
| 
 | ||||
| 	v3statusgrpc "github.com/envoyproxy/go-control-plane/envoy/service/status/v3" | ||||
| ) | ||||
| 
 | ||||
| func init() { | ||||
|  |  | |||
		Loading…
	
		Reference in New Issue