Skip to content

Commit

Permalink
roachtest: fix multitenant-upgrade
Browse files Browse the repository at this point in the history
In #71040 we added disk spilling for tenants which added the following
call to the `mt start-sql` code path:

https://github.com/cockroachdb/cockroach/blob/af5a5a5065ce80c5e6568b4b422bf5c3a179e173/pkg/cli/mt_start_sql.go#L90-L89

The defaults for the store match that of a regular CockroachDB node, and
the tenant will thus attempt to clean up temp dirs for `cockroach-data`
if no store is specified:

https://github.com/cockroachdb/cockroach/blob/6999e5fded43f59eb5839dc8b943fd1e2a33a3fd/pkg/cli/start.go#L223-L227

In the `multitenant-upgrade` roachtest, as it happens there was actually
a cockroach host instance running under `cockroach-data`, and so the
tenant would fail to try to remove its (locked) temp dirs.

Fix that by passing the `--store` flag to the tenants in this test. This
is mildly annoying since the predecessor version doesn't understand it
and so the test has to figure out when it is legal to pass it. Anyway,
it is done now.

I will point out that it isn't the greatest choice to have tenants
default to `cockroach-data` as the resulting interaction with a CRDB
server results in an unfortunate UX. I filed #71603 to that effect.

Release note: None
  • Loading branch information
