xds/client: move watchers from xdsclient to a separate struct (#4963)

This commit is contained in:
Menghan Li 2021-11-09 10:32:16 -08:00 committed by GitHub
parent bac0a7e47f
commit 59e024e4c7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
36 changed files with 1203 additions and 978 deletions

View File

@ -174,7 +174,7 @@ func nodeProtoToV3(n proto.Message) *v3corepb.Node {
return node
}
func dumpToGenericXdsConfig(typeURL string, dumpF func() (string, map[string]xdsclient.UpdateWithMD)) []*v3statuspb.ClientConfig_GenericXdsConfig {
func dumpToGenericXdsConfig(typeURL string, dumpF func() (string, map[string]xdsresource.UpdateWithMD)) []*v3statuspb.ClientConfig_GenericXdsConfig {
_, dump := dumpF()
ret := make([]*v3statuspb.ClientConfig_GenericXdsConfig, 0, len(dump))
for name, d := range dump {

View File

@ -431,11 +431,11 @@ func (b *cdsBalancer) run() {
func (b *cdsBalancer) handleErrorFromUpdate(err error, fromParent bool) {
// This is not necessary today, because xds client never sends connection
// errors.
if fromParent && xdsclient.ErrType(err) == xdsclient.ErrorTypeResourceNotFound {
if fromParent && xdsresource.ErrType(err) == xdsresource.ErrorTypeResourceNotFound {
b.clusterHandler.close()
}
if b.childLB != nil {
if xdsclient.ErrType(err) != xdsclient.ErrorTypeConnection {
if xdsresource.ErrType(err) != xdsresource.ErrorTypeConnection {
// Connection errors will be sent to the child balancers directly.
// There's no need to forward them.
b.childLB.ResolverError(err)

View File

@ -472,7 +472,7 @@ func (s) TestHandleClusterUpdateError(t *testing.T) {
}
// Push a resource-not-found-error this time around.
resourceErr := xdsclient.NewErrorf(xdsclient.ErrorTypeResourceNotFound, "cdsBalancer resource not found error")
resourceErr := xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "cdsBalancer resource not found error")
xdsC.InvokeWatchClusterCallback(xdsresource.ClusterUpdate{}, resourceErr)
// Make sure that the watch is not cancelled. This error indicates that the
// request cluster resource is not found. We should continue to watch it.
@ -557,7 +557,7 @@ func (s) TestResolverError(t *testing.T) {
}
// Push a resource-not-found-error this time around.
resourceErr := xdsclient.NewErrorf(xdsclient.ErrorTypeResourceNotFound, "cdsBalancer resource not found error")
resourceErr := xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "cdsBalancer resource not found error")
cdsB.ResolverError(resourceErr)
// Make sure the registered watch is cancelled.
if _, err := xdsC.WaitForCancelClusterWatch(ctx); err != nil {

View File

@ -36,6 +36,7 @@ import (
"google.golang.org/grpc/serviceconfig"
"google.golang.org/grpc/xds/internal/balancer/priority"
"google.golang.org/grpc/xds/internal/xdsclient"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
// Name is the name of the cluster_resolver balancer.
@ -244,7 +245,7 @@ func (b *clusterResolverBalancer) updateChildConfig() error {
// In both cases, the sub-balancers will be receive the error.
func (b *clusterResolverBalancer) handleErrorFromUpdate(err error, fromParent bool) {
b.logger.Warningf("Received error: %v", err)
if fromParent && xdsclient.ErrType(err) == xdsclient.ErrorTypeResourceNotFound {
if fromParent && xdsresource.ErrType(err) == xdsresource.ErrorTypeResourceNotFound {
// This is an error from the parent ClientConn (can be the parent CDS
// balancer), and is a resource-not-found error. This means the resource
// (can be either LDS or CDS) was removed. Stop the EDS watch.

View File

@ -280,7 +280,7 @@ func (s) TestErrorFromXDSClientUpdate(t *testing.T) {
t.Fatalf("EDS impl got unexpected update: %v", err)
}
connectionErr := xdsclient.NewErrorf(xdsclient.ErrorTypeConnection, "connection error")
connectionErr := xdsresource.NewErrorf(xdsresource.ErrorTypeConnection, "connection error")
xdsC.InvokeWatchEDSCallback("", xdsresource.EndpointsUpdate{}, connectionErr)
sCtx, sCancel := context.WithTimeout(context.Background(), defaultTestShortTimeout)
@ -298,7 +298,7 @@ func (s) TestErrorFromXDSClientUpdate(t *testing.T) {
t.Fatalf("want resolver error, got %v", err)
}
resourceErr := xdsclient.NewErrorf(xdsclient.ErrorTypeResourceNotFound, "clusterResolverBalancer resource not found error")
resourceErr := xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "clusterResolverBalancer resource not found error")
xdsC.InvokeWatchEDSCallback("", xdsresource.EndpointsUpdate{}, resourceErr)
// Even if error is resource not found, watch shouldn't be canceled, because
// this is an EDS resource removed (and xds client actually never sends this
@ -369,7 +369,7 @@ func (s) TestErrorFromResolver(t *testing.T) {
t.Fatalf("EDS impl got unexpected update: %v", err)
}
connectionErr := xdsclient.NewErrorf(xdsclient.ErrorTypeConnection, "connection error")
connectionErr := xdsresource.NewErrorf(xdsresource.ErrorTypeConnection, "connection error")
edsB.ResolverError(connectionErr)
sCtx, sCancel := context.WithTimeout(context.Background(), defaultTestShortTimeout)
@ -387,7 +387,7 @@ func (s) TestErrorFromResolver(t *testing.T) {
t.Fatalf("want resolver error, got %v", err)
}
resourceErr := xdsclient.NewErrorf(xdsclient.ErrorTypeResourceNotFound, "clusterResolverBalancer resource not found error")
resourceErr := xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "clusterResolverBalancer resource not found error")
edsB.ResolverError(resourceErr)
if _, err := xdsC.WaitForCancelEDSWatch(ctx); err != nil {
t.Fatalf("want watch to be canceled, waitForCancel failed: %v", err)

View File

@ -94,7 +94,7 @@ func (w *serviceUpdateWatcher) handleLDSResp(update xdsresource.ListenerUpdate,
// type we check is ResourceNotFound, which indicates the LDS resource
// was removed, and besides sending the error to callback, we also
// cancel the RDS watch.
if xdsclient.ErrType(err) == xdsclient.ErrorTypeResourceNotFound && w.rdsCancel != nil {
if xdsresource.ErrType(err) == xdsresource.ErrorTypeResourceNotFound && w.rdsCancel != nil {
w.rdsCancel()
w.rdsName = ""
w.rdsCancel = nil

View File

@ -32,6 +32,7 @@ import (
"google.golang.org/grpc/resolver"
"google.golang.org/grpc/xds/internal/xdsclient"
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
const xdsScheme = "xds"
@ -234,7 +235,7 @@ func (r *xdsResolver) run() {
case update := <-r.updateCh:
if update.err != nil {
r.logger.Warningf("Watch error on resource %v from xds-client %p, %v", r.target.Endpoint, r.client, update.err)
if xdsclient.ErrType(update.err) == xdsclient.ErrorTypeResourceNotFound {
if xdsresource.ErrType(update.err) == xdsresource.ErrorTypeResourceNotFound {
// If error is resource-not-found, it means the LDS
// resource was removed. Ultimately send an empty service
// config, which picks pick-first, with no address, and

View File

@ -688,7 +688,7 @@ func (s) TestXDSResolverRemovedWithRPCs(t *testing.T) {
}
// Delete the resource
suErr := xdsclient.NewErrorf(xdsclient.ErrorTypeResourceNotFound, "resource removed error")
suErr := xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "resource removed error")
xdsC.InvokeWatchRouteConfigCallback("", xdsresource.RouteConfigUpdate{}, suErr)
if _, err = tcc.stateCh.Receive(ctx); err != nil {
@ -764,7 +764,7 @@ func (s) TestXDSResolverRemovedResource(t *testing.T) {
// Delete the resource. The channel should receive a service config with the
// original cluster but with an erroring config selector.
suErr := xdsclient.NewErrorf(xdsclient.ErrorTypeResourceNotFound, "resource removed error")
suErr := xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "resource removed error")
xdsC.InvokeWatchRouteConfigCallback("", xdsresource.RouteConfigUpdate{}, suErr)
if gotState, err = tcc.stateCh.Receive(ctx); err != nil {
@ -1164,7 +1164,7 @@ func (s) TestXDSResolverResourceNotFoundError(t *testing.T) {
// Invoke the watchAPI callback with a bad service update and wait for the
// ReportError method to be called on the ClientConn.
suErr := xdsclient.NewErrorf(xdsclient.ErrorTypeResourceNotFound, "resource removed error")
suErr := xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "resource removed error")
xdsC.InvokeWatchRouteConfigCallback("", xdsresource.RouteConfigUpdate{}, suErr)
if gotErrVal, gotErr := tcc.errorCh.Receive(ctx); gotErr != context.DeadlineExceeded {

View File

@ -36,7 +36,6 @@ import (
internalgrpclog "google.golang.org/grpc/internal/grpclog"
"google.golang.org/grpc/internal/grpcsync"
"google.golang.org/grpc/internal/xds/env"
"google.golang.org/grpc/xds/internal/xdsclient"
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
@ -365,7 +364,7 @@ func (l *listenerWrapper) handleRDSUpdate(update rdsHandlerUpdate) {
}
if update.err != nil {
l.logger.Warningf("Received error for rds names specified in resource %q: %+v", l.name, update.err)
if xdsclient.ErrType(update.err) == xdsclient.ErrorTypeResourceNotFound {
if xdsresource.ErrType(update.err) == xdsresource.ErrorTypeResourceNotFound {
l.switchMode(nil, connectivity.ServingModeNotServing, update.err)
}
// For errors which are anything other than "resource-not-found", we
@ -381,7 +380,7 @@ func (l *listenerWrapper) handleRDSUpdate(update rdsHandlerUpdate) {
func (l *listenerWrapper) handleLDSUpdate(update ldsUpdateWithError) {
if update.err != nil {
l.logger.Warningf("Received error for resource %q: %+v", l.name, update.err)
if xdsclient.ErrType(update.err) == xdsclient.ErrorTypeResourceNotFound {
if xdsresource.ErrType(update.err) == xdsresource.ErrorTypeResourceNotFound {
l.switchMode(nil, connectivity.ServingModeNotServing, update.err)
}
// For errors which are anything other than "resource-not-found", we

View File

@ -38,10 +38,10 @@ type XDSClient interface {
WatchEndpoints(clusterName string, edsCb func(xdsresource.EndpointsUpdate, error)) (cancel func())
ReportLoad(server string) (*load.Store, func())
DumpLDS() (string, map[string]UpdateWithMD)
DumpRDS() (string, map[string]UpdateWithMD)
DumpCDS() (string, map[string]UpdateWithMD)
DumpEDS() (string, map[string]UpdateWithMD)
DumpLDS() (string, map[string]xdsresource.UpdateWithMD)
DumpRDS() (string, map[string]xdsresource.UpdateWithMD)
DumpCDS() (string, map[string]xdsresource.UpdateWithMD)
DumpEDS() (string, map[string]xdsresource.UpdateWithMD)
BootstrapConfig() *bootstrap.Config
Close()

View File

@ -19,129 +19,16 @@
package xdsclient
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 (c *clientImpl) scheduleCallback(wi *watchInfo, update interface{}, err error) {
c.updateCh.Put(&watcherInfoWithUpdate{
wi: wi,
update: update,
err: err,
})
}
func (c *clientImpl) callCallback(wiu *watcherInfoWithUpdate) {
c.mu.Lock()
// Use a closure to capture the callback and type assertion, to save one
// more switch case.
//
// The callback must be called without c.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 ListenerResource:
if s, ok := c.ldsWatchers[wiu.wi.target]; ok && s[wiu.wi] {
ccb = func() { wiu.wi.ldsCallback(wiu.update.(xdsresource.ListenerUpdate), wiu.err) }
}
case RouteConfigResource:
if s, ok := c.rdsWatchers[wiu.wi.target]; ok && s[wiu.wi] {
ccb = func() { wiu.wi.rdsCallback(wiu.update.(xdsresource.RouteConfigUpdate), wiu.err) }
}
case ClusterResource:
if s, ok := c.cdsWatchers[wiu.wi.target]; ok && s[wiu.wi] {
ccb = func() { wiu.wi.cdsCallback(wiu.update.(xdsresource.ClusterUpdate), wiu.err) }
}
case EndpointsResource:
if s, ok := c.edsWatchers[wiu.wi.target]; ok && s[wiu.wi] {
ccb = func() { wiu.wi.edsCallback(wiu.update.(xdsresource.EndpointsUpdate), wiu.err) }
}
}
c.mu.Unlock()
if ccb != nil {
ccb()
}
}
// NewListeners is called by the underlying xdsAPIClient when it receives an
// xDS response.
//
// A response can contain multiple resources. They will be parsed and put in a
// map from resource name to the resource content.
func (c *clientImpl) NewListeners(updates map[string]xdsresource.ListenerUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
c.mu.Lock()
defer c.mu.Unlock()
c.ldsVersion = metadata.Version
if metadata.ErrState != nil {
c.ldsVersion = metadata.ErrState.Version
}
for name, uErr := range updates {
if s, ok := c.ldsWatchers[name]; ok {
if uErr.Err != nil {
// On error, keep previous version for each resource. But update
// status and error.
mdCopy := c.ldsMD[name]
mdCopy.ErrState = metadata.ErrState
mdCopy.Status = metadata.Status
c.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 := c.ldsCache[name]; !ok || !proto.Equal(cur.Raw, uErr.Update.Raw) {
for wi := range s {
wi.newUpdate(uErr.Update)
}
}
// Sync cache.
c.logger.Debugf("LDS resource with name %v, value %+v added to cache", name, pretty.ToJSON(uErr))
c.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
}
c.ldsMD[name] = mdCopy
}
}
// Resources not in the new update were removed by the server, so delete
// them.
for name := range c.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(c.ldsCache, name)
c.ldsMD[name] = xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusNotExist}
for wi := range c.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.
c.pubsub.NewListeners(updates, metadata)
}
// NewRouteConfigs is called by the underlying xdsAPIClient when it receives an
@ -150,51 +37,7 @@ func (c *clientImpl) NewListeners(updates map[string]xdsresource.ListenerUpdateE
// A response can contain multiple resources. They will be parsed and put in a
// map from resource name to the resource content.
func (c *clientImpl) NewRouteConfigs(updates map[string]xdsresource.RouteConfigUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
c.mu.Lock()
defer c.mu.Unlock()
// If no error received, the status is ACK.
c.rdsVersion = metadata.Version
if metadata.ErrState != nil {
c.rdsVersion = metadata.ErrState.Version
}
for name, uErr := range updates {
if s, ok := c.rdsWatchers[name]; ok {
if uErr.Err != nil {
// On error, keep previous version for each resource. But update
// status and error.
mdCopy := c.rdsMD[name]
mdCopy.ErrState = metadata.ErrState
mdCopy.Status = metadata.Status
c.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 := c.rdsCache[name]; !ok || !proto.Equal(cur.Raw, uErr.Update.Raw) {
for wi := range s {
wi.newUpdate(uErr.Update)
}
}
// Sync cache.
c.logger.Debugf("RDS resource with name %v, value %+v added to cache", name, pretty.ToJSON(uErr))
c.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
}
c.rdsMD[name] = mdCopy
}
}
c.pubsub.NewRouteConfigs(updates, metadata)
}
// NewClusters is called by the underlying xdsAPIClient when it receives an xDS
@ -203,69 +46,7 @@ func (c *clientImpl) NewRouteConfigs(updates map[string]xdsresource.RouteConfigU
// A response can contain multiple resources. They will be parsed and put in a
// map from resource name to the resource content.
func (c *clientImpl) NewClusters(updates map[string]xdsresource.ClusterUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
c.mu.Lock()
defer c.mu.Unlock()
c.cdsVersion = metadata.Version
if metadata.ErrState != nil {
c.cdsVersion = metadata.ErrState.Version
}
for name, uErr := range updates {
if s, ok := c.cdsWatchers[name]; ok {
if uErr.Err != nil {
// On error, keep previous version for each resource. But update
// status and error.
mdCopy := c.cdsMD[name]
mdCopy.ErrState = metadata.ErrState
mdCopy.Status = metadata.Status
c.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 := c.cdsCache[name]; !ok || !proto.Equal(cur.Raw, uErr.Update.Raw) {
for wi := range s {
wi.newUpdate(uErr.Update)
}
}
// Sync cache.
c.logger.Debugf("CDS resource with name %v, value %+v added to cache", name, pretty.ToJSON(uErr))
c.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
}
c.cdsMD[name] = mdCopy
}
}
// Resources not in the new update were removed by the server, so delete
// them.
for name := range c.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(c.cdsCache, name)
c.ldsMD[name] = xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusNotExist}
for wi := range c.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.
c.pubsub.NewClusters(updates, metadata)
}
// NewEndpoints is called by the underlying xdsAPIClient when it receives an
@ -274,63 +55,11 @@ func (c *clientImpl) NewClusters(updates map[string]xdsresource.ClusterUpdateErr
// A response can contain multiple resources. They will be parsed and put in a
// map from resource name to the resource content.
func (c *clientImpl) NewEndpoints(updates map[string]xdsresource.EndpointsUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
c.mu.Lock()
defer c.mu.Unlock()
c.edsVersion = metadata.Version
if metadata.ErrState != nil {
c.edsVersion = metadata.ErrState.Version
}
for name, uErr := range updates {
if s, ok := c.edsWatchers[name]; ok {
if uErr.Err != nil {
// On error, keep previous version for each resource. But update
// status and error.
mdCopy := c.edsMD[name]
mdCopy.ErrState = metadata.ErrState
mdCopy.Status = metadata.Status
c.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 := c.edsCache[name]; !ok || !proto.Equal(cur.Raw, uErr.Update.Raw) {
for wi := range s {
wi.newUpdate(uErr.Update)
}
}
// Sync cache.
c.logger.Debugf("EDS resource with name %v, value %+v added to cache", name, pretty.ToJSON(uErr))
c.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
}
c.edsMD[name] = mdCopy
}
}
c.pubsub.NewEndpoints(updates, metadata)
}
// NewConnectionError is called by the underlying xdsAPIClient when it receives
// a connection error. The error will be forwarded to all the resource watchers.
func (c *clientImpl) NewConnectionError(err error) {
c.mu.Lock()
defer c.mu.Unlock()
for _, s := range c.edsWatchers {
for wi := range s {
wi.newError(NewErrorf(ErrorTypeConnection, "xds: error received from xDS stream: %v", err))
}
}
c.pubsub.NewConnectionError(err)
}

View File

@ -30,13 +30,13 @@ import (
"github.com/golang/protobuf/proto"
"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/internal/grpcsync"
"google.golang.org/grpc/keepalive"
"google.golang.org/grpc/xds/internal/version"
"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/xdsresource"
)
@ -103,11 +103,11 @@ type APIClientBuilder interface {
// will still keep this interface for testing purposes.
type APIClient interface {
// AddWatch adds a watch for an xDS resource given its type and name.
AddWatch(ResourceType, string)
AddWatch(xdsresource.ResourceType, string)
// RemoveWatch cancels an already registered watch for an xDS resource
// given its type and name.
RemoveWatch(ResourceType, string)
RemoveWatch(xdsresource.ResourceType, string)
// reportLoad starts an LRS stream to periodically report load using the
// provided ClientConn, which represent a connection to the management
@ -157,35 +157,13 @@ var newAPIClient = func(apiVersion version.TransportAPI, cc *grpc.ClientConn, op
// style of ccBalancerWrapper so that the Client type does not implement these
// exported methods.
type clientImpl struct {
done *grpcsync.Event
config *bootstrap.Config
cc *grpc.ClientConn // Connection to the management server.
apiClient APIClient
watchExpiryTimeout time.Duration
done *grpcsync.Event
config *bootstrap.Config
cc *grpc.ClientConn // Connection to the management server.
apiClient APIClient
logger *grpclog.PrefixLogger
updateCh *buffer.Unbounded // chan *watcherInfoWithUpdate
// All the following maps are to keep the updates/metadata in a cache.
// TODO: move them to a separate struct/package, to cleanup the xds_client.
// And CSDS handler can be implemented directly by the cache.
mu sync.Mutex
ldsWatchers map[string]map[*watchInfo]bool
ldsVersion string // Only used in CSDS.
ldsCache map[string]xdsresource.ListenerUpdate
ldsMD map[string]xdsresource.UpdateMetadata
rdsWatchers map[string]map[*watchInfo]bool
rdsVersion string // Only used in CSDS.
rdsCache map[string]xdsresource.RouteConfigUpdate
rdsMD map[string]xdsresource.UpdateMetadata
cdsWatchers map[string]map[*watchInfo]bool
cdsVersion string // Only used in CSDS.
cdsCache map[string]xdsresource.ClusterUpdate
cdsMD map[string]xdsresource.UpdateMetadata
edsWatchers map[string]map[*watchInfo]bool
edsVersion string // Only used in CSDS.
edsCache map[string]xdsresource.EndpointsUpdate
edsMD map[string]xdsresource.UpdateMetadata
pubsub *pubsub.Pubsub
// Changes to map lrsClients and the lrsClient inside the map need to be
// protected by lrsMu.
@ -194,7 +172,7 @@ type clientImpl struct {
}
// newWithConfig returns a new xdsClient with the given config.
func newWithConfig(config *bootstrap.Config, watchExpiryTimeout time.Duration) (*clientImpl, error) {
func newWithConfig(config *bootstrap.Config, watchExpiryTimeout time.Duration) (_ *clientImpl, retErr error) {
switch {
case config.XDSServer == nil:
return nil, errors.New("xds: no xds_server provided")
@ -215,35 +193,36 @@ func newWithConfig(config *bootstrap.Config, watchExpiryTimeout time.Duration) (
}
c := &clientImpl{
done: grpcsync.NewEvent(),
config: config,
watchExpiryTimeout: watchExpiryTimeout,
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),
lrsClients: make(map[string]*lrsClient),
done: grpcsync.NewEvent(),
config: config,
lrsClients: make(map[string]*lrsClient),
}
defer func() {
if retErr != nil {
if c.cc != nil {
c.cc.Close()
}
if c.pubsub != nil {
c.pubsub.Close()
}
if c.apiClient != nil {
c.apiClient.Close()
}
}
}()
cc, err := grpc.Dial(config.XDSServer.ServerURI, dopts...)
if err != nil {
// An error from a non-blocking dial indicates something serious.
return nil, fmt.Errorf("xds: failed to dial balancer {%s}: %v", config.XDSServer.ServerURI, err)
}
c.cc = cc
c.logger = prefixLogger((c))
c.logger = prefixLogger(c)
c.logger.Infof("Created ClientConn to xDS management server: %s", config.XDSServer)
c.pubsub = pubsub.New(watchExpiryTimeout, c.logger)
apiClient, err := newAPIClient(config.XDSServer.TransportAPI, cc, BuildOptions{
Parent: c,
Validator: c.updateValidator,
@ -252,12 +231,10 @@ func newWithConfig(config *bootstrap.Config, watchExpiryTimeout time.Duration) (
Logger: c.logger,
})
if err != nil {
cc.Close()
return nil, err
}
c.apiClient = apiClient
c.logger.Infof("Created")
go c.run()
return c, nil
}
@ -267,27 +244,6 @@ func (c *clientRefCounted) BootstrapConfig() *bootstrap.Config {
return c.config
}
// 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 (c *clientImpl) run() {
for {
select {
case t := <-c.updateCh.Get():
c.updateCh.Load()
if c.done.HasFired() {
return
}
c.callCallback(t.(*watcherInfoWithUpdate))
case <-c.done.Done():
return
}
}
}
// Close closes the gRPC connection to the management server.
func (c *clientImpl) Close() {
if c.done.HasFired() {
@ -298,6 +254,7 @@ func (c *clientImpl) Close() {
// the client is closed?
c.apiClient.Close()
c.cc.Close()
c.pubsub.Close()
c.logger.Infof("Shutdown")
}
@ -342,35 +299,3 @@ func (c *clientImpl) updateValidator(u interface{}) error {
}
return nil
}
// ResourceType identifies resources in a transport protocol agnostic way. These
// will be used in transport version agnostic code, while the versioned API
// clients will map these to appropriate version URLs.
type ResourceType int
// Version agnostic resource type constants.
const (
UnknownResource ResourceType = iota
ListenerResource
HTTPConnManagerResource
RouteConfigResource
ClusterResource
EndpointsResource
)
func (r ResourceType) String() string {
switch r {
case ListenerResource:
return "ListenerResource"
case HTTPConnManagerResource:
return "HTTPConnManagerResource"
case RouteConfigResource:
return "RouteConfigResource"
case ClusterResource:
return "ClusterResource"
case EndpointsResource:
return "EndpointsResource"
default:
return "UnknownResource"
}
}

View File

@ -81,8 +81,8 @@ func clientOpts(balancerName string, overrideWatchExpiryTimeout bool) (*bootstra
type testAPIClient struct {
done *grpcsync.Event
addWatches map[ResourceType]*testutils.Channel
removeWatches map[ResourceType]*testutils.Channel
addWatches map[xdsresource.ResourceType]*testutils.Channel
removeWatches map[xdsresource.ResourceType]*testutils.Channel
}
func overrideNewAPIClient() (*testutils.Channel, func()) {
@ -97,17 +97,17 @@ func overrideNewAPIClient() (*testutils.Channel, func()) {
}
func newTestAPIClient() *testAPIClient {
addWatches := map[ResourceType]*testutils.Channel{
ListenerResource: testutils.NewChannel(),
RouteConfigResource: testutils.NewChannel(),
ClusterResource: testutils.NewChannel(),
EndpointsResource: testutils.NewChannel(),
addWatches := map[xdsresource.ResourceType]*testutils.Channel{
xdsresource.ListenerResource: testutils.NewChannel(),
xdsresource.RouteConfigResource: testutils.NewChannel(),
xdsresource.ClusterResource: testutils.NewChannel(),
xdsresource.EndpointsResource: testutils.NewChannel(),
}
removeWatches := map[ResourceType]*testutils.Channel{
ListenerResource: testutils.NewChannel(),
RouteConfigResource: testutils.NewChannel(),
ClusterResource: testutils.NewChannel(),
EndpointsResource: testutils.NewChannel(),
removeWatches := map[xdsresource.ResourceType]*testutils.Channel{
xdsresource.ListenerResource: testutils.NewChannel(),
xdsresource.RouteConfigResource: testutils.NewChannel(),
xdsresource.ClusterResource: testutils.NewChannel(),
xdsresource.EndpointsResource: testutils.NewChannel(),
}
return &testAPIClient{
done: grpcsync.NewEvent(),
@ -116,11 +116,11 @@ func newTestAPIClient() *testAPIClient {
}
}
func (c *testAPIClient) AddWatch(resourceType ResourceType, resourceName string) {
func (c *testAPIClient) AddWatch(resourceType xdsresource.ResourceType, resourceName string) {
c.addWatches[resourceType].Send(resourceName)
}
func (c *testAPIClient) RemoveWatch(resourceType ResourceType, resourceName string) {
func (c *testAPIClient) RemoveWatch(resourceType xdsresource.ResourceType, resourceName string) {
c.removeWatches[resourceType].Send(resourceName)
}
@ -158,12 +158,12 @@ func (s) TestWatchCallAnotherWatch(t *testing.T) {
// Calls another watch inline, to ensure there's deadlock.
client.WatchCluster("another-random-name", func(xdsresource.ClusterUpdate, error) {})
if _, err := apiClient.addWatches[ClusterResource].Receive(ctx); firstTime && err != nil {
if _, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(ctx); firstTime && err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
firstTime = false
})
if _, err := apiClient.addWatches[ClusterResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}

View File

@ -19,108 +19,25 @@
package xdsclient
import (
anypb "github.com/golang/protobuf/ptypes/any"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
// UpdateWithMD contains the raw message of the update and the metadata,
// including version, raw message, timestamp.
//
// This is to be used for config dump and CSDS, not directly by users (like
// resolvers/balancers).
type UpdateWithMD struct {
MD xdsresource.UpdateMetadata
Raw *anypb.Any
}
func rawFromCache(s string, cache interface{}) *anypb.Any {
switch c := cache.(type) {
case map[string]xdsresource.ListenerUpdate:
v, ok := c[s]
if !ok {
return nil
}
return v.Raw
case map[string]xdsresource.RouteConfigUpdate:
v, ok := c[s]
if !ok {
return nil
}
return v.Raw
case map[string]xdsresource.ClusterUpdate:
v, ok := c[s]
if !ok {
return nil
}
return v.Raw
case map[string]xdsresource.EndpointsUpdate:
v, ok := c[s]
if !ok {
return nil
}
return v.Raw
default:
return nil
}
}
func (c *clientImpl) dump(t ResourceType) (string, map[string]UpdateWithMD) {
c.mu.Lock()
defer c.mu.Unlock()
var (
version string
md map[string]xdsresource.UpdateMetadata
cache interface{}
)
switch t {
case ListenerResource:
version = c.ldsVersion
md = c.ldsMD
cache = c.ldsCache
case RouteConfigResource:
version = c.rdsVersion
md = c.rdsMD
cache = c.rdsCache
case ClusterResource:
version = c.cdsVersion
md = c.cdsMD
cache = c.cdsCache
case EndpointsResource:
version = c.edsVersion
md = c.edsMD
cache = c.edsCache
default:
c.logger.Errorf("dumping resource of unknown type: %v", t)
return "", nil
}
ret := make(map[string]UpdateWithMD, len(md))
for s, md := range md {
ret[s] = UpdateWithMD{
MD: md,
Raw: rawFromCache(s, cache),
}
}
return version, ret
}
// DumpLDS returns the status and contents of LDS.
func (c *clientImpl) DumpLDS() (string, map[string]UpdateWithMD) {
return c.dump(ListenerResource)
func (c *clientImpl) DumpLDS() (string, map[string]xdsresource.UpdateWithMD) {
return c.pubsub.Dump(xdsresource.ListenerResource)
}
// DumpRDS returns the status and contents of RDS.
func (c *clientImpl) DumpRDS() (string, map[string]UpdateWithMD) {
return c.dump(RouteConfigResource)
func (c *clientImpl) DumpRDS() (string, map[string]xdsresource.UpdateWithMD) {
return c.pubsub.Dump(xdsresource.RouteConfigResource)
}
// DumpCDS returns the status and contents of CDS.
func (c *clientImpl) DumpCDS() (string, map[string]UpdateWithMD) {
return c.dump(ClusterResource)
func (c *clientImpl) DumpCDS() (string, map[string]xdsresource.UpdateWithMD) {
return c.pubsub.Dump(xdsresource.ClusterResource)
}
// DumpEDS returns the status and contents of EDS.
func (c *clientImpl) DumpEDS() (string, map[string]UpdateWithMD) {
return c.dump(EndpointsResource)
func (c *clientImpl) DumpEDS() (string, map[string]xdsresource.UpdateWithMD) {
return c.pubsub.Dump(xdsresource.EndpointsResource)
}

View File

@ -89,15 +89,15 @@ func (s) TestLDSConfigDump(t *testing.T) {
updateHandler := client.(xdsclient.UpdateHandler)
// Expected unknown.
if err := compareDump(client.DumpLDS, "", map[string]xdsclient.UpdateWithMD{}); err != nil {
if err := compareDump(client.DumpLDS, "", map[string]xdsresource.UpdateWithMD{}); err != nil {
t.Fatalf(err.Error())
}
wantRequested := make(map[string]xdsclient.UpdateWithMD)
wantRequested := make(map[string]xdsresource.UpdateWithMD)
for _, n := range ldsTargets {
cancel := client.WatchListener(n, func(update xdsresource.ListenerUpdate, err error) {})
defer cancel()
wantRequested[n] = xdsclient.UpdateWithMD{MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}
wantRequested[n] = xdsresource.UpdateWithMD{MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}
}
// Expected requested.
if err := compareDump(client.DumpLDS, "", wantRequested); err != nil {
@ -105,10 +105,10 @@ func (s) TestLDSConfigDump(t *testing.T) {
}
update0 := make(map[string]xdsresource.ListenerUpdateErrTuple)
want0 := make(map[string]xdsclient.UpdateWithMD)
want0 := make(map[string]xdsresource.UpdateWithMD)
for n, r := range listenerRaws {
update0[n] = xdsresource.ListenerUpdateErrTuple{Update: xdsresource.ListenerUpdate{Raw: r}}
want0[n] = xdsclient.UpdateWithMD{
want0[n] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: testVersion},
Raw: r,
}
@ -137,10 +137,10 @@ func (s) TestLDSConfigDump(t *testing.T) {
)
// Expect NACK for [0], but old ACK for [1].
wantDump := make(map[string]xdsclient.UpdateWithMD)
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]] = xdsclient.UpdateWithMD{
wantDump[ldsTargets[0]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
Version: testVersion,
@ -152,7 +152,7 @@ func (s) TestLDSConfigDump(t *testing.T) {
Raw: listenerRaws[ldsTargets[0]],
}
wantDump[ldsTargets[1]] = xdsclient.UpdateWithMD{
wantDump[ldsTargets[1]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: nackVersion},
Raw: listenerRaws[ldsTargets[1]],
}
@ -205,15 +205,15 @@ func (s) TestRDSConfigDump(t *testing.T) {
updateHandler := client.(xdsclient.UpdateHandler)
// Expected unknown.
if err := compareDump(client.DumpRDS, "", map[string]xdsclient.UpdateWithMD{}); err != nil {
if err := compareDump(client.DumpRDS, "", map[string]xdsresource.UpdateWithMD{}); err != nil {
t.Fatalf(err.Error())
}
wantRequested := make(map[string]xdsclient.UpdateWithMD)
wantRequested := make(map[string]xdsresource.UpdateWithMD)
for _, n := range rdsTargets {
cancel := client.WatchRouteConfig(n, func(update xdsresource.RouteConfigUpdate, err error) {})
defer cancel()
wantRequested[n] = xdsclient.UpdateWithMD{MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}
wantRequested[n] = xdsresource.UpdateWithMD{MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}
}
// Expected requested.
if err := compareDump(client.DumpRDS, "", wantRequested); err != nil {
@ -221,10 +221,10 @@ func (s) TestRDSConfigDump(t *testing.T) {
}
update0 := make(map[string]xdsresource.RouteConfigUpdateErrTuple)
want0 := make(map[string]xdsclient.UpdateWithMD)
want0 := make(map[string]xdsresource.UpdateWithMD)
for n, r := range routeRaws {
update0[n] = xdsresource.RouteConfigUpdateErrTuple{Update: xdsresource.RouteConfigUpdate{Raw: r}}
want0[n] = xdsclient.UpdateWithMD{
want0[n] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: testVersion},
Raw: r,
}
@ -253,10 +253,10 @@ func (s) TestRDSConfigDump(t *testing.T) {
)
// Expect NACK for [0], but old ACK for [1].
wantDump := make(map[string]xdsclient.UpdateWithMD)
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]] = xdsclient.UpdateWithMD{
wantDump[rdsTargets[0]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
Version: testVersion,
@ -267,7 +267,7 @@ func (s) TestRDSConfigDump(t *testing.T) {
},
Raw: routeRaws[rdsTargets[0]],
}
wantDump[rdsTargets[1]] = xdsclient.UpdateWithMD{
wantDump[rdsTargets[1]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: nackVersion},
Raw: routeRaws[rdsTargets[1]],
}
@ -321,15 +321,15 @@ func (s) TestCDSConfigDump(t *testing.T) {
updateHandler := client.(xdsclient.UpdateHandler)
// Expected unknown.
if err := compareDump(client.DumpCDS, "", map[string]xdsclient.UpdateWithMD{}); err != nil {
if err := compareDump(client.DumpCDS, "", map[string]xdsresource.UpdateWithMD{}); err != nil {
t.Fatalf(err.Error())
}
wantRequested := make(map[string]xdsclient.UpdateWithMD)
wantRequested := make(map[string]xdsresource.UpdateWithMD)
for _, n := range cdsTargets {
cancel := client.WatchCluster(n, func(update xdsresource.ClusterUpdate, err error) {})
defer cancel()
wantRequested[n] = xdsclient.UpdateWithMD{MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}
wantRequested[n] = xdsresource.UpdateWithMD{MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}
}
// Expected requested.
if err := compareDump(client.DumpCDS, "", wantRequested); err != nil {
@ -337,10 +337,10 @@ func (s) TestCDSConfigDump(t *testing.T) {
}
update0 := make(map[string]xdsresource.ClusterUpdateErrTuple)
want0 := make(map[string]xdsclient.UpdateWithMD)
want0 := make(map[string]xdsresource.UpdateWithMD)
for n, r := range clusterRaws {
update0[n] = xdsresource.ClusterUpdateErrTuple{Update: xdsresource.ClusterUpdate{Raw: r}}
want0[n] = xdsclient.UpdateWithMD{
want0[n] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: testVersion},
Raw: r,
}
@ -369,10 +369,10 @@ func (s) TestCDSConfigDump(t *testing.T) {
)
// Expect NACK for [0], but old ACK for [1].
wantDump := make(map[string]xdsclient.UpdateWithMD)
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]] = xdsclient.UpdateWithMD{
wantDump[cdsTargets[0]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
Version: testVersion,
@ -383,7 +383,7 @@ func (s) TestCDSConfigDump(t *testing.T) {
},
Raw: clusterRaws[cdsTargets[0]],
}
wantDump[cdsTargets[1]] = xdsclient.UpdateWithMD{
wantDump[cdsTargets[1]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: nackVersion},
Raw: clusterRaws[cdsTargets[1]],
}
@ -423,15 +423,15 @@ func (s) TestEDSConfigDump(t *testing.T) {
updateHandler := client.(xdsclient.UpdateHandler)
// Expected unknown.
if err := compareDump(client.DumpEDS, "", map[string]xdsclient.UpdateWithMD{}); err != nil {
if err := compareDump(client.DumpEDS, "", map[string]xdsresource.UpdateWithMD{}); err != nil {
t.Fatalf(err.Error())
}
wantRequested := make(map[string]xdsclient.UpdateWithMD)
wantRequested := make(map[string]xdsresource.UpdateWithMD)
for _, n := range edsTargets {
cancel := client.WatchEndpoints(n, func(update xdsresource.EndpointsUpdate, err error) {})
defer cancel()
wantRequested[n] = xdsclient.UpdateWithMD{MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}
wantRequested[n] = xdsresource.UpdateWithMD{MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}}
}
// Expected requested.
if err := compareDump(client.DumpEDS, "", wantRequested); err != nil {
@ -439,10 +439,10 @@ func (s) TestEDSConfigDump(t *testing.T) {
}
update0 := make(map[string]xdsresource.EndpointsUpdateErrTuple)
want0 := make(map[string]xdsclient.UpdateWithMD)
want0 := make(map[string]xdsresource.UpdateWithMD)
for n, r := range endpointRaws {
update0[n] = xdsresource.EndpointsUpdateErrTuple{Update: xdsresource.EndpointsUpdate{Raw: r}}
want0[n] = xdsclient.UpdateWithMD{
want0[n] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: testVersion},
Raw: r,
}
@ -471,10 +471,10 @@ func (s) TestEDSConfigDump(t *testing.T) {
)
// Expect NACK for [0], but old ACK for [1].
wantDump := make(map[string]xdsclient.UpdateWithMD)
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]] = xdsclient.UpdateWithMD{
wantDump[edsTargets[0]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{
Status: xdsresource.ServiceStatusNACKed,
Version: testVersion,
@ -485,7 +485,7 @@ func (s) TestEDSConfigDump(t *testing.T) {
},
Raw: endpointRaws[edsTargets[0]],
}
wantDump[edsTargets[1]] = xdsclient.UpdateWithMD{
wantDump[edsTargets[1]] = xdsresource.UpdateWithMD{
MD: xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusACKed, Version: nackVersion},
Raw: endpointRaws[edsTargets[1]],
}
@ -494,7 +494,7 @@ func (s) TestEDSConfigDump(t *testing.T) {
}
}
func compareDump(dumpFunc func() (string, map[string]xdsclient.UpdateWithMD), wantVersion string, wantDump interface{}) error {
func compareDump(dumpFunc func() (string, map[string]xdsresource.UpdateWithMD), wantVersion string, wantDump interface{}) error {
v, dump := dumpFunc()
if v != wantVersion {
return fmt.Errorf("Dump() returned version %q, want %q", v, wantVersion)

View File

@ -0,0 +1,92 @@
/*
*
* 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) (string, map[string]xdsresource.UpdateWithMD) {
pb.mu.Lock()
defer pb.mu.Unlock()
var (
version string
md map[string]xdsresource.UpdateMetadata
cache interface{}
)
switch t {
case xdsresource.ListenerResource:
version = pb.ldsVersion
md = pb.ldsMD
cache = pb.ldsCache
case xdsresource.RouteConfigResource:
version = pb.rdsVersion
md = pb.rdsMD
cache = pb.rdsCache
case xdsresource.ClusterResource:
version = pb.cdsVersion
md = pb.cdsMD
cache = pb.cdsCache
case xdsresource.EndpointsResource:
version = pb.edsVersion
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 version, ret
}

View File

@ -0,0 +1,186 @@
/*
*
* 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
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
ldsVersion string // Only used in CSDS.
ldsCache map[string]xdsresource.ListenerUpdate
ldsMD map[string]xdsresource.UpdateMetadata
rdsWatchers map[string]map[*watchInfo]bool
rdsVersion string // Only used in CSDS.
rdsCache map[string]xdsresource.RouteConfigUpdate
rdsMD map[string]xdsresource.UpdateMetadata
cdsWatchers map[string]map[*watchInfo]bool
cdsVersion string // Only used in CSDS.
cdsCache map[string]xdsresource.ClusterUpdate
cdsMD map[string]xdsresource.UpdateMetadata
edsWatchers map[string]map[*watchInfo]bool
edsVersion string // Only used in CSDS.
edsCache map[string]xdsresource.EndpointsUpdate
edsMD map[string]xdsresource.UpdateMetadata
}
// New creates a new Pubsub.
func New(watchExpiryTimeout time.Duration, logger *grpclog.PrefixLogger) *Pubsub {
pb := &Pubsub{
done: grpcsync.NewEvent(),
logger: logger,
watchExpiryTimeout: watchExpiryTimeout,
updateCh: buffer.NewUnbounded(),
ldsWatchers: make(map[string]map[*watchInfo]bool),
ldsCache: make(map[string]xdsresource.ListenerUpdate),
ldsMD: make(map[string]xdsresource.UpdateMetadata),
rdsWatchers: make(map[string]map[*watchInfo]bool),
rdsCache: make(map[string]xdsresource.RouteConfigUpdate),
rdsMD: make(map[string]xdsresource.UpdateMetadata),
cdsWatchers: make(map[string]map[*watchInfo]bool),
cdsCache: make(map[string]xdsresource.ClusterUpdate),
cdsMD: make(map[string]xdsresource.UpdateMetadata),
edsWatchers: make(map[string]map[*watchInfo]bool),
edsCache: make(map[string]xdsresource.EndpointsUpdate),
edsMD: make(map[string]xdsresource.UpdateMetadata),
}
go pb.run()
return pb
}
// WatchListener registers a watcher for the LDS resource.
//
// It also returns whether this is the first watch for this resource.
func (pb *Pubsub) WatchListener(serviceName string, cb func(xdsresource.ListenerUpdate, error)) (first bool, cancel func() bool) {
wi := &watchInfo{
c: pb,
rType: xdsresource.ListenerResource,
target: serviceName,
ldsCallback: cb,
}
wi.expiryTimer = time.AfterFunc(pb.watchExpiryTimeout, func() {
wi.timeout()
})
return pb.watch(wi)
}
// WatchRouteConfig register a watcher for the RDS resource.
//
// It also returns whether this is the first watch for this resource.
func (pb *Pubsub) WatchRouteConfig(routeName string, cb func(xdsresource.RouteConfigUpdate, error)) (first bool, cancel func() bool) {
wi := &watchInfo{
c: pb,
rType: xdsresource.RouteConfigResource,
target: routeName,
rdsCallback: cb,
}
wi.expiryTimer = time.AfterFunc(pb.watchExpiryTimeout, func() {
wi.timeout()
})
return pb.watch(wi)
}
// WatchCluster register a watcher for the CDS resource.
//
// It also returns whether this is the first watch for this resource.
func (pb *Pubsub) WatchCluster(clusterName string, cb func(xdsresource.ClusterUpdate, error)) (first bool, cancel func() bool) {
wi := &watchInfo{
c: pb,
rType: xdsresource.ClusterResource,
target: clusterName,
cdsCallback: cb,
}
wi.expiryTimer = time.AfterFunc(pb.watchExpiryTimeout, func() {
wi.timeout()
})
return pb.watch(wi)
}
// WatchEndpoints registers a watcher for the EDS resource.
//
// It also returns whether this is the first watch for this resource.
func (pb *Pubsub) WatchEndpoints(clusterName string, cb func(xdsresource.EndpointsUpdate, error)) (first bool, cancel func() bool) {
wi := &watchInfo{
c: pb,
rType: xdsresource.EndpointsResource,
target: clusterName,
edsCallback: cb,
}
wi.expiryTimer = time.AfterFunc(pb.watchExpiryTimeout, func() {
wi.timeout()
})
return pb.watch(wi)
}
// Close closes the pubsub.
func (pb *Pubsub) Close() {
if pb.done.HasFired() {
return
}
pb.done.Fire()
}
// run is a goroutine for all the callbacks.
//
// Callback can be called in watch(), if an item is found in cache. Without this
// goroutine, the callback will be called inline, which might cause a deadlock
// in user's code. Callbacks also cannot be simple `go callback()` because the
// order matters.
func (pb *Pubsub) run() {
for {
select {
case t := <-pb.updateCh.Get():
pb.updateCh.Load()
if pb.done.HasFired() {
return
}
pb.callCallback(t.(*watcherInfoWithUpdate))
case <-pb.done.Done():
return
}
}
}

View File

@ -0,0 +1,319 @@
/*
*
* 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()
pb.ldsVersion = metadata.Version
if metadata.ErrState != nil {
pb.ldsVersion = metadata.ErrState.Version
}
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.
pb.rdsVersion = metadata.Version
if metadata.ErrState != nil {
pb.rdsVersion = metadata.ErrState.Version
}
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()
pb.cdsVersion = metadata.Version
if metadata.ErrState != nil {
pb.cdsVersion = metadata.ErrState.Version
}
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.ldsMD[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()
pb.edsVersion = metadata.Version
if metadata.ErrState != nil {
pb.edsVersion = metadata.ErrState.Version
}
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.edsWatchers {
for wi := range s {
wi.newError(xdsresource.NewErrorf(xdsresource.ErrorTypeConnection, "xds: error received from xDS stream: %v", err))
}
}
}

View File

@ -0,0 +1,232 @@
/*
*
* 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{}
}
wi.c.scheduleCallback(wi, u, err)
}
func (wi *watchInfo) cancel() {
wi.mu.Lock()
defer wi.mu.Unlock()
if wi.state == watchInfoStateCanceled {
return
}
wi.expiryTimer.Stop()
wi.state = watchInfoStateCanceled
}
func (pb *Pubsub) watch(wi *watchInfo) (first bool, cancel func() bool) {
pb.mu.Lock()
defer pb.mu.Unlock()
pb.logger.Debugf("new watch for type %v, resource name %v", wi.rType, wi.target)
var (
watchers map[string]map[*watchInfo]bool
mds map[string]xdsresource.UpdateMetadata
)
switch wi.rType {
case xdsresource.ListenerResource:
watchers = pb.ldsWatchers
mds = pb.ldsMD
case xdsresource.RouteConfigResource:
watchers = pb.rdsWatchers
mds = pb.rdsMD
case xdsresource.ClusterResource:
watchers = pb.cdsWatchers
mds = pb.cdsMD
case xdsresource.EndpointsResource:
watchers = pb.edsWatchers
mds = pb.edsMD
default:
pb.logger.Errorf("unknown watch type: %v", wi.rType)
return false, nil
}
var firstWatcher bool
resourceName := wi.target
s, ok := watchers[wi.target]
if !ok {
// If this is a new watcher, will ask lower level to send a new request
// with the resource name.
//
// If this (type+name) is already being watched, will not notify the
// underlying versioned apiClient.
pb.logger.Debugf("first watch for type %v, resource name %v, will send a new xDS request", wi.rType, wi.target)
s = make(map[*watchInfo]bool)
watchers[resourceName] = s
mds[resourceName] = xdsresource.UpdateMetadata{Status: xdsresource.ServiceStatusRequested}
firstWatcher = true
}
// No matter what, add the new watcher to the set, so it's callback will be
// call for new responses.
s[wi] = true
// If the resource is in cache, call the callback with the value.
switch wi.rType {
case xdsresource.ListenerResource:
if v, ok := pb.ldsCache[resourceName]; ok {
pb.logger.Debugf("LDS resource with name %v found in cache: %+v", wi.target, pretty.ToJSON(v))
wi.newUpdate(v)
}
case xdsresource.RouteConfigResource:
if v, ok := pb.rdsCache[resourceName]; ok {
pb.logger.Debugf("RDS resource with name %v found in cache: %+v", wi.target, pretty.ToJSON(v))
wi.newUpdate(v)
}
case xdsresource.ClusterResource:
if v, ok := pb.cdsCache[resourceName]; ok {
pb.logger.Debugf("CDS resource with name %v found in cache: %+v", wi.target, pretty.ToJSON(v))
wi.newUpdate(v)
}
case xdsresource.EndpointsResource:
if v, ok := pb.edsCache[resourceName]; ok {
pb.logger.Debugf("EDS resource with name %v found in cache: %+v", wi.target, pretty.ToJSON(v))
wi.newUpdate(v)
}
}
return firstWatcher, func() bool {
pb.logger.Debugf("watch for type %v, resource name %v canceled", wi.rType, wi.target)
wi.cancel()
pb.mu.Lock()
defer pb.mu.Unlock()
var lastWatcher bool
if s := watchers[resourceName]; s != nil {
// Remove this watcher, so it's callback will not be called in the
// future.
delete(s, wi)
if len(s) == 0 {
pb.logger.Debugf("last watch for type %v, resource name %v canceled, will send a new xDS request", wi.rType, wi.target)
// If this was the last watcher, also tell xdsv2Client to stop
// watching this resource.
delete(watchers, resourceName)
delete(mds, resourceName)
lastWatcher = true
// Remove the resource from cache. When a watch for this
// resource is added later, it will trigger a xDS request with
// resource names, and client will receive new xDS responses.
switch wi.rType {
case xdsresource.ListenerResource:
delete(pb.ldsCache, resourceName)
case xdsresource.RouteConfigResource:
delete(pb.rdsCache, resourceName)
case xdsresource.ClusterResource:
delete(pb.cdsCache, resourceName)
case xdsresource.EndpointsResource:
delete(pb.edsCache, resourceName)
}
}
}
return lastWatcher
}
}

View File

@ -25,6 +25,7 @@ import (
"github.com/golang/protobuf/proto"
"google.golang.org/grpc/xds/internal/xdsclient/load"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
"google.golang.org/grpc"
"google.golang.org/grpc/internal/buffer"
@ -52,7 +53,7 @@ type VersionedClient interface {
// SendRequest constructs and sends out a DiscoveryRequest message specific
// to the underlying transport protocol version.
SendRequest(s grpc.ClientStream, resourceNames []string, rType ResourceType, version, nonce, errMsg string) error
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.
@ -65,7 +66,7 @@ type VersionedClient interface {
// If the provided protobuf message contains a resource type which is not
// supported, implementations must return an error of type
// ErrResourceTypeUnsupported.
HandleResponse(proto.Message) (ResourceType, string, string, error)
HandleResponse(proto.Message) (xdsresource.ResourceType, string, string, error)
// NewLoadStatsStream returns a new LRS client stream specific to the underlying
// transport protocol version.
@ -116,14 +117,14 @@ type TransportHelper struct {
// messages. When the user of this client object cancels a watch call,
// these are set to nil. All accesses to the map protected and any value
// inside the map should be protected with the above mutex.
watchMap map[ResourceType]map[string]bool
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, becaues the versions for different resource types should
// be independent.
versionMap map[ResourceType]string
versionMap map[xdsresource.ResourceType]string
// nonceMap contains the nonce from the most recent received response.
nonceMap map[ResourceType]string
nonceMap map[xdsresource.ResourceType]string
}
// NewTransportHelper creates a new transport helper to be used by versioned
@ -138,9 +139,9 @@ func NewTransportHelper(vc VersionedClient, logger *grpclog.PrefixLogger, backof
streamCh: make(chan grpc.ClientStream, 1),
sendCh: buffer.NewUnbounded(),
watchMap: make(map[ResourceType]map[string]bool),
versionMap: make(map[ResourceType]string),
nonceMap: make(map[ResourceType]string),
watchMap: make(map[xdsresource.ResourceType]map[string]bool),
versionMap: make(map[xdsresource.ResourceType]string),
nonceMap: make(map[xdsresource.ResourceType]string),
}
go t.run(ctx)
@ -148,7 +149,7 @@ func NewTransportHelper(vc VersionedClient, logger *grpclog.PrefixLogger, backof
}
// AddWatch adds a watch for an xDS resource given its type and name.
func (t *TransportHelper) AddWatch(rType ResourceType, resourceName string) {
func (t *TransportHelper) AddWatch(rType xdsresource.ResourceType, resourceName string) {
t.sendCh.Put(&watchAction{
rType: rType,
remove: false,
@ -158,7 +159,7 @@ func (t *TransportHelper) AddWatch(rType ResourceType, resourceName string) {
// RemoveWatch cancels an already registered watch for an xDS resource
// given its type and name.
func (t *TransportHelper) RemoveWatch(rType ResourceType, resourceName string) {
func (t *TransportHelper) RemoveWatch(rType xdsresource.ResourceType, resourceName string) {
t.sendCh.Put(&watchAction{
rType: rType,
remove: true,
@ -250,7 +251,7 @@ func (t *TransportHelper) send(ctx context.Context) {
var (
target []string
rType ResourceType
rType xdsresource.ResourceType
version, nonce, errMsg string
send bool
)
@ -292,8 +293,8 @@ func (t *TransportHelper) sendExisting(stream grpc.ClientStream) bool {
defer t.mu.Unlock()
// Reset the ack versions when the stream restarts.
t.versionMap = make(map[ResourceType]string)
t.nonceMap = make(map[ResourceType]string)
t.versionMap = make(map[xdsresource.ResourceType]string)
t.nonceMap = make(map[xdsresource.ResourceType]string)
for rType, s := range t.watchMap {
if err := t.vClient.SendRequest(stream, mapToSlice(s), rType, "", "", ""); err != nil {
@ -351,7 +352,7 @@ func mapToSlice(m map[string]bool) []string {
}
type watchAction struct {
rType ResourceType
rType xdsresource.ResourceType
remove bool // Whether this is to remove watch for the resource.
resource string
}
@ -359,7 +360,7 @@ type watchAction struct {
// processWatchInfo pulls the fields needed by the request from a watchAction.
//
// It also updates the watch map.
func (t *TransportHelper) processWatchInfo(w *watchAction) (target []string, rType ResourceType, ver, nonce string) {
func (t *TransportHelper) processWatchInfo(w *watchAction) (target []string, rType xdsresource.ResourceType, ver, nonce string) {
t.mu.Lock()
defer t.mu.Unlock()
@ -390,7 +391,7 @@ func (t *TransportHelper) processWatchInfo(w *watchAction) (target []string, rTy
}
type ackAction struct {
rType ResourceType
rType xdsresource.ResourceType
version string // NACK if version is an empty string.
nonce string
errMsg string // Empty unless it's a NACK.
@ -403,13 +404,13 @@ type ackAction struct {
// 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 *TransportHelper) processAckInfo(ack *ackAction, stream grpc.ClientStream) (target []string, rType ResourceType, version, nonce string, send bool) {
func (t *TransportHelper) 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, UnknownResource, "", "", false
return nil, xdsresource.UnknownResource, "", "", false
}
rType = ack.rType
@ -429,7 +430,7 @@ func (t *TransportHelper) processAckInfo(ack *ackAction, stream grpc.ClientStrea
// 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, UnknownResource, "", "", false
return nil, xdsresource.UnknownResource, "", "", false
}
send = true
target = mapToSlice(s)

View File

@ -33,7 +33,6 @@ import (
"google.golang.org/grpc/internal/testutils"
"google.golang.org/grpc/xds/internal/testutils/fakeserver"
"google.golang.org/grpc/xds/internal/version"
"google.golang.org/grpc/xds/internal/xdsclient"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
@ -48,22 +47,22 @@ func startXDSV2Client(t *testing.T, cc *grpc.ClientConn) (v2c *client, cbLDS, cb
cbCDS = testutils.NewChannel()
cbEDS = testutils.NewChannel()
v2c, err := newV2Client(&testUpdateReceiver{
f: func(rType xdsclient.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata) {
f: func(rType xdsresource.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata) {
t.Logf("Received %v callback with {%+v}", rType, d)
switch rType {
case xdsclient.ListenerResource:
case xdsresource.ListenerResource:
if _, ok := d[goodLDSTarget1]; ok {
cbLDS.Send(struct{}{})
}
case xdsclient.RouteConfigResource:
case xdsresource.RouteConfigResource:
if _, ok := d[goodRouteName1]; ok {
cbRDS.Send(struct{}{})
}
case xdsclient.ClusterResource:
case xdsresource.ClusterResource:
if _, ok := d[goodClusterName1]; ok {
cbCDS.Send(struct{}{})
}
case xdsclient.EndpointsResource:
case xdsresource.EndpointsResource:
if _, ok := d[goodEDSName]; ok {
cbEDS.Send(struct{}{})
}
@ -118,16 +117,16 @@ func sendXDSRespWithVersion(ch chan<- *fakeserver.Response, respWithoutVersion *
// 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 xdsclient.ResourceType, v2c *client, reqChan *testutils.Channel, req *xdspb.DiscoveryRequest, preVersion string, preNonce string) {
func startXDS(ctx context.Context, t *testing.T, rType xdsresource.ResourceType, v2c *client, reqChan *testutils.Channel, req *xdspb.DiscoveryRequest, preVersion string, preNonce string) {
nameToWatch := ""
switch rType {
case xdsclient.ListenerResource:
case xdsresource.ListenerResource:
nameToWatch = goodLDSTarget1
case xdsclient.RouteConfigResource:
case xdsresource.RouteConfigResource:
nameToWatch = goodRouteName1
case xdsclient.ClusterResource:
case xdsresource.ClusterResource:
nameToWatch = goodClusterName1
case xdsclient.EndpointsResource:
case xdsresource.EndpointsResource:
nameToWatch = goodEDSName
}
v2c.AddWatch(rType, nameToWatch)
@ -143,7 +142,7 @@ func startXDS(ctx context.Context, t *testing.T, rType xdsclient.ResourceType, v
//
// 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 xdsclient.ResourceType, fakeServer *fakeserver.Server, ver int, goodResp *xdspb.DiscoveryResponse, wantReq *xdspb.DiscoveryRequest, callbackCh *testutils.Channel) (string, error) {
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)
@ -163,16 +162,16 @@ func sendGoodResp(ctx context.Context, t *testing.T, rType xdsclient.ResourceTyp
// 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 xdsclient.ResourceType, fakeServer *fakeserver.Server, ver int, wantReq *xdspb.DiscoveryRequest) error {
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 xdsclient.ListenerResource:
case xdsresource.ListenerResource:
typeURL = version.V2ListenerURL
case xdsclient.RouteConfigResource:
case xdsresource.RouteConfigResource:
typeURL = version.V2RouteConfigURL
case xdsclient.ClusterResource:
case xdsresource.ClusterResource:
typeURL = version.V2ClusterURL
case xdsclient.EndpointsResource:
case xdsresource.EndpointsResource:
typeURL = version.V2EndpointsURL
}
nonce := sendXDSRespWithVersion(fakeServer.XDSResponseChan, &xdspb.DiscoveryResponse{
@ -209,59 +208,59 @@ func (s) TestV2ClientAck(t *testing.T) {
defer cancel()
// Start the watch, send a good response, and check for ack.
startXDS(ctx, t, xdsclient.ListenerResource, v2c, fakeServer.XDSRequestChan, goodLDSRequest, "", "")
if _, err := sendGoodResp(ctx, t, xdsclient.ListenerResource, fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS); err != nil {
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, xdsclient.RouteConfigResource, v2c, fakeServer.XDSRequestChan, goodRDSRequest, "", "")
if _, err := sendGoodResp(ctx, t, xdsclient.RouteConfigResource, fakeServer, versionRDS, goodRDSResponse1, goodRDSRequest, cbRDS); err != nil {
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, xdsclient.ClusterResource, v2c, fakeServer.XDSRequestChan, goodCDSRequest, "", "")
if _, err := sendGoodResp(ctx, t, xdsclient.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS); err != nil {
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, xdsclient.EndpointsResource, v2c, fakeServer.XDSRequestChan, goodEDSRequest, "", "")
if _, err := sendGoodResp(ctx, t, xdsclient.EndpointsResource, fakeServer, versionEDS, goodEDSResponse1, goodEDSRequest, cbEDS); err != nil {
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, xdsclient.ListenerResource, fakeServer, versionLDS, goodLDSRequest); err != nil {
if err := sendBadResp(ctx, t, xdsresource.ListenerResource, fakeServer, versionLDS, goodLDSRequest); err != nil {
t.Fatal(err)
}
versionLDS++
if err := sendBadResp(ctx, t, xdsclient.RouteConfigResource, fakeServer, versionRDS, goodRDSRequest); err != nil {
if err := sendBadResp(ctx, t, xdsresource.RouteConfigResource, fakeServer, versionRDS, goodRDSRequest); err != nil {
t.Fatal(err)
}
versionRDS++
if err := sendBadResp(ctx, t, xdsclient.ClusterResource, fakeServer, versionCDS, goodCDSRequest); err != nil {
if err := sendBadResp(ctx, t, xdsresource.ClusterResource, fakeServer, versionCDS, goodCDSRequest); err != nil {
t.Fatal(err)
}
versionCDS++
if err := sendBadResp(ctx, t, xdsclient.EndpointsResource, fakeServer, versionEDS, goodEDSRequest); err != nil {
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, xdsclient.ListenerResource, fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS); err != nil {
if _, err := sendGoodResp(ctx, t, xdsresource.ListenerResource, fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS); err != nil {
t.Fatal(err)
}
versionLDS++
if _, err := sendGoodResp(ctx, t, xdsclient.RouteConfigResource, fakeServer, versionRDS, goodRDSResponse1, goodRDSRequest, cbRDS); err != nil {
if _, err := sendGoodResp(ctx, t, xdsresource.RouteConfigResource, fakeServer, versionRDS, goodRDSResponse1, goodRDSRequest, cbRDS); err != nil {
t.Fatal(err)
}
versionRDS++
if _, err := sendGoodResp(ctx, t, xdsclient.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS); err != nil {
if _, err := sendGoodResp(ctx, t, xdsresource.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS); err != nil {
t.Fatal(err)
}
versionCDS++
if _, err := sendGoodResp(ctx, t, xdsclient.EndpointsResource, fakeServer, versionEDS, goodEDSResponse1, goodEDSRequest, cbEDS); err != nil {
if _, err := sendGoodResp(ctx, t, xdsresource.EndpointsResource, fakeServer, versionEDS, goodEDSResponse1, goodEDSRequest, cbEDS); err != nil {
t.Fatal(err)
}
versionEDS++
@ -282,7 +281,7 @@ func (s) TestV2ClientAckFirstIsNack(t *testing.T) {
defer cancel()
// Start the watch, send a good response, and check for ack.
startXDS(ctx, t, xdsclient.ListenerResource, v2c, fakeServer.XDSRequestChan, goodLDSRequest, "", "")
startXDS(ctx, t, xdsresource.ListenerResource, v2c, fakeServer.XDSRequestChan, goodLDSRequest, "", "")
nonce := sendXDSRespWithVersion(fakeServer.XDSResponseChan, &xdspb.DiscoveryResponse{
Resources: []*anypb.Any{{}},
@ -298,7 +297,7 @@ func (s) TestV2ClientAckFirstIsNack(t *testing.T) {
t.Logf("Bad response nacked")
versionLDS++
sendGoodResp(ctx, t, xdsclient.ListenerResource, fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS)
sendGoodResp(ctx, t, xdsresource.ListenerResource, fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS)
versionLDS++
}
@ -317,14 +316,14 @@ func (s) TestV2ClientAckNackAfterNewWatch(t *testing.T) {
defer cancel()
// Start the watch, send a good response, and check for ack.
startXDS(ctx, t, xdsclient.ListenerResource, v2c, fakeServer.XDSRequestChan, goodLDSRequest, "", "")
nonce, err := sendGoodResp(ctx, t, xdsclient.ListenerResource, fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS)
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, xdsclient.ListenerResource, v2c, fakeServer.XDSRequestChan, goodLDSRequest, strconv.Itoa(versionLDS), nonce)
startXDS(ctx, t, xdsresource.ListenerResource, v2c, fakeServer.XDSRequestChan, goodLDSRequest, strconv.Itoa(versionLDS), nonce)
versionLDS++
// This is an invalid response after the new watch.
@ -341,7 +340,7 @@ func (s) TestV2ClientAckNackAfterNewWatch(t *testing.T) {
t.Logf("Bad response nacked")
versionLDS++
if _, err := sendGoodResp(ctx, t, xdsclient.ListenerResource, fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS); err != nil {
if _, err := sendGoodResp(ctx, t, xdsresource.ListenerResource, fakeServer, versionLDS, goodLDSResponse1, goodLDSRequest, cbLDS); err != nil {
t.Fatal(err)
}
versionLDS++
@ -362,42 +361,42 @@ func (s) TestV2ClientAckNewWatchAfterCancel(t *testing.T) {
defer cancel()
// Start a CDS watch.
v2c.AddWatch(xdsclient.ClusterResource, goodClusterName1)
v2c.AddWatch(xdsresource.ClusterResource, goodClusterName1)
if err := compareXDSRequest(ctx, fakeServer.XDSRequestChan, goodCDSRequest, "", "", false); err != nil {
t.Fatal(err)
}
t.Logf("FakeServer received %v request...", xdsclient.ClusterResource)
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, xdsclient.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS)
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(xdsclient.ClusterResource, goodClusterName1)
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", xdsclient.ClusterResource, err)
t.Fatalf("Failed to receive %v request: %v", xdsresource.ClusterResource, err)
}
v2c.AddWatch(xdsclient.ClusterResource, goodClusterName1)
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", xdsclient.ClusterResource, err)
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, xdsclient.ClusterResource, fakeServer, versionCDS, goodCDSRequest); err != nil {
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, xdsclient.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS); err != nil {
if _, err := sendGoodResp(ctx, t, xdsresource.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS); err != nil {
t.Fatal(err)
}
versionCDS++
@ -420,25 +419,25 @@ func (s) TestV2ClientAckCancelResponseRace(t *testing.T) {
defer cancel()
// Start a CDS watch.
v2c.AddWatch(xdsclient.ClusterResource, goodClusterName1)
v2c.AddWatch(xdsresource.ClusterResource, goodClusterName1)
if err := compareXDSRequest(ctx, fakeServer.XDSRequestChan, goodCDSRequest, "", "", false); err != nil {
t.Fatalf("Failed to receive %v request: %v", xdsclient.ClusterResource, err)
t.Fatalf("Failed to receive %v request: %v", xdsresource.ClusterResource, err)
}
t.Logf("FakeServer received %v request...", xdsclient.ClusterResource)
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, xdsclient.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS)
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(xdsclient.ClusterResource, goodClusterName1)
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", xdsclient.ClusterResource, err)
t.Fatalf("Failed to receive %v request: %v", xdsresource.ClusterResource, err)
}
versionCDS++
@ -450,7 +449,7 @@ func (s) TestV2ClientAckCancelResponseRace(t *testing.T) {
// Send a good response.
nonce = sendXDSRespWithVersion(fakeServer.XDSResponseChan, goodCDSResponse1, versionCDS)
t.Logf("Good %v response pushed to fakeServer...", xdsclient.ClusterResource)
t.Logf("Good %v response pushed to fakeServer...", xdsresource.ClusterResource)
// Expect no ACK because watch was canceled.
sCtx, sCancel = context.WithTimeout(context.Background(), defaultTestShortTimeout)
@ -461,24 +460,24 @@ func (s) TestV2ClientAckCancelResponseRace(t *testing.T) {
// Still expected an callback update, because response was good.
if _, err := cbCDS.Receive(ctx); err != nil {
t.Fatalf("Timeout when expecting %v update", xdsclient.ClusterResource)
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(xdsclient.ClusterResource, goodClusterName1)
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", xdsclient.ClusterResource, err)
t.Fatalf("Failed to receive %v request: %v", xdsresource.ClusterResource, err)
}
// Send a bad response with the next version.
if err := sendBadResp(ctx, t, xdsclient.ClusterResource, fakeServer, versionCDS, goodCDSRequest); err != nil {
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, xdsclient.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS); err != nil {
if _, err := sendGoodResp(ctx, t, xdsresource.ClusterResource, fakeServer, versionCDS, goodCDSResponse1, goodCDSRequest, cbCDS); err != nil {
t.Fatal(err)
}
versionCDS++

View File

@ -28,7 +28,6 @@ import (
"github.com/google/go-cmp/cmp/cmpopts"
"google.golang.org/grpc/internal/testutils"
"google.golang.org/grpc/xds/internal/version"
"google.golang.org/grpc/xds/internal/xdsclient"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
@ -175,7 +174,7 @@ func (s) TestCDSHandleResponse(t *testing.T) {
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
testWatchHandle(t, &watchHandleTestcase{
rType: xdsclient.ClusterResource,
rType: xdsresource.ClusterResource,
resourceName: goodClusterName1,
responseToHandle: test.cdsResponse,
@ -195,7 +194,7 @@ func (s) TestCDSHandleResponseWithoutWatch(t *testing.T) {
defer cleanup()
v2c, err := newV2Client(&testUpdateReceiver{
f: func(xdsclient.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) {},
f: func(xdsresource.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) {},
}, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
if err != nil {
t.Fatal(err)

View File

@ -43,11 +43,11 @@ func init() {
}
var (
resourceTypeToURL = map[xdsclient.ResourceType]string{
xdsclient.ListenerResource: version.V2ListenerURL,
xdsclient.RouteConfigResource: version.V2RouteConfigURL,
xdsclient.ClusterResource: version.V2ClusterURL,
xdsclient.EndpointsResource: version.V2EndpointsURL,
resourceTypeToURL = map[xdsresource.ResourceType]string{
xdsresource.ListenerResource: version.V2ListenerURL,
xdsresource.RouteConfigResource: version.V2RouteConfigURL,
xdsresource.ClusterResource: version.V2ClusterURL,
xdsresource.EndpointsResource: version.V2EndpointsURL,
}
)
@ -109,7 +109,7 @@ func (v2c *client) NewStream(ctx context.Context) (grpc.ClientStream, error) {
// - 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 xdsclient.ResourceType, version, nonce, errMsg string) error {
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)
@ -151,8 +151,8 @@ func (v2c *client) RecvResponse(s grpc.ClientStream) (proto.Message, error) {
return resp, nil
}
func (v2c *client) HandleResponse(r proto.Message) (xdsclient.ResourceType, string, string, error) {
rType := xdsclient.UnknownResource
func (v2c *client) HandleResponse(r proto.Message) (xdsresource.ResourceType, string, string, error) {
rType := xdsresource.UnknownResource
resp, ok := r.(*v2xdspb.DiscoveryResponse)
if !ok {
return rType, "", "", fmt.Errorf("xds: unsupported message type: %T", resp)
@ -167,16 +167,16 @@ func (v2c *client) HandleResponse(r proto.Message) (xdsclient.ResourceType, stri
switch {
case xdsresource.IsListenerResource(url):
err = v2c.handleLDSResponse(resp)
rType = xdsclient.ListenerResource
rType = xdsresource.ListenerResource
case xdsresource.IsRouteConfigResource(url):
err = v2c.handleRDSResponse(resp)
rType = xdsclient.RouteConfigResource
rType = xdsresource.RouteConfigResource
case xdsresource.IsClusterResource(url):
err = v2c.handleCDSResponse(resp)
rType = xdsclient.ClusterResource
rType = xdsresource.ClusterResource
case xdsresource.IsEndpointsResource(url):
err = v2c.handleEDSResponse(resp)
rType = xdsclient.EndpointsResource
rType = xdsresource.EndpointsResource
default:
return rType, "", "", xdsclient.ErrResourceTypeUnsupported{
ErrStr: fmt.Sprintf("Resource type %v unknown in response from server", resp.GetTypeUrl()),

View File

@ -288,7 +288,7 @@ var (
)
type watchHandleTestcase struct {
rType xdsclient.ResourceType
rType xdsresource.ResourceType
resourceName string
responseToHandle *xdspb.DiscoveryResponse
@ -299,7 +299,7 @@ type watchHandleTestcase struct {
}
type testUpdateReceiver struct {
f func(rType xdsclient.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata)
f func(rType xdsresource.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata)
}
func (t *testUpdateReceiver) NewListeners(d map[string]xdsresource.ListenerUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
@ -307,7 +307,7 @@ func (t *testUpdateReceiver) NewListeners(d map[string]xdsresource.ListenerUpdat
for k, v := range d {
dd[k] = v
}
t.newUpdate(xdsclient.ListenerResource, dd, metadata)
t.newUpdate(xdsresource.ListenerResource, dd, metadata)
}
func (t *testUpdateReceiver) NewRouteConfigs(d map[string]xdsresource.RouteConfigUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
@ -315,7 +315,7 @@ func (t *testUpdateReceiver) NewRouteConfigs(d map[string]xdsresource.RouteConfi
for k, v := range d {
dd[k] = v
}
t.newUpdate(xdsclient.RouteConfigResource, dd, metadata)
t.newUpdate(xdsresource.RouteConfigResource, dd, metadata)
}
func (t *testUpdateReceiver) NewClusters(d map[string]xdsresource.ClusterUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
@ -323,7 +323,7 @@ func (t *testUpdateReceiver) NewClusters(d map[string]xdsresource.ClusterUpdateE
for k, v := range d {
dd[k] = v
}
t.newUpdate(xdsclient.ClusterResource, dd, metadata)
t.newUpdate(xdsresource.ClusterResource, dd, metadata)
}
func (t *testUpdateReceiver) NewEndpoints(d map[string]xdsresource.EndpointsUpdateErrTuple, metadata xdsresource.UpdateMetadata) {
@ -331,12 +331,12 @@ func (t *testUpdateReceiver) NewEndpoints(d map[string]xdsresource.EndpointsUpda
for k, v := range d {
dd[k] = v
}
t.newUpdate(xdsclient.EndpointsResource, dd, metadata)
t.newUpdate(xdsresource.EndpointsResource, dd, metadata)
}
func (t *testUpdateReceiver) NewConnectionError(error) {}
func (t *testUpdateReceiver) newUpdate(rType xdsclient.ResourceType, d map[string]interface{}, metadata xdsresource.UpdateMetadata) {
func (t *testUpdateReceiver) newUpdate(rType xdsresource.ResourceType, d map[string]interface{}, metadata xdsresource.UpdateMetadata) {
t.f(rType, d, metadata)
}
@ -360,28 +360,28 @@ func testWatchHandle(t *testing.T, test *watchHandleTestcase) {
gotUpdateCh := testutils.NewChannel()
v2c, err := newV2Client(&testUpdateReceiver{
f: func(rType xdsclient.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata) {
f: func(rType xdsresource.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata) {
if rType == test.rType {
switch test.rType {
case xdsclient.ListenerResource:
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 xdsclient.RouteConfigResource:
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 xdsclient.ClusterResource:
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 xdsclient.EndpointsResource:
case xdsresource.EndpointsResource:
dd := make(map[string]xdsresource.EndpointsUpdateErrTuple)
for n, u := range d {
dd[n] = u.(xdsresource.EndpointsUpdateErrTuple)
@ -416,13 +416,13 @@ func testWatchHandle(t *testing.T, test *watchHandleTestcase) {
// request channel afterwards.
var handleXDSResp func(response *xdspb.DiscoveryResponse) error
switch test.rType {
case xdsclient.ListenerResource:
case xdsresource.ListenerResource:
handleXDSResp = v2c.handleLDSResponse
case xdsclient.RouteConfigResource:
case xdsresource.RouteConfigResource:
handleXDSResp = v2c.handleRDSResponse
case xdsclient.ClusterResource:
case xdsresource.ClusterResource:
handleXDSResp = v2c.handleCDSResponse
case xdsclient.EndpointsResource:
case xdsresource.EndpointsResource:
handleXDSResp = v2c.handleEDSResponse
}
if err := handleXDSResp(test.responseToHandle); (err != nil) != test.wantHandleErr {
@ -504,7 +504,7 @@ func (s) TestV2ClientBackoffAfterRecvError(t *testing.T) {
callbackCh := make(chan struct{})
v2c, err := newV2Client(&testUpdateReceiver{
f: func(xdsclient.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) { close(callbackCh) },
f: func(xdsresource.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) { close(callbackCh) },
}, cc, goodNodeProto, clientBackoff, nil)
if err != nil {
t.Fatal(err)
@ -512,7 +512,7 @@ func (s) TestV2ClientBackoffAfterRecvError(t *testing.T) {
defer v2c.Close()
t.Log("Started xds v2Client...")
v2c.AddWatch(xdsclient.ListenerResource, goodLDSTarget1)
v2c.AddWatch(xdsresource.ListenerResource, goodLDSTarget1)
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
if _, err := fakeServer.XDSRequestChan.Receive(ctx); err != nil {
@ -549,8 +549,8 @@ func (s) TestV2ClientRetriesAfterBrokenStream(t *testing.T) {
callbackCh := testutils.NewChannel()
v2c, err := newV2Client(&testUpdateReceiver{
f: func(rType xdsclient.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata) {
if rType == xdsclient.ListenerResource {
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{}{})
@ -564,7 +564,7 @@ func (s) TestV2ClientRetriesAfterBrokenStream(t *testing.T) {
defer v2c.Close()
t.Log("Started xds v2Client...")
v2c.AddWatch(xdsclient.ListenerResource, goodLDSTarget1)
v2c.AddWatch(xdsresource.ListenerResource, goodLDSTarget1)
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
defer cancel()
if _, err := fakeServer.XDSRequestChan.Receive(ctx); err != nil {
@ -621,8 +621,8 @@ func (s) TestV2ClientWatchWithoutStream(t *testing.T) {
callbackCh := testutils.NewChannel()
v2c, err := newV2Client(&testUpdateReceiver{
f: func(rType xdsclient.ResourceType, d map[string]interface{}, md xdsresource.UpdateMetadata) {
if rType == xdsclient.ListenerResource {
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)
@ -638,7 +638,7 @@ func (s) TestV2ClientWatchWithoutStream(t *testing.T) {
// This watch is started when the xds-ClientConn is in Transient Failure,
// and no xds stream is created.
v2c.AddWatch(xdsclient.ListenerResource, goodLDSTarget1)
v2c.AddWatch(xdsresource.ListenerResource, goodLDSTarget1)
// The watcher should receive an update, with a timeout error in it.
sCtx, sCancel := context.WithTimeout(context.Background(), defaultTestShortTimeout)

View File

@ -29,7 +29,6 @@ import (
"google.golang.org/grpc/xds/internal"
xtestutils "google.golang.org/grpc/xds/internal/testutils"
"google.golang.org/grpc/xds/internal/version"
"google.golang.org/grpc/xds/internal/xdsclient"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
@ -165,7 +164,7 @@ func (s) TestEDSHandleResponse(t *testing.T) {
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
testWatchHandle(t, &watchHandleTestcase{
rType: xdsclient.EndpointsResource,
rType: xdsresource.EndpointsResource,
resourceName: goodEDSName,
responseToHandle: test.edsResponse,
wantHandleErr: test.wantErr,
@ -184,7 +183,7 @@ func (s) TestEDSHandleResponseWithoutWatch(t *testing.T) {
defer cleanup()
v2c, err := newV2Client(&testUpdateReceiver{
f: func(xdsclient.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) {},
f: func(xdsresource.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) {},
}, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
if err != nil {
t.Fatal(err)

View File

@ -24,7 +24,6 @@ import (
v2xdspb "github.com/envoyproxy/go-control-plane/envoy/api/v2"
"github.com/google/go-cmp/cmp/cmpopts"
"google.golang.org/grpc/xds/internal/xdsclient"
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
)
@ -163,7 +162,7 @@ func (s) TestLDSHandleResponse(t *testing.T) {
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
testWatchHandle(t, &watchHandleTestcase{
rType: xdsclient.ListenerResource,
rType: xdsresource.ListenerResource,
resourceName: goodLDSTarget1,
responseToHandle: test.ldsResponse,
wantHandleErr: test.wantErr,
@ -182,7 +181,7 @@ func (s) TestLDSHandleResponseWithoutWatch(t *testing.T) {
defer cleanup()
v2c, err := newV2Client(&testUpdateReceiver{
f: func(xdsclient.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) {},
f: func(xdsresource.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) {},
}, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
if err != nil {
t.Fatal(err)

View File

@ -36,7 +36,7 @@ import (
// pre-requirement for RDS, and RDS handle would fail without an existing LDS
// watch.
func doLDS(ctx context.Context, t *testing.T, v2c xdsclient.APIClient, fakeServer *fakeserver.Server) {
v2c.AddWatch(xdsclient.ListenerResource, goodLDSTarget1)
v2c.AddWatch(xdsresource.ListenerResource, goodLDSTarget1)
if _, err := fakeServer.XDSRequestChan.Receive(ctx); err != nil {
t.Fatalf("Timeout waiting for LDS request: %v", err)
}
@ -164,7 +164,7 @@ func (s) TestRDSHandleResponseWithRouting(t *testing.T) {
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
testWatchHandle(t, &watchHandleTestcase{
rType: xdsclient.RouteConfigResource,
rType: xdsresource.RouteConfigResource,
resourceName: goodRouteName1,
responseToHandle: test.rdsResponse,
wantHandleErr: test.wantErr,
@ -183,7 +183,7 @@ func (s) TestRDSHandleResponseWithoutRDSWatch(t *testing.T) {
defer cleanup()
v2c, err := newV2Client(&testUpdateReceiver{
f: func(xdsclient.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) {},
f: func(xdsresource.ResourceType, map[string]interface{}, xdsresource.UpdateMetadata) {},
}, cc, goodNodeProto, func(int) time.Duration { return 0 }, nil)
if err != nil {
t.Fatal(err)

View File

@ -43,11 +43,11 @@ func init() {
}
var (
resourceTypeToURL = map[xdsclient.ResourceType]string{
xdsclient.ListenerResource: version.V3ListenerURL,
xdsclient.RouteConfigResource: version.V3RouteConfigURL,
xdsclient.ClusterResource: version.V3ClusterURL,
xdsclient.EndpointsResource: version.V3EndpointsURL,
resourceTypeToURL = map[xdsresource.ResourceType]string{
xdsresource.ListenerResource: version.V3ListenerURL,
xdsresource.RouteConfigResource: version.V3RouteConfigURL,
xdsresource.ClusterResource: version.V3ClusterURL,
xdsresource.EndpointsResource: version.V3EndpointsURL,
}
)
@ -109,7 +109,7 @@ func (v3c *client) NewStream(ctx context.Context) (grpc.ClientStream, error) {
// - 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 xdsclient.ResourceType, version, nonce, errMsg string) error {
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)
@ -151,8 +151,8 @@ func (v3c *client) RecvResponse(s grpc.ClientStream) (proto.Message, error) {
return resp, nil
}
func (v3c *client) HandleResponse(r proto.Message) (xdsclient.ResourceType, string, string, error) {
rType := xdsclient.UnknownResource
func (v3c *client) HandleResponse(r proto.Message) (xdsresource.ResourceType, string, string, error) {
rType := xdsresource.UnknownResource
resp, ok := r.(*v3discoverypb.DiscoveryResponse)
if !ok {
return rType, "", "", fmt.Errorf("xds: unsupported message type: %T", resp)
@ -167,16 +167,16 @@ func (v3c *client) HandleResponse(r proto.Message) (xdsclient.ResourceType, stri
switch {
case xdsresource.IsListenerResource(url):
err = v3c.handleLDSResponse(resp)
rType = xdsclient.ListenerResource
rType = xdsresource.ListenerResource
case xdsresource.IsRouteConfigResource(url):
err = v3c.handleRDSResponse(resp)
rType = xdsclient.RouteConfigResource
rType = xdsresource.RouteConfigResource
case xdsresource.IsClusterResource(url):
err = v3c.handleCDSResponse(resp)
rType = xdsclient.ClusterResource
rType = xdsresource.ClusterResource
case xdsresource.IsEndpointsResource(url):
err = v3c.handleEDSResponse(resp)
rType = xdsclient.EndpointsResource
rType = xdsresource.EndpointsResource
default:
return rType, "", "", xdsclient.ErrResourceTypeUnsupported{
ErrStr: fmt.Sprintf("Resource type %v unknown in response from server", resp.GetTypeUrl()),

View File

@ -13,239 +13,29 @@
* 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"
"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 *clientImpl
rType 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(NewErrorf(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 ListenerResource:
u = xdsresource.ListenerUpdate{}
case RouteConfigResource:
u = xdsresource.RouteConfigUpdate{}
case ClusterResource:
u = xdsresource.ClusterUpdate{}
case EndpointsResource:
u = xdsresource.EndpointsUpdate{}
}
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 (c *clientImpl) watch(wi *watchInfo) (cancel func()) {
c.mu.Lock()
defer c.mu.Unlock()
c.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 ListenerResource:
watchers = c.ldsWatchers
mds = c.ldsMD
case RouteConfigResource:
watchers = c.rdsWatchers
mds = c.rdsMD
case ClusterResource:
watchers = c.cdsWatchers
mds = c.cdsMD
case EndpointsResource:
watchers = c.edsWatchers
mds = c.edsMD
default:
c.logger.Errorf("unknown watch type: %v", wi.rType)
return nil
}
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.
c.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}
c.apiClient.AddWatch(wi.rType, resourceName)
}
// 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 ListenerResource:
if v, ok := c.ldsCache[resourceName]; ok {
c.logger.Debugf("LDS resource with name %v found in cache: %+v", wi.target, pretty.ToJSON(v))
wi.newUpdate(v)
}
case RouteConfigResource:
if v, ok := c.rdsCache[resourceName]; ok {
c.logger.Debugf("RDS resource with name %v found in cache: %+v", wi.target, pretty.ToJSON(v))
wi.newUpdate(v)
}
case ClusterResource:
if v, ok := c.cdsCache[resourceName]; ok {
c.logger.Debugf("CDS resource with name %v found in cache: %+v", wi.target, pretty.ToJSON(v))
wi.newUpdate(v)
}
case EndpointsResource:
if v, ok := c.edsCache[resourceName]; ok {
c.logger.Debugf("EDS resource with name %v found in cache: %+v", wi.target, pretty.ToJSON(v))
wi.newUpdate(v)
}
}
return func() {
c.logger.Debugf("watch for type %v, resource name %v canceled", wi.rType, wi.target)
wi.cancel()
c.mu.Lock()
defer c.mu.Unlock()
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 {
c.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)
c.apiClient.RemoveWatch(wi.rType, resourceName)
// 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 ListenerResource:
delete(c.ldsCache, resourceName)
case RouteConfigResource:
delete(c.rdsCache, resourceName)
case ClusterResource:
delete(c.cdsCache, resourceName)
case EndpointsResource:
delete(c.edsCache, resourceName)
}
}
}
}
}
// WatchListener uses LDS to discover information about the provided listener.
//
// 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()) {
wi := &watchInfo{
c: c,
rType: ListenerResource,
target: serviceName,
ldsCallback: cb,
first, cancelF := c.pubsub.WatchListener(serviceName, cb)
if first {
c.apiClient.AddWatch(xdsresource.ListenerResource, serviceName)
}
return func() {
if cancelF() {
c.apiClient.RemoveWatch(xdsresource.ListenerResource, serviceName)
}
}
wi.expiryTimer = time.AfterFunc(c.watchExpiryTimeout, func() {
wi.timeout()
})
return c.watch(wi)
}
// WatchRouteConfig starts a listener watcher for the service..
@ -254,17 +44,15 @@ func (c *clientImpl) WatchListener(serviceName string, cb func(xdsresource.Liste
// 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()) {
wi := &watchInfo{
c: c,
rType: RouteConfigResource,
target: routeName,
rdsCallback: cb,
first, cancelF := c.pubsub.WatchRouteConfig(routeName, cb)
if first {
c.apiClient.AddWatch(xdsresource.RouteConfigResource, routeName)
}
return func() {
if cancelF() {
c.apiClient.RemoveWatch(xdsresource.RouteConfigResource, routeName)
}
}
wi.expiryTimer = time.AfterFunc(c.watchExpiryTimeout, func() {
wi.timeout()
})
return c.watch(wi)
}
// WatchCluster uses CDS to discover information about the provided
@ -277,17 +65,15 @@ func (c *clientImpl) WatchRouteConfig(routeName string, cb func(xdsresource.Rout
// 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()) {
wi := &watchInfo{
c: c,
rType: ClusterResource,
target: clusterName,
cdsCallback: cb,
first, cancelF := c.pubsub.WatchCluster(clusterName, cb)
if first {
c.apiClient.AddWatch(xdsresource.ClusterResource, clusterName)
}
return func() {
if cancelF() {
c.apiClient.RemoveWatch(xdsresource.ClusterResource, clusterName)
}
}
wi.expiryTimer = time.AfterFunc(c.watchExpiryTimeout, func() {
wi.timeout()
})
return c.watch(wi)
}
// WatchEndpoints uses EDS to discover endpoints in the provided clusterName.
@ -299,15 +85,13 @@ func (c *clientImpl) WatchCluster(clusterName string, cb func(xdsresource.Cluste
// 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()) {
wi := &watchInfo{
c: c,
rType: EndpointsResource,
target: clusterName,
edsCallback: cb,
first, cancelF := c.pubsub.WatchEndpoints(clusterName, cb)
if first {
c.apiClient.AddWatch(xdsresource.EndpointsResource, clusterName)
}
return func() {
if cancelF() {
c.apiClient.RemoveWatch(xdsresource.EndpointsResource, clusterName)
}
}
wi.expiryTimer = time.AfterFunc(c.watchExpiryTimeout, func() {
wi.timeout()
})
return c.watch(wi)
}

View File

@ -56,7 +56,7 @@ func (s) TestClusterWatch(t *testing.T) {
cancelWatch := client.WatchCluster(testCDSName, func(update xdsresource.ClusterUpdate, err error) {
clusterUpdateCh.Send(xdsresource.ClusterUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[ClusterResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -122,7 +122,7 @@ func (s) TestClusterTwoWatchSameResourceName(t *testing.T) {
if i == 0 {
// A new watch is registered on the underlying API client only for
// the first iteration because we are using the same resource name.
if _, err := apiClient.addWatches[ClusterResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
}
@ -194,7 +194,7 @@ func (s) TestClusterThreeWatchDifferentResourceName(t *testing.T) {
if i == 0 {
// A new watch is registered on the underlying API client only for
// the first iteration because we are using the same resource name.
if _, err := apiClient.addWatches[ClusterResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
}
@ -205,7 +205,7 @@ func (s) TestClusterThreeWatchDifferentResourceName(t *testing.T) {
client.WatchCluster(testCDSName+"2", func(update xdsresource.ClusterUpdate, err error) {
clusterUpdateCh2.Send(xdsresource.ClusterUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[ClusterResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -250,7 +250,7 @@ func (s) TestClusterWatchAfterCache(t *testing.T) {
client.WatchCluster(testCDSName, func(update xdsresource.ClusterUpdate, err error) {
clusterUpdateCh.Send(xdsresource.ClusterUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[ClusterResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -269,7 +269,7 @@ func (s) TestClusterWatchAfterCache(t *testing.T) {
})
sCtx, sCancel := context.WithTimeout(ctx, defaultTestShortTimeout)
defer sCancel()
if n, err := apiClient.addWatches[ClusterResource].Receive(sCtx); err != context.DeadlineExceeded {
if n, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(sCtx); err != context.DeadlineExceeded {
t.Fatalf("want no new watch to start (recv timeout), got resource name: %v error %v", n, err)
}
@ -311,7 +311,7 @@ func (s) TestClusterWatchExpiryTimer(t *testing.T) {
client.WatchCluster(testCDSName, func(u xdsresource.ClusterUpdate, err error) {
clusterUpdateCh.Send(xdsresource.ClusterUpdateErrTuple{Update: u, Err: err})
})
if _, err := apiClient.addWatches[ClusterResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -350,7 +350,7 @@ func (s) TestClusterWatchExpiryTimerStop(t *testing.T) {
client.WatchCluster(testCDSName, func(u xdsresource.ClusterUpdate, err error) {
clusterUpdateCh.Send(xdsresource.ClusterUpdateErrTuple{Update: u, Err: err})
})
if _, err := apiClient.addWatches[ClusterResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -398,7 +398,7 @@ func (s) TestClusterResourceRemoved(t *testing.T) {
client.WatchCluster(testCDSName+"1", func(update xdsresource.ClusterUpdate, err error) {
clusterUpdateCh1.Send(xdsresource.ClusterUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[ClusterResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -407,7 +407,7 @@ func (s) TestClusterResourceRemoved(t *testing.T) {
client.WatchCluster(testCDSName+"2", func(update xdsresource.ClusterUpdate, err error) {
clusterUpdateCh2.Send(xdsresource.ClusterUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[ClusterResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -428,7 +428,7 @@ func (s) TestClusterResourceRemoved(t *testing.T) {
client.NewClusters(map[string]xdsresource.ClusterUpdateErrTuple{testCDSName + "2": {Update: wantUpdate2}}, xdsresource.UpdateMetadata{})
// Watcher 1 should get an error.
if u, err := clusterUpdateCh1.Receive(ctx); err != nil || ErrType(u.(xdsresource.ClusterUpdateErrTuple).Err) != ErrorTypeResourceNotFound {
if u, err := clusterUpdateCh1.Receive(ctx); err != nil || xdsresource.ErrType(u.(xdsresource.ClusterUpdateErrTuple).Err) != xdsresource.ErrorTypeResourceNotFound {
t.Errorf("unexpected clusterUpdate: %v, error receiving from channel: %v, want update with error resource not found", u, err)
}
@ -482,7 +482,7 @@ func (s) TestClusterWatchNACKError(t *testing.T) {
clusterUpdateCh.Send(xdsresource.ClusterUpdateErrTuple{Update: update, Err: err})
})
defer cancelWatch()
if _, err := apiClient.addWatches[ClusterResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -527,11 +527,11 @@ func (s) TestClusterWatchPartialValid(t *testing.T) {
})
defer func() {
cancelWatch()
if _, err := apiClient.removeWatches[ClusterResource].Receive(ctx); err != nil {
if _, err := apiClient.removeWatches[xdsresource.ClusterResource].Receive(ctx); err != nil {
t.Fatalf("want watch to be canceled, got err: %v", err)
}
}()
if _, err := apiClient.addWatches[ClusterResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ClusterResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
updateChs[name] = clusterUpdateCh

View File

@ -74,7 +74,7 @@ func (s) TestEndpointsWatch(t *testing.T) {
cancelWatch := client.WatchEndpoints(testCDSName, func(update xdsresource.EndpointsUpdate, err error) {
endpointsUpdateCh.Send(xdsresource.EndpointsUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[EndpointsResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.EndpointsResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -142,7 +142,7 @@ func (s) TestEndpointsTwoWatchSameResourceName(t *testing.T) {
if i == 0 {
// A new watch is registered on the underlying API client only for
// the first iteration because we are using the same resource name.
if _, err := apiClient.addWatches[EndpointsResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.EndpointsResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
}
@ -214,7 +214,7 @@ func (s) TestEndpointsThreeWatchDifferentResourceName(t *testing.T) {
if i == 0 {
// A new watch is registered on the underlying API client only for
// the first iteration because we are using the same resource name.
if _, err := apiClient.addWatches[EndpointsResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.EndpointsResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
}
@ -225,7 +225,7 @@ func (s) TestEndpointsThreeWatchDifferentResourceName(t *testing.T) {
client.WatchEndpoints(testCDSName+"2", func(update xdsresource.EndpointsUpdate, err error) {
endpointsUpdateCh2.Send(xdsresource.EndpointsUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[EndpointsResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.EndpointsResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -270,7 +270,7 @@ func (s) TestEndpointsWatchAfterCache(t *testing.T) {
client.WatchEndpoints(testCDSName, func(update xdsresource.EndpointsUpdate, err error) {
endpointsUpdateCh.Send(xdsresource.EndpointsUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[EndpointsResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.EndpointsResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -287,7 +287,7 @@ func (s) TestEndpointsWatchAfterCache(t *testing.T) {
})
sCtx, sCancel := context.WithTimeout(ctx, defaultTestShortTimeout)
defer sCancel()
if n, err := apiClient.addWatches[EndpointsResource].Receive(sCtx); err != context.DeadlineExceeded {
if n, err := apiClient.addWatches[xdsresource.EndpointsResource].Receive(sCtx); err != context.DeadlineExceeded {
t.Fatalf("want no new watch to start (recv timeout), got resource name: %v error %v", n, err)
}
@ -329,7 +329,7 @@ func (s) TestEndpointsWatchExpiryTimer(t *testing.T) {
client.WatchEndpoints(testCDSName, func(update xdsresource.EndpointsUpdate, err error) {
endpointsUpdateCh.Send(xdsresource.EndpointsUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[EndpointsResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.EndpointsResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -368,7 +368,7 @@ func (s) TestEndpointsWatchNACKError(t *testing.T) {
endpointsUpdateCh.Send(xdsresource.EndpointsUpdateErrTuple{Update: update, Err: err})
})
defer cancelWatch()
if _, err := apiClient.addWatches[EndpointsResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.EndpointsResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -411,11 +411,11 @@ func (s) TestEndpointsWatchPartialValid(t *testing.T) {
})
defer func() {
cancelWatch()
if _, err := apiClient.removeWatches[EndpointsResource].Receive(ctx); err != nil {
if _, err := apiClient.removeWatches[xdsresource.EndpointsResource].Receive(ctx); err != nil {
t.Fatalf("want watch to be canceled, got err: %v", err)
}
}()
if _, err := apiClient.addWatches[EndpointsResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.EndpointsResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
updateChs[name] = endpointsUpdateCh

View File

@ -56,7 +56,7 @@ func (s) TestLDSWatch(t *testing.T) {
cancelWatch := client.WatchListener(testLDSName, func(update xdsresource.ListenerUpdate, err error) {
ldsUpdateCh.Send(xdsresource.ListenerUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[ListenerResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ListenerResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -124,7 +124,7 @@ func (s) TestLDSTwoWatchSameResourceName(t *testing.T) {
if i == 0 {
// A new watch is registered on the underlying API client only for
// the first iteration because we are using the same resource name.
if _, err := apiClient.addWatches[ListenerResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ListenerResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
}
@ -197,7 +197,7 @@ func (s) TestLDSThreeWatchDifferentResourceName(t *testing.T) {
if i == 0 {
// A new watch is registered on the underlying API client only for
// the first iteration because we are using the same resource name.
if _, err := apiClient.addWatches[ListenerResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ListenerResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
}
@ -208,7 +208,7 @@ func (s) TestLDSThreeWatchDifferentResourceName(t *testing.T) {
client.WatchListener(testLDSName+"2", func(update xdsresource.ListenerUpdate, err error) {
ldsUpdateCh2.Send(xdsresource.ListenerUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[ListenerResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ListenerResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -253,7 +253,7 @@ func (s) TestLDSWatchAfterCache(t *testing.T) {
client.WatchListener(testLDSName, func(update xdsresource.ListenerUpdate, err error) {
ldsUpdateCh.Send(xdsresource.ListenerUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[ListenerResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ListenerResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -270,7 +270,7 @@ func (s) TestLDSWatchAfterCache(t *testing.T) {
})
sCtx, sCancel := context.WithTimeout(ctx, defaultTestShortTimeout)
defer sCancel()
if n, err := apiClient.addWatches[ListenerResource].Receive(sCtx); err != context.DeadlineExceeded {
if n, err := apiClient.addWatches[xdsresource.ListenerResource].Receive(sCtx); err != context.DeadlineExceeded {
t.Fatalf("want no new watch to start (recv timeout), got resource name: %v error %v", n, err)
}
@ -315,7 +315,7 @@ func (s) TestLDSResourceRemoved(t *testing.T) {
client.WatchListener(testLDSName+"1", func(update xdsresource.ListenerUpdate, err error) {
ldsUpdateCh1.Send(xdsresource.ListenerUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[ListenerResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ListenerResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
// Another watch for a different name.
@ -323,7 +323,7 @@ func (s) TestLDSResourceRemoved(t *testing.T) {
client.WatchListener(testLDSName+"2", func(update xdsresource.ListenerUpdate, err error) {
ldsUpdateCh2.Send(xdsresource.ListenerUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[ListenerResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ListenerResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -344,7 +344,7 @@ func (s) TestLDSResourceRemoved(t *testing.T) {
client.NewListeners(map[string]xdsresource.ListenerUpdateErrTuple{testLDSName + "2": {Update: wantUpdate2}}, xdsresource.UpdateMetadata{})
// Watcher 1 should get an error.
if u, err := ldsUpdateCh1.Receive(ctx); err != nil || ErrType(u.(xdsresource.ListenerUpdateErrTuple).Err) != ErrorTypeResourceNotFound {
if u, err := ldsUpdateCh1.Receive(ctx); err != nil || xdsresource.ErrType(u.(xdsresource.ListenerUpdateErrTuple).Err) != xdsresource.ErrorTypeResourceNotFound {
t.Errorf("unexpected ListenerUpdate: %v, error receiving from channel: %v, want update with error resource not found", u, err)
}
@ -398,7 +398,7 @@ func (s) TestListenerWatchNACKError(t *testing.T) {
ldsUpdateCh.Send(xdsresource.ListenerUpdateErrTuple{Update: update, Err: err})
})
defer cancelWatch()
if _, err := apiClient.addWatches[ListenerResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ListenerResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -441,11 +441,11 @@ func (s) TestListenerWatchPartialValid(t *testing.T) {
})
defer func() {
cancelWatch()
if _, err := apiClient.removeWatches[ListenerResource].Receive(ctx); err != nil {
if _, err := apiClient.removeWatches[xdsresource.ListenerResource].Receive(ctx); err != nil {
t.Fatalf("want watch to be canceled, got err: %v", err)
}
}()
if _, err := apiClient.addWatches[ListenerResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ListenerResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
updateChs[name] = ldsUpdateCh
@ -493,7 +493,7 @@ func (s) TestListenerWatch_RedundantUpdateSupression(t *testing.T) {
client.WatchListener(testLDSName, func(update xdsresource.ListenerUpdate, err error) {
ldsUpdateCh.Send(xdsresource.ListenerUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[ListenerResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.ListenerResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}

View File

@ -56,7 +56,7 @@ func (s) TestRDSWatch(t *testing.T) {
cancelWatch := client.WatchRouteConfig(testRDSName, func(update xdsresource.RouteConfigUpdate, err error) {
rdsUpdateCh.Send(xdsresource.RouteConfigUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[RouteConfigResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.RouteConfigResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -131,7 +131,7 @@ func (s) TestRDSTwoWatchSameResourceName(t *testing.T) {
if i == 0 {
// A new watch is registered on the underlying API client only for
// the first iteration because we are using the same resource name.
if _, err := apiClient.addWatches[RouteConfigResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.RouteConfigResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
}
@ -211,7 +211,7 @@ func (s) TestRDSThreeWatchDifferentResourceName(t *testing.T) {
if i == 0 {
// A new watch is registered on the underlying API client only for
// the first iteration because we are using the same resource name.
if _, err := apiClient.addWatches[RouteConfigResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.RouteConfigResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
}
@ -222,7 +222,7 @@ func (s) TestRDSThreeWatchDifferentResourceName(t *testing.T) {
client.WatchRouteConfig(testRDSName+"2", func(update xdsresource.RouteConfigUpdate, err error) {
rdsUpdateCh2.Send(xdsresource.RouteConfigUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[RouteConfigResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.RouteConfigResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -281,7 +281,7 @@ func (s) TestRDSWatchAfterCache(t *testing.T) {
client.WatchRouteConfig(testRDSName, func(update xdsresource.RouteConfigUpdate, err error) {
rdsUpdateCh.Send(xdsresource.RouteConfigUpdateErrTuple{Update: update, Err: err})
})
if _, err := apiClient.addWatches[RouteConfigResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.RouteConfigResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -305,7 +305,7 @@ func (s) TestRDSWatchAfterCache(t *testing.T) {
})
sCtx, sCancel := context.WithTimeout(ctx, defaultTestShortTimeout)
defer sCancel()
if n, err := apiClient.addWatches[RouteConfigResource].Receive(sCtx); err != context.DeadlineExceeded {
if n, err := apiClient.addWatches[xdsresource.RouteConfigResource].Receive(sCtx); err != context.DeadlineExceeded {
t.Fatalf("want no new watch to start (recv timeout), got resource name: %v error %v", n, err)
}
@ -347,7 +347,7 @@ func (s) TestRouteWatchNACKError(t *testing.T) {
rdsUpdateCh.Send(xdsresource.RouteConfigUpdateErrTuple{Update: update, Err: err})
})
defer cancelWatch()
if _, err := apiClient.addWatches[RouteConfigResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.RouteConfigResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
@ -390,11 +390,11 @@ func (s) TestRouteWatchPartialValid(t *testing.T) {
})
defer func() {
cancelWatch()
if _, err := apiClient.removeWatches[RouteConfigResource].Receive(ctx); err != nil {
if _, err := apiClient.removeWatches[xdsresource.RouteConfigResource].Receive(ctx); err != nil {
t.Fatalf("want watch to be canceled, got err: %v", err)
}
}()
if _, err := apiClient.addWatches[RouteConfigResource].Receive(ctx); err != nil {
if _, err := apiClient.addWatches[xdsresource.RouteConfigResource].Receive(ctx); err != nil {
t.Fatalf("want new watch to start, got error %v", err)
}
updateChs[name] = rdsUpdateCh

View File

@ -16,7 +16,7 @@
*
*/
package xdsclient
package xdsresource
import "fmt"

View File

@ -21,6 +21,7 @@ import (
"time"
"google.golang.org/grpc/xds/internal/version"
"google.golang.org/protobuf/types/known/anypb"
)
// UpdateValidatorFunc performs validations on update structs using
@ -105,3 +106,45 @@ type UpdateErrorMetadata struct {
// Timestamp is when the NACKed response was received.
Timestamp time.Time
}
// UpdateWithMD contains the raw message of the update and the metadata,
// including version, raw message, timestamp.
//
// This is to be used for config dump and CSDS, not directly by users (like
// resolvers/balancers).
type UpdateWithMD struct {
MD UpdateMetadata
Raw *anypb.Any
}
// ResourceType identifies resources in a transport protocol agnostic way. These
// will be used in transport version agnostic code, while the versioned API
// clients will map these to appropriate version URLs.
type ResourceType int
// Version agnostic resource type constants.
const (
UnknownResource ResourceType = iota
ListenerResource
HTTPConnManagerResource
RouteConfigResource
ClusterResource
EndpointsResource
)
func (r ResourceType) String() string {
switch r {
case ListenerResource:
return "ListenerResource"
case HTTPConnManagerResource:
return "HTTPConnManagerResource"
case RouteConfigResource:
return "RouteConfigResource"
case ClusterResource:
return "ClusterResource"
case EndpointsResource:
return "EndpointsResource"
default:
return "UnknownResource"
}
}

View File

@ -430,7 +430,7 @@ func (s) TestServeSuccess(t *testing.T) {
// Push an error to the registered listener watch callback and make sure
// that Serve does not return.
client.InvokeWatchListenerCallback(xdsresource.ListenerUpdate{}, xdsclient.NewErrorf(xdsclient.ErrorTypeResourceNotFound, "LDS resource not found"))
client.InvokeWatchListenerCallback(xdsresource.ListenerUpdate{}, xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "LDS resource not found"))
sCtx, sCancel := context.WithTimeout(context.Background(), defaultTestShortTimeout)
defer sCancel()
if _, err := serveDone.Receive(sCtx); err != context.DeadlineExceeded {