Skip to content

Commit

Permalink
roachtest: create secure clusters by default
Browse files Browse the repository at this point in the history
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
  • Loading branch information
DarrylWong committed Feb 2, 2024
1 parent 0abc774 commit c1ae680
Show file tree
Hide file tree
Showing 67 changed files with 419 additions and 416 deletions.
6 changes: 0 additions & 6 deletions pkg/cmd/roachtest/clusterstats/helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (
"sort"

"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/roachprod/prometheus"
"github.com/cockroachdb/cockroach/pkg/util/search"
Expand All @@ -37,12 +36,7 @@ func SetupCollectorPromClient(
if err != nil {
return nil, err
}
httpclient, err := roachtestutil.DefaultHttpClientWithSessionCookie(ctx, c, l, c.Node(1), fmt.Sprintf("http://%s:9090/api/v1/query", prometheusNodeIP[0]))
if err != nil {
return nil, err
}
client, err := promapi.NewClient(promapi.Config{
Client: &httpclient,
Address: fmt.Sprintf("http://%s:9090", prometheusNodeIP[0]),
})
if err != nil {
Expand Down
8 changes: 6 additions & 2 deletions pkg/cmd/roachtest/roachtestutil/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,9 +39,13 @@ func SystemInterfaceSystemdUnitName() string {
// DefaultPGUrl is a wrapper over roachprod.PgUrl that calls it with the arguments
// that *almost* all roachtests want: single tenant and only a single node.
func DefaultPGUrl(
ctx context.Context, c cluster.Cluster, l *logger.Logger, node option.NodeListOption,
ctx context.Context,
c cluster.Cluster,
l *logger.Logger,
node option.NodeListOption,
auth install.PGAuthMode,
) (string, error) {
opts := roachprod.PGURLOptions{Secure: c.IsSecure()}
opts := roachprod.PGURLOptions{Auth: auth, Secure: c.IsSecure()}
pgurl, err := roachprod.PgURL(ctx, l, c.MakeNodes(node), "certs", opts)
if err != nil {
return "", err
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/roachtest/tests/activerecord.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,8 @@ func registerActiveRecord(r registry.Registry) {
t.Status("setting up cockroach")
startOpts := option.DefaultStartOptsInMemory()
startOpts.RoachprodOpts.SQLPort = config.DefaultSQLPort
c.Start(ctx, t.L(), startOpts, install.MakeClusterSettings(), c.All())
// Activerecord uses root user with ssl disabled.
c.Start(ctx, t.L(), startOpts, install.MakeClusterSettings(install.SecureOption(false)), c.All())

version, err := fetchCockroachVersion(ctx, t.L(), c, node[0])
if err != nil {
Expand Down
9 changes: 4 additions & 5 deletions pkg/cmd/roachtest/tests/admission_control_elastic_io.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,17 +65,16 @@ func registerElasticIO(r registry.Registry) {
WithGrafanaDashboardJSON(grafana.ChangefeedAdmissionControlGrafana)
err := c.StartGrafana(ctx, t.L(), promCfg)
require.NoError(t, err)
promClient, err := clusterstats.SetupCollectorPromClient(ctx, c, t.L(), promCfg)
require.NoError(t, err)
statCollector := clusterstats.NewStatsCollector(ctx, promClient)

c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(workAndPromNode))
startOpts := option.DefaultStartOptsNoBackups()
roachtestutil.SetDefaultAdminUIPort(c, &startOpts.RoachprodOpts)
startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs,
"--vmodule=io_load_listener=2")
settings := install.MakeClusterSettings()
c.Start(ctx, t.L(), startOpts, settings, c.Range(1, crdbNodes))
promClient, err := clusterstats.SetupCollectorPromClient(ctx, c, t.L(), promCfg)
require.NoError(t, err)
statCollector := clusterstats.NewStatsCollector(ctx, promClient)
setAdmissionControl(ctx, t, c, true)
duration := 30 * time.Minute
t.Status("running workload")
Expand All @@ -85,7 +84,7 @@ func registerElasticIO(r registry.Registry) {
url := fmt.Sprintf(" {pgurl:1-%d}", crdbNodes)
cmd := "./workload run kv --init --histograms=perf/stats.json --concurrency=512 " +
"--splits=1000 --read-percent=0 --min-block-bytes=65536 --max-block-bytes=65536 " +
"--txn-qos=background --tolerate-errors" + dur + url
"--txn-qos=background --tolerate-errors --secure" + dur + url
c.Run(ctx, option.WithNodes(c.Node(workAndPromNode)), cmd)
return nil
})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,17 +64,18 @@ func registerIntentResolutionOverload(r registry.Registry) {
WithGrafanaDashboardJSON(grafana.ChangefeedAdmissionControlGrafana)
err := c.StartGrafana(ctx, t.L(), promCfg)
require.NoError(t, err)
promClient, err := clusterstats.SetupCollectorPromClient(ctx, c, t.L(), promCfg)
require.NoError(t, err)
statCollector := clusterstats.NewStatsCollector(ctx, promClient)

startOpts := option.DefaultStartOptsNoBackups()
startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs,
"--vmodule=io_load_listener=2")
roachtestutil.SetDefaultSQLPort(c, &startOpts.RoachprodOpts)
roachtestutil.SetDefaultAdminUIPort(c, &startOpts.RoachprodOpts)
settings := install.MakeClusterSettings()
c.Start(ctx, t.L(), startOpts, settings, c.Range(1, crdbNodes))

promClient, err := clusterstats.SetupCollectorPromClient(ctx, c, t.L(), promCfg)
require.NoError(t, err)
statCollector := clusterstats.NewStatsCollector(ctx, promClient)

setAdmissionControl(ctx, t, c, true)
t.Status("running txn")
m := c.NewMonitor(ctx, c.Range(1, crdbNodes))
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/admission_control_tpcc_overload.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,7 @@ func verifyNodeLiveness(
if err := retry.WithMaxAttempts(ctx, retry.Options{
MaxBackoff: 500 * time.Millisecond,
}, 60, func() (err error) {
response, err = getMetrics(ctx, adminURLs[0], now.Add(-runDuration), now, []tsQuery{
response, err = getMetrics(ctx, c, t, adminURLs[0], now.Add(-runDuration), now, []tsQuery{
{
name: "cr.node.liveness.heartbeatfailures",
queryType: total,
Expand Down
18 changes: 4 additions & 14 deletions pkg/cmd/roachtest/tests/allocator.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/clusterstats"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
Expand All @@ -48,17 +47,12 @@ func registerAllocator(r registry.Registry) {
db := c.Conn(ctx, t.L(), 1)
defer db.Close()

pgurl, err := roachtestutil.DefaultPGUrl(ctx, c, t.L(), c.Nodes(1))
if err != nil {
t.Fatal(err)
}

m := c.NewMonitor(ctx, c.Range(1, start))
m.Go(func(ctx context.Context) error {
t.Status("loading fixture")
if err := c.RunE(
ctx, option.WithNodes(c.Node(1)),
"./cockroach", "workload", "fixtures", "import", "tpch", "--scale-factor", "10", pgurl,
"./cockroach", "workload", "fixtures", "import", "tpch", "--scale-factor", "10", "{pgurl:1}",
); err != nil {
t.Fatal(err)
}
Expand All @@ -73,7 +67,7 @@ func registerAllocator(r registry.Registry) {
WithCluster(clusNodes.InstallNodes()).
WithPrometheusNode(promNode.InstallNodes()[0])

err = c.StartGrafana(ctx, t.L(), cfg)
err := c.StartGrafana(ctx, t.L(), cfg)
require.NoError(t, err)

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

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

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

// Decommission a node. The ranges should down-replicate to 7 replicas.
Expand Down
8 changes: 1 addition & 7 deletions pkg/cmd/roachtest/tests/alterpk.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
Expand Down Expand Up @@ -105,14 +104,9 @@ func registerAlterPK(r registry.Registry) {
const duration = 10 * time.Minute

roachNodes, loadNode := setupTest(ctx, t, c)
pgurl, err := roachtestutil.DefaultPGUrl(ctx, c, t.L(), c.Node(1))
if err != nil {
t.Fatal(err)
}
cmd := fmt.Sprintf(
"./cockroach workload fixtures import tpcc --warehouses=%d --db=tpcc '%s'",
"./cockroach workload fixtures import tpcc --warehouses=%d --db=tpcc {pgurl:1}",
warehouses,
pgurl,
)
if err := c.RunE(ctx, option.WithNodes(c.Node(roachNodes[0])), cmd); err != nil {
t.Fatal(err)
Expand Down
9 changes: 5 additions & 4 deletions pkg/cmd/roachtest/tests/asyncpg.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package tests

import (
"context"
"fmt"
"regexp"

"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
Expand All @@ -22,11 +23,11 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
)

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

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

Expand Down
5 changes: 3 additions & 2 deletions pkg/cmd/roachtest/tests/asyncpg_blocklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,9 @@ var asyncpgBlocklist = blocklist{
`test_codecs.TestCodecs.test_unknown_type_text_fallback`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/54516",
`test_codecs.TestCodecs.test_void`: "unknown",
`test_connect.TestSettings.test_get_settings_01`: "unknown",
`test_copy.TestCopyFrom.test_copy_from_query_cancellation_explicit`: "known asyncpg ssl issue - https://github.com/MagicStack/asyncpg/issues/240",
`test_copy.TestCopyFrom.test_copy_from_query_timeout_1`: "known asyncpg ssl issue - https://github.com/MagicStack/asyncpg/issues/240",
`test_copy.TestCopyFrom.test_copy_from_query_to_sink`: "known asyncpg ssl issue - https://github.com/MagicStack/asyncpg/issues/240",
`test_copy.TestCopyFrom.test_copy_from_table_basics`: "no support for COPY TO - https://github.com/cockroachdb/cockroach/issues/85571",
`test_copy.TestCopyTo.test_copy_to_table_basics`: "unknown",
`test_cursor.TestCursor.test_cursor_02`: "unknown",
Expand All @@ -60,15 +63,13 @@ var asyncpgBlocklist = blocklist{
`test_introspection.TestIntrospection.test_introspection_on_large_db`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/22456",
`test_introspection.TestIntrospection.test_introspection_retries_after_cache_bust`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/27796",
`test_introspection.TestIntrospection.test_introspection_sticks_for_ps`: "unknown type: pg_catalog.json",
`test_listeners.TestListeners.test_dangling_listener_warns`: "LISTEN - https://github.com/cockroachdb/cockroach/issues/41522",
`test_listeners.TestListeners.test_listen_01`: "LISTEN - https://github.com/cockroachdb/cockroach/issues/41522",
`test_listeners.TestListeners.test_listen_02`: "LISTEN - https://github.com/cockroachdb/cockroach/issues/41522",
`test_listeners.TestListeners.test_listen_notletters`: "LISTEN - https://github.com/cockroachdb/cockroach/issues/41522",
`test_listeners.TestLogListeners.test_log_listener_01`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511",
`test_listeners.TestLogListeners.test_log_listener_02`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511",
`test_listeners.TestLogListeners.test_log_listener_03`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511",
`test_pool.TestPool.test_pool_remote_close`: "unsupported pg_terminate_backend() function",
`test_prepare.TestPrepare.test_prepare_08_big_result`: "unknown",
`test_prepare.TestPrepare.test_prepare_09_raise_error`: "unsupported feature - https://github.com/cockroachdb/cockroach/issues/17511",
`test_prepare.TestPrepare.test_prepare_14_explain`: "unknown",
`test_prepare.TestPrepare.test_prepare_16_command_result`: "unknown",
Expand Down
7 changes: 1 addition & 6 deletions pkg/cmd/roachtest/tests/autoupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/testutils"
Expand Down Expand Up @@ -74,12 +73,8 @@ func registerAutoUpgrade(r registry.Registry) {

decommissionAndStop := func(node int) error {
t.WorkerStatus("decommission")
pgurl, err := roachtestutil.DefaultPGUrl(ctx, c, t.L(), c.Node(node))
if err != nil {
return err
}
if err := c.RunE(ctx, option.WithNodes(c.Node(node)),
fmt.Sprintf("./cockroach node decommission %d --insecure --url=%s", node, pgurl)); err != nil {
fmt.Sprintf("./cockroach node decommission %d --certs-dir=certs --port={pgport%s}", node, c.Node(node))); err != nil {
return err
}
t.WorkerStatus("stop")
Expand Down
8 changes: 3 additions & 5 deletions pkg/cmd/roachtest/tests/awsdms.go
Original file line number Diff line number Diff line change
Expand Up @@ -766,11 +766,9 @@ func setupDMSEndpointsAndTask(
PostgreSQLSettings: &dmstypes.PostgreSQLSettings{
DatabaseName: proto.String(awsdmsCRDBDatabase),
Username: proto.String(awsdmsCRDBUser),
// Password is a required field, but CockroachDB doesn't take passwords in
// --insecure mode. As such, put in some garbage.
Password: proto.String("garbage"),
Port: proto.Int32(26257),
ServerName: proto.String(externalCRDBAddr[0]),
Password: proto.String(awsdmsPassword),
Port: proto.Int32(26257),
ServerName: proto.String(externalCRDBAddr[0]),
},
},
endpoint: dmsEndpoints.defaultTarget,
Expand Down
6 changes: 1 addition & 5 deletions pkg/cmd/roachtest/tests/backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -335,15 +335,11 @@ func registerBackup(r registry.Registry) {
m := c.NewMonitor(ctx)
m.Go(func(ctx context.Context) error {
t.Status(`running backup`)
pgurl, err := roachtestutil.DefaultPGUrl(ctx, c, t.L(), c.Node(1))
if err != nil {
return err
}
// Tick once before starting the backup, and once after to capture the
// total elapsed time. This is used by roachperf to compute and display
// the average MB/sec per node.
tick()
c.Run(ctx, option.WithNodes(c.Node(1)), `./cockroach sql --insecure --url=`+pgurl+` -e "
c.Run(ctx, option.WithNodes(c.Node(1)), `./cockroach sql --url={pgurl:1} -e "
BACKUP bank.bank TO 'gs://`+backupTestingBucket+`/`+dest+`?AUTH=implicit'"`)
tick()

Expand Down
8 changes: 6 additions & 2 deletions pkg/cmd/roachtest/tests/build_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,11 +12,11 @@ package tests

import (
"context"
"net/http"
"strings"

"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
Expand All @@ -33,7 +33,11 @@ func RunBuildInfo(ctx context.Context, t test.Test, c cluster.Cluster) {
t.Fatal(err)
}
url := `http://` + adminUIAddrs[0] + `/_status/details/local`
err = httputil.GetJSON(http.Client{}, url, &details)
client, err := roachtestutil.DefaultHttpClientWithSessionCookie(ctx, c, t.L(), c.Node(1), url)
if err != nil {
t.Fatal(err)
}
err = httputil.GetJSON(client, url, &details)
if err != nil {
t.Fatal(err)
}
Expand Down
14 changes: 4 additions & 10 deletions pkg/cmd/roachtest/tests/clearrange.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
Expand Down Expand Up @@ -66,16 +65,12 @@ func registerClearRange(r registry.Registry) {
func runClearRange(ctx context.Context, t test.Test, c cluster.Cluster, aggressiveChecks bool) {
t.Status("restoring fixture")
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings())
pgurl, err := roachtestutil.DefaultPGUrl(ctx, c, t.L(), c.Nodes(1))
if err != nil {
t.Fatal(err)
}
m := c.NewMonitor(ctx)
m.Go(func(ctx context.Context) error {
// NB: on a 10 node cluster, this should take well below 3h.
tBegin := timeutil.Now()
c.Run(ctx, option.WithNodes(c.Node(1)), "./cockroach", "workload", "fixtures", "import", "bank",
"--payload-bytes=10240", "--ranges=10", "--rows=65104166", "--seed=4", "--db=bigbank", pgurl)
"--payload-bytes=10240", "--ranges=10", "--rows=65104166", "--seed=4", "--db=bigbank", "{pgurl:1}")
t.L().Printf("import took %.2fs", timeutil.Since(tBegin).Seconds())
return nil
})
Expand Down Expand Up @@ -104,10 +99,9 @@ func runClearRange(ctx context.Context, t test.Test, c cluster.Cluster, aggressi
// Use a 120s connect timeout to work around the fact that the server will
// declare itself ready before it's actually 100% ready. See:
// https://github.com/cockroachdb/cockroach/issues/34897#issuecomment-465089057
c.Run(ctx, option.WithNodes(c.Node(1)), fmt.Sprintf(
`COCKROACH_CONNECT_TIMEOUT=120 ./cockroach sql --url=%s --insecure -e "DROP DATABASE IF EXISTS tinybank"`, pgurl))
c.Run(ctx, option.WithNodes(c.Node(1)), `COCKROACH_CONNECT_TIMEOUT=120 ./cockroach sql --url={pgurl:1} -e "DROP DATABASE IF EXISTS tinybank"`)
c.Run(ctx, option.WithNodes(c.Node(1)), "./cockroach", "workload", "fixtures", "import", "bank", "--db=tinybank",
"--payload-bytes=100", "--ranges=10", "--rows=800", "--seed=1", pgurl)
"--payload-bytes=100", "--ranges=10", "--rows=800", "--seed=1", "{pgurl:1}")

t.Status()

Expand Down Expand Up @@ -139,7 +133,7 @@ ORDER BY raw_start_key ASC LIMIT 1`,
}()

m.Go(func(ctx context.Context) error {
c.Run(ctx, option.WithNodes(c.Node(1)), `./cockroach workload init kv`, pgurl)
c.Run(ctx, option.WithNodes(c.Node(1)), `./cockroach workload init kv {pgurl:1}`)
c.Run(ctx, option.WithNodes(c.All()), fmt.Sprintf(`./cockroach workload run kv --concurrency=32 --duration=1h --tolerate-errors {pgurl%s}`, c.All()))
return nil
})
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/cli.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ func runCLINodeStatus(ctx context.Context, t test.Test, c cluster.Cluster) {
}

nodeStatus := func() (_ string, _ []string, err error) {
result, err := c.RunWithDetailsSingleNode(ctx, t.L(), option.WithNodes(c.Node(1)), "./cockroach node status --insecure -p {pgport:1}")
result, err := c.RunWithDetailsSingleNode(ctx, t.L(), option.WithNodes(c.Node(1)), "./cockroach node status --certs-dir=certs -p {pgport:1}")
if err != nil {
return "", nil, err
}
Expand Down
Loading

0 comments on commit c1ae680

Please sign in to comment.