Skip to content

Commit 7c3d18a

Browse files
committed
roachtest: create secure clusters by default
In order to run on secure clusters, many tests were changed to explicity specify a pgurl or certs directory to authenticate with. Currently most tests authenticate with the root user, but in the future we want to use a non root user when possible. This change also fixes roachtests to use the new defaultHTTPClient helper to disable cert verification and automatically retrieve and use sessionID for auth. Release note: None
1 parent 7c6c6e4 commit 7c3d18a

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

85 files changed

+423
-465
lines changed

pkg/cmd/roachtest/cluster.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1428,7 +1428,7 @@ func (c *clusterImpl) HealthStatus(
14281428
return nil, errors.WithDetail(err, "Unable to get admin UI address(es)")
14291429
}
14301430
getStatus := func(ctx context.Context, node int) *HealthStatusResult {
1431-
url := fmt.Sprintf(`http://%s/health?ready=1`, adminAddrs[node-1])
1431+
url := fmt.Sprintf(`https://%s/health?ready=1`, adminAddrs[node-1])
14321432
resp, err := httputil.Get(ctx, url)
14331433
if err != nil {
14341434
return newHealthStatusResult(node, 0, nil, err)

pkg/cmd/roachtest/clusterstats/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,6 @@ go_library(
1515
deps = [
1616
"//pkg/cmd/roachtest/cluster",
1717
"//pkg/cmd/roachtest/option",
18-
"//pkg/cmd/roachtest/roachtestutil",
1918
"//pkg/cmd/roachtest/test",
2019
"//pkg/roachprod/logger",
2120
"//pkg/roachprod/prometheus",

pkg/cmd/roachtest/roachtestutil/BUILD.bazel

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,7 @@ go_library(
66
"commandbuilder.go",
77
"disk_usage.go",
88
"health_checker.go",
9+
"httpclient.go",
910
"jaeger.go",
1011
"utils.go",
1112
"validation_check.go",
@@ -25,9 +26,10 @@ go_library(
2526
"//pkg/util/httputil",
2627
"//pkg/util/humanizeutil",
2728
"//pkg/util/log",
29+
"//pkg/util/protoutil",
30+
"//pkg/util/syncutil",
2831
"//pkg/util/timeutil",
2932
"@com_github_cockroachdb_errors//:errors",
30-
"@com_github_pkg_errors//:errors",
3133
],
3234
)
3335

pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -141,9 +141,7 @@ var (
141141
// defaultClusterSettings is the set of cluster settings always
142142
// passed to `clusterupgrade.StartWithSettings` when (re)starting
143143
// nodes in a cluster.
144-
defaultClusterSettings = []install.ClusterSettingOption{
145-
install.SecureOption(true),
146-
}
144+
defaultClusterSettings = []install.ClusterSettingOption{}
147145

148146
// minSupportedARM64Version is the minimum version for which there
149147
// is a published ARM64 build. If we are running a mixedversion test

pkg/cmd/roachtest/roachtestutil/utils.go

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -30,9 +30,13 @@ func SystemInterfaceSystemdUnitName() string {
3030
// DefaultPGUrl is a wrapper over roachprod.PgUrl that calls it with the arguments
3131
// that *almost* all roachtests want: single tenant and only a single node.
3232
func DefaultPGUrl(
33-
ctx context.Context, c cluster.Cluster, l *logger.Logger, node option.NodeListOption,
33+
ctx context.Context,
34+
c cluster.Cluster,
35+
l *logger.Logger,
36+
node option.NodeListOption,
37+
auth install.PGAuthMode,
3438
) (string, error) {
35-
opts := roachprod.PGURLOptions{Secure: c.IsSecure()}
39+
opts := roachprod.PGURLOptions{Auth: auth, Secure: c.IsSecure()}
3640
pgurl, err := roachprod.PgURL(ctx, l, c.MakeNodes(node), "certs", opts)
3741
if err != nil {
3842
return "", err

pkg/cmd/roachtest/tests/activerecord.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,8 @@ func registerActiveRecord(r registry.Registry) {
5151
t.Status("setting up cockroach")
5252
startOpts := option.DefaultStartOptsInMemory()
5353
startOpts.RoachprodOpts.SQLPort = config.DefaultSQLPort
54-
c.Start(ctx, t.L(), startOpts, install.MakeClusterSettings(), c.All())
54+
// Activerecord uses root user with ssl disabled.
55+
c.Start(ctx, t.L(), startOpts, install.MakeClusterSettings(install.SecureOption(false)), c.All())
5556

5657
version, err := fetchCockroachVersion(ctx, t.L(), c, node[0])
5758
if err != nil {

pkg/cmd/roachtest/tests/admission_control_elastic_io.go

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -65,17 +65,16 @@ func registerElasticIO(r registry.Registry) {
6565
WithGrafanaDashboardJSON(grafana.ChangefeedAdmissionControlGrafana)
6666
err := c.StartGrafana(ctx, t.L(), promCfg)
6767
require.NoError(t, err)
68-
promClient, err := clusterstats.SetupCollectorPromClient(ctx, c, t.L(), promCfg)
69-
require.NoError(t, err)
70-
statCollector := clusterstats.NewStatsCollector(ctx, promClient)
71-
7268
c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(workAndPromNode))
7369
startOpts := option.DefaultStartOptsNoBackups()
7470
roachtestutil.SetDefaultAdminUIPort(c, &startOpts.RoachprodOpts)
7571
startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs,
7672
"--vmodule=io_load_listener=2")
7773
settings := install.MakeClusterSettings()
7874
c.Start(ctx, t.L(), startOpts, settings, c.Range(1, crdbNodes))
75+
promClient, err := clusterstats.SetupCollectorPromClient(ctx, c, t.L(), promCfg)
76+
require.NoError(t, err)
77+
statCollector := clusterstats.NewStatsCollector(ctx, promClient)
7978
setAdmissionControl(ctx, t, c, true)
8079
duration := 30 * time.Minute
8180
t.Status("running workload")
@@ -85,7 +84,7 @@ func registerElasticIO(r registry.Registry) {
8584
url := fmt.Sprintf(" {pgurl:1-%d}", crdbNodes)
8685
cmd := "./workload run kv --init --histograms=perf/stats.json --concurrency=512 " +
8786
"--splits=1000 --read-percent=0 --min-block-bytes=65536 --max-block-bytes=65536 " +
88-
"--txn-qos=background --tolerate-errors" + dur + url
87+
"--txn-qos=background --tolerate-errors --secure" + dur + url
8988
c.Run(ctx, option.WithNodes(c.Node(workAndPromNode)), cmd)
9089
return nil
9190
})

pkg/cmd/roachtest/tests/admission_control_intent_resolution.go

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -64,17 +64,18 @@ func registerIntentResolutionOverload(r registry.Registry) {
6464
WithGrafanaDashboardJSON(grafana.ChangefeedAdmissionControlGrafana)
6565
err := c.StartGrafana(ctx, t.L(), promCfg)
6666
require.NoError(t, err)
67-
promClient, err := clusterstats.SetupCollectorPromClient(ctx, c, t.L(), promCfg)
68-
require.NoError(t, err)
69-
statCollector := clusterstats.NewStatsCollector(ctx, promClient)
7067

7168
startOpts := option.DefaultStartOptsNoBackups()
7269
startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs,
7370
"--vmodule=io_load_listener=2")
74-
roachtestutil.SetDefaultSQLPort(c, &startOpts.RoachprodOpts)
7571
roachtestutil.SetDefaultAdminUIPort(c, &startOpts.RoachprodOpts)
7672
settings := install.MakeClusterSettings()
7773
c.Start(ctx, t.L(), startOpts, settings, c.Range(1, crdbNodes))
74+
75+
promClient, err := clusterstats.SetupCollectorPromClient(ctx, c, t.L(), promCfg)
76+
require.NoError(t, err)
77+
statCollector := clusterstats.NewStatsCollector(ctx, promClient)
78+
7879
setAdmissionControl(ctx, t, c, true)
7980
t.Status("running txn")
8081
m := c.NewMonitor(ctx, c.Range(1, crdbNodes))

pkg/cmd/roachtest/tests/admission_control_multitenant_fairness.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -145,7 +145,7 @@ func runMultiTenantFairness(
145145
t.L().Printf("starting cockroach securely (<%s)", time.Minute)
146146
c.Start(ctx, t.L(),
147147
option.DefaultStartOptsNoBackups(),
148-
install.MakeClusterSettings(install.SecureOption(true)),
148+
install.MakeClusterSettings(),
149149
crdbNode,
150150
)
151151

pkg/cmd/roachtest/tests/admission_control_tpcc_overload.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -124,7 +124,7 @@ func verifyNodeLiveness(
124124
if err := retry.WithMaxAttempts(ctx, retry.Options{
125125
MaxBackoff: 500 * time.Millisecond,
126126
}, 60, func() (err error) {
127-
response, err = getMetrics(ctx, adminURLs[0], now.Add(-runDuration), now, []tsQuery{
127+
response, err = getMetrics(ctx, c, t, adminURLs[0], now.Add(-runDuration), now, []tsQuery{
128128
{
129129
name: "cr.node.liveness.heartbeatfailures",
130130
queryType: total,

pkg/cmd/roachtest/tests/allocator.go

Lines changed: 4 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@ import (
2121
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/clusterstats"
2222
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
2323
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
24-
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
2524
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
2625
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
2726
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
@@ -48,17 +47,12 @@ func registerAllocator(r registry.Registry) {
4847
db := c.Conn(ctx, t.L(), 1)
4948
defer db.Close()
5049

51-
pgurl, err := roachtestutil.DefaultPGUrl(ctx, c, t.L(), c.Nodes(1))
52-
if err != nil {
53-
t.Fatal(err)
54-
}
55-
5650
m := c.NewMonitor(ctx, c.Range(1, start))
5751
m.Go(func(ctx context.Context) error {
5852
t.Status("loading fixture")
5953
if err := c.RunE(
6054
ctx, option.WithNodes(c.Node(1)),
61-
"./cockroach", "workload", "fixtures", "import", "tpch", "--scale-factor", "10", pgurl,
55+
"./cockroach", "workload", "fixtures", "import", "tpch", "--scale-factor", "10", "{pgurl:1}",
6256
); err != nil {
6357
t.Fatal(err)
6458
}
@@ -73,7 +67,7 @@ func registerAllocator(r registry.Registry) {
7367
WithCluster(clusNodes.InstallNodes()).
7468
WithPrometheusNode(promNode.InstallNodes()[0])
7569

76-
err = c.StartGrafana(ctx, t.L(), cfg)
70+
err := c.StartGrafana(ctx, t.L(), cfg)
7771
require.NoError(t, err)
7872

7973
cleanupFunc := func() {
@@ -91,7 +85,7 @@ func registerAllocator(r registry.Registry) {
9185

9286
// Start the remaining nodes to kick off upreplication/rebalancing.
9387
c.Start(ctx, t.L(), startOpts, install.MakeClusterSettings(), c.Range(start+1, nodes))
94-
c.Run(ctx, option.WithNodes(c.Node(1)), fmt.Sprintf("./cockroach workload init kv --drop '%s'", pgurl))
88+
c.Run(ctx, option.WithNodes(c.Node(1)), "./cockroach workload init kv --drop {pgurl:1}")
9589
for node := 1; node <= nodes; node++ {
9690
node := node
9791
// TODO(dan): Ideally, the test would fail if this queryload failed,
@@ -457,13 +451,9 @@ FROM crdb_internal.kv_store_status
457451
t.Fatalf("expected 0 mis-replicated ranges, but found %d", n)
458452
}
459453

460-
pgurl, err := roachtestutil.DefaultPGUrl(ctx, c, t.L(), c.Nodes(1))
461-
if err != nil {
462-
t.Fatal(err)
463-
}
464454
decom := func(id int) {
465455
c.Run(ctx, option.WithNodes(c.Node(1)),
466-
fmt.Sprintf("./cockroach node decommission --insecure --url=%s --wait=none %d", pgurl, id))
456+
fmt.Sprintf("./cockroach node decommission --certs-dir=certs --port={pgport%s} --wait=none %d", c.Node(id), id))
467457
}
468458

469459
// Decommission a node. The ranges should down-replicate to 7 replicas.

pkg/cmd/roachtest/tests/alterpk.go

Lines changed: 1 addition & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@ import (
1818
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
1919
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
2020
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
21-
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
2221
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
2322
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
2423
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
@@ -105,14 +104,9 @@ func registerAlterPK(r registry.Registry) {
105104
const duration = 10 * time.Minute
106105

107106
roachNodes, loadNode := setupTest(ctx, t, c)
108-
pgurl, err := roachtestutil.DefaultPGUrl(ctx, c, t.L(), c.Node(1))
109-
if err != nil {
110-
t.Fatal(err)
111-
}
112107
cmd := fmt.Sprintf(
113-
"./cockroach workload fixtures import tpcc --warehouses=%d --db=tpcc '%s'",
108+
"./cockroach workload fixtures import tpcc --warehouses=%d --db=tpcc {pgurl:1}",
114109
warehouses,
115-
pgurl,
116110
)
117111
if err := c.RunE(ctx, option.WithNodes(c.Node(roachNodes[0])), cmd); err != nil {
118112
t.Fatal(err)

pkg/cmd/roachtest/tests/asyncpg.go

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@ package tests
1212

1313
import (
1414
"context"
15+
"fmt"
1516
"regexp"
1617

1718
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
@@ -22,11 +23,11 @@ import (
2223
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
2324
)
2425

25-
const asyncpgRunTestCmd = `
26-
source venv/bin/activate &&
26+
var asyncpgRunTestCmd = fmt.Sprintf(`
27+
source venv/bin/activate &&
2728
cd /mnt/data1/asyncpg &&
28-
PGPORT={pgport:1} PGHOST=localhost PGUSER=test_admin PGDATABASE=defaultdb python3 setup.py test > asyncpg.stdout
29-
`
29+
PGPORT={pgport:1} PGHOST=localhost PGUSER=%s PGPASSWORD=%s PGSSLROOTCERT=$HOME/certs/ca.crt PGSSLMODE=require PGDATABASE=defaultdb python3 setup.py test > asyncpg.stdout
30+
`, install.DefaultUser, install.DefaultPassword)
3031

3132
var asyncpgReleaseTagRegex = regexp.MustCompile(`^(?P<major>v\d+)\.(?P<minor>\d+)\.(?P<point>\d+)$`)
3233

pkg/cmd/roachtest/tests/asyncpg_blocklist.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,9 @@ var asyncpgBlocklist = blocklist{
3838
`test_codecs.TestCodecs.test_unknown_type_text_fallback`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/54516",
3939
`test_codecs.TestCodecs.test_void`: "unknown",
4040
`test_connect.TestSettings.test_get_settings_01`: "unknown",
41+
`test_copy.TestCopyFrom.test_copy_from_query_cancellation_explicit`: "known asyncpg ssl issue - https://github.com/MagicStack/asyncpg/issues/240",
42+
`test_copy.TestCopyFrom.test_copy_from_query_timeout_1`: "known asyncpg ssl issue - https://github.com/MagicStack/asyncpg/issues/240",
43+
`test_copy.TestCopyFrom.test_copy_from_query_to_sink`: "known asyncpg ssl issue - https://github.com/MagicStack/asyncpg/issues/240",
4144
`test_copy.TestCopyFrom.test_copy_from_table_basics`: "no support for COPY TO - https://github.com/cockroachdb/cockroach/issues/85571",
4245
`test_copy.TestCopyTo.test_copy_to_table_basics`: "unknown",
4346
`test_cursor.TestCursor.test_cursor_02`: "unknown",
@@ -60,15 +63,13 @@ var asyncpgBlocklist = blocklist{
6063
`test_introspection.TestIntrospection.test_introspection_on_large_db`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/22456",
6164
`test_introspection.TestIntrospection.test_introspection_retries_after_cache_bust`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/27796",
6265
`test_introspection.TestIntrospection.test_introspection_sticks_for_ps`: "unknown type: pg_catalog.json",
63-
`test_listeners.TestListeners.test_dangling_listener_warns`: "LISTEN - https://github.com/cockroachdb/cockroach/issues/41522",
6466
`test_listeners.TestListeners.test_listen_01`: "LISTEN - https://github.com/cockroachdb/cockroach/issues/41522",
6567
`test_listeners.TestListeners.test_listen_02`: "LISTEN - https://github.com/cockroachdb/cockroach/issues/41522",
6668
`test_listeners.TestListeners.test_listen_notletters`: "LISTEN - https://github.com/cockroachdb/cockroach/issues/41522",
6769
`test_listeners.TestLogListeners.test_log_listener_01`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511",
6870
`test_listeners.TestLogListeners.test_log_listener_02`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511",
6971
`test_listeners.TestLogListeners.test_log_listener_03`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511",
7072
`test_pool.TestPool.test_pool_remote_close`: "unsupported pg_terminate_backend() function",
71-
`test_prepare.TestPrepare.test_prepare_08_big_result`: "unknown",
7273
`test_prepare.TestPrepare.test_prepare_09_raise_error`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511",
7374
`test_prepare.TestPrepare.test_prepare_14_explain`: "unknown",
7475
`test_prepare.TestPrepare.test_prepare_16_command_result`: "unknown",

pkg/cmd/roachtest/tests/autoupgrade.go

Lines changed: 1 addition & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@ import (
1818
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
1919
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
2020
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
21-
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
2221
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
2322
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
2423
"github.com/cockroachdb/cockroach/pkg/testutils"
@@ -74,12 +73,8 @@ func registerAutoUpgrade(r registry.Registry) {
7473

7574
decommissionAndStop := func(node int) error {
7675
t.WorkerStatus("decommission")
77-
pgurl, err := roachtestutil.DefaultPGUrl(ctx, c, t.L(), c.Node(node))
78-
if err != nil {
79-
return err
80-
}
8176
if err := c.RunE(ctx, option.WithNodes(c.Node(node)),
82-
fmt.Sprintf("./cockroach node decommission %d --insecure --url=%s", node, pgurl)); err != nil {
77+
fmt.Sprintf("./cockroach node decommission %d --certs-dir=certs --port={pgport%s}", node, c.Node(node))); err != nil {
8378
return err
8479
}
8580
t.WorkerStatus("stop")

pkg/cmd/roachtest/tests/awsdms.go

Lines changed: 3 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -766,11 +766,9 @@ func setupDMSEndpointsAndTask(
766766
PostgreSQLSettings: &dmstypes.PostgreSQLSettings{
767767
DatabaseName: proto.String(awsdmsCRDBDatabase),
768768
Username: proto.String(awsdmsCRDBUser),
769-
// Password is a required field, but CockroachDB doesn't take passwords in
770-
// --insecure mode. As such, put in some garbage.
771-
Password: proto.String("garbage"),
772-
Port: proto.Int32(26257),
773-
ServerName: proto.String(externalCRDBAddr[0]),
769+
Password: proto.String(awsdmsPassword),
770+
Port: proto.Int32(26257),
771+
ServerName: proto.String(externalCRDBAddr[0]),
774772
},
775773
},
776774
endpoint: dmsEndpoints.defaultTarget,

pkg/cmd/roachtest/tests/backup.go

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -335,15 +335,11 @@ func registerBackup(r registry.Registry) {
335335
m := c.NewMonitor(ctx)
336336
m.Go(func(ctx context.Context) error {
337337
t.Status(`running backup`)
338-
pgurl, err := roachtestutil.DefaultPGUrl(ctx, c, t.L(), c.Node(1))
339-
if err != nil {
340-
return err
341-
}
342338
// Tick once before starting the backup, and once after to capture the
343339
// total elapsed time. This is used by roachperf to compute and display
344340
// the average MB/sec per node.
345341
tick()
346-
c.Run(ctx, option.WithNodes(c.Node(1)), `./cockroach sql --insecure --url=`+pgurl+` -e "
342+
c.Run(ctx, option.WithNodes(c.Node(1)), `./cockroach sql --url={pgurl:1} -e "
347343
BACKUP bank.bank TO 'gs://`+backupTestingBucket+`/`+dest+`?AUTH=implicit'"`)
348344
tick()
349345

pkg/cmd/roachtest/tests/backup_restore_roundtrip.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -110,7 +110,7 @@ func backupRestoreRoundTrip(
110110
"COCKROACH_MIN_RANGE_MAX_BYTES=1",
111111
})
112112

113-
c.Start(ctx, t.L(), maybeUseMemoryBudget(t, 50), install.MakeClusterSettings(install.SecureOption(true), envOption), roachNodes)
113+
c.Start(ctx, t.L(), maybeUseMemoryBudget(t, 50), install.MakeClusterSettings(envOption), roachNodes)
114114
m := c.NewMonitor(ctx, roachNodes)
115115

116116
m.Go(func(ctx context.Context) error {

pkg/cmd/roachtest/tests/build_info.go

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -12,15 +12,14 @@ package tests
1212

1313
import (
1414
"context"
15-
"net/http"
1615
"strings"
1716

1817
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
1918
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
19+
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
2020
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
2121
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
2222
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
23-
"github.com/cockroachdb/cockroach/pkg/util/httputil"
2423
)
2524

2625
// RunBuildInfo is a test that sanity checks the build info.
@@ -32,8 +31,9 @@ func RunBuildInfo(ctx context.Context, t test.Test, c cluster.Cluster) {
3231
if err != nil {
3332
t.Fatal(err)
3433
}
35-
url := `http://` + adminUIAddrs[0] + `/_status/details/local`
36-
err = httputil.GetJSON(http.Client{}, url, &details)
34+
url := `https://` + adminUIAddrs[0] + `/_status/details/local`
35+
client := roachtestutil.DefaultHTTPClient(c, t.L())
36+
err = client.GetJSON(ctx, url, &details)
3737
if err != nil {
3838
t.Fatal(err)
3939
}

0 commit comments

Comments
 (0)