Merge pull request #4584 from hashicorp/refactor_cluster_config (#16517)

NET-2841: PART 1 - refactor NewPeeringCluster to support custom config
This commit is contained in:
Anita Akaeze 2023-03-02 18:21:25 -05:00 committed by GitHub
parent 139bb51736
commit f1d16adda8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 200 additions and 199 deletions

View File

@ -6,6 +6,7 @@ import (
"testing" "testing"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/testcontainers/testcontainers-go"
"github.com/hashicorp/consul/api" "github.com/hashicorp/consul/api"
@ -43,16 +44,26 @@ func BasicPeeringTwoClustersSetup(
consulVersion string, consulVersion string,
peeringThroughMeshgateway bool, peeringThroughMeshgateway bool,
) (*BuiltCluster, *BuiltCluster) { ) (*BuiltCluster, *BuiltCluster) {
// acceptingCluster, acceptingCtx, acceptingClient := NewPeeringCluster(t, "dc1", 3, consulVersion, true) acceptingCluster, acceptingCtx, acceptingClient := NewCluster(t, &ClusterConfig{
acceptingCluster, acceptingCtx, acceptingClient := NewPeeringCluster(t, 3, 1, &libcluster.BuildOptions{ NumServers: 3,
Datacenter: "dc1", NumClients: 1,
ConsulVersion: consulVersion, BuildOpts: &libcluster.BuildOptions{
InjectAutoEncryption: true, Datacenter: "dc1",
ConsulVersion: consulVersion,
InjectAutoEncryption: true,
},
ApplyDefaultProxySettings: true,
}) })
dialingCluster, dialingCtx, dialingClient := NewPeeringCluster(t, 1, 1, &libcluster.BuildOptions{
Datacenter: "dc2", dialingCluster, dialingCtx, dialingClient := NewCluster(t, &ClusterConfig{
ConsulVersion: consulVersion, NumServers: 1,
InjectAutoEncryption: true, NumClients: 1,
BuildOpts: &libcluster.BuildOptions{
Datacenter: "dc2",
ConsulVersion: consulVersion,
InjectAutoEncryption: true,
},
ApplyDefaultProxySettings: true,
}) })
// Create the mesh gateway for dataplane traffic and peering control plane traffic (if enabled) // Create the mesh gateway for dataplane traffic and peering control plane traffic (if enabled)
@ -151,92 +162,68 @@ func BasicPeeringTwoClustersSetup(
} }
} }
// NewDialingCluster creates a cluster for peering with a single dev agent type ClusterConfig struct {
// TODO: note: formerly called CreatingPeeringClusterAndSetup NumServers int
// NumClients int
// Deprecated: use NewPeeringCluster mostly ApplyDefaultProxySettings bool
func NewDialingCluster( BuildOpts *libcluster.BuildOptions
t *testing.T, Cmd string
version string, LogConsumer *TestLogConsumer
dialingPeerName string, Ports []int
) (*libcluster.Cluster, *api.Client, libservice.Service) {
t.Helper()
t.Logf("creating the dialing cluster")
opts := libcluster.BuildOptions{
Datacenter: "dc2",
InjectAutoEncryption: true,
InjectGossipEncryption: true,
AllowHTTPAnyway: true,
ConsulVersion: version,
}
ctx := libcluster.NewBuildContext(t, opts)
conf := libcluster.NewConfigBuilder(ctx).
Peering(true).
ToAgentConfig(t)
t.Logf("dc2 server config: \n%s", conf.JSON)
cluster, err := libcluster.NewN(t, *conf, 1)
require.NoError(t, err)
node := cluster.Agents[0]
client := node.GetClient()
libcluster.WaitForLeader(t, cluster, client)
libcluster.WaitForMembers(t, client, 1)
// Default Proxy Settings
ok, err := utils.ApplyDefaultProxySettings(client)
require.NoError(t, err)
require.True(t, ok)
// Create the mesh gateway for dataplane traffic
_, err = libservice.NewGatewayService(context.Background(), "mesh", "mesh", node)
require.NoError(t, err)
// Create a service and proxy instance
clientProxyService, err := libservice.CreateAndRegisterStaticClientSidecar(node, dialingPeerName, true)
require.NoError(t, err)
libassert.CatalogServiceExists(t, client, "static-client-sidecar-proxy", nil)
return cluster, client, clientProxyService
} }
// NewPeeringCluster creates a cluster with peering enabled. It also creates // NewCluster creates a cluster with peering enabled. It also creates
// and registers a mesh-gateway at the client agent. The API client returned is // and registers a mesh-gateway at the client agent. The API client returned is
// pointed at the client agent. // pointed at the client agent.
// - proxy-defaults.protocol = tcp // - proxy-defaults.protocol = tcp
func NewPeeringCluster( func NewCluster(
t *testing.T, t *testing.T,
numServers int, config *ClusterConfig,
numClients int,
buildOpts *libcluster.BuildOptions,
) (*libcluster.Cluster, *libcluster.BuildContext, *api.Client) { ) (*libcluster.Cluster, *libcluster.BuildContext, *api.Client) {
require.NotEmpty(t, buildOpts.Datacenter) var (
require.True(t, numServers > 0) cluster *libcluster.Cluster
err error
)
require.NotEmpty(t, config.BuildOpts.Datacenter)
require.True(t, config.NumServers > 0)
opts := libcluster.BuildOptions{ opts := libcluster.BuildOptions{
Datacenter: buildOpts.Datacenter, Datacenter: config.BuildOpts.Datacenter,
InjectAutoEncryption: buildOpts.InjectAutoEncryption, InjectAutoEncryption: config.BuildOpts.InjectAutoEncryption,
InjectGossipEncryption: true, InjectGossipEncryption: true,
AllowHTTPAnyway: true, AllowHTTPAnyway: true,
ConsulVersion: buildOpts.ConsulVersion, ConsulVersion: config.BuildOpts.ConsulVersion,
ACLEnabled: buildOpts.ACLEnabled, ACLEnabled: config.BuildOpts.ACLEnabled,
} }
ctx := libcluster.NewBuildContext(t, opts) ctx := libcluster.NewBuildContext(t, opts)
serverConf := libcluster.NewConfigBuilder(ctx). serverConf := libcluster.NewConfigBuilder(ctx).
Bootstrap(numServers). Bootstrap(config.NumServers).
Peering(true). Peering(true).
ToAgentConfig(t) ToAgentConfig(t)
t.Logf("%s server config: \n%s", opts.Datacenter, serverConf.JSON) t.Logf("%s server config: \n%s", opts.Datacenter, serverConf.JSON)
cluster, err := libcluster.NewN(t, *serverConf, numServers) // optional
if config.LogConsumer != nil {
serverConf.LogConsumer = config.LogConsumer
}
t.Logf("Cluster config:\n%s", serverConf.JSON)
// optional custom cmd
if config.Cmd != "" {
serverConf.Cmd = append(serverConf.Cmd, config.Cmd)
}
if config.Ports != nil {
cluster, err = libcluster.New(t, []libcluster.Config{*serverConf}, config.Ports...)
} else {
cluster, err = libcluster.NewN(t, *serverConf, config.NumServers)
}
require.NoError(t, err) require.NoError(t, err)
var retryJoin []string var retryJoin []string
for i := 0; i < numServers; i++ { for i := 0; i < config.NumServers; i++ {
retryJoin = append(retryJoin, fmt.Sprintf("agent-%d", i)) retryJoin = append(retryJoin, fmt.Sprintf("agent-%d", i))
} }
@ -248,18 +235,33 @@ func NewPeeringCluster(
clientConf := configbuiilder.ToAgentConfig(t) clientConf := configbuiilder.ToAgentConfig(t)
t.Logf("%s client config: \n%s", opts.Datacenter, clientConf.JSON) t.Logf("%s client config: \n%s", opts.Datacenter, clientConf.JSON)
require.NoError(t, cluster.AddN(*clientConf, numClients, true)) require.NoError(t, cluster.AddN(*clientConf, config.NumClients, true))
// Use the client agent as the HTTP endpoint since we will not rotate it in many tests. // Use the client agent as the HTTP endpoint since we will not rotate it in many tests.
clientNode := cluster.Agents[numServers] var client *api.Client
client := clientNode.GetClient() if config.NumClients > 0 {
clientNode := cluster.Agents[config.NumServers]
client = clientNode.GetClient()
} else {
client = cluster.Agents[0].GetClient()
}
libcluster.WaitForLeader(t, cluster, client) libcluster.WaitForLeader(t, cluster, client)
libcluster.WaitForMembers(t, client, numServers+numClients) libcluster.WaitForMembers(t, client, config.NumServers+config.NumClients)
// Default Proxy Settings // Default Proxy Settings
ok, err := utils.ApplyDefaultProxySettings(client) if config.ApplyDefaultProxySettings {
require.NoError(t, err) ok, err := utils.ApplyDefaultProxySettings(client)
require.True(t, ok) require.NoError(t, err)
require.True(t, ok)
}
return cluster, ctx, client return cluster, ctx, client
} }
type TestLogConsumer struct {
Msgs []string
}
func (g *TestLogConsumer) Accept(l testcontainers.Log) {
g.Msgs = append(g.Msgs, string(l.Content))
}

View File

@ -24,14 +24,18 @@ import (
func TestBasicConnectService(t *testing.T) { func TestBasicConnectService(t *testing.T) {
t.Parallel() t.Parallel()
buildOptions := &libcluster.BuildOptions{ cluster, _, _ := topology.NewCluster(t, &topology.ClusterConfig{
Datacenter: "dc1", NumServers: 1,
InjectAutoEncryption: true, NumClients: 1,
InjectGossipEncryption: true, ApplyDefaultProxySettings: true,
// TODO(rb): fix the test to not need the service/envoy stack to use :8500 BuildOpts: &libcluster.BuildOptions{
AllowHTTPAnyway: true, Datacenter: "dc1",
} InjectAutoEncryption: true,
cluster, _, _ := topology.NewPeeringCluster(t, 1, 1, buildOptions) InjectGossipEncryption: true,
// TODO(rb): fix the test to not need the service/envoy stack to use :8500
AllowHTTPAnyway: true,
},
})
clientService := createServices(t, cluster) clientService := createServices(t, cluster)
_, port := clientService.GetAddr() _, port := clientService.GetAddr()

View File

@ -1,70 +0,0 @@
package test
import (
"testing"
libcluster "github.com/hashicorp/consul/test/integration/consul-container/libs/cluster"
"github.com/hashicorp/consul/test/integration/consul-container/libs/utils"
"github.com/stretchr/testify/require"
"github.com/testcontainers/testcontainers-go"
)
type TestLogConsumer struct {
Msgs []string
}
func (g *TestLogConsumer) Accept(l testcontainers.Log) {
g.Msgs = append(g.Msgs, string(l.Content))
}
// Creates a cluster with options for basic customization. All args except t
// are optional and will use sensible defaults when not provided.
func CreateCluster(
t *testing.T,
cmd string,
logConsumer *TestLogConsumer,
buildOptions *libcluster.BuildOptions,
applyDefaultProxySettings bool,
ports ...int,
) *libcluster.Cluster {
// optional
if buildOptions == nil {
buildOptions = &libcluster.BuildOptions{
InjectAutoEncryption: true,
InjectGossipEncryption: true,
}
}
ctx := libcluster.NewBuildContext(t, *buildOptions)
conf := libcluster.NewConfigBuilder(ctx).ToAgentConfig(t)
// optional
if logConsumer != nil {
conf.LogConsumer = logConsumer
}
t.Logf("Cluster config:\n%s", conf.JSON)
// optional custom cmd
if cmd != "" {
conf.Cmd = append(conf.Cmd, cmd)
}
cluster, err := libcluster.New(t, []libcluster.Config{*conf}, ports...)
require.NoError(t, err)
node := cluster.Agents[0]
client := node.GetClient()
libcluster.WaitForLeader(t, cluster, client)
libcluster.WaitForMembers(t, client, 1)
if applyDefaultProxySettings {
ok, err := utils.ApplyDefaultProxySettings(client)
require.NoError(t, err)
require.True(t, ok)
}
return cluster
}

View File

@ -14,8 +14,8 @@ import (
libassert "github.com/hashicorp/consul/test/integration/consul-container/libs/assert" libassert "github.com/hashicorp/consul/test/integration/consul-container/libs/assert"
libcluster "github.com/hashicorp/consul/test/integration/consul-container/libs/cluster" libcluster "github.com/hashicorp/consul/test/integration/consul-container/libs/cluster"
libservice "github.com/hashicorp/consul/test/integration/consul-container/libs/service" libservice "github.com/hashicorp/consul/test/integration/consul-container/libs/service"
libtopology "github.com/hashicorp/consul/test/integration/consul-container/libs/topology"
"github.com/hashicorp/consul/test/integration/consul-container/libs/utils" "github.com/hashicorp/consul/test/integration/consul-container/libs/utils"
"github.com/hashicorp/consul/test/integration/consul-container/test"
"github.com/hashicorp/go-cleanhttp" "github.com/hashicorp/go-cleanhttp"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
@ -31,13 +31,20 @@ func TestAPIGatewayCreate(t *testing.T) {
t.Parallel() t.Parallel()
listenerPortOne := 6000 listenerPortOne := 6000
buildOpts := &libcluster.BuildOptions{ clusterConfig := &libtopology.ClusterConfig{
Datacenter: "dc1", NumServers: 1,
InjectAutoEncryption: true, NumClients: 1,
InjectGossipEncryption: true, BuildOpts: &libcluster.BuildOptions{
AllowHTTPAnyway: true, Datacenter: "dc1",
InjectAutoEncryption: true,
InjectGossipEncryption: true,
AllowHTTPAnyway: true,
},
Ports: []int{listenerPortOne},
ApplyDefaultProxySettings: true,
} }
cluster := test.CreateCluster(t, "", nil, buildOpts, true, listenerPortOne)
cluster, _, _ := libtopology.NewCluster(t, clusterConfig)
client := cluster.APIClient(0) client := cluster.APIClient(0)
// add api gateway config // add api gateway config

View File

@ -5,16 +5,15 @@ import (
"crypto/rand" "crypto/rand"
"encoding/hex" "encoding/hex"
"fmt" "fmt"
"testing"
"time"
"github.com/hashicorp/consul/api" "github.com/hashicorp/consul/api"
libassert "github.com/hashicorp/consul/test/integration/consul-container/libs/assert" libassert "github.com/hashicorp/consul/test/integration/consul-container/libs/assert"
libcluster "github.com/hashicorp/consul/test/integration/consul-container/libs/cluster" libcluster "github.com/hashicorp/consul/test/integration/consul-container/libs/cluster"
libservice "github.com/hashicorp/consul/test/integration/consul-container/libs/service" libservice "github.com/hashicorp/consul/test/integration/consul-container/libs/service"
"github.com/hashicorp/consul/test/integration/consul-container/test" libtopology "github.com/hashicorp/consul/test/integration/consul-container/libs/topology"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"testing"
"time"
) )
func getNamespace() string { func getNamespace() string {
@ -43,14 +42,21 @@ func TestHTTPRouteFlattening(t *testing.T) {
//infrastructure set up //infrastructure set up
listenerPort := 6000 listenerPort := 6000
//create cluster
buildOpts := &libcluster.BuildOptions{ clusterConfig := &libtopology.ClusterConfig{
Datacenter: "dc1", NumServers: 1,
InjectAutoEncryption: true, NumClients: 1,
InjectGossipEncryption: true, BuildOpts: &libcluster.BuildOptions{
AllowHTTPAnyway: true, Datacenter: "dc1",
InjectAutoEncryption: true,
InjectGossipEncryption: true,
AllowHTTPAnyway: true,
},
Ports: []int{listenerPort},
ApplyDefaultProxySettings: true,
} }
cluster := test.CreateCluster(t, "", nil, buildOpts, true, listenerPort)
cluster, _, _ := libtopology.NewCluster(t, clusterConfig)
client := cluster.Agents[0].GetClient() client := cluster.Agents[0].GetClient()
service1ResponseCode := 200 service1ResponseCode := 200

View File

@ -45,9 +45,14 @@ func TestAccessLogs(t *testing.T) {
t.Skip() t.Skip()
} }
cluster, _, _ := topology.NewPeeringCluster(t, 1, 1, &libcluster.BuildOptions{ cluster, _, _ := topology.NewCluster(t, &topology.ClusterConfig{
Datacenter: "dc1", NumServers: 1,
InjectAutoEncryption: true, NumClients: 1,
ApplyDefaultProxySettings: true,
BuildOpts: &libcluster.BuildOptions{
Datacenter: "dc1",
InjectAutoEncryption: true,
},
}) })
// Turn on access logs. Do this before starting the sidecars so that they inherit the configuration // Turn on access logs. Do this before starting the sidecars so that they inherit the configuration

View File

@ -11,7 +11,7 @@ import (
"github.com/hashicorp/consul/api" "github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/sdk/testutil/retry" "github.com/hashicorp/consul/sdk/testutil/retry"
libcluster "github.com/hashicorp/consul/test/integration/consul-container/libs/cluster" libcluster "github.com/hashicorp/consul/test/integration/consul-container/libs/cluster"
"github.com/hashicorp/consul/test/integration/consul-container/test" libtopology "github.com/hashicorp/consul/test/integration/consul-container/libs/topology"
) )
const ( const (
@ -128,8 +128,20 @@ func TestServerRequestRateLimit(t *testing.T) {
for _, tc := range testCases { for _, tc := range testCases {
t.Run(tc.description, func(t *testing.T) { t.Run(tc.description, func(t *testing.T) {
logConsumer := &test.TestLogConsumer{} clusterConfig := &libtopology.ClusterConfig{
cluster := test.CreateCluster(t, tc.cmd, logConsumer, nil, false) NumServers: 1,
NumClients: 0,
Cmd: tc.cmd,
LogConsumer: &libtopology.TestLogConsumer{},
BuildOpts: &libcluster.BuildOptions{
Datacenter: "dc1",
InjectAutoEncryption: true,
InjectGossipEncryption: true,
},
ApplyDefaultProxySettings: false,
}
cluster, _, _ := libtopology.NewCluster(t, clusterConfig)
defer terminate(t, cluster) defer terminate(t, cluster)
client, err := cluster.GetClient(nil, true) client, err := cluster.GetClient(nil, true)
@ -137,7 +149,7 @@ func TestServerRequestRateLimit(t *testing.T) {
// perform actions and validate returned errors to client // perform actions and validate returned errors to client
for _, op := range tc.operations { for _, op := range tc.operations {
err = op.action.function(client) err := op.action.function(client)
if len(op.expectedErrorMsg) > 0 { if len(op.expectedErrorMsg) > 0 {
require.Error(t, err) require.Error(t, err)
require.Equal(t, op.expectedErrorMsg, err.Error()) require.Equal(t, op.expectedErrorMsg, err.Error())
@ -165,7 +177,7 @@ func TestServerRequestRateLimit(t *testing.T) {
// validate logs // validate logs
// putting this last as there are cases where logs // putting this last as there are cases where logs
// were not present in consumer when assertion was made. // were not present in consumer when assertion was made.
checkLogsForMessage(r, logConsumer.Msgs, checkLogsForMessage(r, clusterConfig.LogConsumer.Msgs,
fmt.Sprintf("[DEBUG] agent.server.rpc-rate-limit: RPC exceeded allowed rate limit: rpc=%s", op.action.rateLimitOperation), fmt.Sprintf("[DEBUG] agent.server.rpc-rate-limit: RPC exceeded allowed rate limit: rpc=%s", op.action.rateLimitOperation),
op.action.rateLimitOperation, "exceeded", op.expectExceededLog) op.action.rateLimitOperation, "exceeded", op.expectExceededLog)

View File

@ -18,9 +18,14 @@ import (
func TestTroubleshootProxy(t *testing.T) { func TestTroubleshootProxy(t *testing.T) {
t.Parallel() t.Parallel()
cluster, _, _ := topology.NewPeeringCluster(t, 1, 1, &libcluster.BuildOptions{ cluster, _, _ := topology.NewCluster(t, &topology.ClusterConfig{
Datacenter: "dc1", NumServers: 1,
InjectAutoEncryption: true, NumClients: 1,
BuildOpts: &libcluster.BuildOptions{
Datacenter: "dc1",
InjectAutoEncryption: true,
},
ApplyDefaultProxySettings: true,
}) })
serverService, clientService := topology.CreateServices(t, cluster) serverService, clientService := topology.CreateServices(t, cluster)

View File

@ -36,11 +36,16 @@ func TestACL_Upgrade_Node_Token(t *testing.T) {
run := func(t *testing.T, tc testcase) { run := func(t *testing.T, tc testcase) {
// NOTE: Disable auto.encrypt due to its conflict with ACL token during bootstrap // NOTE: Disable auto.encrypt due to its conflict with ACL token during bootstrap
cluster, _, _ := libtopology.NewPeeringCluster(t, 1, 1, &libcluster.BuildOptions{ cluster, _, _ := libtopology.NewCluster(t, &libtopology.ClusterConfig{
Datacenter: "dc1", NumServers: 1,
ConsulVersion: tc.oldversion, NumClients: 1,
InjectAutoEncryption: false, BuildOpts: &libcluster.BuildOptions{
ACLEnabled: true, Datacenter: "dc1",
ConsulVersion: tc.oldversion,
InjectAutoEncryption: false,
ACLEnabled: true,
},
ApplyDefaultProxySettings: true,
}) })
agentToken, err := cluster.CreateAgentToken("dc1", agentToken, err := cluster.CreateAgentToken("dc1",

View File

@ -1,6 +1,7 @@
package upgrade package upgrade
import ( import (
"context"
"testing" "testing"
"time" "time"
@ -9,6 +10,7 @@ import (
"github.com/hashicorp/consul/api" "github.com/hashicorp/consul/api"
libcluster "github.com/hashicorp/consul/test/integration/consul-container/libs/cluster" libcluster "github.com/hashicorp/consul/test/integration/consul-container/libs/cluster"
libservice "github.com/hashicorp/consul/test/integration/consul-container/libs/service" libservice "github.com/hashicorp/consul/test/integration/consul-container/libs/service"
libtopology "github.com/hashicorp/consul/test/integration/consul-container/libs/topology"
"github.com/hashicorp/consul/test/integration/consul-container/libs/utils" "github.com/hashicorp/consul/test/integration/consul-container/libs/utils"
) )
@ -21,9 +23,22 @@ func TestTargetServersWithLatestGAClients(t *testing.T) {
numClients = 1 numClients = 1
) )
cluster := serversCluster(t, numServers, utils.TargetImageName, utils.TargetVersion) clusterConfig := &libtopology.ClusterConfig{
NumServers: numServers,
NumClients: numClients,
BuildOpts: &libcluster.BuildOptions{
Datacenter: "dc1",
ConsulVersion: utils.TargetVersion,
},
ApplyDefaultProxySettings: true,
}
libservice.ClientsCreate(t, numClients, utils.LatestImageName, utils.LatestVersion, cluster) cluster, _, _ := libtopology.NewCluster(t, clusterConfig)
// change the version of Client agent to latest version
config := cluster.Agents[3].GetConfig()
config.Version = utils.LatestVersion
cluster.Agents[3].Upgrade(context.Background(), config)
client := cluster.APIClient(0) client := cluster.APIClient(0)

View File

@ -49,10 +49,15 @@ func TestIngressGateway_UpgradeToTarget_fromLatest(t *testing.T) {
run := func(t *testing.T, oldVersion, targetVersion string) { run := func(t *testing.T, oldVersion, targetVersion string) {
// setup // setup
// TODO? we don't need a peering cluster, so maybe this is overkill // TODO? we don't need a peering cluster, so maybe this is overkill
cluster, _, client := topology.NewPeeringCluster(t, 1, 2, &libcluster.BuildOptions{ cluster, _, client := topology.NewCluster(t, &topology.ClusterConfig{
Datacenter: "dc1", NumServers: 1,
ConsulVersion: oldVersion, NumClients: 2,
// TODO? InjectAutoEncryption: true, BuildOpts: &libcluster.BuildOptions{
Datacenter: "dc1",
ConsulVersion: oldVersion,
// TODO? InjectAutoEncryption: true,
},
ApplyDefaultProxySettings: true,
}) })
// upsert config entry making http default protocol for global // upsert config entry making http default protocol for global

View File

@ -335,7 +335,12 @@ func TestTrafficManagement_ServiceResolver(t *testing.T) {
if oldVersionTmp.LessThan(libutils.Version_1_14) { if oldVersionTmp.LessThan(libutils.Version_1_14) {
buildOpts.InjectAutoEncryption = false buildOpts.InjectAutoEncryption = false
} }
cluster, _, _ := topology.NewPeeringCluster(t, 1, 1, buildOpts) cluster, _, _ := topology.NewCluster(t, &topology.ClusterConfig{
NumServers: 1,
NumClients: 1,
BuildOpts: buildOpts,
ApplyDefaultProxySettings: true,
})
node := cluster.Agents[0] node := cluster.Agents[0]
client := node.GetClient() client := node.GetClient()