tbg committed Oct 15, 2021
1 parent cb725a6 commit 465113b
Show file tree
Hide file tree
Showing 2 changed files with 31 additions and 30 deletions.
6 changes: 2 additions & 4 deletions pkg/cmd/roachtest/tests/multitenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,15 +34,13 @@ func runAcceptanceMultitenant(ctx context.Context, t test.Test, c cluster.Cluste
kvAddrs, err := c.ExternalAddr(ctx, c.All())
require.NoError(t, err)

tenantCtx, cancel := context.WithCancel(ctx)
defer cancel()
const (
tenantHTTPPort = 8081
tenantSQLPort = 30258
)
const tenantNode = 1
tenant := createTenantNode(tenantCtx, t, c, "./cockroach", kvAddrs,
tenantID, tenantNode, tenantHTTPPort, tenantSQLPort)
tenant := createTenantNode(kvAddrs, tenantID, tenantNode, tenantHTTPPort, tenantSQLPort)
tenant.start(ctx, t, c, "./cockroach")

t.Status("checking that a client can connect to the tenant server")

Expand Down
55 changes: 29 additions & 26 deletions pkg/cmd/roachtest/tests/multitenant_upgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,30 +48,23 @@ type tenantNode struct {
kvAddrs []string
pgURL string

binary string
binary string // the binary last passed to start()
errCh chan error
node int

// When starting a <=21.1 tenant, this needs to be set
// to avoid passing unsupported flags during start().
doesNotSupportStoreFlag bool
}

func createTenantNode(
ctx context.Context,
t test.Test,
c cluster.Cluster,
binary string,
kvAddrs []string,
tenantID int,
node int,
httpPort, sqlPort int,
) *tenantNode {
func createTenantNode(kvAddrs []string, tenantID, node, httpPort, sqlPort int) *tenantNode {
tn := &tenantNode{
tenantID: tenantID,
httpPort: httpPort,
kvAddrs: kvAddrs,
binary: binary,
node: node,
sqlPort: sqlPort,
}
tn.start(ctx, t, c, binary)
return tn
}

Expand All @@ -91,12 +84,20 @@ func (tn *tenantNode) logDir() string {
return fmt.Sprintf("logs/mt-%d", tn.tenantID)
}

func (tn *tenantNode) storeDir() string {
return fmt.Sprintf("cockroach-data-mt-%d", tn.tenantID)
}

func (tn *tenantNode) start(ctx context.Context, t test.Test, c cluster.Cluster, binary string) {
tn.binary = binary
extraArgs := []string{"--log-dir=" + tn.logDir()}
if !tn.doesNotSupportStoreFlag {
extraArgs = append(extraArgs, "--store="+tn.storeDir())
}
tn.errCh = startTenantServer(
ctx, c, c.Node(tn.node), binary, tn.kvAddrs, tn.tenantID,
tn.httpPort, tn.sqlPort,
"--log-dir="+tn.logDir(),
extraArgs...,
)
externalUrls, err := c.ExternalPGUrl(ctx, c.Node(tn.node))
require.NoError(t, err)
Expand Down Expand Up @@ -184,8 +185,9 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster,
runner.QueryRow(t, "SHOW CLUSTER SETTING version").Scan(&initialVersion)

const tenantNode = 2
tenant11 := createTenantNode(ctx, t, c, predecessorBinary, kvAddrs,
tenant11ID, tenantNode, tenant11HTTPPort, tenant11SQLPort)
tenant11 := createTenantNode(kvAddrs, tenant11ID, tenantNode, tenant11HTTPPort, tenant11SQLPort)
tenant11.doesNotSupportStoreFlag = true // can be removed when predecessorBinary is 21.2
tenant11.start(ctx, t, c, predecessorBinary)
defer tenant11.stop(ctx, t, c)

t.Status("checking that a client can connect to the tenant 11 server")
Expand Down Expand Up @@ -250,8 +252,9 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster,
runner.Exec(t, `SELECT crdb_internal.create_tenant($1)`, tenant12ID)

t.Status("starting tenant 12 server with older binary")
tenant12 := createTenantNode(ctx, t, c, predecessorBinary, kvAddrs,
tenant12ID, tenantNode, tenant12HTTPPort, tenant12SQLPort)
tenant12 := createTenantNode(kvAddrs, tenant12ID, tenantNode, tenant12HTTPPort, tenant12SQLPort)
tenant12.doesNotSupportStoreFlag = true // can be removed when predecessorBinary is 21.2
tenant12.start(ctx, t, c, predecessorBinary)
defer tenant12.stop(ctx, t, c)

t.Status("verifying that the tenant 12 server works and is at the earlier version")
Expand All @@ -272,8 +275,8 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster,
runner.Exec(t, `SELECT crdb_internal.create_tenant($1)`, tenant13ID)

t.Status("starting tenant 13 server with new binary")
tenant13 := createTenantNode(ctx, t, c, currentBinary, kvAddrs,
tenant13ID, tenantNode, tenant13HTTPPort, tenant13SQLPort)
tenant13 := createTenantNode(kvAddrs, tenant13ID, tenantNode, tenant13HTTPPort, tenant13SQLPort)
tenant13.start(ctx, t, c, currentBinary)
defer tenant13.stop(ctx, t, c)

t.Status("verifying that the tenant 13 server works and is at the earlier version")
Expand All @@ -291,6 +294,7 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster,
tenant11.stop(ctx, t, c)

t.Status("starting the tenant 11 server with the current binary")
tenant11.doesNotSupportStoreFlag = false
tenant11.start(ctx, t, c, currentBinary)

if needsWorkaround {
Expand Down Expand Up @@ -332,6 +336,7 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster,
tenant12.stop(ctx, t, c)

t.Status("starting the tenant 12 server with the current binary")
tenant12.doesNotSupportStoreFlag = false
tenant12.start(ctx, t, c, currentBinary)

t.Status("verify tenant 12 server works with the new binary")
Expand Down Expand Up @@ -388,8 +393,8 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster,
runner.Exec(t, `SELECT crdb_internal.create_tenant($1)`, tenant14ID)

t.Status("verifying the tenant 14 works and has the proper version")
tenant14 := createTenantNode(ctx, t, c, currentBinary, kvAddrs,
tenant14ID, tenantNode, tenant14HTTPPort, tenant14SQLPort)
tenant14 := createTenantNode(kvAddrs, tenant14ID, tenantNode, tenant14HTTPPort, tenant14SQLPort)
tenant14.start(ctx, t, c, currentBinary)
defer tenant14.stop(ctx, t, c)
verifySQL(t, tenant14.pgURL,
mkStmt(`CREATE TABLE foo (id INT PRIMARY KEY, v STRING)`),
Expand Down Expand Up @@ -420,7 +425,7 @@ func startTenantServer(
tenantID int,
httpPort int,
sqlPort int,
logFlags string,
extraFlags ...string,
) chan error {

args := []string{
Expand All @@ -433,9 +438,7 @@ func startTenantServer(
// Don't bind to external interfaces when running locally.
"--sql-addr", ifLocal(c, "127.0.0.1", "0.0.0.0") + ":" + strconv.Itoa(sqlPort),
}
if logFlags != "" {
args = append(args, logFlags)
}
args = append(args, extraFlags...)
errCh := make(chan error, 1)
go func() {
errCh <- c.RunE(tenantCtx, node,
Expand Down

0 comments on commit 465113b

Please sign in to comment.