mirror of https://github.com/grpc/grpc-go.git
				
				
				
			xds/internal/xdsclient: Add counter metrics for valid and invalid resource updates (#8038)
This commit is contained in:
		
							parent
							
								
									f227ba9ba0
								
							
						
					
					
						commit
						267a09bb5d
					
				|  | @ -64,6 +64,9 @@ var ( | |||
| 	// gRPC server. An xDS-enabled server needs to know what type of credentials
 | ||||
| 	// is configured on the underlying gRPC server. This is set by server.go.
 | ||||
| 	GetServerCredentials any // func (*grpc.Server) credentials.TransportCredentials
 | ||||
| 	// MetricsRecorderForServer returns the MetricsRecorderList derived from a
 | ||||
| 	// server's stats handlers.
 | ||||
| 	MetricsRecorderForServer any // func (*grpc.Server) estats.MetricsRecorder
 | ||||
| 	// CanonicalString returns the canonical string of the code defined here:
 | ||||
| 	// https://github.com/grpc/grpc/blob/master/doc/statuscodes.md.
 | ||||
| 	//
 | ||||
|  |  | |||
|  | @ -30,6 +30,7 @@ import ( | |||
| 
 | ||||
| 	"google.golang.org/grpc/attributes" | ||||
| 	"google.golang.org/grpc/credentials" | ||||
| 	"google.golang.org/grpc/experimental/stats" | ||||
| 	"google.golang.org/grpc/internal" | ||||
| 	"google.golang.org/grpc/serviceconfig" | ||||
| ) | ||||
|  | @ -175,6 +176,8 @@ type BuildOptions struct { | |||
| 	// Authority is the effective authority of the clientconn for which the
 | ||||
| 	// resolver is built.
 | ||||
| 	Authority string | ||||
| 	// MetricsRecorder is the metrics recorder to do recording.
 | ||||
| 	MetricsRecorder stats.MetricsRecorder | ||||
| } | ||||
| 
 | ||||
| // An Endpoint is one network endpoint, or server, which may have multiple
 | ||||
|  |  | |||
|  | @ -77,6 +77,7 @@ func (ccr *ccResolverWrapper) start() error { | |||
| 			CredsBundle:          ccr.cc.dopts.copts.CredsBundle, | ||||
| 			Dialer:               ccr.cc.dopts.copts.Dialer, | ||||
| 			Authority:            ccr.cc.authority, | ||||
| 			MetricsRecorder:      ccr.cc.metricsRecorderList, | ||||
| 		} | ||||
| 		var err error | ||||
| 		// The delegating resolver is used unless:
 | ||||
|  |  | |||
|  | @ -37,12 +37,14 @@ import ( | |||
| 	"google.golang.org/grpc/credentials" | ||||
| 	"google.golang.org/grpc/encoding" | ||||
| 	"google.golang.org/grpc/encoding/proto" | ||||
| 	estats "google.golang.org/grpc/experimental/stats" | ||||
| 	"google.golang.org/grpc/grpclog" | ||||
| 	"google.golang.org/grpc/internal" | ||||
| 	"google.golang.org/grpc/internal/binarylog" | ||||
| 	"google.golang.org/grpc/internal/channelz" | ||||
| 	"google.golang.org/grpc/internal/grpcsync" | ||||
| 	"google.golang.org/grpc/internal/grpcutil" | ||||
| 	istats "google.golang.org/grpc/internal/stats" | ||||
| 	"google.golang.org/grpc/internal/transport" | ||||
| 	"google.golang.org/grpc/keepalive" | ||||
| 	"google.golang.org/grpc/mem" | ||||
|  | @ -82,6 +84,9 @@ func init() { | |||
| 	internal.BinaryLogger = binaryLogger | ||||
| 	internal.JoinServerOptions = newJoinServerOption | ||||
| 	internal.BufferPool = bufferPool | ||||
| 	internal.MetricsRecorderForServer = func(srv *Server) estats.MetricsRecorder { | ||||
| 		return istats.NewMetricsRecorderList(srv.opts.statsHandlers) | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| var statusOK = status.New(codes.OK, "") | ||||
|  |  | |||
|  | @ -26,5 +26,5 @@ var ( | |||
| 	NewWRR any // func() wrr.WRR
 | ||||
| 
 | ||||
| 	// NewXDSClient is the function used to create a new xDS client.
 | ||||
| 	NewXDSClient any // func(string) (xdsclient.XDSClient, func(), error)
 | ||||
| 	NewXDSClient any // func(string, estats.MetricsRecorder) (xdsclient.XDSClient, func(), error)
 | ||||
| ) | ||||
|  |  | |||
|  | @ -25,6 +25,7 @@ import ( | |||
| 	rand "math/rand/v2" | ||||
| 	"sync/atomic" | ||||
| 
 | ||||
| 	estats "google.golang.org/grpc/experimental/stats" | ||||
| 	"google.golang.org/grpc/internal" | ||||
| 	"google.golang.org/grpc/internal/grpclog" | ||||
| 	"google.golang.org/grpc/internal/grpcsync" | ||||
|  | @ -50,13 +51,16 @@ const Scheme = "xds" | |||
| // the provided config and a new xDS client in that pool.
 | ||||
| func newBuilderWithConfigForTesting(config []byte) (resolver.Builder, error) { | ||||
| 	return &xdsResolverBuilder{ | ||||
| 		newXDSClient: func(name string) (xdsclient.XDSClient, func(), error) { | ||||
| 		newXDSClient: func(name string, mr estats.MetricsRecorder) (xdsclient.XDSClient, func(), error) { | ||||
| 			config, err := bootstrap.NewConfigFromContents(config) | ||||
| 			if err != nil { | ||||
| 				return nil, nil, err | ||||
| 			} | ||||
| 			pool := xdsclient.NewPool(config) | ||||
| 			return pool.NewClientForTesting(xdsclient.OptionsForTesting{Name: name}) | ||||
| 			return pool.NewClientForTesting(xdsclient.OptionsForTesting{ | ||||
| 				Name:            name, | ||||
| 				MetricsRecorder: mr, | ||||
| 			}) | ||||
| 		}, | ||||
| 	}, nil | ||||
| } | ||||
|  | @ -66,8 +70,11 @@ func newBuilderWithConfigForTesting(config []byte) (resolver.Builder, error) { | |||
| // specific xds client pool being used.
 | ||||
| func newBuilderWithPoolForTesting(pool *xdsclient.Pool) (resolver.Builder, error) { | ||||
| 	return &xdsResolverBuilder{ | ||||
| 		newXDSClient: func(name string) (xdsclient.XDSClient, func(), error) { | ||||
| 			return pool.NewClientForTesting(xdsclient.OptionsForTesting{Name: name}) | ||||
| 		newXDSClient: func(name string, mr estats.MetricsRecorder) (xdsclient.XDSClient, func(), error) { | ||||
| 			return pool.NewClientForTesting(xdsclient.OptionsForTesting{ | ||||
| 				Name:            name, | ||||
| 				MetricsRecorder: mr, | ||||
| 			}) | ||||
| 		}, | ||||
| 	}, nil | ||||
| } | ||||
|  | @ -82,7 +89,7 @@ func init() { | |||
| } | ||||
| 
 | ||||
| type xdsResolverBuilder struct { | ||||
| 	newXDSClient func(string) (xdsclient.XDSClient, func(), error) | ||||
| 	newXDSClient func(string, estats.MetricsRecorder) (xdsclient.XDSClient, func(), error) | ||||
| } | ||||
| 
 | ||||
| // Build helps implement the resolver.Builder interface.
 | ||||
|  | @ -115,11 +122,11 @@ func (b *xdsResolverBuilder) Build(target resolver.Target, cc resolver.ClientCon | |||
| 	r.serializerCancel = cancel | ||||
| 
 | ||||
| 	// Initialize the xDS client.
 | ||||
| 	newXDSClient := rinternal.NewXDSClient.(func(string) (xdsclient.XDSClient, func(), error)) | ||||
| 	newXDSClient := rinternal.NewXDSClient.(func(string, estats.MetricsRecorder) (xdsclient.XDSClient, func(), error)) | ||||
| 	if b.newXDSClient != nil { | ||||
| 		newXDSClient = b.newXDSClient | ||||
| 	} | ||||
| 	client, closeFn, err := newXDSClient(target.String()) | ||||
| 	client, closeFn, err := newXDSClient(target.String(), opts.MetricsRecorder) | ||||
| 	if err != nil { | ||||
| 		return nil, fmt.Errorf("xds: failed to create xds-client: %v", err) | ||||
| 	} | ||||
|  |  | |||
|  | @ -32,6 +32,7 @@ import ( | |||
| 	"github.com/google/go-cmp/cmp" | ||||
| 	"github.com/google/uuid" | ||||
| 	"google.golang.org/grpc/codes" | ||||
| 	estats "google.golang.org/grpc/experimental/stats" | ||||
| 	"google.golang.org/grpc/internal" | ||||
| 	iresolver "google.golang.org/grpc/internal/resolver" | ||||
| 	"google.golang.org/grpc/internal/testutils" | ||||
|  | @ -257,7 +258,7 @@ func (s) TestResolverCloseClosesXDSClient(t *testing.T) { | |||
| 	// client is closed.
 | ||||
| 	origNewClient := rinternal.NewXDSClient | ||||
| 	closeCh := make(chan struct{}) | ||||
| 	rinternal.NewXDSClient = func(string) (xdsclient.XDSClient, func(), error) { | ||||
| 	rinternal.NewXDSClient = func(string, estats.MetricsRecorder) (xdsclient.XDSClient, func(), error) { | ||||
| 		bc := e2e.DefaultBootstrapContents(t, uuid.New().String(), "dummy-management-server-address") | ||||
| 		config, err := bootstrap.NewConfigFromContents(bc) | ||||
| 		if err != nil { | ||||
|  |  | |||
|  | @ -23,6 +23,7 @@ import ( | |||
| 	"sync" | ||||
| 	"sync/atomic" | ||||
| 
 | ||||
| 	"google.golang.org/grpc/experimental/stats" | ||||
| 	"google.golang.org/grpc/grpclog" | ||||
| 	igrpclog "google.golang.org/grpc/internal/grpclog" | ||||
| 	"google.golang.org/grpc/internal/grpcsync" | ||||
|  | @ -87,6 +88,8 @@ type authority struct { | |||
| 	xdsClientSerializer       *grpcsync.CallbackSerializer // Serializer to run call ins from the xDS client, owned by this authority.
 | ||||
| 	xdsClientSerializerClose  func()                       // Function to close the above serializer.
 | ||||
| 	logger                    *igrpclog.PrefixLogger       // Logger for this authority.
 | ||||
| 	target                    string                       // The gRPC Channel target.
 | ||||
| 	metricsRecorder           stats.MetricsRecorder        // The metrics recorder used for emitting metrics.
 | ||||
| 
 | ||||
| 	// The below defined fields must only be accessed in the context of the
 | ||||
| 	// serializer callback, owned by this authority.
 | ||||
|  | @ -120,6 +123,8 @@ type authorityBuildOptions struct { | |||
| 	serializer       *grpcsync.CallbackSerializer // Callback serializer for invoking watch callbacks
 | ||||
| 	getChannelForADS xdsChannelForADS             // Function to acquire a reference to an xdsChannel
 | ||||
| 	logPrefix        string                       // Prefix for logging
 | ||||
| 	target           string                       // Target for the gRPC Channel that owns xDS Client/Authority
 | ||||
| 	metricsRecorder  stats.MetricsRecorder        // metricsRecorder to emit metrics
 | ||||
| } | ||||
| 
 | ||||
| // newAuthority creates a new authority instance with the provided
 | ||||
|  | @ -143,6 +148,8 @@ func newAuthority(args authorityBuildOptions) *authority { | |||
| 		xdsClientSerializerClose:  cancel, | ||||
| 		logger:                    igrpclog.NewPrefixLogger(l, logPrefix), | ||||
| 		resources:                 make(map[xdsresource.Type]map[string]*resourceState), | ||||
| 		target:                    args.target, | ||||
| 		metricsRecorder:           args.metricsRecorder, | ||||
| 	} | ||||
| 
 | ||||
| 	// Create an ordered list of xdsChannels with their server configs. The
 | ||||
|  | @ -358,6 +365,7 @@ func (a *authority) handleADSResourceUpdate(serverConfig *bootstrap.ServerConfig | |||
| 		// On error, keep previous version of the resource. But update status
 | ||||
| 		// and error.
 | ||||
| 		if uErr.Err != nil { | ||||
| 			xdsClientResourceUpdatesInvalidMetric.Record(a.metricsRecorder, 1, a.target, serverConfig.ServerURI(), rType.TypeName()) | ||||
| 			state.md.ErrState = md.ErrState | ||||
| 			state.md.Status = md.Status | ||||
| 			for watcher := range state.watchers { | ||||
|  | @ -369,6 +377,8 @@ func (a *authority) handleADSResourceUpdate(serverConfig *bootstrap.ServerConfig | |||
| 			continue | ||||
| 		} | ||||
| 
 | ||||
| 		xdsClientResourceUpdatesValidMetric.Record(a.metricsRecorder, 1, a.target, serverConfig.ServerURI(), rType.TypeName()) | ||||
| 
 | ||||
| 		if state.deletionIgnored { | ||||
| 			state.deletionIgnored = false | ||||
| 			a.logger.Infof("A valid update was received for resource %q of type %q after previously ignoring a deletion", name, rType.TypeName()) | ||||
|  |  | |||
|  | @ -25,6 +25,7 @@ import ( | |||
| 
 | ||||
| 	"github.com/google/uuid" | ||||
| 	"google.golang.org/grpc/internal/testutils" | ||||
| 	"google.golang.org/grpc/internal/testutils/stats" | ||||
| 	"google.golang.org/grpc/internal/testutils/xds/e2e" | ||||
| 	"google.golang.org/grpc/internal/xds/bootstrap" | ||||
| ) | ||||
|  | @ -60,7 +61,7 @@ func (s) TestClientNew_Single(t *testing.T) { | |||
| 	defer func() { xdsClientImplCloseHook = origClientImplCloseHook }() | ||||
| 
 | ||||
| 	// The first call to New() should create a new client.
 | ||||
| 	_, closeFunc, err := pool.NewClient(t.Name()) | ||||
| 	_, closeFunc, err := pool.NewClient(t.Name(), &stats.NoopMetricsRecorder{}) | ||||
| 	if err != nil { | ||||
| 		t.Fatalf("Failed to create xDS client: %v", err) | ||||
| 	} | ||||
|  | @ -76,7 +77,7 @@ func (s) TestClientNew_Single(t *testing.T) { | |||
| 	closeFuncs := make([]func(), count) | ||||
| 	for i := 0; i < count; i++ { | ||||
| 		func() { | ||||
| 			_, closeFuncs[i], err = pool.NewClient(t.Name()) | ||||
| 			_, closeFuncs[i], err = pool.NewClient(t.Name(), &stats.NoopMetricsRecorder{}) | ||||
| 			if err != nil { | ||||
| 				t.Fatalf("%d-th call to New() failed with error: %v", i, err) | ||||
| 			} | ||||
|  | @ -114,7 +115,7 @@ func (s) TestClientNew_Single(t *testing.T) { | |||
| 
 | ||||
| 	// Calling New() again, after the previous Client was actually closed,
 | ||||
| 	// should create a new one.
 | ||||
| 	_, closeFunc, err = pool.NewClient(t.Name()) | ||||
| 	_, closeFunc, err = pool.NewClient(t.Name(), &stats.NoopMetricsRecorder{}) | ||||
| 	if err != nil { | ||||
| 		t.Fatalf("Failed to create xDS client: %v", err) | ||||
| 	} | ||||
|  | @ -156,7 +157,7 @@ func (s) TestClientNew_Multiple(t *testing.T) { | |||
| 
 | ||||
| 	// Create two xDS clients.
 | ||||
| 	client1Name := t.Name() + "-1" | ||||
| 	_, closeFunc1, err := pool.NewClient(client1Name) | ||||
| 	_, closeFunc1, err := pool.NewClient(client1Name, &stats.NoopMetricsRecorder{}) | ||||
| 	if err != nil { | ||||
| 		t.Fatalf("Failed to create xDS client: %v", err) | ||||
| 	} | ||||
|  | @ -171,7 +172,7 @@ func (s) TestClientNew_Multiple(t *testing.T) { | |||
| 	} | ||||
| 
 | ||||
| 	client2Name := t.Name() + "-2" | ||||
| 	_, closeFunc2, err := pool.NewClient(client2Name) | ||||
| 	_, closeFunc2, err := pool.NewClient(client2Name, &stats.NoopMetricsRecorder{}) | ||||
| 	if err != nil { | ||||
| 		t.Fatalf("Failed to create xDS client: %v", err) | ||||
| 	} | ||||
|  | @ -193,7 +194,7 @@ func (s) TestClientNew_Multiple(t *testing.T) { | |||
| 		defer wg.Done() | ||||
| 		for i := 0; i < count; i++ { | ||||
| 			var err error | ||||
| 			_, closeFuncs1[i], err = pool.NewClient(client1Name) | ||||
| 			_, closeFuncs1[i], err = pool.NewClient(client1Name, &stats.NoopMetricsRecorder{}) | ||||
| 			if err != nil { | ||||
| 				t.Errorf("%d-th call to New() failed with error: %v", i, err) | ||||
| 			} | ||||
|  | @ -203,7 +204,7 @@ func (s) TestClientNew_Multiple(t *testing.T) { | |||
| 		defer wg.Done() | ||||
| 		for i := 0; i < count; i++ { | ||||
| 			var err error | ||||
| 			_, closeFuncs2[i], err = pool.NewClient(client2Name) | ||||
| 			_, closeFuncs2[i], err = pool.NewClient(client2Name, &stats.NoopMetricsRecorder{}) | ||||
| 			if err != nil { | ||||
| 				t.Errorf("%d-th call to New() failed with error: %v", i, err) | ||||
| 			} | ||||
|  |  | |||
|  | @ -27,6 +27,7 @@ import ( | |||
| 	"time" | ||||
| 
 | ||||
| 	v3statuspb "github.com/envoyproxy/go-control-plane/envoy/service/status/v3" | ||||
| 	estats "google.golang.org/grpc/experimental/stats" | ||||
| 	"google.golang.org/grpc/internal" | ||||
| 	"google.golang.org/grpc/internal/backoff" | ||||
| 	"google.golang.org/grpc/internal/grpclog" | ||||
|  | @ -60,6 +61,21 @@ var ( | |||
| 	xdsClientImplCloseHook  = func(string) {} | ||||
| 
 | ||||
| 	defaultExponentialBackoff = backoff.DefaultExponential.Backoff | ||||
| 
 | ||||
| 	xdsClientResourceUpdatesValidMetric = estats.RegisterInt64Count(estats.MetricDescriptor{ | ||||
| 		Name:        "grpc.xds_client.resource_updates_valid", | ||||
| 		Description: "A counter of resources received that were considered valid. The counter will be incremented even for resources that have not changed.", | ||||
| 		Unit:        "resource", | ||||
| 		Labels:      []string{"grpc.target", "grpc.xds.server", "grpc.xds.resource_type"}, | ||||
| 		Default:     false, | ||||
| 	}) | ||||
| 	xdsClientResourceUpdatesInvalidMetric = estats.RegisterInt64Count(estats.MetricDescriptor{ | ||||
| 		Name:        "grpc.xds_client.resource_updates_invalid", | ||||
| 		Description: "A counter of resources received that were considered invalid.", | ||||
| 		Unit:        "resource", | ||||
| 		Labels:      []string{"grpc.target", "grpc.xds.server", "grpc.xds.resource_type"}, | ||||
| 		Default:     false, | ||||
| 	}) | ||||
| ) | ||||
| 
 | ||||
| // clientImpl is the real implementation of the xDS client. The exported Client
 | ||||
|  | @ -78,6 +94,8 @@ type clientImpl struct { | |||
| 	serializer         *grpcsync.CallbackSerializer // Serializer for invoking resource watcher callbacks.
 | ||||
| 	serializerClose    func()                       // Function to close the serializer.
 | ||||
| 	logger             *grpclog.PrefixLogger        // Logger for this client.
 | ||||
| 	metricsRecorder    estats.MetricsRecorder       // Metrics recorder for metrics.
 | ||||
| 	target             string                       // The gRPC target for this client.
 | ||||
| 
 | ||||
| 	// The clientImpl owns a bunch of channels to individual xDS servers
 | ||||
| 	// specified in the bootstrap configuration. Authorities acquire references
 | ||||
|  | @ -111,9 +129,11 @@ func init() { | |||
| } | ||||
| 
 | ||||
| // newClientImpl returns a new xdsClient with the given config.
 | ||||
| func newClientImpl(config *bootstrap.Config, watchExpiryTimeout time.Duration, streamBackoff func(int) time.Duration) (*clientImpl, error) { | ||||
| func newClientImpl(config *bootstrap.Config, watchExpiryTimeout time.Duration, streamBackoff func(int) time.Duration, mr estats.MetricsRecorder, target string) (*clientImpl, error) { | ||||
| 	ctx, cancel := context.WithCancel(context.Background()) | ||||
| 	c := &clientImpl{ | ||||
| 		metricsRecorder:    mr, | ||||
| 		target:             target, | ||||
| 		done:               grpcsync.NewEvent(), | ||||
| 		authorities:        make(map[string]*authority), | ||||
| 		config:             config, | ||||
|  | @ -139,6 +159,8 @@ func newClientImpl(config *bootstrap.Config, watchExpiryTimeout time.Duration, s | |||
| 			serializer:       c.serializer, | ||||
| 			getChannelForADS: c.getChannelForADS, | ||||
| 			logPrefix:        clientPrefix(c), | ||||
| 			target:           target, | ||||
| 			metricsRecorder:  c.metricsRecorder, | ||||
| 		}) | ||||
| 	} | ||||
| 	c.topLevelAuthority = newAuthority(authorityBuildOptions{ | ||||
|  | @ -147,6 +169,8 @@ func newClientImpl(config *bootstrap.Config, watchExpiryTimeout time.Duration, s | |||
| 		serializer:       c.serializer, | ||||
| 		getChannelForADS: c.getChannelForADS, | ||||
| 		logPrefix:        clientPrefix(c), | ||||
| 		target:           target, | ||||
| 		metricsRecorder:  c.metricsRecorder, | ||||
| 	}) | ||||
| 	c.logger = prefixLogger(c) | ||||
| 	return c, nil | ||||
|  |  | |||
|  | @ -0,0 +1,149 @@ | |||
| /* | ||||
|  * | ||||
|  * Copyright 2025 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 ( | ||||
| 	"context" | ||||
| 	"encoding/json" | ||||
| 	"fmt" | ||||
| 	"testing" | ||||
| 
 | ||||
| 	"github.com/google/uuid" | ||||
| 	"google.golang.org/grpc/internal/testutils" | ||||
| 	"google.golang.org/grpc/internal/testutils/stats" | ||||
| 	"google.golang.org/grpc/internal/testutils/xds/e2e" | ||||
| 	"google.golang.org/grpc/internal/xds/bootstrap" | ||||
| 	"google.golang.org/grpc/xds/internal/xdsclient/xdsresource" | ||||
| 
 | ||||
| 	v3listenerpb "github.com/envoyproxy/go-control-plane/envoy/config/listener/v3" | ||||
| ) | ||||
| 
 | ||||
| type noopListenerWatcher struct{} | ||||
| 
 | ||||
| func (noopListenerWatcher) OnUpdate(_ *xdsresource.ListenerResourceData, onDone xdsresource.OnDoneFunc) { | ||||
| 	onDone() | ||||
| } | ||||
| 
 | ||||
| func (noopListenerWatcher) OnError(_ error, onDone xdsresource.OnDoneFunc) { | ||||
| 	onDone() | ||||
| } | ||||
| 
 | ||||
| func (noopListenerWatcher) OnResourceDoesNotExist(onDone xdsresource.OnDoneFunc) { | ||||
| 	onDone() | ||||
| } | ||||
| 
 | ||||
| // TestResourceUpdateMetrics configures an xDS client, and a management server
 | ||||
| // to send valid and invalid LDS updates, and verifies that the expected metrics
 | ||||
| // for both good and bad updates are emitted.
 | ||||
| func (s) TestResourceUpdateMetrics(t *testing.T) { | ||||
| 	ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) | ||||
| 	defer cancel() | ||||
| 
 | ||||
| 	tmr := stats.NewTestMetricsRecorder() | ||||
| 	l, err := testutils.LocalTCPListener() | ||||
| 	if err != nil { | ||||
| 		t.Fatalf("net.Listen() failed: %v", err) | ||||
| 	} | ||||
| 
 | ||||
| 	mgmtServer := e2e.StartManagementServer(t, e2e.ManagementServerOptions{Listener: l}) | ||||
| 	const listenerResourceName = "test-listener-resource" | ||||
| 	const routeConfigurationName = "test-route-configuration-resource" | ||||
| 	nodeID := uuid.New().String() | ||||
| 	resources := e2e.UpdateOptions{ | ||||
| 		NodeID:         nodeID, | ||||
| 		Listeners:      []*v3listenerpb.Listener{e2e.DefaultClientListener(listenerResourceName, routeConfigurationName)}, | ||||
| 		SkipValidation: true, | ||||
| 	} | ||||
| 	if err := mgmtServer.Update(ctx, resources); err != nil { | ||||
| 		t.Fatalf("Failed to update management server with resources: %v, err: %v", resources, err) | ||||
| 	} | ||||
| 
 | ||||
| 	bootstrapContents, err := bootstrap.NewContentsForTesting(bootstrap.ConfigOptionsForTesting{ | ||||
| 		Servers: []byte(fmt.Sprintf(`[{ | ||||
| 			"server_uri": %q, | ||||
| 			"channel_creds": [{"type": "insecure"}] | ||||
| 		}]`, mgmtServer.Address)), | ||||
| 		Node: []byte(fmt.Sprintf(`{"id": "%s"}`, nodeID)), | ||||
| 		Authorities: map[string]json.RawMessage{ | ||||
| 			"authority": []byte("{}"), | ||||
| 		}, | ||||
| 	}) | ||||
| 	if err != nil { | ||||
| 		t.Fatalf("Failed to create bootstrap configuration: %v", err) | ||||
| 	} | ||||
| 
 | ||||
| 	config, err := bootstrap.NewConfigFromContents(bootstrapContents) | ||||
| 	if err != nil { | ||||
| 		t.Fatalf("Failed to parse bootstrap contents: %s, %v", string(bootstrapContents), err) | ||||
| 	} | ||||
| 	pool := NewPool(config) | ||||
| 	client, close, err := pool.NewClientForTesting(OptionsForTesting{ | ||||
| 		Name:               t.Name(), | ||||
| 		WatchExpiryTimeout: defaultTestWatchExpiryTimeout, | ||||
| 		MetricsRecorder:    tmr, | ||||
| 	}) | ||||
| 	if err != nil { | ||||
| 		t.Fatalf("Failed to create an xDS client: %v", err) | ||||
| 	} | ||||
| 	defer close() | ||||
| 
 | ||||
| 	// Watch the valid listener configured on the management server. This should
 | ||||
| 	// cause a resource updates valid count to emit eventually.
 | ||||
| 	xdsresource.WatchListener(client, listenerResourceName, noopListenerWatcher{}) | ||||
| 	mdWant := stats.MetricsData{ | ||||
| 		Handle:    xdsClientResourceUpdatesValidMetric.Descriptor(), | ||||
| 		IntIncr:   1, | ||||
| 		LabelKeys: []string{"grpc.target", "grpc.xds.server", "grpc.xds.resource_type"}, | ||||
| 		LabelVals: []string{"Test/ResourceUpdateMetrics", mgmtServer.Address, "ListenerResource"}, | ||||
| 	} | ||||
| 	if err := tmr.WaitForInt64Count(ctx, mdWant); err != nil { | ||||
| 		t.Fatal(err.Error()) | ||||
| 	} | ||||
| 	// Invalid should have no recording point.
 | ||||
| 	if got, _ := tmr.Metric("grpc.xds_client.resource_updates_invalid"); got != 0 { | ||||
| 		t.Fatalf("Unexpected data for metric \"grpc.xds_client.resource_updates_invalid\", got: %v, want: %v", got, 0) | ||||
| 	} | ||||
| 
 | ||||
| 	// Update management server with a bad update. Eventually, tmr should
 | ||||
| 	// receive an invalid count received metric. The successful metric should
 | ||||
| 	// stay the same.
 | ||||
| 	resources = e2e.UpdateOptions{ | ||||
| 		NodeID:         nodeID, | ||||
| 		Listeners:      []*v3listenerpb.Listener{e2e.DefaultClientListener(listenerResourceName, routeConfigurationName)}, | ||||
| 		SkipValidation: true, | ||||
| 	} | ||||
| 	resources.Listeners[0].ApiListener = nil | ||||
| 	if err := mgmtServer.Update(ctx, resources); err != nil { | ||||
| 		t.Fatalf("Failed to update management server with resources: %v, err: %v", resources, err) | ||||
| 	} | ||||
| 
 | ||||
| 	mdWant = stats.MetricsData{ | ||||
| 		Handle:    xdsClientResourceUpdatesInvalidMetric.Descriptor(), | ||||
| 		IntIncr:   1, | ||||
| 		LabelKeys: []string{"grpc.target", "grpc.xds.server", "grpc.xds.resource_type"}, | ||||
| 		LabelVals: []string{"Test/ResourceUpdateMetrics", mgmtServer.Address, "ListenerResource"}, | ||||
| 	} | ||||
| 	if err := tmr.WaitForInt64Count(ctx, mdWant); err != nil { | ||||
| 		t.Fatal(err.Error()) | ||||
| 	} | ||||
| 	// Valid should stay the same at 1.
 | ||||
| 	if got, _ := tmr.Metric("grpc.xds_client.resource_updates_valid"); got != 1 { | ||||
| 		t.Fatalf("Unexpected data for metric \"grpc.xds_client.resource_updates_invalid\", got: %v, want: %v", got, 1) | ||||
| 	} | ||||
| } | ||||
|  | @ -24,7 +24,9 @@ import ( | |||
| 	"time" | ||||
| 
 | ||||
| 	v3statuspb "github.com/envoyproxy/go-control-plane/envoy/service/status/v3" | ||||
| 	estats "google.golang.org/grpc/experimental/stats" | ||||
| 	"google.golang.org/grpc/internal/backoff" | ||||
| 	istats "google.golang.org/grpc/internal/stats" | ||||
| 	"google.golang.org/grpc/internal/xds/bootstrap" | ||||
| ) | ||||
| 
 | ||||
|  | @ -59,6 +61,10 @@ type OptionsForTesting struct { | |||
| 	// backoff duration after stream failures.
 | ||||
| 	// If unspecified, uses the default value used in non-test code.
 | ||||
| 	StreamBackoffAfterFailure func(int) time.Duration | ||||
| 
 | ||||
| 	// MetricsRecorder is the metrics recorder the xDS Client will use. If
 | ||||
| 	// unspecified, uses a no-op MetricsRecorder.
 | ||||
| 	MetricsRecorder estats.MetricsRecorder | ||||
| } | ||||
| 
 | ||||
| // NewPool creates a new xDS client pool with the given bootstrap config.
 | ||||
|  | @ -82,8 +88,8 @@ func NewPool(config *bootstrap.Config) *Pool { | |||
| // The second return value represents a close function which the caller is
 | ||||
| // expected to invoke once they are done using the client.  It is safe for the
 | ||||
| // caller to invoke this close function multiple times.
 | ||||
| func (p *Pool) NewClient(name string) (XDSClient, func(), error) { | ||||
| 	return p.newRefCounted(name, defaultWatchExpiryTimeout, backoff.DefaultExponential.Backoff) | ||||
| func (p *Pool) NewClient(name string, metricsRecorder estats.MetricsRecorder) (XDSClient, func(), error) { | ||||
| 	return p.newRefCounted(name, defaultWatchExpiryTimeout, backoff.DefaultExponential.Backoff, metricsRecorder) | ||||
| } | ||||
| 
 | ||||
| // NewClientForTesting returns an xDS client configured with the provided
 | ||||
|  | @ -107,7 +113,10 @@ func (p *Pool) NewClientForTesting(opts OptionsForTesting) (XDSClient, func(), e | |||
| 	if opts.StreamBackoffAfterFailure == nil { | ||||
| 		opts.StreamBackoffAfterFailure = defaultExponentialBackoff | ||||
| 	} | ||||
| 	return p.newRefCounted(opts.Name, opts.WatchExpiryTimeout, opts.StreamBackoffAfterFailure) | ||||
| 	if opts.MetricsRecorder == nil { | ||||
| 		opts.MetricsRecorder = istats.NewMetricsRecorderList(nil) | ||||
| 	} | ||||
| 	return p.newRefCounted(opts.Name, opts.WatchExpiryTimeout, opts.StreamBackoffAfterFailure, opts.MetricsRecorder) | ||||
| } | ||||
| 
 | ||||
| // GetClientForTesting returns an xDS client created earlier using the given
 | ||||
|  | @ -206,7 +215,7 @@ func (p *Pool) clientRefCountedClose(name string) { | |||
| // newRefCounted creates a new reference counted xDS client implementation for
 | ||||
| // name, if one does not exist already. If an xDS client for the given name
 | ||||
| // exists, it gets a reference to it and returns it.
 | ||||
| func (p *Pool) newRefCounted(name string, watchExpiryTimeout time.Duration, streamBackoff func(int) time.Duration) (XDSClient, func(), error) { | ||||
| func (p *Pool) newRefCounted(name string, watchExpiryTimeout time.Duration, streamBackoff func(int) time.Duration, metricsRecorder estats.MetricsRecorder) (XDSClient, func(), error) { | ||||
| 	p.mu.Lock() | ||||
| 	defer p.mu.Unlock() | ||||
| 
 | ||||
|  | @ -219,7 +228,7 @@ func (p *Pool) newRefCounted(name string, watchExpiryTimeout time.Duration, stre | |||
| 		return c, sync.OnceFunc(func() { p.clientRefCountedClose(name) }), nil | ||||
| 	} | ||||
| 
 | ||||
| 	c, err := newClientImpl(p.config, watchExpiryTimeout, streamBackoff) | ||||
| 	c, err := newClientImpl(p.config, watchExpiryTimeout, streamBackoff, metricsRecorder, name) | ||||
| 	if err != nil { | ||||
| 		return nil, nil, err | ||||
| 	} | ||||
|  |  | |||
|  | @ -27,9 +27,12 @@ import ( | |||
| 	"google.golang.org/grpc" | ||||
| 	"google.golang.org/grpc/codes" | ||||
| 	"google.golang.org/grpc/connectivity" | ||||
| 	estats "google.golang.org/grpc/experimental/stats" | ||||
| 	"google.golang.org/grpc/internal" | ||||
| 	internalgrpclog "google.golang.org/grpc/internal/grpclog" | ||||
| 	"google.golang.org/grpc/internal/grpcsync" | ||||
| 	iresolver "google.golang.org/grpc/internal/resolver" | ||||
| 	istats "google.golang.org/grpc/internal/stats" | ||||
| 	"google.golang.org/grpc/internal/transport" | ||||
| 	"google.golang.org/grpc/internal/xds/bootstrap" | ||||
| 	"google.golang.org/grpc/metadata" | ||||
|  | @ -87,6 +90,12 @@ func NewGRPCServer(opts ...grpc.ServerOption) (*GRPCServer, error) { | |||
| 	} | ||||
| 	s.handleServerOptions(opts) | ||||
| 
 | ||||
| 	var mrl estats.MetricsRecorder | ||||
| 	mrl = istats.NewMetricsRecorderList(nil) | ||||
| 	if srv, ok := s.gs.(*grpc.Server); ok { // Will hit in prod but not for testing.
 | ||||
| 		mrl = internal.MetricsRecorderForServer.(func(*grpc.Server) estats.MetricsRecorder)(srv) | ||||
| 	} | ||||
| 
 | ||||
| 	// Initializing the xDS client upfront (instead of at serving time)
 | ||||
| 	// simplifies the code by eliminating the need for a mutex to protect the
 | ||||
| 	// xdsC and xdsClientClose fields.
 | ||||
|  | @ -94,7 +103,7 @@ func NewGRPCServer(opts ...grpc.ServerOption) (*GRPCServer, error) { | |||
| 	if s.opts.clientPoolForTesting != nil { | ||||
| 		pool = s.opts.clientPoolForTesting | ||||
| 	} | ||||
| 	xdsClient, xdsClientClose, err := pool.NewClient(xdsclient.NameForServer) | ||||
| 	xdsClient, xdsClientClose, err := pool.NewClient(xdsclient.NameForServer, mrl) | ||||
| 	if err != nil { | ||||
| 		return nil, fmt.Errorf("xDS client creation failed: %v", err) | ||||
| 	} | ||||
|  |  | |||
		Loading…
	
		Reference in New Issue