Skip to content

Commit

Permalink
logictest: support mixed binary version tests with upgrades
Browse files Browse the repository at this point in the history
The logictest framework can now use the cockroach-go testserver to test
a multi-node cluster in which each node is running a different binary
version.

Previously, only mixed logical versions could be tested.

Release note: None
  • Loading branch information
rafiss committed Nov 14, 2022
1 parent 15612ab commit ba973ea
Show file tree
Hide file tree
Showing 13 changed files with 529 additions and 43 deletions.
6 changes: 6 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -378,6 +378,8 @@ ALL_TESTS = [
"//pkg/sql/lexbase:lexbase_test",
"//pkg/sql/logictest/tests/5node-disk:5node-disk_test",
"//pkg/sql/logictest/tests/5node:5node_test",
"//pkg/sql/logictest/tests/cockroach-go-testserver-22.1-22.2:cockroach-go-testserver-22_1-22_2_test",
"//pkg/sql/logictest/tests/cockroach-go-testserver:cockroach-go-testserver_test",
"//pkg/sql/logictest/tests/fakedist-disk:fakedist-disk_test",
"//pkg/sql/logictest/tests/fakedist-vec-off:fakedist-vec-off_test",
"//pkg/sql/logictest/tests/fakedist:fakedist_test",
Expand Down Expand Up @@ -1537,6 +1539,8 @@ GO_TARGETS = [
"//pkg/sql/logictest/logictestbase:logictestbase",
"//pkg/sql/logictest/tests/5node-disk:5node-disk_test",
"//pkg/sql/logictest/tests/5node:5node_test",
"//pkg/sql/logictest/tests/cockroach-go-testserver-22.1-22.2:cockroach-go-testserver-22_1-22_2_test",
"//pkg/sql/logictest/tests/cockroach-go-testserver:cockroach-go-testserver_test",
"//pkg/sql/logictest/tests/fakedist-disk:fakedist-disk_test",
"//pkg/sql/logictest/tests/fakedist-vec-off:fakedist-vec-off_test",
"//pkg/sql/logictest/tests/fakedist:fakedist_test",
Expand Down Expand Up @@ -2713,6 +2717,8 @@ GET_X_DATA_TARGETS = [
"//pkg/sql/logictest/logictestbase:get_x_data",
"//pkg/sql/logictest/tests/5node:get_x_data",
"//pkg/sql/logictest/tests/5node-disk:get_x_data",
"//pkg/sql/logictest/tests/cockroach-go-testserver:get_x_data",
"//pkg/sql/logictest/tests/cockroach-go-testserver-22.1-22.2:get_x_data",
"//pkg/sql/logictest/tests/fakedist:get_x_data",
"//pkg/sql/logictest/tests/fakedist-disk:get_x_data",
"//pkg/sql/logictest/tests/fakedist-vec-off:get_x_data",
Expand Down
2 changes: 2 additions & 0 deletions pkg/cmd/generate-logictest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ type testFileTemplateConfig struct {
CclLogicTest bool
ExecBuildLogicTest bool
SqliteLogicTest bool
CockroachGoTestserverTest bool
Ccl bool
ForceProductionValues bool
Package, TestRuleName, RelDir string
Expand Down Expand Up @@ -160,6 +161,7 @@ func (t *testdir) dump() error {
tplCfg.Package = strings.ReplaceAll(strings.ReplaceAll(cfg.Name, "-", "_"), ".", "")
tplCfg.RelDir = t.relPathToParent
tplCfg.TestCount = testCount
tplCfg.CockroachGoTestserverTest = cfg.UseCockroachGoTestserver
// The NumCPU calculation is a guess pulled out of thin air to
// allocate the tests which use 3-node clusters 2 vCPUs, and
// the ones which use more a bit more.
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/generate-logictest/templates.go
Original file line number Diff line number Diff line change
Expand Up @@ -258,7 +258,8 @@ go_test(
"@com_github_cockroachdb_sqllogictest//:testfiles", # keep{{ end }}{{ if .CclLogicTest }}
"//pkg/ccl/logictestccl:testdata", # keep{{ end }}{{ if .LogicTest }}
"//pkg/sql/logictest:testdata", # keep{{ end }}{{ if .ExecBuildLogicTest }}
"//pkg/sql/opt/exec/execbuilder:testdata", # keep{{ end }}
"//pkg/sql/opt/exec/execbuilder:testdata", # keep{{ end }}{{ if .CockroachGoTestserverTest }}
"//pkg/cmd/cockroach-short", # keep{{ end }}
],
shard_count = {{ if gt .TestCount 16 }}16{{ else }}{{ .TestCount }}{{end}},
tags = ["cpu:{{ if gt .NumCPU 4 }}4{{ else }}{{ .NumCPU }}{{ end }}"],
Expand Down
7 changes: 6 additions & 1 deletion pkg/sql/logictest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,15 @@ filegroup(
go_library(
name = "logictest",
testonly = 1,
srcs = ["logic.go"],
srcs = [
"logic.go",
"ports_helper.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/logictest",
visibility = ["//visibility:public"],
deps = [
"//pkg/base",
"//pkg/build/bazel",
"//pkg/cloud/externalconn/providers",
"//pkg/clusterversion",
"//pkg/col/coldata",
Expand Down Expand Up @@ -60,6 +64,7 @@ go_library(
"//pkg/testutils/sqlutils",
"//pkg/upgrade",
"//pkg/util",
"//pkg/util/binfetcher",
"//pkg/util/envutil",
"//pkg/util/log",
"//pkg/util/randutil",
Expand Down
219 changes: 178 additions & 41 deletions pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,8 +37,9 @@ import (
"time"
"unicode/utf8"

_ "github.com/cockroachdb/cockroach-go/v2/testserver" // placeholder until mixed-version functionality is added.
"github.com/cockroachdb/cockroach-go/v2/testserver"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/build/bazel"
_ "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/providers" // imported to register ExternalConnection providers
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
Expand Down Expand Up @@ -72,6 +73,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/upgrade"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/binfetcher"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
Expand Down Expand Up @@ -359,6 +361,10 @@ import (
// in the cluster with index N (note this is 0-indexed, while
// node IDs themselves are 1-indexed).
//
// - upgrade N
// When using a cockroach-go/testserver logictest, upgrades the node at
// index N to the version specified by the logictest config.
//
// A "host-cluster-" prefix can be prepended to the user, which will force
// the user session to be against the host cluster (useful for multi-tenant
// configurations).
Expand Down Expand Up @@ -535,7 +541,6 @@ var (
"declarative-corpus", "",
"enables generation and storage of a declarative schema changer corpus",
)

// globalMVCCRangeTombstone will write a global MVCC range tombstone across
// the entire user keyspace during cluster bootstrapping. This should not
// semantically affect the test data written above it, but will activate MVCC
Expand Down Expand Up @@ -951,6 +956,8 @@ type logicTest struct {
// cluster is the test cluster against which we are testing. This cluster
// may be reset during the lifetime of the test.
cluster serverutils.TestClusterInterface
// testserverCluster is the testserver cluster. This uses real binaries.
testserverCluster testserver.TestServer
// sharedIODir is the ExternalIO directory that is shared between all clusters
// created in the same logicTest. It is populated during setup() of the logic
// test.
Expand Down Expand Up @@ -1148,13 +1155,24 @@ func (t *logicTest) setUser(user string, nodeIdxOverride int) func() {
nodeIdx = nodeIdxOverride
}

addr := t.cluster.Server(nodeIdx).ServingSQLAddr()
if len(t.tenantAddrs) > 0 && !strings.HasPrefix(user, "host-cluster-") {
addr = t.tenantAddrs[nodeIdx]
var addr string
var pgURL url.URL
var pgUser string
var cleanupFunc func()
pgUser = strings.TrimPrefix(user, "host-cluster-")
if t.cfg.UseCockroachGoTestserver {
pgURL = *t.testserverCluster.PGURL()
pgURL.User = url.User(pgUser)
cleanupFunc = func() {}
} else {
addr = t.cluster.Server(nodeIdx).ServingSQLAddr()
if len(t.tenantAddrs) > 0 && !strings.HasPrefix(user, "host-cluster-") {
addr = t.tenantAddrs[nodeIdx]
}
pgURL, cleanupFunc = sqlutils.PGUrl(t.rootT, addr, "TestLogic", url.User(pgUser))
pgURL.Path = "test"
}
pgUser := strings.TrimPrefix(user, "host-cluster-")
pgURL, cleanupFunc := sqlutils.PGUrl(t.rootT, addr, "TestLogic", url.User(pgUser))
pgURL.Path = "test"

db := t.openDB(pgURL)

// The default value for extra_float_digits assumed by tests is
Expand Down Expand Up @@ -1196,6 +1214,44 @@ func (t *logicTest) openDB(pgURL url.URL) *gosql.DB {
return gosql.OpenDB(connector)
}

func (t *logicTest) newTestServerCluster(bootstrapBinaryPath string, upgradeBinaryPath string) {
// During config initialization, NumNodes is required to be 3.
ports := make([]int, t.cfg.NumNodes)
for i := 0; i < len(ports); i++ {
port, err := getFreePort()
if err != nil {
t.Fatal(err)
}
ports[i] = port
}

opts := []testserver.TestServerOpt{
testserver.ThreeNodeOpt(),
testserver.StoreOnDiskOpt(),
testserver.CockroachBinaryPathOpt(bootstrapBinaryPath),
testserver.UpgradeCockroachBinaryPathOpt(upgradeBinaryPath),
testserver.AddListenAddrPortOpt(ports[0]),
testserver.AddListenAddrPortOpt(ports[1]),
testserver.AddListenAddrPortOpt(ports[2]),
}
if strings.Contains(upgradeBinaryPath, "cockroach-short") {
// If we're using a cockroach-short binary, that means it was locally
// built, so we need to opt-in to development upgrades.
opts = append(opts, testserver.EnvVarOpt([]string{
"COCKROACH_UPGRADE_TO_DEV_VERSION=true",
}))
}

ts, err := testserver.NewTestServer(opts...)
if err != nil {
t.Fatal(err)
}

t.testserverCluster = ts
t.clusterCleanupFuncs = append(t.clusterCleanupFuncs, t.setUser(username.RootUser, 0 /* nodeIdxOverride */))
t.clusterCleanupFuncs = append(t.clusterCleanupFuncs, ts.Stop)
}

// newCluster creates a new cluster. It should be called after the logic tests's
// server args are configured. That is, either during setup() when creating the
// initial cluster to be used in a test, or when creating additional test
Expand Down Expand Up @@ -1723,7 +1779,45 @@ func (t *logicTest) setup(
t.sharedIODir = tempExternalIODir
t.testCleanupFuncs = append(t.testCleanupFuncs, tempExternalIODirCleanup)

t.newCluster(serverArgs, t.clusterOpts, t.knobOpts, t.tenantClusterSettingOverrideOpts)
if cfg.UseCockroachGoTestserver {
if !bazel.BuiltWithBazel() {
skip.IgnoreLint(t.t(), "cockroach-go/testserver can only be uzed in bazel builds")
}
if cfg.NumNodes != 3 {
t.Fatal("cockroach-go testserver tests must use 3 nodes")
}
if cfg.CockroachGoBootstrapVersion == "" {
t.Fatal("cockroach-go testserver tests must specify CockroachGoBootstrapVersion")
}
bootstrapBinaryPath, err := binfetcher.Download(context.Background(), binfetcher.Options{
Binary: "cockroach",
Dir: tempExternalIODir,
Version: cfg.CockroachGoBootstrapVersion,
GOOS: runtime.GOOS,
})
if err != nil {
t.Fatal(err)
}
localBinaryPath, found := bazel.FindBinary("pkg/cmd/cockroach-short/cockroach-short_/", "cockroach-short")
if !found {
t.Fatal(errors.New("cockroach binary not found"))
}
upgradeBinaryPath := localBinaryPath
if cfg.CockroachGoUpgradeVersion != "" {
upgradeBinaryPath, err = binfetcher.Download(context.Background(), binfetcher.Options{
Binary: "cockroach",
Dir: tempExternalIODir,
Version: cfg.CockroachGoUpgradeVersion,
GOOS: runtime.GOOS,
})
if err != nil {
t.Fatal(err)
}
}
t.newTestServerCluster(bootstrapBinaryPath, upgradeBinaryPath)
} else {
t.newCluster(serverArgs, t.clusterOpts, t.knobOpts, t.tenantClusterSettingOverrideOpts)
}

// Only create the test database on the initial cluster, since cluster restore
// expects an empty cluster.
Expand All @@ -1733,19 +1827,8 @@ CREATE DATABASE test; USE test;
t.Fatal(err)
}

if !t.cfg.BootstrapVersion.Equal(roachpb.Version{}) && t.cfg.BootstrapVersion.Less(clusterversion.ByKey(clusterversion.V22_2SetSystemUsersUserIDColumnNotNull)) {
// Hacky way to create user with an ID if we're on a
// bootstrapped binary less than 22.2. The version gate
// causes the regular CREATE USER to fail since it will not
// insert an ID.
if _, err := t.db.Exec(`INSERT INTO system.users VALUES ($1, '', false, $2);`,
username.TestUser, 100); err != nil {
t.Fatal(err)
}
} else {
if _, err := t.db.Exec(fmt.Sprintf("CREATE USER %s;", username.TestUser)); err != nil {
t.Fatal(err)
}
if _, err := t.db.Exec(fmt.Sprintf("CREATE USER %s;", username.TestUser)); err != nil {
t.Fatal(err)
}

t.labelMap = make(map[string]string)
Expand Down Expand Up @@ -2267,6 +2350,10 @@ func fetchSubtests(path string) ([]subtestDetails, error) {
}

func (t *logicTest) purgeZoneConfig() {
if t.cluster == nil {
// We can only purge zone configs for in-memory test clusters.
return
}
for i := 0; i < t.cluster.NumServers(); i++ {
sysconfigProvider := t.cluster.Server(i).SystemConfigProvider()
sysconfig := sysconfigProvider.GetSystemConfig()
Expand Down Expand Up @@ -2902,6 +2989,43 @@ func (t *logicTest) processSubtest(
}
t.traceStop()

case "upgrade":
if len(fields) != 2 {
return errors.Errorf("upgrade requires a node num argument, found: %v", fields)
}
if t.testserverCluster == nil {
return errors.Errorf(`could not perform "upgrade", not a cockroach-go/testserver cluster`)
}
nodeNum, err := strconv.Atoi(fields[1])
if err != nil {
t.Fatal(err)
}
if err := t.testserverCluster.UpgradeNode(nodeNum); err != nil {
t.Fatal(err)
}

successfulPing := false
for i := 0; i < 50; i++ {
url := t.testserverCluster.PGURLForNode(nodeNum)
db, err := gosql.Open("postgres", url.String())
if err != nil {
return err
}
t.clusterCleanupFuncs = append(t.clusterCleanupFuncs, func() { _ = db.Close() })
if err = db.Ping(); err == nil {
successfulPing = true
break
}
time.Sleep(time.Millisecond * 1000)
}

if !successfulPing {
return errors.New("node did not finish init")
}

if err := t.testserverCluster.WaitForInitFinishForNode(nodeNum); err != nil {
t.Fatal(err)
}
default:
return errors.Errorf("%s:%d: unknown command: %s",
path, s.Line+subtest.lineLineIndexIntoFile, cmd,
Expand Down Expand Up @@ -3142,13 +3266,21 @@ func (t *logicTest) execQuery(query logicQuery) error {
db := t.db
var closeDB func()
if query.nodeIdx != 0 {
addr := t.cluster.Server(query.nodeIdx).ServingSQLAddr()
if len(t.tenantAddrs) > 0 {
addr = t.tenantAddrs[query.nodeIdx]
var pgURL url.URL
if t.testserverCluster != nil {
pgURL = *t.testserverCluster.PGURLForNode(query.nodeIdx)
pgURL.User = url.User(t.user)
pgURL.Path = "test"
} else {
addr := t.cluster.Server(query.nodeIdx).ServingSQLAddr()
if len(t.tenantAddrs) > 0 {
addr = t.tenantAddrs[query.nodeIdx]
}
var cleanupFunc func()
pgURL, cleanupFunc = sqlutils.PGUrl(t.rootT, addr, "TestLogic", url.User(t.user))
defer cleanupFunc()
pgURL.Path = "test"
}
pgURL, cleanupFunc := sqlutils.PGUrl(t.rootT, addr, "TestLogic", url.User(t.user))
defer cleanupFunc()
pgURL.Path = "test"

db = t.openDB(pgURL)
closeDB = func() {
Expand Down Expand Up @@ -3717,8 +3849,12 @@ func (t *logicTest) validateAfterTestCompletion() error {

// Ensure that all of the created descriptors can round-trip through json.
{
rows, err := t.db.Query(
`
// If `useCockroachGoTestserver` is true and we do an upgrade,
// this may fail if we're in between migrations that
// upgrade the descriptors.
if !t.cfg.UseCockroachGoTestserver {
rows, err := t.db.Query(
`
SELECT encode(descriptor, 'hex') AS descriptor
FROM system.descriptor
WHERE descriptor
Expand All @@ -3731,17 +3867,18 @@ SELECT encode(descriptor, 'hex') AS descriptor
)
);
`,
)
if err != nil {
return errors.Wrap(err, "failed to test for descriptor JSON round-trip")
}
rowsMat, err := sqlutils.RowsToStrMatrix(rows)
if err != nil {
return errors.Wrap(err, "failed read rows from descriptor JSON round-trip")
}
if len(rowsMat) > 0 {
return errors.Errorf("some descriptors did not round-trip:\n%s",
sqlutils.MatrixToStr(rowsMat))
)
if err != nil {
return errors.Wrap(err, "failed to test for descriptor JSON round-trip")
}
rowsMat, err := sqlutils.RowsToStrMatrix(rows)
if err != nil {
return errors.Wrap(err, "failed read rows from descriptor JSON round-trip")
}
if len(rowsMat) > 0 {
return errors.Errorf("some descriptors did not round-trip:\n%s",
sqlutils.MatrixToStr(rowsMat))
}
}
}

Expand Down
Loading

0 comments on commit ba973ea

Please sign in to comment.