mirror of https://github.com/grpc/grpc-go.git
xds: make comparison of server configs in bootstrap more reliable (#6112)
This commit is contained in:
parent
7507ea6bcb
commit
52ca957106
|
@ -31,19 +31,19 @@ import (
|
|||
"time"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/google"
|
||||
"google.golang.org/grpc/grpclog"
|
||||
"google.golang.org/grpc/internal/envconfig"
|
||||
"google.golang.org/grpc/internal/googlecloud"
|
||||
internalgrpclog "google.golang.org/grpc/internal/grpclog"
|
||||
"google.golang.org/grpc/internal/grpcrand"
|
||||
"google.golang.org/grpc/resolver"
|
||||
_ "google.golang.org/grpc/xds" // To register xds resolvers and balancers.
|
||||
"google.golang.org/grpc/xds/internal/xdsclient"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
"google.golang.org/protobuf/types/known/structpb"
|
||||
|
||||
v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
|
||||
|
||||
_ "google.golang.org/grpc/xds" // To register xds resolvers and balancers.
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -116,9 +116,14 @@ func (c2pResolverBuilder) Build(t resolver.Target, cc resolver.ClientConn, opts
|
|||
if balancerName == "" {
|
||||
balancerName = tdURL
|
||||
}
|
||||
serverConfig := &bootstrap.ServerConfig{
|
||||
ServerURI: balancerName,
|
||||
Creds: grpc.WithCredentialsBundle(google.NewDefaultCredentials()),
|
||||
serverConfig, err := bootstrap.ServerConfigFromJSON([]byte(fmt.Sprintf(`
|
||||
{
|
||||
"server_uri": "%s",
|
||||
"channel_creds": [{"type": "google_default"}],
|
||||
"server_features": ["xds_v3"]
|
||||
}`, balancerName)))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to build bootstrap configuration: %v", err)
|
||||
}
|
||||
config := &bootstrap.Config{
|
||||
XDSServer: serverConfig,
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package googledirectpath
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strconv"
|
||||
"strings"
|
||||
"testing"
|
||||
|
@ -212,15 +213,20 @@ func TestBuildXDS(t *testing.T) {
|
|||
},
|
||||
}
|
||||
}
|
||||
serverConfig := &bootstrap.ServerConfig{
|
||||
ServerURI: tdURL,
|
||||
wantServerConfig, err := bootstrap.ServerConfigFromJSON([]byte(fmt.Sprintf(`{
|
||||
"server_uri": "%s",
|
||||
"channel_creds": [{"type": "google_default"}],
|
||||
"server_features": ["xds_v3"]
|
||||
}`, tdURL)))
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to build server bootstrap config: %v", err)
|
||||
}
|
||||
wantConfig := &bootstrap.Config{
|
||||
XDSServer: serverConfig,
|
||||
XDSServer: wantServerConfig,
|
||||
ClientDefaultListenerResourceNameTemplate: "%s",
|
||||
Authorities: map[string]*bootstrap.Authority{
|
||||
"traffic-director-c2p.xds.googleapis.com": {
|
||||
XDSServer: serverConfig,
|
||||
XDSServer: wantServerConfig,
|
||||
},
|
||||
},
|
||||
NodeProto: wantNode,
|
||||
|
@ -234,9 +240,9 @@ func TestBuildXDS(t *testing.T) {
|
|||
protocmp.Transform(),
|
||||
}
|
||||
select {
|
||||
case c := <-configCh:
|
||||
if diff := cmp.Diff(c, wantConfig, cmpOpts); diff != "" {
|
||||
t.Fatalf("%v", diff)
|
||||
case gotConfig := <-configCh:
|
||||
if diff := cmp.Diff(wantConfig, gotConfig, cmpOpts); diff != "" {
|
||||
t.Fatalf("Unexpected diff in bootstrap config (-want +got):\n%s", diff)
|
||||
}
|
||||
case <-time.After(time.Second):
|
||||
t.Fatalf("timeout waiting for client config")
|
||||
|
|
|
@ -53,7 +53,9 @@ const (
|
|||
var (
|
||||
defaultTestAuthorityServerConfig = &bootstrap.ServerConfig{
|
||||
ServerURI: "self_server",
|
||||
CredsType: "self_creds",
|
||||
Creds: bootstrap.ChannelCreds{
|
||||
Type: "insecure",
|
||||
},
|
||||
}
|
||||
noopODLBCfg = outlierdetection.LBConfig{
|
||||
Interval: 1<<63 - 1,
|
||||
|
|
|
@ -59,7 +59,9 @@ var (
|
|||
}
|
||||
testLRSServerConfig = &bootstrap.ServerConfig{
|
||||
ServerURI: "trafficdirector.googleapis.com:443",
|
||||
CredsType: "google_default",
|
||||
Creds: bootstrap.ChannelCreds{
|
||||
Type: "google_default",
|
||||
},
|
||||
}
|
||||
|
||||
cmpOpts = cmp.Options{
|
||||
|
@ -720,7 +722,9 @@ func (s) TestUpdateLRSServer(t *testing.T) {
|
|||
|
||||
testLRSServerConfig2 := &bootstrap.ServerConfig{
|
||||
ServerURI: "trafficdirector-another.googleapis.com:443",
|
||||
CredsType: "google_default",
|
||||
Creds: bootstrap.ChannelCreds{
|
||||
Type: "google_default",
|
||||
},
|
||||
}
|
||||
// Update LRS server to a different name.
|
||||
if err := b.UpdateClientConnState(balancer.ClientConnState{
|
||||
|
|
|
@ -186,7 +186,7 @@ func (b *clusterImplBalancer) updateLoadStore(newConfig *LBConfig) error {
|
|||
} else {
|
||||
// Old is not nil, new is not nil, compare string values, if
|
||||
// different, stop old and start new.
|
||||
if *b.lrsServer != *newConfig.LoadReportingServer {
|
||||
if !b.lrsServer.Equal(newConfig.LoadReportingServer) {
|
||||
b.lrsServer = newConfig.LoadReportingServer
|
||||
stopOldLoadReport = true
|
||||
startNewLoadReport = true
|
||||
|
|
|
@ -0,0 +1,166 @@
|
|||
/*
|
||||
*
|
||||
* Copyright 2023 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 clusterimpl_test
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net"
|
||||
"strconv"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/grpctest"
|
||||
"google.golang.org/grpc/internal/stubserver"
|
||||
"google.golang.org/grpc/internal/testutils/xds/e2e"
|
||||
"google.golang.org/grpc/status"
|
||||
|
||||
v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
|
||||
v3endpointpb "github.com/envoyproxy/go-control-plane/envoy/config/endpoint/v3"
|
||||
testgrpc "google.golang.org/grpc/test/grpc_testing"
|
||||
testpb "google.golang.org/grpc/test/grpc_testing"
|
||||
|
||||
_ "google.golang.org/grpc/xds"
|
||||
)
|
||||
|
||||
const (
|
||||
defaultTestTimeout = 5 * time.Second
|
||||
defaultTestShortTimeout = 100 * time.Millisecond
|
||||
)
|
||||
|
||||
type s struct {
|
||||
grpctest.Tester
|
||||
}
|
||||
|
||||
func Test(t *testing.T) {
|
||||
grpctest.RunSubTests(t, s{})
|
||||
}
|
||||
|
||||
// TestConfigUpdateWithSameLoadReportingServerConfig tests the scenario where
|
||||
// the clusterimpl LB policy receives a config update with no change in the load
|
||||
// reporting server configuration. The test verifies that the existing load
|
||||
// repoting stream is not terminated and that a new load reporting stream is not
|
||||
// created.
|
||||
func (s) TestConfigUpdateWithSameLoadReportingServerConfig(t *testing.T) {
|
||||
// Create an xDS management server that serves ADS and LRS requests.
|
||||
opts := e2e.ManagementServerOptions{SupportLoadReportingService: true}
|
||||
mgmtServer, nodeID, _, resolver, mgmtServerCleanup := e2e.SetupManagementServer(t, opts)
|
||||
defer mgmtServerCleanup()
|
||||
|
||||
// Start a server backend exposing the test service.
|
||||
backend := &stubserver.StubServer{
|
||||
EmptyCallF: func(context.Context, *testpb.Empty) (*testpb.Empty, error) { return &testpb.Empty{}, nil },
|
||||
}
|
||||
backend.StartServer()
|
||||
defer backend.Stop()
|
||||
|
||||
// Extract the host and port where the server backend is running.
|
||||
_, p, err := net.SplitHostPort(backend.Address)
|
||||
if err != nil {
|
||||
t.Fatalf("Invalid serving address for server backend: %v", err)
|
||||
}
|
||||
port, err := strconv.ParseUint(p, 10, 32)
|
||||
if err != nil {
|
||||
t.Fatalf("Invalid serving port for server backend: %v", err)
|
||||
}
|
||||
t.Logf("Started server backend at %q", backend.Address)
|
||||
|
||||
// Configure the xDS management server with default resources. Override the
|
||||
// default cluster to include an LRS server config pointing to self.
|
||||
const serviceName = "my-test-xds-service"
|
||||
resources := e2e.DefaultClientResources(e2e.ResourceParams{
|
||||
DialTarget: serviceName,
|
||||
NodeID: nodeID,
|
||||
Host: "localhost",
|
||||
Port: uint32(port),
|
||||
SecLevel: e2e.SecurityLevelNone,
|
||||
})
|
||||
resources.Clusters[0].LrsServer = &v3corepb.ConfigSource{
|
||||
ConfigSourceSpecifier: &v3corepb.ConfigSource_Self{
|
||||
Self: &v3corepb.SelfConfigSource{},
|
||||
},
|
||||
}
|
||||
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
|
||||
defer cancel()
|
||||
if err := mgmtServer.Update(ctx, resources); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// Create a ClientConn and make a successful RPC.
|
||||
cc, err := grpc.Dial(fmt.Sprintf("xds:///%s", serviceName), grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(resolver))
|
||||
if err != nil {
|
||||
t.Fatalf("failed to dial local test server: %v", err)
|
||||
}
|
||||
defer cc.Close()
|
||||
|
||||
client := testgrpc.NewTestServiceClient(cc)
|
||||
if _, err := client.EmptyCall(ctx, &testpb.Empty{}); err != nil {
|
||||
t.Fatalf("rpc EmptyCall() failed: %v", err)
|
||||
}
|
||||
|
||||
// Ensure that an LRS stream is created.
|
||||
if _, err := mgmtServer.LRSServer.LRSStreamOpenChan.Receive(ctx); err != nil {
|
||||
t.Fatalf("Failure when waiting for an LRS stream to be opened: %v", err)
|
||||
}
|
||||
|
||||
// Configure a new resource on the management server with drop config that
|
||||
// drops all RPCs, but with no change in the load reporting server config.
|
||||
resources.Endpoints = []*v3endpointpb.ClusterLoadAssignment{
|
||||
e2e.EndpointResourceWithOptions(e2e.EndpointOptions{
|
||||
ClusterName: "endpoints-" + serviceName,
|
||||
Host: "localhost",
|
||||
Ports: []uint32{uint32(port)},
|
||||
DropPercents: map[string]int{"test-drop-everything": 100},
|
||||
}),
|
||||
}
|
||||
if err := mgmtServer.Update(ctx, resources); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// Repeatedly send RPCs until we sees that they are getting dropped, or the
|
||||
// test context deadline expires. The former indicates that new config with
|
||||
// drops has been applied.
|
||||
for ; ctx.Err() == nil; <-time.After(defaultTestShortTimeout) {
|
||||
_, err := client.EmptyCall(ctx, &testpb.Empty{})
|
||||
if err != nil && status.Code(err) == codes.Unavailable && strings.Contains(err.Error(), "RPC is dropped") {
|
||||
break
|
||||
}
|
||||
}
|
||||
if ctx.Err() != nil {
|
||||
t.Fatalf("Timeout when waiting for RPCs to be dropped after config update")
|
||||
}
|
||||
|
||||
// Ensure that the old LRS stream is not closed.
|
||||
sCtx, sCancel := context.WithTimeout(ctx, defaultTestShortTimeout)
|
||||
defer sCancel()
|
||||
if _, err := mgmtServer.LRSServer.LRSStreamCloseChan.Receive(sCtx); err == nil {
|
||||
t.Fatal("LRS stream closed when expected not to")
|
||||
}
|
||||
|
||||
// Also ensure that a new LRS stream is not created.
|
||||
sCtx, sCancel = context.WithTimeout(ctx, defaultTestShortTimeout)
|
||||
defer sCancel()
|
||||
if _, err := mgmtServer.LRSServer.LRSStreamOpenChan.Receive(sCtx); err == nil {
|
||||
t.Fatal("New LRS stream created when expected not to")
|
||||
}
|
||||
}
|
|
@ -161,7 +161,9 @@ const (
|
|||
|
||||
var testLRSServerConfig = &bootstrap.ServerConfig{
|
||||
ServerURI: "trafficdirector.googleapis.com:443",
|
||||
CredsType: "google_default",
|
||||
Creds: bootstrap.ChannelCreds{
|
||||
Type: "google_default",
|
||||
},
|
||||
}
|
||||
|
||||
func TestParseConfig(t *testing.T) {
|
||||
|
|
|
@ -31,7 +31,6 @@ import (
|
|||
"github.com/envoyproxy/go-control-plane/pkg/wellknown"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/uuid"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/credentials"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
|
@ -53,6 +52,7 @@ import (
|
|||
"google.golang.org/grpc/xds/internal/balancer/ringhash"
|
||||
"google.golang.org/grpc/xds/internal/httpfilter"
|
||||
"google.golang.org/grpc/xds/internal/httpfilter/router"
|
||||
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/testutils/fakeclient"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
|
@ -213,10 +213,7 @@ func (s) TestResolverBuilder_DifferentBootstrapConfigs(t *testing.T) {
|
|||
|
||||
// Add top-level xDS server config corresponding to the above
|
||||
// management server.
|
||||
test.bootstrapCfg.XDSServer = &bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
}
|
||||
test.bootstrapCfg.XDSServer = xdstestutils.ServerConfigForAddress(t, mgmtServer.Address)
|
||||
|
||||
// Override xDS client creation to use bootstrap configuration
|
||||
// specified by the test.
|
||||
|
@ -538,10 +535,7 @@ func (s) TestResolverWatchCallbackAfterClose(t *testing.T) {
|
|||
// closes the xDS client.
|
||||
func (s) TestResolverCloseClosesXDSClient(t *testing.T) {
|
||||
bootstrapCfg := &bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: "dummy-management-server-address",
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, "dummy-management-server-address"),
|
||||
}
|
||||
|
||||
// Override xDS client creation to use bootstrap configuration pointing to a
|
||||
|
|
|
@ -19,6 +19,10 @@
|
|||
package testutils
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
|
||||
)
|
||||
|
@ -45,3 +49,20 @@ func BuildResourceName(typ xdsresource.ResourceType, auth, id string, ctxParams
|
|||
ContextParams: ctxParams,
|
||||
}).String()
|
||||
}
|
||||
|
||||
// ServerConfigForAddress returns a bootstrap.ServerConfig for the given address
|
||||
// with default values of insecure channel_creds and v3 server_features.
|
||||
func ServerConfigForAddress(t *testing.T, addr string) *bootstrap.ServerConfig {
|
||||
t.Helper()
|
||||
|
||||
jsonCfg := fmt.Sprintf(`{
|
||||
"server_uri": "%s",
|
||||
"channel_creds": [{"type": "insecure"}],
|
||||
"server_features": ["xds_v3"]
|
||||
}`, addr)
|
||||
sc, err := bootstrap.ServerConfigFromJSON([]byte(jsonCfg))
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to create server config from JSON %s: %v", jsonCfg, err)
|
||||
}
|
||||
return sc
|
||||
}
|
||||
|
|
|
@ -24,19 +24,20 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
|
||||
v3listenerpb "github.com/envoyproxy/go-control-plane/envoy/config/listener/v3"
|
||||
v3discoverypb "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
|
||||
"github.com/google/uuid"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/testutils/xds/e2e"
|
||||
"google.golang.org/grpc/xds/internal"
|
||||
_ "google.golang.org/grpc/xds/internal/httpfilter/router"
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
|
||||
|
||||
v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
|
||||
v3listenerpb "github.com/envoyproxy/go-control-plane/envoy/config/listener/v3"
|
||||
v3discoverypb "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
|
||||
|
||||
_ "google.golang.org/grpc/xds/internal/httpfilter/router" // Register the router filter.
|
||||
)
|
||||
|
||||
var emptyServerOpts = e2e.ManagementServerOptions{}
|
||||
|
@ -64,11 +65,7 @@ func setupTest(ctx context.Context, t *testing.T, opts e2e.ManagementServerOptio
|
|||
}
|
||||
|
||||
a, err := newAuthority(authorityArgs{
|
||||
serverCfg: &bootstrap.ServerConfig{
|
||||
ServerURI: ms.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
},
|
||||
serverCfg: xdstestutils.ServerConfigForAddress(t, ms.Address),
|
||||
bootstrapCfg: &bootstrap.Config{
|
||||
NodeProto: &v3corepb.Node{Id: nodeID},
|
||||
},
|
||||
|
|
|
@ -84,19 +84,66 @@ func (d *googleDefaultCredsBuilder) Name() string {
|
|||
return "google_default"
|
||||
}
|
||||
|
||||
// ChannelCreds contains the credentials to be used while communicating with an
|
||||
// xDS server. It is also used to dedup servers with the same server URI.
|
||||
type ChannelCreds struct {
|
||||
// Type contains a unique name identifying the credentials type. The only
|
||||
// supported types currently are "google_default" and "insecure".
|
||||
Type string
|
||||
// Config contains the JSON configuration associated with the credentials.
|
||||
Config json.RawMessage
|
||||
}
|
||||
|
||||
// Equal reports whether cc and other are considered equal.
|
||||
func (cc ChannelCreds) Equal(other ChannelCreds) bool {
|
||||
return cc.Type == other.Type && bytes.Equal(cc.Config, other.Config)
|
||||
}
|
||||
|
||||
// String returns a string representation of the credentials. It contains the
|
||||
// type and the config (if non-nil) separated by a "-".
|
||||
func (cc ChannelCreds) String() string {
|
||||
if cc.Config == nil {
|
||||
return cc.Type
|
||||
}
|
||||
|
||||
// We do not expect the Marshal call to fail since we wrote to cc.Config
|
||||
// after a successful unmarshaling from JSON configuration. Therefore,
|
||||
// it is safe to ignore the error here.
|
||||
b, _ := json.Marshal(cc.Config)
|
||||
return cc.Type + "-" + string(b)
|
||||
}
|
||||
|
||||
// ServerConfig contains the configuration to connect to a server, including
|
||||
// URI, creds, and transport API version (e.g. v2 or v3).
|
||||
//
|
||||
// It contains unexported fields that are initialized when unmarshaled from JSON
|
||||
// using either the UnmarshalJSON() method or the ServerConfigFromJSON()
|
||||
// function. Hence users are strongly encouraged not to use a literal struct
|
||||
// initialization to create an instance of this type, but instead unmarshal from
|
||||
// JSON using one of the two available options.
|
||||
type ServerConfig struct {
|
||||
// ServerURI is the management server to connect to.
|
||||
//
|
||||
// The bootstrap file contains an ordered list of xDS servers to contact for
|
||||
// this authority. The first one is picked.
|
||||
ServerURI string
|
||||
// Creds contains the credentials to be used while talking to the xDS
|
||||
// server, as a grpc.DialOption.
|
||||
Creds grpc.DialOption
|
||||
// CredsType is the type of the creds. It will be used to dedup servers.
|
||||
CredsType string
|
||||
// Creds contains the credentials to be used while communicationg with this
|
||||
// xDS server. It is also used to dedup servers with the same server URI.
|
||||
Creds ChannelCreds
|
||||
// ServerFeatures contains a list of features supported by this xDS server.
|
||||
// It is also used to dedup servers with the same server URI and creds.
|
||||
ServerFeatures []string
|
||||
|
||||
// As part of unmarshaling the JSON config into this struct, we ensure that
|
||||
// the credentials config is valid by building an instance of the specified
|
||||
// credentials and store it here as a grpc.DialOption for easy access when
|
||||
// dialing this xDS server.
|
||||
credsDialOption grpc.DialOption
|
||||
}
|
||||
|
||||
// CredsDialOption returns the configured credentials as a grpc dial option.
|
||||
func (sc *ServerConfig) CredsDialOption() grpc.DialOption {
|
||||
return sc.credsDialOption
|
||||
}
|
||||
|
||||
// String returns the string representation of the ServerConfig.
|
||||
|
@ -109,17 +156,17 @@ type ServerConfig struct {
|
|||
// content. It doesn't cover NodeProto because NodeProto isn't used by
|
||||
// federation.
|
||||
func (sc *ServerConfig) String() string {
|
||||
var ver = "xDSv3"
|
||||
return strings.Join([]string{sc.ServerURI, sc.CredsType, ver}, "-")
|
||||
features := strings.Join(sc.ServerFeatures, "-")
|
||||
return strings.Join([]string{sc.ServerURI, sc.Creds.String(), features}, "-")
|
||||
}
|
||||
|
||||
// MarshalJSON marshals the ServerConfig to json.
|
||||
func (sc ServerConfig) MarshalJSON() ([]byte, error) {
|
||||
server := xdsServer{
|
||||
ServerURI: sc.ServerURI,
|
||||
ChannelCreds: []channelCreds{{Type: sc.CredsType, Config: nil}},
|
||||
ServerURI: sc.ServerURI,
|
||||
ChannelCreds: []channelCreds{{Type: sc.Creds.Type, Config: sc.Creds.Config}},
|
||||
ServerFeatures: sc.ServerFeatures,
|
||||
}
|
||||
server.ServerFeatures = []string{serverFeaturesV3}
|
||||
return json.Marshal(server)
|
||||
}
|
||||
|
||||
|
@ -129,10 +176,11 @@ func (sc *ServerConfig) UnmarshalJSON(data []byte) error {
|
|||
if err := json.Unmarshal(data, &server); err != nil {
|
||||
return fmt.Errorf("xds: json.Unmarshal(data) for field ServerConfig failed during bootstrap: %v", err)
|
||||
}
|
||||
|
||||
sc.ServerURI = server.ServerURI
|
||||
sc.ServerFeatures = server.ServerFeatures
|
||||
for _, cc := range server.ChannelCreds {
|
||||
// We stop at the first credential type that we support.
|
||||
sc.CredsType = cc.Type
|
||||
c := bootstrap.GetCredentials(cc.Type)
|
||||
if c == nil {
|
||||
continue
|
||||
|
@ -141,12 +189,56 @@ func (sc *ServerConfig) UnmarshalJSON(data []byte) error {
|
|||
if err != nil {
|
||||
return fmt.Errorf("failed to build credentials bundle from bootstrap for %q: %v", cc.Type, err)
|
||||
}
|
||||
sc.Creds = grpc.WithCredentialsBundle(bundle)
|
||||
sc.Creds = ChannelCreds{
|
||||
Type: cc.Type,
|
||||
Config: cc.Config,
|
||||
}
|
||||
sc.credsDialOption = grpc.WithCredentialsBundle(bundle)
|
||||
break
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ServerConfigFromJSON creates a new ServerConfig from the given JSON
|
||||
// configuration. This is the preferred way of creating a ServerConfig when
|
||||
// hand-crafting the JSON configuration.
|
||||
func ServerConfigFromJSON(data []byte) (*ServerConfig, error) {
|
||||
sc := new(ServerConfig)
|
||||
if err := sc.UnmarshalJSON(data); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return sc, nil
|
||||
}
|
||||
|
||||
// Equal reports whether sc and other are considered equal.
|
||||
func (sc *ServerConfig) Equal(other *ServerConfig) bool {
|
||||
switch {
|
||||
case sc == nil && other == nil:
|
||||
return true
|
||||
case (sc != nil) != (other != nil):
|
||||
return false
|
||||
case sc.ServerURI != other.ServerURI:
|
||||
return false
|
||||
case !sc.Creds.Equal(other.Creds):
|
||||
return false
|
||||
case !equalStringSlice(sc.ServerFeatures, other.ServerFeatures):
|
||||
return false
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func equalStringSlice(a, b []string) bool {
|
||||
if len(a) != len(b) {
|
||||
return false
|
||||
}
|
||||
for i := range a {
|
||||
if a[i] != b[i] {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
// unmarshalJSONServerConfigSlice unmarshals JSON to a slice.
|
||||
func unmarshalJSONServerConfigSlice(data []byte) ([]*ServerConfig, error) {
|
||||
var servers []*ServerConfig
|
||||
|
@ -422,7 +514,7 @@ func newConfigFromContents(data []byte) (*Config, error) {
|
|||
if config.XDSServer.ServerURI == "" {
|
||||
return nil, fmt.Errorf("xds: required field %q not found in bootstrap %s", "xds_servers.server_uri", jsonData["xds_servers"])
|
||||
}
|
||||
if config.XDSServer.Creds == nil {
|
||||
if config.XDSServer.CredsDialOption() == nil {
|
||||
return nil, fmt.Errorf("xds: required field %q doesn't contain valid value in bootstrap %s", "xds_servers.channel_creds", jsonData["xds_servers"])
|
||||
}
|
||||
// Post-process the authorities' client listener resource template field:
|
||||
|
|
|
@ -28,10 +28,7 @@ import (
|
|||
"github.com/golang/protobuf/proto"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/google"
|
||||
"google.golang.org/grpc/credentials/tls/certprovider"
|
||||
"google.golang.org/grpc/internal"
|
||||
"google.golang.org/grpc/internal/envconfig"
|
||||
|
@ -56,7 +53,7 @@ var (
|
|||
"channel_creds": [
|
||||
{ "type": "google_default" }
|
||||
],
|
||||
"server_features" : ["foo", "bar"]
|
||||
"server_features" : ["xds_v3"]
|
||||
}]
|
||||
}`,
|
||||
"serverFeaturesExcludesXDSV3": `
|
||||
|
@ -71,8 +68,7 @@ var (
|
|||
"server_uri": "trafficdirector.googleapis.com:443",
|
||||
"channel_creds": [
|
||||
{ "type": "google_default" }
|
||||
],
|
||||
"server_features" : ["foo", "bar", "xds_v3"]
|
||||
]
|
||||
}]
|
||||
}`,
|
||||
"emptyNodeProto": `
|
||||
|
@ -145,7 +141,8 @@ var (
|
|||
"channel_creds": [
|
||||
{ "type": "not-google-default" },
|
||||
{ "type": "google_default" }
|
||||
]
|
||||
],
|
||||
"server_features": ["xds_v3"]
|
||||
}]
|
||||
}`,
|
||||
"goodBootstrap": `
|
||||
|
@ -160,7 +157,8 @@ var (
|
|||
"server_uri": "trafficdirector.googleapis.com:443",
|
||||
"channel_creds": [
|
||||
{ "type": "google_default" }
|
||||
]
|
||||
],
|
||||
"server_features": ["xds_v3"]
|
||||
}]
|
||||
}`,
|
||||
"multipleXDSServers": `
|
||||
|
@ -174,7 +172,8 @@ var (
|
|||
"xds_servers" : [
|
||||
{
|
||||
"server_uri": "trafficdirector.googleapis.com:443",
|
||||
"channel_creds": [{ "type": "google_default" }]
|
||||
"channel_creds": [{ "type": "google_default" }],
|
||||
"server_features": ["xds_v3"]
|
||||
},
|
||||
{
|
||||
"server_uri": "backup.never.use.com:1234",
|
||||
|
@ -198,19 +197,35 @@ var (
|
|||
ClientFeatures: []string{clientFeatureNoOverprovisioning, clientFeatureResourceWrapper},
|
||||
}
|
||||
nilCredsConfigV3 = &Config{
|
||||
XDSServer: &ServerConfig{
|
||||
ServerURI: "trafficdirector.googleapis.com:443",
|
||||
Creds: ChannelCreds{Type: "insecure"},
|
||||
ServerFeatures: []string{"xds_v3"},
|
||||
},
|
||||
NodeProto: v3NodeProto,
|
||||
ClientDefaultListenerResourceNameTemplate: "%s",
|
||||
}
|
||||
nilCredsConfigNoServerFeatures = &Config{
|
||||
XDSServer: &ServerConfig{
|
||||
ServerURI: "trafficdirector.googleapis.com:443",
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
Creds: ChannelCreds{Type: "insecure"},
|
||||
},
|
||||
NodeProto: v3NodeProto,
|
||||
ClientDefaultListenerResourceNameTemplate: "%s",
|
||||
}
|
||||
nonNilCredsConfigV3 = &Config{
|
||||
XDSServer: &ServerConfig{
|
||||
ServerURI: "trafficdirector.googleapis.com:443",
|
||||
Creds: ChannelCreds{Type: "google_default"},
|
||||
ServerFeatures: []string{"xds_v3"},
|
||||
},
|
||||
NodeProto: v3NodeProto,
|
||||
ClientDefaultListenerResourceNameTemplate: "%s",
|
||||
}
|
||||
nonNilCredsConfigNoServerFeatures = &Config{
|
||||
XDSServer: &ServerConfig{
|
||||
ServerURI: "trafficdirector.googleapis.com:443",
|
||||
Creds: grpc.WithCredentialsBundle(google.NewComputeEngineCredentials()),
|
||||
CredsType: "google_default",
|
||||
Creds: ChannelCreds{Type: "google_default"},
|
||||
},
|
||||
NodeProto: v3NodeProto,
|
||||
ClientDefaultListenerResourceNameTemplate: "%s",
|
||||
|
@ -218,14 +233,13 @@ var (
|
|||
)
|
||||
|
||||
func (c *Config) compare(want *Config) error {
|
||||
if diff := cmp.Diff(c, want,
|
||||
if diff := cmp.Diff(want, c,
|
||||
cmpopts.EquateEmpty(),
|
||||
cmp.AllowUnexported(ServerConfig{}),
|
||||
cmp.Comparer(proto.Equal),
|
||||
cmp.Comparer(func(a, b grpc.DialOption) bool { return (a != nil) == (b != nil) }),
|
||||
cmp.Transformer("certproviderconfigstring", func(a *certprovider.BuildableConfig) string { return a.String() }),
|
||||
); diff != "" {
|
||||
return fmt.Errorf("diff: %v", diff)
|
||||
return fmt.Errorf("unexpected diff in config (-want, +got):\n%s", diff)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -251,7 +265,6 @@ func setupBootstrapOverride(bootstrapFileMap map[string]string) func() {
|
|||
// This function overrides the bootstrap file NAME env variable, to test the
|
||||
// code that reads file with the given fileName.
|
||||
func testNewConfigWithFileNameEnv(t *testing.T, fileName string, wantError bool, wantConfig *Config) {
|
||||
t.Helper()
|
||||
origBootstrapFileName := envconfig.XDSBootstrapFileName
|
||||
envconfig.XDSBootstrapFileName = fileName
|
||||
defer func() { envconfig.XDSBootstrapFileName = origBootstrapFileName }()
|
||||
|
@ -373,8 +386,7 @@ func TestNewConfigV3ProtoSuccess(t *testing.T) {
|
|||
"emptyNodeProto", &Config{
|
||||
XDSServer: &ServerConfig{
|
||||
ServerURI: "trafficdirector.googleapis.com:443",
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
Creds: ChannelCreds{Type: "insecure"},
|
||||
},
|
||||
NodeProto: &v3corepb.Node{
|
||||
UserAgentName: gRPCUserAgentName,
|
||||
|
@ -384,9 +396,9 @@ func TestNewConfigV3ProtoSuccess(t *testing.T) {
|
|||
ClientDefaultListenerResourceNameTemplate: "%s",
|
||||
},
|
||||
},
|
||||
{"unknownTopLevelFieldInFile", nilCredsConfigV3},
|
||||
{"unknownFieldInNodeProto", nilCredsConfigV3},
|
||||
{"unknownFieldInXdsServer", nilCredsConfigV3},
|
||||
{"unknownTopLevelFieldInFile", nilCredsConfigNoServerFeatures},
|
||||
{"unknownFieldInNodeProto", nilCredsConfigNoServerFeatures},
|
||||
{"unknownFieldInXdsServer", nilCredsConfigNoServerFeatures},
|
||||
{"multipleChannelCreds", nonNilCredsConfigV3},
|
||||
{"goodBootstrap", nonNilCredsConfigV3},
|
||||
{"multipleXDSServers", nonNilCredsConfigV3},
|
||||
|
@ -400,34 +412,12 @@ func TestNewConfigV3ProtoSuccess(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
// TestNewConfigV3Support verifies bootstrap functionality involving support for
|
||||
// the xDS v3 transport protocol. Here the client ends up using v2 or v3 based
|
||||
// on what the server supports.
|
||||
func TestNewConfigV3Support(t *testing.T) {
|
||||
cancel := setupBootstrapOverride(v3BootstrapFileMap)
|
||||
defer cancel()
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
wantConfig *Config
|
||||
}{
|
||||
{"serverFeaturesIncludesXDSV3", nonNilCredsConfigV3},
|
||||
{"serverFeaturesExcludesXDSV3", nonNilCredsConfigV3},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
testNewConfigWithFileNameEnv(t, test.name, false, test.wantConfig)
|
||||
testNewConfigWithFileContentEnv(t, test.name, false, test.wantConfig)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// TestNewConfigBootstrapEnvPriority tests that the two env variables are read
|
||||
// in correct priority.
|
||||
//
|
||||
// the case where the bootstrap file
|
||||
// environment variable is not set.
|
||||
// "GRPC_XDS_BOOTSTRAP" which specifies the file name containing the bootstrap
|
||||
// configuration takes precedence over "GRPC_XDS_BOOTSTRAP_CONFIG", which
|
||||
// directly specifies the bootstrap configuration in itself.
|
||||
func TestNewConfigBootstrapEnvPriority(t *testing.T) {
|
||||
oldFileReadFunc := bootstrapFileReadFunc
|
||||
bootstrapFileReadFunc = func(filename string) ([]byte, error) {
|
||||
|
@ -440,7 +430,7 @@ func TestNewConfigBootstrapEnvPriority(t *testing.T) {
|
|||
|
||||
goodFileName2 := "serverFeaturesExcludesXDSV3"
|
||||
goodFileContent2 := v3BootstrapFileMap[goodFileName2]
|
||||
goodConfig2 := nonNilCredsConfigV3
|
||||
goodConfig2 := nonNilCredsConfigNoServerFeatures
|
||||
|
||||
origBootstrapFileName := envconfig.XDSBootstrapFileName
|
||||
envconfig.XDSBootstrapFileName = ""
|
||||
|
@ -458,21 +448,33 @@ func TestNewConfigBootstrapEnvPriority(t *testing.T) {
|
|||
// When one of them is set, it should be used.
|
||||
envconfig.XDSBootstrapFileName = goodFileName1
|
||||
envconfig.XDSBootstrapFileContent = ""
|
||||
if c, err := NewConfig(); err != nil || c.compare(goodConfig1) != nil {
|
||||
t.Errorf("NewConfig() = %v, %v, want: %v, %v", c, err, goodConfig1, nil)
|
||||
c, err := NewConfig()
|
||||
if err != nil {
|
||||
t.Errorf("NewConfig() failed: %v", err)
|
||||
}
|
||||
if err := c.compare(goodConfig1); err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
||||
envconfig.XDSBootstrapFileName = ""
|
||||
envconfig.XDSBootstrapFileContent = goodFileContent2
|
||||
if c, err := NewConfig(); err != nil || c.compare(goodConfig2) != nil {
|
||||
t.Errorf("NewConfig() = %v, %v, want: %v, %v", c, err, goodConfig1, nil)
|
||||
c, err = NewConfig()
|
||||
if err != nil {
|
||||
t.Errorf("NewConfig() failed: %v", err)
|
||||
}
|
||||
if err := c.compare(goodConfig2); err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
||||
// Set both, file name should be read.
|
||||
envconfig.XDSBootstrapFileName = goodFileName1
|
||||
envconfig.XDSBootstrapFileContent = goodFileContent2
|
||||
if c, err := NewConfig(); err != nil || c.compare(goodConfig1) != nil {
|
||||
t.Errorf("NewConfig() = %v, %v, want: %v, %v", c, err, goodConfig1, nil)
|
||||
c, err = NewConfig()
|
||||
if err != nil {
|
||||
t.Errorf("NewConfig() failed: %v", err)
|
||||
}
|
||||
if err := c.compare(goodConfig1); err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -559,7 +561,7 @@ func TestNewConfigWithCertificateProviders(t *testing.T) {
|
|||
"channel_creds": [
|
||||
{ "type": "google_default" }
|
||||
],
|
||||
"server_features" : ["foo", "bar", "xds_v3"]
|
||||
"server_features" : ["xds_v3"]
|
||||
}],
|
||||
"certificate_providers": {
|
||||
"unknownProviderInstance1": {
|
||||
|
@ -585,7 +587,7 @@ func TestNewConfigWithCertificateProviders(t *testing.T) {
|
|||
"channel_creds": [
|
||||
{ "type": "google_default" }
|
||||
],
|
||||
"server_features" : ["foo", "bar", "xds_v3"],
|
||||
"server_features" : ["xds_v3"],
|
||||
}],
|
||||
"certificate_providers": {
|
||||
"unknownProviderInstance": {
|
||||
|
@ -609,9 +611,9 @@ func TestNewConfigWithCertificateProviders(t *testing.T) {
|
|||
"xds_servers" : [{
|
||||
"server_uri": "trafficdirector.googleapis.com:443",
|
||||
"channel_creds": [
|
||||
{ "type": "google_default" }
|
||||
{ "type": "insecure" }
|
||||
],
|
||||
"server_features" : ["foo", "bar", "xds_v3"]
|
||||
"server_features" : ["xds_v3"]
|
||||
}],
|
||||
"certificate_providers": {
|
||||
"unknownProviderInstance": {
|
||||
|
@ -639,12 +641,19 @@ func TestNewConfigWithCertificateProviders(t *testing.T) {
|
|||
cancel := setupBootstrapOverride(bootstrapFileMap)
|
||||
defer cancel()
|
||||
|
||||
// Cannot use xdstestutils.ServerConfigForAddress here, as it would lead to
|
||||
// a cyclic dependency.
|
||||
jsonCfg := `{
|
||||
"server_uri": "trafficdirector.googleapis.com:443",
|
||||
"channel_creds": [{"type": "insecure"}],
|
||||
"server_features": ["xds_v3"]
|
||||
}`
|
||||
serverCfg, err := ServerConfigFromJSON([]byte(jsonCfg))
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to create server config from JSON %s: %v", jsonCfg, err)
|
||||
}
|
||||
goodConfig := &Config{
|
||||
XDSServer: &ServerConfig{
|
||||
ServerURI: "trafficdirector.googleapis.com:443",
|
||||
Creds: grpc.WithCredentialsBundle(google.NewComputeEngineCredentials()),
|
||||
CredsType: "google_default",
|
||||
},
|
||||
XDSServer: serverCfg,
|
||||
NodeProto: v3NodeProto,
|
||||
CertProviderConfigs: map[string]*certprovider.BuildableConfig{
|
||||
"fakeProviderInstance": wantCfg,
|
||||
|
@ -735,8 +744,7 @@ func TestNewConfigWithServerListenerResourceNameTemplate(t *testing.T) {
|
|||
wantConfig: &Config{
|
||||
XDSServer: &ServerConfig{
|
||||
ServerURI: "trafficdirector.googleapis.com:443",
|
||||
Creds: grpc.WithCredentialsBundle(google.NewComputeEngineCredentials()),
|
||||
CredsType: "google_default",
|
||||
Creds: ChannelCreds{Type: "google_default"},
|
||||
},
|
||||
NodeProto: v3NodeProto,
|
||||
ServerListenerResourceNameTemplate: "grpc/server?xds.resource.listening_address=%s",
|
||||
|
@ -801,7 +809,7 @@ func TestNewConfigWithFederation(t *testing.T) {
|
|||
"xds_servers": [{
|
||||
"server_uri": "td.com",
|
||||
"channel_creds": [ { "type": "google_default" } ],
|
||||
"server_features" : ["foo", "bar", "xds_v3"]
|
||||
"server_features" : ["xds_v3"]
|
||||
}]
|
||||
}
|
||||
}
|
||||
|
@ -884,8 +892,7 @@ func TestNewConfigWithFederation(t *testing.T) {
|
|||
wantConfig: &Config{
|
||||
XDSServer: &ServerConfig{
|
||||
ServerURI: "trafficdirector.googleapis.com:443",
|
||||
Creds: grpc.WithCredentialsBundle(google.NewComputeEngineCredentials()),
|
||||
CredsType: "google_default",
|
||||
Creds: ChannelCreds{Type: "google_default"},
|
||||
},
|
||||
NodeProto: v3NodeProto,
|
||||
ServerListenerResourceNameTemplate: "xdstp://xds.example.com/envoy.config.listener.v3.Listener/grpc/server?listening_address=%s",
|
||||
|
@ -894,9 +901,9 @@ func TestNewConfigWithFederation(t *testing.T) {
|
|||
"xds.td.com": {
|
||||
ClientListenerResourceNameTemplate: "xdstp://xds.td.com/envoy.config.listener.v3.Listener/%s",
|
||||
XDSServer: &ServerConfig{
|
||||
ServerURI: "td.com",
|
||||
Creds: grpc.WithCredentialsBundle(google.NewComputeEngineCredentials()),
|
||||
CredsType: "google_default",
|
||||
ServerURI: "td.com",
|
||||
Creds: ChannelCreds{Type: "google_default"},
|
||||
ServerFeatures: []string{"xds_v3"},
|
||||
},
|
||||
},
|
||||
},
|
||||
|
@ -907,8 +914,7 @@ func TestNewConfigWithFederation(t *testing.T) {
|
|||
wantConfig: &Config{
|
||||
XDSServer: &ServerConfig{
|
||||
ServerURI: "trafficdirector.googleapis.com:443",
|
||||
Creds: grpc.WithCredentialsBundle(google.NewComputeEngineCredentials()),
|
||||
CredsType: "google_default",
|
||||
Creds: ChannelCreds{Type: "google_default"},
|
||||
},
|
||||
NodeProto: v3NodeProto,
|
||||
ClientDefaultListenerResourceNameTemplate: "%s",
|
||||
|
@ -919,8 +925,7 @@ func TestNewConfigWithFederation(t *testing.T) {
|
|||
wantConfig: &Config{
|
||||
XDSServer: &ServerConfig{
|
||||
ServerURI: "trafficdirector.googleapis.com:443",
|
||||
Creds: grpc.WithCredentialsBundle(google.NewComputeEngineCredentials()),
|
||||
CredsType: "google_default",
|
||||
Creds: ChannelCreds{Type: "google_default"},
|
||||
},
|
||||
NodeProto: v3NodeProto,
|
||||
ClientDefaultListenerResourceNameTemplate: "xdstp://xds.example.com/envoy.config.listener.v3.Listener/%s",
|
||||
|
@ -939,8 +944,7 @@ func TestNewConfigWithFederation(t *testing.T) {
|
|||
wantConfig: &Config{
|
||||
XDSServer: &ServerConfig{
|
||||
ServerURI: "trafficdirector.googleapis.com:443",
|
||||
Creds: grpc.WithCredentialsBundle(google.NewComputeEngineCredentials()),
|
||||
CredsType: "google_default",
|
||||
Creds: ChannelCreds{Type: "google_default"},
|
||||
},
|
||||
NodeProto: v3NodeProto,
|
||||
ClientDefaultListenerResourceNameTemplate: "xdstp://xds.example.com/envoy.config.listener.v3.Listener/%s",
|
||||
|
@ -966,22 +970,26 @@ func TestNewConfigWithFederation(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestServerConfigMarshalAndUnmarshal(t *testing.T) {
|
||||
c := ServerConfig{
|
||||
ServerURI: "test-server",
|
||||
Creds: nil,
|
||||
CredsType: "test-creds",
|
||||
jsonCfg := `{
|
||||
"server_uri": "test-server",
|
||||
"channel_creds": [{"type": "insecure"}],
|
||||
"server_features": ["xds_v3"]
|
||||
}`
|
||||
origConfig, err := ServerConfigFromJSON([]byte(jsonCfg))
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to create server config from JSON %s: %v", jsonCfg, err)
|
||||
}
|
||||
|
||||
bs, err := json.Marshal(c)
|
||||
bs, err := json.Marshal(origConfig)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to marshal: %v", err)
|
||||
}
|
||||
var cUnmarshal ServerConfig
|
||||
if err := json.Unmarshal(bs, &cUnmarshal); err != nil {
|
||||
|
||||
unmarshaledConfig := new(ServerConfig)
|
||||
if err := json.Unmarshal(bs, unmarshaledConfig); err != nil {
|
||||
t.Fatalf("failed to unmarshal: %v", err)
|
||||
}
|
||||
if diff := cmp.Diff(cUnmarshal, c); diff != "" {
|
||||
t.Fatalf("diff (-got +want): %v", diff)
|
||||
if diff := cmp.Diff(origConfig, unmarshaledConfig); diff != "" {
|
||||
t.Fatalf("Unexpected diff in server config (-want, +got):\n%s", diff)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -24,11 +24,10 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/testutils/xds/fakeserver"
|
||||
"google.golang.org/grpc/status"
|
||||
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
"google.golang.org/protobuf/testing/protocmp"
|
||||
|
||||
|
@ -43,17 +42,15 @@ const (
|
|||
)
|
||||
|
||||
func (s) TestLRSClient(t *testing.T) {
|
||||
fs, sCleanup, err := fakeserver.StartServer(nil)
|
||||
fs1, sCleanup, err := fakeserver.StartServer(nil)
|
||||
if err != nil {
|
||||
t.Fatalf("failed to start fake xDS server: %v", err)
|
||||
}
|
||||
defer sCleanup()
|
||||
|
||||
serverCfg1 := xdstestutils.ServerConfigForAddress(t, fs1.Address)
|
||||
xdsC, close, err := NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: fs.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: serverCfg1,
|
||||
NodeProto: &v3corepb.Node{},
|
||||
}, defaultClientWatchExpiryTimeout, time.Duration(0))
|
||||
if err != nil {
|
||||
|
@ -62,24 +59,18 @@ func (s) TestLRSClient(t *testing.T) {
|
|||
defer close()
|
||||
|
||||
// Report to the same address should not create new ClientConn.
|
||||
store1, lrsCancel1 := xdsC.ReportLoad(
|
||||
&bootstrap.ServerConfig{
|
||||
ServerURI: fs.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
},
|
||||
)
|
||||
store1, lrsCancel1 := xdsC.ReportLoad(serverCfg1)
|
||||
defer lrsCancel1()
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout)
|
||||
defer cancel()
|
||||
if u, err := fs.NewConnChan.Receive(ctx); err != nil {
|
||||
if u, err := fs1.NewConnChan.Receive(ctx); err != nil {
|
||||
t.Errorf("unexpected timeout: %v, %v, want NewConn", u, err)
|
||||
}
|
||||
|
||||
sCtx, sCancel := context.WithTimeout(context.Background(), defaultTestShortTimeout)
|
||||
defer sCancel()
|
||||
if u, err := fs.NewConnChan.Receive(sCtx); err != context.DeadlineExceeded {
|
||||
if u, err := fs1.NewConnChan.Receive(sCtx); err != context.DeadlineExceeded {
|
||||
t.Errorf("unexpected NewConn: %v, %v, want channel recv timeout", u, err)
|
||||
}
|
||||
|
||||
|
@ -90,13 +81,8 @@ func (s) TestLRSClient(t *testing.T) {
|
|||
defer sCleanup2()
|
||||
|
||||
// Report to a different address should create new ClientConn.
|
||||
store2, lrsCancel2 := xdsC.ReportLoad(
|
||||
&bootstrap.ServerConfig{
|
||||
ServerURI: fs2.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
},
|
||||
)
|
||||
serverCgf2 := xdstestutils.ServerConfigForAddress(t, fs2.Address)
|
||||
store2, lrsCancel2 := xdsC.ReportLoad(serverCgf2)
|
||||
defer lrsCancel2()
|
||||
if u, err := fs2.NewConnChan.Receive(ctx); err != nil {
|
||||
t.Errorf("unexpected timeout: %v, %v, want NewConn", u, err)
|
||||
|
|
|
@ -24,8 +24,6 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/testutils"
|
||||
"google.golang.org/grpc/internal/testutils/xds/e2e"
|
||||
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
|
||||
|
@ -94,20 +92,12 @@ func setupForAuthorityTests(ctx context.Context, t *testing.T, idleTimeout time.
|
|||
// config, which points to the above management server.
|
||||
nodeID := uuid.New().String()
|
||||
client, close, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: defaultAuthorityServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, defaultAuthorityServer.Address),
|
||||
NodeProto: &v3corepb.Node{Id: nodeID},
|
||||
Authorities: map[string]*bootstrap.Authority{
|
||||
testAuthority1: {},
|
||||
testAuthority2: {},
|
||||
testAuthority3: {
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: nonDefaultAuthorityServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
},
|
||||
testAuthority3: {XDSServer: xdstestutils.ServerConfigForAddress(t, nonDefaultAuthorityServer.Address)},
|
||||
},
|
||||
}, defaultTestWatchExpiryTimeout, idleTimeout)
|
||||
if err != nil {
|
||||
|
|
|
@ -28,11 +28,10 @@ import (
|
|||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"github.com/google/uuid"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/grpcsync"
|
||||
"google.golang.org/grpc/internal/testutils"
|
||||
"google.golang.org/grpc/internal/testutils/xds/e2e"
|
||||
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
|
||||
|
@ -535,10 +534,7 @@ func (s) TestCDSWatch_ExpiryTimerFiresBeforeResponse(t *testing.T) {
|
|||
defer mgmtServer.Stop()
|
||||
|
||||
client, close, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{},
|
||||
}, defaultTestWatchExpiryTimeout, time.Duration(0))
|
||||
if err != nil {
|
||||
|
@ -581,10 +577,7 @@ func (s) TestCDSWatch_ValidResponseCancelsExpiryTimerBehavior(t *testing.T) {
|
|||
// Create an xDS client talking to the above management server.
|
||||
nodeID := uuid.New().String()
|
||||
client, close, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{Id: nodeID},
|
||||
}, defaultTestWatchExpiryTimeout, time.Duration(0))
|
||||
if err != nil {
|
||||
|
|
|
@ -28,12 +28,11 @@ import (
|
|||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"github.com/google/uuid"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/grpcsync"
|
||||
"google.golang.org/grpc/internal/testutils"
|
||||
"google.golang.org/grpc/internal/testutils/xds/e2e"
|
||||
"google.golang.org/grpc/xds/internal"
|
||||
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
|
||||
|
@ -592,10 +591,7 @@ func (s) TestEDSWatch_ExpiryTimerFiresBeforeResponse(t *testing.T) {
|
|||
defer mgmtServer.Stop()
|
||||
|
||||
client, close, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{},
|
||||
}, defaultTestWatchExpiryTimeout, time.Duration(0))
|
||||
if err != nil {
|
||||
|
@ -638,10 +634,7 @@ func (s) TestEDSWatch_ValidResponseCancelsExpiryTimerBehavior(t *testing.T) {
|
|||
// Create an xDS client talking to the above management server.
|
||||
nodeID := uuid.New().String()
|
||||
client, close, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{Id: nodeID},
|
||||
}, defaultTestWatchExpiryTimeout, time.Duration(0))
|
||||
if err != nil {
|
||||
|
|
|
@ -29,13 +29,12 @@ import (
|
|||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"github.com/google/uuid"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/envconfig"
|
||||
"google.golang.org/grpc/internal/grpcsync"
|
||||
"google.golang.org/grpc/internal/grpctest"
|
||||
"google.golang.org/grpc/internal/testutils"
|
||||
"google.golang.org/grpc/internal/testutils/xds/e2e"
|
||||
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
|
||||
|
@ -584,10 +583,7 @@ func (s) TestLDSWatch_ExpiryTimerFiresBeforeResponse(t *testing.T) {
|
|||
defer mgmtServer.Stop()
|
||||
|
||||
client, close, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{},
|
||||
}, defaultTestWatchExpiryTimeout, time.Duration(0))
|
||||
if err != nil {
|
||||
|
@ -630,10 +626,7 @@ func (s) TestLDSWatch_ValidResponseCancelsExpiryTimerBehavior(t *testing.T) {
|
|||
// Create an xDS client talking to the above management server.
|
||||
nodeID := uuid.New().String()
|
||||
client, close, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{Id: nodeID},
|
||||
}, defaultTestWatchExpiryTimeout, time.Duration(0))
|
||||
if err != nil {
|
||||
|
|
|
@ -28,11 +28,10 @@ import (
|
|||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"github.com/google/uuid"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/grpcsync"
|
||||
"google.golang.org/grpc/internal/testutils"
|
||||
"google.golang.org/grpc/internal/testutils/xds/e2e"
|
||||
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
|
||||
|
@ -626,10 +625,7 @@ func (s) TestRDSWatch_ExpiryTimerFiresBeforeResponse(t *testing.T) {
|
|||
|
||||
// Create an xDS client talking to a non-existent management server.
|
||||
client, close, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{},
|
||||
}, defaultTestWatchExpiryTimeout, time.Duration(0))
|
||||
if err != nil {
|
||||
|
@ -672,10 +668,7 @@ func (s) TestRDSWatch_ValidResponseCancelsExpiryTimerBehavior(t *testing.T) {
|
|||
// Create an xDS client talking to the above management server.
|
||||
nodeID := uuid.New().String()
|
||||
client, close, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{Id: nodeID},
|
||||
}, defaultTestWatchExpiryTimeout, time.Duration(0))
|
||||
if err != nil {
|
||||
|
|
|
@ -28,12 +28,11 @@ import (
|
|||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"github.com/google/uuid"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/testutils"
|
||||
"google.golang.org/grpc/internal/testutils/xds/e2e"
|
||||
"google.golang.org/grpc/internal/testutils/xds/fakeserver"
|
||||
"google.golang.org/grpc/xds/internal"
|
||||
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource"
|
||||
|
@ -243,11 +242,7 @@ func (s) TestHandleListenerResponseFromManagementServer(t *testing.T) {
|
|||
// Create an xDS client talking to the above management server.
|
||||
nodeID := uuid.New().String()
|
||||
client, close, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{Id: nodeID},
|
||||
}, defaultTestWatchExpiryTimeout, time.Duration(0))
|
||||
if err != nil {
|
||||
|
@ -509,11 +504,7 @@ func (s) TestHandleRouteConfigResponseFromManagementServer(t *testing.T) {
|
|||
// Create an xDS client talking to the above management server.
|
||||
nodeID := uuid.New().String()
|
||||
client, close, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{Id: nodeID},
|
||||
}, defaultTestWatchExpiryTimeout, time.Duration(0))
|
||||
if err != nil {
|
||||
|
@ -751,11 +742,7 @@ func (s) TestHandleClusterResponseFromManagementServer(t *testing.T) {
|
|||
// Create an xDS client talking to the above management server.
|
||||
nodeID := uuid.New().String()
|
||||
client, close, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{Id: nodeID},
|
||||
}, defaultTestWatchExpiryTimeout, time.Duration(0))
|
||||
if err != nil {
|
||||
|
@ -1076,11 +1063,7 @@ func (s) TestHandleEndpointsResponseFromManagementServer(t *testing.T) {
|
|||
// Create an xDS client talking to the above management server.
|
||||
nodeID := uuid.New().String()
|
||||
client, close, err := xdsclient.NewWithConfigForTesting(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{Id: nodeID},
|
||||
}, defaultTestWatchExpiryTimeout, time.Duration(0))
|
||||
if err != nil {
|
||||
|
|
|
@ -25,10 +25,8 @@ import (
|
|||
v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/uuid"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/testutils/xds/fakeserver"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/transport"
|
||||
"google.golang.org/protobuf/testing/protocmp"
|
||||
"google.golang.org/protobuf/types/known/durationpb"
|
||||
|
@ -43,17 +41,10 @@ func (s) TestReportLoad(t *testing.T) {
|
|||
defer cleanup()
|
||||
t.Logf("Started xDS management server on %s", mgmtServer.Address)
|
||||
|
||||
// Construct the server config to represent the management server.
|
||||
nodeProto := &v3corepb.Node{Id: uuid.New().String()}
|
||||
serverCfg := bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
}
|
||||
|
||||
// Create a transport to the fake management server.
|
||||
nodeProto := &v3corepb.Node{Id: uuid.New().String()}
|
||||
tr, err := transport.New(transport.Options{
|
||||
ServerCfg: serverCfg,
|
||||
ServerCfg: *testutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: nodeProto,
|
||||
OnRecvHandler: func(transport.ResourceUpdate) error { return nil }, // No ADS validation.
|
||||
OnErrorHandler: func(error) {}, // No ADS stream error handling.
|
||||
|
|
|
@ -177,7 +177,7 @@ func New(opts Options) (*Transport, error) {
|
|||
switch {
|
||||
case opts.ServerCfg.ServerURI == "":
|
||||
return nil, errors.New("missing server URI when creating a new transport")
|
||||
case opts.ServerCfg.Creds == nil:
|
||||
case opts.ServerCfg.CredsDialOption() == nil:
|
||||
return nil, errors.New("missing credentials when creating a new transport")
|
||||
case opts.OnRecvHandler == nil:
|
||||
return nil, errors.New("missing OnRecv callback handler when creating a new transport")
|
||||
|
@ -189,7 +189,7 @@ func New(opts Options) (*Transport, error) {
|
|||
|
||||
// Dial the xDS management with the passed in credentials.
|
||||
dopts := []grpc.DialOption{
|
||||
opts.ServerCfg.Creds,
|
||||
opts.ServerCfg.CredsDialOption(),
|
||||
grpc.WithKeepaliveParams(keepalive.ClientParameters{
|
||||
// We decided to use these sane defaults in all languages, and
|
||||
// kicked the can down the road as far making these configurable.
|
||||
|
|
|
@ -26,12 +26,10 @@ import (
|
|||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"github.com/google/uuid"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/testutils"
|
||||
"google.golang.org/grpc/internal/testutils/xds/e2e"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/transport"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
|
||||
"google.golang.org/protobuf/proto"
|
||||
|
@ -135,16 +133,9 @@ func (s) TestSimpleAckAndNack(t *testing.T) {
|
|||
SkipValidation: true,
|
||||
})
|
||||
|
||||
// Construct the server config to represent the management server.
|
||||
serverCfg := bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
}
|
||||
|
||||
// Create a new transport.
|
||||
tr, err := transport.New(transport.Options{
|
||||
ServerCfg: serverCfg,
|
||||
ServerCfg: *xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
OnRecvHandler: dataModelValidator,
|
||||
OnErrorHandler: func(err error) {},
|
||||
OnSendHandler: func(*transport.ResourceSendInfo) {},
|
||||
|
@ -322,16 +313,9 @@ func (s) TestInvalidFirstResponse(t *testing.T) {
|
|||
SkipValidation: true,
|
||||
})
|
||||
|
||||
// Construct the server config to represent the management server.
|
||||
serverCfg := bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
}
|
||||
|
||||
// Create a new transport.
|
||||
tr, err := transport.New(transport.Options{
|
||||
ServerCfg: serverCfg,
|
||||
ServerCfg: *xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{Id: nodeID},
|
||||
OnRecvHandler: dataModelValidator,
|
||||
OnErrorHandler: func(err error) {},
|
||||
|
@ -451,16 +435,9 @@ func (s) TestResourceIsNotRequestedAnymore(t *testing.T) {
|
|||
SkipValidation: true,
|
||||
})
|
||||
|
||||
// Construct the server config to represent the management server.
|
||||
serverCfg := bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
}
|
||||
|
||||
// Create a new transport.
|
||||
tr, err := transport.New(transport.Options{
|
||||
ServerCfg: serverCfg,
|
||||
ServerCfg: *xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
NodeProto: &v3corepb.Node{Id: nodeID},
|
||||
OnRecvHandler: dataModelValidator,
|
||||
OnErrorHandler: func(err error) {},
|
||||
|
|
|
@ -27,12 +27,10 @@ import (
|
|||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"github.com/google/uuid"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/connectivity"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/testutils"
|
||||
"google.golang.org/grpc/internal/testutils/xds/e2e"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/transport"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
|
||||
"google.golang.org/protobuf/testing/protocmp"
|
||||
|
@ -98,18 +96,11 @@ func (s) TestTransport_BackoffAfterStreamFailure(t *testing.T) {
|
|||
return 0
|
||||
}
|
||||
|
||||
// Construct the server config to represent the management server.
|
||||
nodeID := uuid.New().String()
|
||||
serverCfg := bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
}
|
||||
|
||||
// Create a new transport. Since we are only testing backoff behavior here,
|
||||
// we can pass a no-op data model layer implementation.
|
||||
nodeID := uuid.New().String()
|
||||
tr, err := transport.New(transport.Options{
|
||||
ServerCfg: serverCfg,
|
||||
ServerCfg: *xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
OnRecvHandler: func(transport.ResourceUpdate) error { return nil }, // No data model layer validation.
|
||||
OnErrorHandler: func(err error) {
|
||||
select {
|
||||
|
@ -267,17 +258,10 @@ func (s) TestTransport_RetriesAfterBrokenStream(t *testing.T) {
|
|||
SkipValidation: true,
|
||||
})
|
||||
|
||||
// Construct the server config to represent the management server.
|
||||
serverCfg := bootstrap.ServerConfig{
|
||||
ServerURI: lis.Addr().String(),
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
}
|
||||
|
||||
// Create a new transport. Since we are only testing backoff behavior here,
|
||||
// we can pass a no-op data model layer implementation.
|
||||
tr, err := transport.New(transport.Options{
|
||||
ServerCfg: serverCfg,
|
||||
ServerCfg: *xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
OnRecvHandler: func(transport.ResourceUpdate) error { return nil }, // No data model layer validation.
|
||||
OnErrorHandler: func(err error) {
|
||||
select {
|
||||
|
@ -405,18 +389,11 @@ func (s) TestTransport_ResourceRequestedBeforeStreamCreation(t *testing.T) {
|
|||
// stream to the management server.
|
||||
lis.Stop()
|
||||
|
||||
// Construct the server config to represent the management server.
|
||||
nodeID := uuid.New().String()
|
||||
serverCfg := bootstrap.ServerConfig{
|
||||
ServerURI: lis.Addr().String(),
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
}
|
||||
|
||||
// Create a new transport. Since we are only testing backoff behavior here,
|
||||
// we can pass a no-op data model layer implementation.
|
||||
nodeID := uuid.New().String()
|
||||
tr, err := transport.New(transport.Options{
|
||||
ServerCfg: serverCfg,
|
||||
ServerCfg: *xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
OnRecvHandler: func(transport.ResourceUpdate) error { return nil }, // No data model layer validation.
|
||||
OnErrorHandler: func(error) {}, // No stream error handling.
|
||||
OnSendHandler: func(*transport.ResourceSendInfo) {}, // No on send handler
|
||||
|
|
|
@ -21,8 +21,7 @@ import (
|
|||
"strings"
|
||||
"testing"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/transport"
|
||||
|
||||
|
@ -49,10 +48,8 @@ func (s) TestNew(t *testing.T) {
|
|||
},
|
||||
{
|
||||
name: "missing onRecv handler",
|
||||
opts: transport.Options{ServerCfg: bootstrap.ServerConfig{
|
||||
ServerURI: "server-address",
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
opts: transport.Options{
|
||||
ServerCfg: *testutils.ServerConfigForAddress(t, "server-address"),
|
||||
NodeProto: &v3corepb.Node{},
|
||||
},
|
||||
wantErrStr: "missing OnRecv callback handler when creating a new transport",
|
||||
|
@ -60,10 +57,7 @@ func (s) TestNew(t *testing.T) {
|
|||
{
|
||||
name: "missing onError handler",
|
||||
opts: transport.Options{
|
||||
ServerCfg: bootstrap.ServerConfig{
|
||||
ServerURI: "server-address",
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
ServerCfg: *testutils.ServerConfigForAddress(t, "server-address"),
|
||||
NodeProto: &v3corepb.Node{},
|
||||
OnRecvHandler: func(transport.ResourceUpdate) error { return nil },
|
||||
OnSendHandler: func(*transport.ResourceSendInfo) {},
|
||||
|
@ -74,10 +68,7 @@ func (s) TestNew(t *testing.T) {
|
|||
{
|
||||
name: "missing onSend handler",
|
||||
opts: transport.Options{
|
||||
ServerCfg: bootstrap.ServerConfig{
|
||||
ServerURI: "server-address",
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
ServerCfg: *testutils.ServerConfigForAddress(t, "server-address"),
|
||||
NodeProto: &v3corepb.Node{},
|
||||
OnRecvHandler: func(transport.ResourceUpdate) error { return nil },
|
||||
OnErrorHandler: func(error) {},
|
||||
|
@ -87,10 +78,7 @@ func (s) TestNew(t *testing.T) {
|
|||
{
|
||||
name: "happy case",
|
||||
opts: transport.Options{
|
||||
ServerCfg: bootstrap.ServerConfig{
|
||||
ServerURI: "server-address",
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
ServerCfg: *testutils.ServerConfigForAddress(t, "server-address"),
|
||||
NodeProto: &v3corepb.Node{},
|
||||
OnRecvHandler: func(transport.ResourceUpdate) error { return nil },
|
||||
OnErrorHandler: func(error) {},
|
||||
|
|
|
@ -26,12 +26,10 @@ import (
|
|||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/uuid"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/grpctest"
|
||||
"google.golang.org/grpc/internal/testutils"
|
||||
"google.golang.org/grpc/internal/testutils/xds/fakeserver"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/transport"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/xdsresource/version"
|
||||
"google.golang.org/protobuf/testing/protocmp"
|
||||
|
@ -176,17 +174,10 @@ func (s) TestHandleResponseFromManagementServer(t *testing.T) {
|
|||
t.Logf("Started xDS management server on %s", mgmtServer.Address)
|
||||
mgmtServer.XDSResponseChan <- &fakeserver.Response{Resp: test.managementServerResponse}
|
||||
|
||||
// Construct the server config to represent the management server.
|
||||
serverCfg := bootstrap.ServerConfig{
|
||||
ServerURI: mgmtServer.Address,
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
CredsType: "insecure",
|
||||
}
|
||||
|
||||
// Create a new transport.
|
||||
resourcesCh := testutils.NewChannel()
|
||||
tr, err := transport.New(transport.Options{
|
||||
ServerCfg: serverCfg,
|
||||
ServerCfg: *xdstestutils.ServerConfigForAddress(t, mgmtServer.Address),
|
||||
// No validation. Simply push received resources on a channel.
|
||||
OnRecvHandler: func(update transport.ResourceUpdate) error {
|
||||
resourcesCh.Send(&resourcesWithTypeURL{
|
||||
|
|
|
@ -22,9 +22,8 @@ import (
|
|||
|
||||
v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/grpc/internal/grpctest"
|
||||
"google.golang.org/grpc/xds/internal/xdsclient/bootstrap"
|
||||
xdstestutils "google.golang.org/grpc/xds/internal/testutils"
|
||||
)
|
||||
|
||||
type s struct {
|
||||
|
@ -48,10 +47,7 @@ func (s) TestNewWithGRPCDial(t *testing.T) {
|
|||
|
||||
// Create a new transport and ensure that the custom dialer was called.
|
||||
opts := Options{
|
||||
ServerCfg: bootstrap.ServerConfig{
|
||||
ServerURI: "server-address",
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
ServerCfg: *xdstestutils.ServerConfigForAddress(t, "server-address"),
|
||||
NodeProto: &v3corepb.Node{},
|
||||
OnRecvHandler: func(ResourceUpdate) error { return nil },
|
||||
OnErrorHandler: func(error) {},
|
||||
|
|
|
@ -317,16 +317,13 @@ func (p *fakeProvider) Close() {
|
|||
|
||||
// setupOverrides sets up overrides for bootstrap config, new xdsClient creation
|
||||
// and new gRPC.Server creation.
|
||||
func setupOverrides() (*fakeGRPCServer, *testutils.Channel, func()) {
|
||||
func setupOverrides(t *testing.T) (*fakeGRPCServer, *testutils.Channel, func()) {
|
||||
clientCh := testutils.NewChannel()
|
||||
origNewXDSClient := newXDSClient
|
||||
newXDSClient = func() (xdsclient.XDSClient, func(), error) {
|
||||
c := fakeclient.NewClient()
|
||||
c.SetBootstrapConfig(&bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: "dummyBalancer",
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, "server-address"),
|
||||
NodeProto: xdstestutils.EmptyNodeProtoV3,
|
||||
ServerListenerResourceNameTemplate: testServerListenerResourceNameTemplate,
|
||||
CertProviderConfigs: certProviderConfigs,
|
||||
|
@ -349,16 +346,13 @@ func setupOverrides() (*fakeGRPCServer, *testutils.Channel, func()) {
|
|||
// one. Tests that use xdsCredentials need a real grpc.Server instead of a fake
|
||||
// one, because the xDS-enabled server needs to read configured creds from the
|
||||
// underlying grpc.Server to confirm whether xdsCreds were configured.
|
||||
func setupOverridesForXDSCreds(includeCertProviderCfg bool) (*testutils.Channel, func()) {
|
||||
func setupOverridesForXDSCreds(t *testing.T, includeCertProviderCfg bool) (*testutils.Channel, func()) {
|
||||
clientCh := testutils.NewChannel()
|
||||
origNewXDSClient := newXDSClient
|
||||
newXDSClient = func() (xdsclient.XDSClient, func(), error) {
|
||||
c := fakeclient.NewClient()
|
||||
bc := &bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: "dummyBalancer",
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, "server-address"),
|
||||
NodeProto: xdstestutils.EmptyNodeProtoV3,
|
||||
ServerListenerResourceNameTemplate: testServerListenerResourceNameTemplate,
|
||||
}
|
||||
|
@ -382,7 +376,7 @@ func setupOverridesForXDSCreds(includeCertProviderCfg bool) (*testutils.Channel,
|
|||
// 4. Push a good response from the xdsClient, and make sure that Serve() on the
|
||||
// underlying grpc.Server is called.
|
||||
func (s) TestServeSuccess(t *testing.T) {
|
||||
fs, clientCh, cleanup := setupOverrides()
|
||||
fs, clientCh, cleanup := setupOverrides(t)
|
||||
defer cleanup()
|
||||
|
||||
// Create a new xDS-enabled gRPC server and pass it a server option to get
|
||||
|
@ -505,7 +499,7 @@ func (s) TestServeSuccess(t *testing.T) {
|
|||
// is received. This should cause Serve() to exit before calling Serve() on the
|
||||
// underlying grpc.Server.
|
||||
func (s) TestServeWithStop(t *testing.T) {
|
||||
fs, clientCh, cleanup := setupOverrides()
|
||||
fs, clientCh, cleanup := setupOverrides(t)
|
||||
defer cleanup()
|
||||
|
||||
// Note that we are not deferring the Stop() here since we explicitly call
|
||||
|
@ -604,10 +598,7 @@ func (s) TestServeBootstrapConfigInvalid(t *testing.T) {
|
|||
{
|
||||
desc: "certificate provider config is missing",
|
||||
bootstrapConfig: &bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: "dummyBalancer",
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, "server-address"),
|
||||
NodeProto: xdstestutils.EmptyNodeProtoV3,
|
||||
ServerListenerResourceNameTemplate: testServerListenerResourceNameTemplate,
|
||||
},
|
||||
|
@ -615,10 +606,7 @@ func (s) TestServeBootstrapConfigInvalid(t *testing.T) {
|
|||
{
|
||||
desc: "server_listener_resource_name_template is missing",
|
||||
bootstrapConfig: &bootstrap.Config{
|
||||
XDSServer: &bootstrap.ServerConfig{
|
||||
ServerURI: "dummyBalancer",
|
||||
Creds: grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
},
|
||||
XDSServer: xdstestutils.ServerConfigForAddress(t, "server-address"),
|
||||
NodeProto: xdstestutils.EmptyNodeProtoV3,
|
||||
CertProviderConfigs: certProviderConfigs,
|
||||
},
|
||||
|
@ -708,7 +696,7 @@ func (s) TestServeNewClientFailure(t *testing.T) {
|
|||
// server is not configured with xDS credentials. Verifies that the security
|
||||
// config received as part of a Listener update is not acted upon.
|
||||
func (s) TestHandleListenerUpdate_NoXDSCreds(t *testing.T) {
|
||||
fs, clientCh, cleanup := setupOverrides()
|
||||
fs, clientCh, cleanup := setupOverrides(t)
|
||||
defer cleanup()
|
||||
|
||||
// Create a server option to get notified about serving mode changes. We don't
|
||||
|
@ -828,7 +816,7 @@ func (s) TestHandleListenerUpdate_NoXDSCreds(t *testing.T) {
|
|||
// server is configured with xDS credentials, but receives a Listener update
|
||||
// with an error. Verifies that no certificate providers are created.
|
||||
func (s) TestHandleListenerUpdate_ErrorUpdate(t *testing.T) {
|
||||
clientCh, cleanup := setupOverridesForXDSCreds(true)
|
||||
clientCh, cleanup := setupOverridesForXDSCreds(t, true)
|
||||
defer cleanup()
|
||||
|
||||
xdsCreds, err := xds.NewServerCredentials(xds.ServerOptions{FallbackCreds: insecure.NewCredentials()})
|
||||
|
|
Loading…
Reference in New Issue