From 58bb285912ef82ab69ab4a069906386d99aed189 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Thu, 20 Jan 2022 09:01:02 +1100 Subject: [PATCH] *: replace `"testdata/..."` with `testutils.TestDataPath` Also try and break down `/` as much as possible. ``` -- fix the slashes ag '"testdata/' -G '_test\.go$' pkg -l | xargs sed -i '' -e 's/"testdata\/\([^"]*\)"/testutils.TestDataPath(t, "\1")/g' git diff --name-only | xargs sed -i '' -e 's_filepath.Join(testutils.TestDataPath(), "\([^/]*\)/\([^"]*\)"_testutils.TestDataPath(t, "\1", "\2")_g' git diff --name-only | xargs sed -i '' -e 's_filepath.Join(testutils.TestDataPath()\(.*,\) "\([^/]*\)/\([^"]*\)"_testutils.TestDataPath(t\1 "\2", "\3")_g' -- crlfmt and make generate ``` Also some manual fixes unfortunately... Release note: None --- pkg/ccl/backupccl/backup_test.go | 2 +- .../backupccl/restore_mid_schema_change_test.go | 4 ++-- pkg/ccl/backupccl/restore_old_sequences_test.go | 5 +++-- pkg/ccl/backupccl/restore_old_versions_test.go | 12 ++++++------ pkg/ccl/backupccl/show_test.go | 4 ++-- pkg/ccl/importccl/import_processor_test.go | 3 ++- pkg/ccl/importccl/mysql_testdata_helpers_test.go | 4 ++-- pkg/ccl/importccl/read_import_avro_test.go | 7 ++++--- .../kvfollowerreadsccl/boundedstaleness_test.go | 2 +- pkg/ccl/sqlproxyccl/frontend_admitter_test.go | 3 ++- pkg/ccl/sqlproxyccl/proxy_handler_test.go | 4 ++-- .../engineccl/pebble_key_manager_test.go | 5 +++-- pkg/cli/doctor_test.go | 9 +++++---- pkg/cli/import_test.go | 10 +++++----- pkg/cli/log_flags_test.go | 3 ++- pkg/cli/statement_bundle_test.go | 4 ++-- pkg/cli/zip_test.go | 14 +++++++------- pkg/cmd/github-pull-request-make/BUILD.bazel | 1 + pkg/cmd/github-pull-request-make/main_test.go | 9 +++++---- pkg/geo/geoindex/BUILD.bazel | 1 + pkg/geo/geoindex/s2_geography_index_test.go | 3 ++- pkg/geo/geoindex/s2_geometry_index_test.go | 5 +++-- .../kvcoord/txn_coord_sender_savepoints_test.go | 3 ++- .../concurrency/concurrency_manager_test.go | 2 +- pkg/kv/kvserver/concurrency/lock_table_test.go | 3 ++- pkg/kv/kvserver/replica_raft_truncation_test.go | 3 ++- pkg/server/debug/pprofui/BUILD.bazel | 1 + pkg/server/debug/pprofui/server_test.go | 3 ++- pkg/server/pagination_test.go | 5 +++-- pkg/server/pgurl/BUILD.bazel | 1 + pkg/server/pgurl/pgurl_test.go | 3 ++- pkg/sql/catalog/nstree/BUILD.bazel | 1 + pkg/sql/catalog/nstree/map_test.go | 3 ++- pkg/sql/catalog/nstree/set_test.go | 3 ++- pkg/sql/catalog/schemaexpr/BUILD.bazel | 1 + .../schemaexpr/computed_column_rewrites_test.go | 3 ++- pkg/sql/conn_executor_savepoints_test.go | 3 ++- pkg/sql/contention/BUILD.bazel | 1 + pkg/sql/contention/registry_test.go | 3 ++- pkg/sql/explain_tree_test.go | 3 ++- pkg/sql/inverted/BUILD.bazel | 1 + pkg/sql/inverted/expression_test.go | 3 ++- pkg/sql/logictest/BUILD.bazel | 1 + pkg/sql/logictest/logic_test.go | 3 ++- pkg/sql/logictest/parallel_test.go | 2 +- pkg/sql/opt/exec/execbuilder/BUILD.bazel | 1 + pkg/sql/opt/exec/execbuilder/builder_test.go | 3 ++- pkg/sql/opt/exec/explain/BUILD.bazel | 1 + pkg/sql/opt/exec/explain/output_test.go | 3 ++- pkg/sql/opt/exec/explain/plan_gist_test.go | 5 +++-- pkg/sql/opt/idxconstraint/BUILD.bazel | 1 + .../opt/idxconstraint/index_constraints_test.go | 3 ++- pkg/sql/opt/memo/expr_test.go | 3 ++- pkg/sql/opt/memo/memo_test.go | 12 ++++++------ pkg/sql/opt/memo/typing_test.go | 3 ++- pkg/sql/opt/norm/BUILD.bazel | 1 + pkg/sql/opt/norm/norm_test.go | 5 +++-- pkg/sql/opt/opbench/BUILD.bazel | 1 + pkg/sql/opt/opbench/opbench_test.go | 3 ++- pkg/sql/opt/optbuilder/BUILD.bazel | 1 + pkg/sql/opt/optbuilder/builder_test.go | 3 ++- pkg/sql/opt/optgen/lang/BUILD.bazel | 1 + pkg/sql/opt/optgen/lang/compiler_test.go | 3 ++- pkg/sql/opt/optgen/lang/parser_test.go | 3 ++- pkg/sql/opt/optgen/lang/scanner_test.go | 3 ++- pkg/sql/opt/partialidx/BUILD.bazel | 1 + pkg/sql/opt/partialidx/implicator_test.go | 3 ++- pkg/sql/opt/xform/BUILD.bazel | 1 + pkg/sql/opt/xform/optimizer_test.go | 13 +++++++------ pkg/sql/pgwire/auth_test.go | 2 +- pkg/sql/pgwire/hba/scanner_test.go | 3 ++- pkg/sql/pgwire/pgtest_test.go | 5 +++-- pkg/sql/sem/tree/pretty_test.go | 2 +- pkg/sql/session_migration_test.go | 3 ++- pkg/sql/tests/system_table_test.go | 2 +- pkg/storage/disk_map_test.go | 4 ++-- pkg/storage/intent_interleaving_iter_test.go | 3 ++- pkg/storage/intent_reader_writer_test.go | 3 ++- pkg/storage/mvcc_history_test.go | 3 ++- pkg/storage/pebble_file_registry_test.go | 3 ++- pkg/storage/pebble_test.go | 3 ++- pkg/util/admission/BUILD.bazel | 1 + pkg/util/admission/granter_test.go | 5 +++-- pkg/util/admission/work_queue_test.go | 3 ++- 84 files changed, 179 insertions(+), 112 deletions(-) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 047b6dbc312b..9564bbc6ed30 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -271,7 +271,7 @@ func TestBackupRestoreDataDriven(t *testing.T) { defer httpServerCleanup() ctx := context.Background() - datadriven.Walk(t, "testdata/backup-restore/", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "backup-restore", ""), func(t *testing.T, path string) { var lastCreatedServer string ds := newDatadrivenTestState() defer ds.cleanup(ctx) diff --git a/pkg/ccl/backupccl/restore_mid_schema_change_test.go b/pkg/ccl/backupccl/restore_mid_schema_change_test.go index 6183665eb5bd..e795c8b3a845 100644 --- a/pkg/ccl/backupccl/restore_mid_schema_change_test.go +++ b/pkg/ccl/backupccl/restore_mid_schema_change_test.go @@ -58,8 +58,8 @@ func TestRestoreMidSchemaChange(t *testing.T) { skip.UnderRaceWithIssue(t, 56584) - const ( - testdataBase = "testdata/restore_mid_schema_change" + var ( + testdataBase = testutils.TestDataPath(t, "restore_mid_schema_change") exportDirs = testdataBase + "/exports" ) for _, isClusterRestore := range []bool{true, false} { diff --git a/pkg/ccl/backupccl/restore_old_sequences_test.go b/pkg/ccl/backupccl/restore_old_sequences_test.go index 86c45dd2c7da..56222c79ca78 100644 --- a/pkg/ccl/backupccl/restore_old_sequences_test.go +++ b/pkg/ccl/backupccl/restore_old_sequences_test.go @@ -15,6 +15,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/stretchr/testify/require" @@ -41,8 +42,8 @@ import ( func TestRestoreOldSequences(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - const ( - testdataBase = "testdata/restore_old_sequences" + var ( + testdataBase = testutils.TestDataPath(t, "restore_old_sequences") exportDirs = testdataBase + "/exports" ) diff --git a/pkg/ccl/backupccl/restore_old_versions_test.go b/pkg/ccl/backupccl/restore_old_versions_test.go index e3dd0e84677e..d29cb2c41e98 100644 --- a/pkg/ccl/backupccl/restore_old_versions_test.go +++ b/pkg/ccl/backupccl/restore_old_versions_test.go @@ -63,8 +63,8 @@ import ( func TestRestoreOldVersions(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - const ( - testdataBase = "testdata/restore_old_versions" + testdataBase := testutils.TestDataPath(t, "restore_old_versions") + var ( exportDirsWithoutInterleave = testdataBase + "/exports-without-interleaved" exportDirs = testdataBase + "/exports" fkRevDirs = testdataBase + "/fk-rev-history" @@ -655,7 +655,7 @@ func TestRestoreOldBackupMissingOfflineIndexes(t *testing.T) { skip.UnderRace(t, "times out under race cause it starts up two test servers") ctx := context.Background() - badBackups, err := filepath.Abs("testdata/restore_old_versions/inc_missing_addsst/v20.2.7") + badBackups, err := filepath.Abs(testutils.TestDataPath(t, "restore_old_versions", "inc_missing_addsst", "v20.2.7")) require.NoError(t, err) args := base.TestServerArgs{ExternalIODir: badBackups} backupDirs := make([]string, 9) @@ -728,10 +728,10 @@ func TestRestoreWithDroppedSchemaCorruption(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() + backupDir := testutils.TestDataPath(t, "restore_with_dropped_schema", "exports", "v20.2.7") const ( - dbName = "foo" - backupDir = "testdata/restore_with_dropped_schema/exports/v20.2.7" - fromDir = "nodelocal://0/" + dbName = "foo" + fromDir = "nodelocal://0/" ) args := base.TestServerArgs{ExternalIODir: backupDir} diff --git a/pkg/ccl/backupccl/show_test.go b/pkg/ccl/backupccl/show_test.go index 3041a48c77ce..02735e980e67 100644 --- a/pkg/ccl/backupccl/show_test.go +++ b/pkg/ccl/backupccl/show_test.go @@ -555,8 +555,8 @@ func TestShowUpgradedForeignKeys(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - const ( - testdataBase = "testdata/restore_old_versions" + var ( + testdataBase = testutils.TestDataPath(t, "restore_old_versions") fkRevDirs = testdataBase + "/fk-rev-history" ) diff --git a/pkg/ccl/importccl/import_processor_test.go b/pkg/ccl/importccl/import_processor_test.go index 78afda1f8496..b38d1ca4e8e9 100644 --- a/pkg/ccl/importccl/import_processor_test.go +++ b/pkg/ccl/importccl/import_processor_test.go @@ -38,6 +38,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" @@ -999,7 +1000,7 @@ func avroFormat(t *testing.T, format roachpb.AvroOptions_Format) roachpb.IOFileF if format != roachpb.AvroOptions_OCF { // Need to load schema for record specific inputs. - bytes, err := ioutil.ReadFile("testdata/avro/simple-schema.json") + bytes, err := ioutil.ReadFile(testutils.TestDataPath(t, "avro", "simple-schema.json")) require.NoError(t, err) avro.SchemaJSON = string(bytes) avro.RecordSeparator = '\n' diff --git a/pkg/ccl/importccl/mysql_testdata_helpers_test.go b/pkg/ccl/importccl/mysql_testdata_helpers_test.go index b3ee8aaaabf3..8304798c6b2a 100644 --- a/pkg/ccl/importccl/mysql_testdata_helpers_test.go +++ b/pkg/ccl/importccl/mysql_testdata_helpers_test.go @@ -191,12 +191,12 @@ func getMysqlOutfileTestdata(t *testing.T) ([]simpleTestRow, []outfileDumpCfg) { } for i := range configs { - configs[i].filename = filepath.Join(`testdata`, `mysqlout`, configs[i].name, `simple.txt`) + configs[i].filename = testutils.TestDataPath(t, `mysqlout`, configs[i].name, `simple.txt`) } if rewriteMysqlTestData { genMysqlTestdata(t, func() { - if err := os.RemoveAll(filepath.Join(`testdata`, `mysqlout`)); err != nil { + if err := os.RemoveAll(testutils.TestDataPath(t, `mysqlout`)); err != nil { t.Fatal(err) } for _, cfg := range configs { diff --git a/pkg/ccl/importccl/read_import_avro_test.go b/pkg/ccl/importccl/read_import_avro_test.go index 98d8f438fff8..ac718c69cece 100644 --- a/pkg/ccl/importccl/read_import_avro_test.go +++ b/pkg/ccl/importccl/read_import_avro_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/linkedin/goavro/v2" @@ -524,7 +525,7 @@ var _ importRowProducer = &limitAvroStream{} func BenchmarkOCFImport(b *testing.B) { benchmarkAvroImport(b, roachpb.AvroOptions{ Format: roachpb.AvroOptions_OCF, - }, "testdata/avro/stock-10000.ocf") + }, testutils.TestDataPath(b, "avro", "stock-10000.ocf")) } // goos: darwin @@ -541,13 +542,13 @@ func BenchmarkOCFImport(b *testing.B) { // BenchmarkBinaryJSONImport-16 500000 3215 ns/op 37.32 MB/s // BenchmarkBinaryJSONImport-16 500000 3235 ns/op 37.09 MB/s func BenchmarkBinaryJSONImport(b *testing.B) { - schemaBytes, err := ioutil.ReadFile("testdata/avro/stock-schema.json") + schemaBytes, err := ioutil.ReadFile(testutils.TestDataPath(b, "avro", "stock-schema.json")) require.NoError(b, err) benchmarkAvroImport(b, roachpb.AvroOptions{ Format: roachpb.AvroOptions_BIN_RECORDS, SchemaJSON: string(schemaBytes), - }, "testdata/avro/stock-10000.bjson") + }, testutils.TestDataPath(b, "avro", "stock-10000.bjson")) } func benchmarkAvroImport(b *testing.B, avroOpts roachpb.AvroOptions, testData string) { diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go index 14a4611d1cb5..947d06bc3141 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness_test.go @@ -280,7 +280,7 @@ func TestBoundedStalenessDataDriven(t *testing.T) { return errorRegexp.ReplaceAllString(s, "$1 XXX") } - datadriven.Walk(t, "testdata/boundedstaleness", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "boundedstaleness"), func(t *testing.T, path string) { tc := testcluster.StartTestCluster(t, 3, clusterArgs) defer tc.Stopper().Stop(ctx) diff --git a/pkg/ccl/sqlproxyccl/frontend_admitter_test.go b/pkg/ccl/sqlproxyccl/frontend_admitter_test.go index 94b13f2c35f6..fa3fe7c1e3f4 100644 --- a/pkg/ccl/sqlproxyccl/frontend_admitter_test.go +++ b/pkg/ccl/sqlproxyccl/frontend_admitter_test.go @@ -13,6 +13,7 @@ import ( "crypto/tls" "fmt" "net" + "path/filepath" "testing" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -23,7 +24,7 @@ import ( ) func tlsConfig() (*tls.Config, error) { - cer, err := tls.LoadX509KeyPair("testdata/testserver.crt", "testdata/testserver.key") + cer, err := tls.LoadX509KeyPair(filepath.Join("testdata", "testserver.crt"), filepath.Join("testdata", "testserver.key")) if err != nil { return nil, err } diff --git a/pkg/ccl/sqlproxyccl/proxy_handler_test.go b/pkg/ccl/sqlproxyccl/proxy_handler_test.go index 4e82abc1cc22..4cc38ece8e98 100644 --- a/pkg/ccl/sqlproxyccl/proxy_handler_test.go +++ b/pkg/ccl/sqlproxyccl/proxy_handler_test.go @@ -1124,8 +1124,8 @@ openssl genrsa -out testdata/testserver.key 2048 openssl req -new -x509 -sha256 -key testdata/testserver.key -out testdata/testserver.crt \ -days 3650 -config testdata/testserver_config.cnf ` - opts.ListenKey = "testdata/testserver.key" - opts.ListenCert = "testdata/testserver.crt" + opts.ListenKey = testutils.TestDataPath(t, "testserver.key") + opts.ListenCert = testutils.TestDataPath(t, "testserver.crt") return newProxyServer(ctx, t, stopper, opts) } diff --git a/pkg/ccl/storageccl/engineccl/pebble_key_manager_test.go b/pkg/ccl/storageccl/engineccl/pebble_key_manager_test.go index d710b0a1e26a..24472902c612 100644 --- a/pkg/ccl/storageccl/engineccl/pebble_key_manager_test.go +++ b/pkg/ccl/storageccl/engineccl/pebble_key_manager_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/ccl/storageccl/engineccl/enginepbccl" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -246,7 +247,7 @@ func TestDataKeyManager(t *testing.T) { return timeutil.Unix(unixTime, 0) } - datadriven.RunTest(t, "testdata/data_key_manager", + datadriven.RunTest(t, testutils.TestDataPath(t, "data_key_manager"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "init": @@ -405,7 +406,7 @@ func TestDataKeyManagerIO(t *testing.T) { var dkm *DataKeyManager - datadriven.RunTest(t, "testdata/data_key_manager_io", + datadriven.RunTest(t, testutils.TestDataPath(t, "data_key_manager_io"), func(t *testing.T, d *datadriven.TestData) string { fmt.Println(d.Pos) buf.Reset() diff --git a/pkg/cli/doctor_test.go b/pkg/cli/doctor_test.go index 303efb095044..76ec85dcc109 100644 --- a/pkg/cli/doctor_test.go +++ b/pkg/cli/doctor_test.go @@ -14,6 +14,7 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/datadriven" ) @@ -43,7 +44,7 @@ func TestDoctorCluster(t *testing.T) { } // Using datadriven allows TESTFLAGS=-rewrite. - datadriven.RunTest(t, "testdata/doctor/test_examine_cluster", func(t *testing.T, td *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "doctor", "test_examine_cluster"), func(t *testing.T, td *datadriven.TestData) string { return out }) }) @@ -62,7 +63,7 @@ func TestDoctorZipDir(t *testing.T) { } // Using datadriven allows TESTFLAGS=-rewrite. - datadriven.RunTest(t, "testdata/doctor/test_examine_zipdir", func(t *testing.T, td *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "doctor", "test_examine_zipdir"), func(t *testing.T, td *datadriven.TestData) string { return out }) }) @@ -74,7 +75,7 @@ func TestDoctorZipDir(t *testing.T) { } // Using datadriven allows TESTFLAGS=-rewrite. - datadriven.RunTest(t, "testdata/doctor/test_recreate_zipdir", func(t *testing.T, td *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "doctor", "test_recreate_zipdir"), func(t *testing.T, td *datadriven.TestData) string { return out }) }) @@ -86,7 +87,7 @@ func TestDoctorZipDir(t *testing.T) { } // Using datadriven allows TESTFLAGS=-rewrite. - datadriven.RunTest(t, "testdata/doctor/test_examine_zipdir_verbose", func(t *testing.T, td *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "doctor", "test_examine_zipdir_verbose"), func(t *testing.T, td *datadriven.TestData) string { return out }) }) diff --git a/pkg/cli/import_test.go b/pkg/cli/import_test.go index 35e1ff86ffc6..639ea38ec81f 100644 --- a/pkg/cli/import_test.go +++ b/pkg/cli/import_test.go @@ -106,7 +106,7 @@ func TestImportCLI(t *testing.T) { { "pgdump", "PGDUMP", - "testdata/import/db.sql", + testutils.TestDataPath(t, "import", "db.sql"), "", "IMPORT PGDUMP 'userfile://defaultdb.public.userfiles_root/db." + "sql' WITH max_row_size='524288'", @@ -117,7 +117,7 @@ func TestImportCLI(t *testing.T) { { "pgdump-with-options", "PGDUMP", - "testdata/import/db.sql", + testutils.TestDataPath(t, "import", "db.sql"), "--max-row-size=1000 --skip-foreign-keys=true --row-limit=10 " + "--ignore-unsupported-statements=true --log-ignored-statements='foo://bar'", "IMPORT PGDUMP 'userfile://defaultdb.public.userfiles_root/db." + @@ -131,7 +131,7 @@ func TestImportCLI(t *testing.T) { { "pgdump-to-target-database", "PGDUMP", - "testdata/import/db.sql", + testutils.TestDataPath(t, "import", "db.sql"), "--ignore-unsupported-statements=true --url=postgresql:///baz", "IMPORT PGDUMP 'userfile://defaultdb.public.userfiles_root/db." + "sql' WITH max_row_size='524288', ignore_unsupported_statements", @@ -142,7 +142,7 @@ func TestImportCLI(t *testing.T) { { "mysql", "MYSQLDUMP", - "testdata/import/db.sql", + testutils.TestDataPath(t, "import", "db.sql"), "", "IMPORT MYSQLDUMP 'userfile://defaultdb.public.userfiles_root/db.sql'", "IMPORT TABLE foo FROM MYSQLDUMP 'userfile://defaultdb.public.userfiles_root/db.sql'", @@ -151,7 +151,7 @@ func TestImportCLI(t *testing.T) { { "mysql-with-options", "MYSQLDUMP", - "testdata/import/db.sql", + testutils.TestDataPath(t, "import", "db.sql"), "--skip-foreign-keys=true --row-limit=10", "IMPORT MYSQLDUMP 'userfile://defaultdb.public.userfiles_root/db." + "sql' WITH skip_foreign_keys, row_limit='10'", diff --git a/pkg/cli/log_flags_test.go b/pkg/cli/log_flags_test.go index ab8e87f39420..f7421debb7de 100644 --- a/pkg/cli/log_flags_test.go +++ b/pkg/cli/log_flags_test.go @@ -19,6 +19,7 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/logconfig" @@ -129,7 +130,7 @@ func TestSetupLogging(t *testing.T) { ctx := context.Background() - datadriven.RunTest(t, "testdata/logflags", func(t *testing.T, td *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "logflags"), func(t *testing.T, td *datadriven.TestData) string { args := strings.Split(td.Input, "\n") initCLIDefaults() diff --git a/pkg/cli/statement_bundle_test.go b/pkg/cli/statement_bundle_test.go index a819fa287f46..794b3ea39051 100644 --- a/pkg/cli/statement_bundle_test.go +++ b/pkg/cli/statement_bundle_test.go @@ -14,7 +14,6 @@ import ( "context" "net/url" "os" - "path/filepath" "testing" "github.com/cockroachdb/cockroach/pkg/base" @@ -23,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cli/clisqlclient" "github.com/cockroachdb/cockroach/pkg/cli/clisqlexec" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -65,7 +65,7 @@ func TestRunExplainCombinations(t *testing.T) { defer cleanupFn() conn := c.ConnCtx.MakeSQLConn(os.Stdout, os.Stdout, pgURL.String()) for _, test := range tests { - bundle, err := loadStatementBundle(filepath.Join("testdata/explain-bundle", test.bundlePath)) + bundle, err := loadStatementBundle(testutils.TestDataPath(t, "explain-bundle", test.bundlePath)) assert.NoError(t, err) // Disable autostats collection, which will override the injected stats. if err := conn.Exec(`SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false`, nil); err != nil { diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index 2e3b9fe68f3c..642bbc94f813 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -146,7 +146,7 @@ func TestZip(t *testing.T) { // We use datadriven simply to read the golden output file; we don't actually // run any commands. Using datadriven allows TESTFLAGS=-rewrite. - datadriven.RunTest(t, "testdata/zip/testzip", func(t *testing.T, td *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "zip", "testzip"), func(t *testing.T, td *datadriven.TestData) string { return out }) } @@ -198,7 +198,7 @@ func TestConcurrentZip(t *testing.T) { // We use datadriven simply to read the golden output file; we don't actually // run any commands. Using datadriven allows TESTFLAGS=-rewrite. - datadriven.RunTest(t, "testdata/zip/testzip_concurrent", func(t *testing.T, td *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "zip", "testzip_concurrent"), func(t *testing.T, td *datadriven.TestData) string { return out }) } @@ -237,7 +237,7 @@ create table defaultdb."../system"(x int); re := regexp.MustCompile(`(?m)^.*(table|database).*$`) out = strings.Join(re.FindAllString(out, -1), "\n") - datadriven.RunTest(t, "testdata/zip/specialnames", + datadriven.RunTest(t, testutils.TestDataPath(t, "zip", "specialnames"), func(t *testing.T, td *datadriven.TestData) string { return out }) @@ -322,7 +322,7 @@ func TestUnavailableZip(t *testing.T) { re := regexp.MustCompile(`(?m)^(requesting ranges.*found|writing: debug/nodes/\d+/ranges).*\n`) out = re.ReplaceAllString(out, ``) - datadriven.RunTest(t, "testdata/zip/unavailable", + datadriven.RunTest(t, testutils.TestDataPath(t, "zip", "unavailable"), func(t *testing.T, td *datadriven.TestData) string { return out }) @@ -401,7 +401,7 @@ func TestPartialZip(t *testing.T) { t.Log(out) out = eraseNonDeterministicZipOutput(out) - datadriven.RunTest(t, "testdata/zip/partial1", + datadriven.RunTest(t, testutils.TestDataPath(t, "zip", "partial1"), func(t *testing.T, td *datadriven.TestData) string { return out }) @@ -413,7 +413,7 @@ func TestPartialZip(t *testing.T) { } out = eraseNonDeterministicZipOutput(out) - datadriven.RunTest(t, "testdata/zip/partial1_excluded", + datadriven.RunTest(t, testutils.TestDataPath(t, "zip", "partial1_excluded"), func(t *testing.T, td *datadriven.TestData) string { return out }) @@ -439,7 +439,7 @@ func TestPartialZip(t *testing.T) { // This last case may take a little while to converge. To make this work with datadriven and at the same // time retain the ability to use the `-rewrite` flag, we use a retry loop within that already checks the // output ahead of time and retries for some time if necessary. - datadriven.RunTest(t, "testdata/zip/partial2", + datadriven.RunTest(t, testutils.TestDataPath(t, "zip", "partial2"), func(t *testing.T, td *datadriven.TestData) string { f := func() string { out, err := c.RunWithCapture("debug zip --concurrency=1 --cpu-profile-duration=0 " + os.DevNull) diff --git a/pkg/cmd/github-pull-request-make/BUILD.bazel b/pkg/cmd/github-pull-request-make/BUILD.bazel index 2b9b0833db21..9188f42cb20b 100644 --- a/pkg/cmd/github-pull-request-make/BUILD.bazel +++ b/pkg/cmd/github-pull-request-make/BUILD.bazel @@ -26,6 +26,7 @@ go_test( data = glob(["testdata/**"]), embed = [":github-pull-request-make_lib"], deps = [ + "//pkg/testutils", "//pkg/testutils/skip", "@com_github_kr_pretty//:pretty", ], diff --git a/pkg/cmd/github-pull-request-make/main_test.go b/pkg/cmd/github-pull-request-make/main_test.go index 0b429d52c3d1..29e495368c94 100644 --- a/pkg/cmd/github-pull-request-make/main_test.go +++ b/pkg/cmd/github-pull-request-make/main_test.go @@ -20,27 +20,28 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/kr/pretty" ) func TestPkgsFromDiff(t *testing.T) { for filename, expPkgs := range map[string]map[string]pkg{ - "testdata/10305.diff": { + testutils.TestDataPath(t, "10305.diff"): { "pkg/roachpb": {tests: []string{"TestLeaseEquivalence"}}, "pkg/storage": {tests: []string{"TestStoreRangeLease", "TestStoreRangeLeaseSwitcheroo"}}, }, - "testdata/skip.diff": { + testutils.TestDataPath(t, "skip.diff"): { "pkg/ccl/storageccl": {tests: []string{"TestPutS3"}}, }, // This PR had some churn and renamed packages. This was formerly problematic // because nonexistent packages would be emitted. - "testdata/27595.diff": { + testutils.TestDataPath(t, "27595.diff"): { "pkg/storage/closedts/transport": {tests: []string{"TestTransportConnectOnRequest", "TestTransportClientReceivesEntries"}}, "pkg/storage/closedts/container": {tests: []string{"TestTwoNodes"}}, "pkg/storage/closedts/storage": {tests: []string{"TestConcurrent"}}, }, - "testdata/removed.diff": {}, + testutils.TestDataPath(t, "removed.diff"): {}, } { t.Run(filename, func(t *testing.T) { f, err := os.Open(filename) diff --git a/pkg/geo/geoindex/BUILD.bazel b/pkg/geo/geoindex/BUILD.bazel index 6d367ad2cfa0..361f6619f931 100644 --- a/pkg/geo/geoindex/BUILD.bazel +++ b/pkg/geo/geoindex/BUILD.bazel @@ -43,6 +43,7 @@ go_test( "//pkg/geo/geopb", "//pkg/geo/geoprojbase", "//pkg/geo/geos", + "//pkg/testutils", "//pkg/util/leaktest", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_golang_geo//s2", diff --git a/pkg/geo/geoindex/s2_geography_index_test.go b/pkg/geo/geoindex/s2_geography_index_test.go index 7849b2c4d4f0..dfd75d4a5f7b 100644 --- a/pkg/geo/geoindex/s2_geography_index_test.go +++ b/pkg/geo/geoindex/s2_geography_index_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/geo" "github.com/cockroachdb/cockroach/pkg/geo/geogfn" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/datadriven" ) @@ -33,7 +34,7 @@ func TestS2GeographyIndexBasic(t *testing.T) { ctx := context.Background() var index GeographyIndex shapes := make(map[string]geo.Geography) - datadriven.RunTest(t, "testdata/s2_geography", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "s2_geography"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "init": cfg := s2Config(t, d) diff --git a/pkg/geo/geoindex/s2_geometry_index_test.go b/pkg/geo/geoindex/s2_geometry_index_test.go index d87616a189ee..d971dd3c96bc 100644 --- a/pkg/geo/geoindex/s2_geometry_index_test.go +++ b/pkg/geo/geoindex/s2_geometry_index_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/geo" "github.com/cockroachdb/cockroach/pkg/geo/geoprojbase" "github.com/cockroachdb/cockroach/pkg/geo/geos" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/datadriven" "github.com/stretchr/testify/require" @@ -30,7 +31,7 @@ func TestS2GeometryIndexBasic(t *testing.T) { ctx := context.Background() var index GeometryIndex shapes := make(map[string]geo.Geometry) - datadriven.RunTest(t, "testdata/s2_geometry", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "s2_geometry"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "init": cfg := s2Config(t, d) @@ -83,7 +84,7 @@ func TestClipByRect(t *testing.T) { var g geo.Geometry var err error - datadriven.RunTest(t, "testdata/clip", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "clip"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "geometry": g, err = geo.ParseGeometry(d.Input) diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints_test.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints_test.go index 6ac9624b0095..dd41d50adf7f 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_savepoints_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/kvclientutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -41,7 +42,7 @@ func TestSavepoints(t *testing.T) { abortKey := roachpb.Key("abort") errKey := roachpb.Key("injectErr") - datadriven.Walk(t, "testdata/savepoints", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "savepoints"), func(t *testing.T, path string) { // We want to inject txn abort errors in some cases. // // We do this by injecting the error from "underneath" the diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go index bb8a2a4f09c9..a959b00a7dcb 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go @@ -84,7 +84,7 @@ func TestConcurrencyManagerBasic(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - datadriven.Walk(t, "testdata/concurrency_manager", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "concurrency_manager"), func(t *testing.T, path string) { c := newCluster() c.enableTxnPushes() m := concurrency.NewManager(c.makeConfig()) diff --git a/pkg/kv/kvserver/concurrency/lock_table_test.go b/pkg/kv/kvserver/concurrency/lock_table_test.go index 31970eb26ead..a6c4d02bd608 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -159,7 +160,7 @@ func TestLockTableBasic(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - datadriven.Walk(t, "testdata/lock_table", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "lock_table"), func(t *testing.T, path string) { var lt lockTable var txnsByName map[string]*enginepb.TxnMeta var txnCounter uint128.Uint128 diff --git a/pkg/kv/kvserver/replica_raft_truncation_test.go b/pkg/kv/kvserver/replica_raft_truncation_test.go index ba25718fe9d0..ae6d1ed0f5e2 100644 --- a/pkg/kv/kvserver/replica_raft_truncation_test.go +++ b/pkg/kv/kvserver/replica_raft_truncation_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -36,7 +37,7 @@ func TestHandleTruncatedStateBelowRaft(t *testing.T) { // truncation code. ctx := context.Background() - datadriven.Walk(t, "testdata/truncated_state", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "truncated_state"), func(t *testing.T, path string) { const rangeID = 12 loader := stateloader.Make(rangeID) eng := storage.NewDefaultInMemForTesting() diff --git a/pkg/server/debug/pprofui/BUILD.bazel b/pkg/server/debug/pprofui/BUILD.bazel index 0ec2299839a9..aec52ad2320d 100644 --- a/pkg/server/debug/pprofui/BUILD.bazel +++ b/pkg/server/debug/pprofui/BUILD.bazel @@ -34,6 +34,7 @@ go_test( deps = [ "//pkg/build/bazel", "//pkg/server/serverpb", + "//pkg/testutils", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/server/debug/pprofui/server_test.go b/pkg/server/debug/pprofui/server_test.go index 8ea1926b5517..c2942fc67aef 100644 --- a/pkg/server/debug/pprofui/server_test.go +++ b/pkg/server/debug/pprofui/server_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/build/bazel" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/stretchr/testify/require" ) @@ -51,7 +52,7 @@ func TestServer(t *testing.T) { mockProfile := func(ctx context.Context, req *serverpb.ProfileRequest) (*serverpb.JSONResponse, error) { require.Equal(t, expectedNodeID, req.NodeId) - b, err := ioutil.ReadFile("testdata/heap.profile") + b, err := ioutil.ReadFile(testutils.TestDataPath(t, "heap.profile")) require.NoError(t, err) return &serverpb.JSONResponse{Data: b}, nil } diff --git a/pkg/server/pagination_test.go b/pkg/server/pagination_test.go index b757d069a19d..5286246a1c30 100644 --- a/pkg/server/pagination_test.go +++ b/pkg/server/pagination_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/datadriven" @@ -48,7 +49,7 @@ func TestSimplePaginate(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - datadriven.RunTest(t, "testdata/simple_paginate", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "simple_paginate"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "paginate": var input interface{} @@ -200,7 +201,7 @@ func TestPaginationState(t *testing.T) { } var state paginationState - datadriven.RunTest(t, "testdata/pagination_state", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "pagination_state"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "define": state = paginationState{} diff --git a/pkg/server/pgurl/BUILD.bazel b/pkg/server/pgurl/BUILD.bazel index 220ed2c261fa..c00ce2df076b 100644 --- a/pkg/server/pgurl/BUILD.bazel +++ b/pkg/server/pgurl/BUILD.bazel @@ -23,6 +23,7 @@ go_test( data = glob(["testdata/**"]), embed = [":pgurl"], deps = [ + "//pkg/testutils", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", diff --git a/pkg/server/pgurl/pgurl_test.go b/pkg/server/pgurl/pgurl_test.go index f18c448748f5..52204fb5fe62 100644 --- a/pkg/server/pgurl/pgurl_test.go +++ b/pkg/server/pgurl/pgurl_test.go @@ -16,13 +16,14 @@ import ( "net/url" "testing" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/datadriven" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) func TestURL(t *testing.T) { - datadriven.RunTest(t, "testdata/url", func(t *testing.T, td *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "url"), func(t *testing.T, td *datadriven.TestData) string { var result bytes.Buffer var u *URL diff --git a/pkg/sql/catalog/nstree/BUILD.bazel b/pkg/sql/catalog/nstree/BUILD.bazel index 9e6a0bd9d236..2171b99c9120 100644 --- a/pkg/sql/catalog/nstree/BUILD.bazel +++ b/pkg/sql/catalog/nstree/BUILD.bazel @@ -33,6 +33,7 @@ go_test( deps = [ "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", + "//pkg/testutils", "//pkg/util/iterutil", "@com_github_cockroachdb_datadriven//:datadriven", ], diff --git a/pkg/sql/catalog/nstree/map_test.go b/pkg/sql/catalog/nstree/map_test.go index 296d1d259026..d58fef7af21f 100644 --- a/pkg/sql/catalog/nstree/map_test.go +++ b/pkg/sql/catalog/nstree/map_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/datadriven" ) @@ -52,7 +53,7 @@ import ( // If no such entry exists, "not found" will be printed. // func TestMapDataDriven(t *testing.T) { - datadriven.Walk(t, "testdata/map", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "map"), func(t *testing.T, path string) { var tr Map datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { return testMapDataDriven(t, d, &tr) diff --git a/pkg/sql/catalog/nstree/set_test.go b/pkg/sql/catalog/nstree/set_test.go index be20a1e52d26..f078ad391c67 100644 --- a/pkg/sql/catalog/nstree/set_test.go +++ b/pkg/sql/catalog/nstree/set_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/datadriven" ) @@ -35,7 +36,7 @@ import ( // Clears the tree. // func TestSetDataDriven(t *testing.T) { - datadriven.Walk(t, "testdata/set", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "set"), func(t *testing.T, path string) { var tr Set datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { return testSetDataDriven(t, d, &tr) diff --git a/pkg/sql/catalog/schemaexpr/BUILD.bazel b/pkg/sql/catalog/schemaexpr/BUILD.bazel index 30c14e801d47..72de4c503d14 100644 --- a/pkg/sql/catalog/schemaexpr/BUILD.bazel +++ b/pkg/sql/catalog/schemaexpr/BUILD.bazel @@ -56,6 +56,7 @@ go_test( "//pkg/sql/sem/builtins", "//pkg/sql/sem/tree", "//pkg/sql/types", + "//pkg/testutils", "//pkg/util/leaktest", "//pkg/util/log", "@com_github_cockroachdb_datadriven//:datadriven", diff --git a/pkg/sql/catalog/schemaexpr/computed_column_rewrites_test.go b/pkg/sql/catalog/schemaexpr/computed_column_rewrites_test.go index fe8b6a535b9b..e4deb5295aac 100644 --- a/pkg/sql/catalog/schemaexpr/computed_column_rewrites_test.go +++ b/pkg/sql/catalog/schemaexpr/computed_column_rewrites_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/datadriven" @@ -26,7 +27,7 @@ func TestParseComputedColumnRewrites(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - path := "testdata/computed_column_rewrites" + path := testutils.TestDataPath(t, "computed_column_rewrites") datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "parse": diff --git a/pkg/sql/conn_executor_savepoints_test.go b/pkg/sql/conn_executor_savepoints_test.go index 9298908f63be..7eada8158eed 100644 --- a/pkg/sql/conn_executor_savepoints_test.go +++ b/pkg/sql/conn_executor_savepoints_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -31,7 +32,7 @@ func TestSavepoints(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - datadriven.Walk(t, "testdata/savepoints", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "savepoints"), func(t *testing.T, path string) { params := base.TestServerArgs{} s, origConn, _ := serverutils.StartServer(t, params) diff --git a/pkg/sql/contention/BUILD.bazel b/pkg/sql/contention/BUILD.bazel index 5216bac4e893..079607680f88 100644 --- a/pkg/sql/contention/BUILD.bazel +++ b/pkg/sql/contention/BUILD.bazel @@ -31,6 +31,7 @@ go_test( "//pkg/roachpb:with-mocks", "//pkg/sql/contentionpb", "//pkg/storage/enginepb", + "//pkg/testutils", "//pkg/util/cache", "//pkg/util/encoding", "//pkg/util/leaktest", diff --git a/pkg/sql/contention/registry_test.go b/pkg/sql/contention/registry_test.go index 4b20b9ca6bb6..fbecb6b2ff51 100644 --- a/pkg/sql/contention/registry_test.go +++ b/pkg/sql/contention/registry_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/contention" "github.com/cockroachdb/cockroach/pkg/sql/contentionpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -70,7 +71,7 @@ func TestRegistry(t *testing.T) { registryMap := make(map[string]*contention.Registry) // registry is the current registry. var registry *contention.Registry - datadriven.RunTest(t, "testdata/contention_registry", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "contention_registry"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "use": var registryKey string diff --git a/pkg/sql/explain_tree_test.go b/pkg/sql/explain_tree_test.go index 30cdfa17bf23..0b936fdfc623 100644 --- a/pkg/sql/explain_tree_test.go +++ b/pkg/sql/explain_tree_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/sessionphase" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -46,7 +47,7 @@ func TestPlanToTreeAndPlanToString(t *testing.T) { USE t; `) - datadriven.RunTest(t, "testdata/explain_tree", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "explain_tree"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "exec": r.Exec(t, d.Input) diff --git a/pkg/sql/inverted/BUILD.bazel b/pkg/sql/inverted/BUILD.bazel index 54a0f836edea..89b280d3fe66 100644 --- a/pkg/sql/inverted/BUILD.bazel +++ b/pkg/sql/inverted/BUILD.bazel @@ -22,6 +22,7 @@ go_test( data = glob(["testdata/**"]), embed = [":inverted"], deps = [ + "//pkg/testutils", "//pkg/util/encoding", "//pkg/util/leaktest", "//pkg/util/treeprinter", diff --git a/pkg/sql/inverted/expression_test.go b/pkg/sql/inverted/expression_test.go index 20ef38901445..cb5f3b36f284 100644 --- a/pkg/sql/inverted/expression_test.go +++ b/pkg/sql/inverted/expression_test.go @@ -16,6 +16,7 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/treeprinter" @@ -139,7 +140,7 @@ func TestExpression(t *testing.T) { defer leaktest.AfterTest(t)() exprsByName := make(map[string]Expression) - datadriven.RunTest(t, "testdata/expression", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "expression"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "new-span-leaf": var name string diff --git a/pkg/sql/logictest/BUILD.bazel b/pkg/sql/logictest/BUILD.bazel index 4cf288169f86..c75251190ec8 100644 --- a/pkg/sql/logictest/BUILD.bazel +++ b/pkg/sql/logictest/BUILD.bazel @@ -78,6 +78,7 @@ go_test( "//pkg/sql/sessiondatapb", "//pkg/sql/stats", "//pkg/sql/tests", + "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/skip", "//pkg/testutils/sqlutils", diff --git a/pkg/sql/logictest/logic_test.go b/pkg/sql/logictest/logic_test.go index b42e1c94751a..76d80a5acf00 100644 --- a/pkg/sql/logictest/logic_test.go +++ b/pkg/sql/logictest/logic_test.go @@ -13,6 +13,7 @@ package logictest import ( "testing" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) @@ -28,7 +29,7 @@ import ( func TestLogic(t *testing.T) { defer leaktest.AfterTest(t)() skip.UnderDeadlock(t, "times out and/or hangs") - RunLogicTest(t, TestServerArgs{}, "testdata/logic_test/[^.]*") + RunLogicTest(t, TestServerArgs{}, testutils.TestDataPath(t, "logic_test", "[^.]*")) } // TestSqlLiteLogic runs the supported SqlLite logic tests. See the comments diff --git a/pkg/sql/logictest/parallel_test.go b/pkg/sql/logictest/parallel_test.go index fadd1761efb8..8a4295985d97 100644 --- a/pkg/sql/logictest/parallel_test.go +++ b/pkg/sql/logictest/parallel_test.go @@ -48,7 +48,7 @@ import ( ) var ( - paralleltestdata = flag.String("partestdata", "testdata/parallel_test/[^.]*", "test data glob") + paralleltestdata = flag.String("partestdata", filepath.Join("testdata", "parallel_test", "[^.]*"), "test data glob") ) type parallelTest struct { diff --git a/pkg/sql/opt/exec/execbuilder/BUILD.bazel b/pkg/sql/opt/exec/execbuilder/BUILD.bazel index aaca89419b73..75ef860d3981 100644 --- a/pkg/sql/opt/exec/execbuilder/BUILD.bazel +++ b/pkg/sql/opt/exec/execbuilder/BUILD.bazel @@ -69,6 +69,7 @@ go_test( "//pkg/sql/logictest", "//pkg/sql/opt", "//pkg/sql/sem/tree", + "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/skip", "//pkg/testutils/testcluster", diff --git a/pkg/sql/opt/exec/execbuilder/builder_test.go b/pkg/sql/opt/exec/execbuilder/builder_test.go index aae841faa15f..9f7655c70674 100644 --- a/pkg/sql/opt/exec/execbuilder/builder_test.go +++ b/pkg/sql/opt/exec/execbuilder/builder_test.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/logictest" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) @@ -29,5 +30,5 @@ func TestExecBuild(t *testing.T) { defer leaktest.AfterTest(t)() defer sql.TestingOverrideExplainEnvVersion("CockroachDB execbuilder test version")() skip.UnderDeadlock(t, "times out and/or hangs") - logictest.RunLogicTest(t, logictest.TestServerArgs{}, "testdata/[^.]*") + logictest.RunLogicTest(t, logictest.TestServerArgs{}, testutils.TestDataPath(t, "[^.]*")) } diff --git a/pkg/sql/opt/exec/explain/BUILD.bazel b/pkg/sql/opt/exec/explain/BUILD.bazel index 8bb74feca4c0..1cff6728b734 100644 --- a/pkg/sql/opt/exec/explain/BUILD.bazel +++ b/pkg/sql/opt/exec/explain/BUILD.bazel @@ -62,6 +62,7 @@ go_test( "//pkg/sql/opt/testutils/testcat", "//pkg/sql/sem/tree", "//pkg/sql/types", + "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/testcluster", "//pkg/util", diff --git a/pkg/sql/opt/exec/explain/output_test.go b/pkg/sql/opt/exec/explain/output_test.go index e0d298c25f34..897832656caf 100644 --- a/pkg/sql/opt/exec/explain/output_test.go +++ b/pkg/sql/opt/exec/explain/output_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/exec/explain" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util" @@ -74,7 +75,7 @@ func TestOutputBuilder(t *testing.T) { return ob } - datadriven.RunTest(t, "testdata/output", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "output"), func(t *testing.T, d *datadriven.TestData) string { var flags explain.Flags for _, arg := range d.CmdArgs { switch arg.Key { diff --git a/pkg/sql/opt/exec/explain/plan_gist_test.go b/pkg/sql/opt/exec/explain/plan_gist_test.go index a30533b07881..186a24bd1840 100644 --- a/pkg/sql/opt/exec/explain/plan_gist_test.go +++ b/pkg/sql/opt/exec/explain/plan_gist_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/opttester" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/testcat" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/datadriven" ) @@ -116,8 +117,8 @@ func TestPlanGistBuilder(t *testing.T) { } } // RFC: should I move this to opt_tester? - datadriven.RunTest(t, "testdata/gists", testGists) - datadriven.RunTest(t, "testdata/gists_tpce", testGists) + datadriven.RunTest(t, testutils.TestDataPath(t, "gists"), testGists) + datadriven.RunTest(t, testutils.TestDataPath(t, "gists_tpce"), testGists) } func TestPlanGistHashEquivalency(t *testing.T) { diff --git a/pkg/sql/opt/idxconstraint/BUILD.bazel b/pkg/sql/opt/idxconstraint/BUILD.bazel index 2930e5d73e23..2cfb9b9e323c 100644 --- a/pkg/sql/opt/idxconstraint/BUILD.bazel +++ b/pkg/sql/opt/idxconstraint/BUILD.bazel @@ -33,6 +33,7 @@ go_test( "//pkg/sql/opt/testutils", "//pkg/sql/parser", "//pkg/sql/sem/tree", + "//pkg/testutils", "//pkg/util/leaktest", "@com_github_cockroachdb_datadriven//:datadriven", ], diff --git a/pkg/sql/opt/idxconstraint/index_constraints_test.go b/pkg/sql/opt/idxconstraint/index_constraints_test.go index 8950398b9eb2..69a1230e120f 100644 --- a/pkg/sql/opt/idxconstraint/index_constraints_test.go +++ b/pkg/sql/opt/idxconstraint/index_constraints_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + tu "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/datadriven" ) @@ -57,7 +58,7 @@ import ( func TestIndexConstraints(t *testing.T) { defer leaktest.AfterTest(t)() - datadriven.Walk(t, testutils.TestDataPath(t), func(t *testing.T, path string) { + datadriven.Walk(t, tu.TestDataPath(t), func(t *testing.T, path string) { semaCtx := tree.MakeSemaContext() evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) diff --git a/pkg/sql/opt/memo/expr_test.go b/pkg/sql/opt/memo/expr_test.go index b1db007c3534..9d261e74b019 100644 --- a/pkg/sql/opt/memo/expr_test.go +++ b/pkg/sql/opt/memo/expr_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" _ "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + tu "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/datadriven" ) @@ -53,7 +54,7 @@ import ( func TestExprIsNeverNull(t *testing.T) { defer leaktest.AfterTest(t)() - datadriven.Walk(t, "testdata/expr", func(t *testing.T, path string) { + datadriven.Walk(t, tu.TestDataPath(t, "expr"), func(t *testing.T, path string) { catalog := testcat.New() datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { diff --git a/pkg/sql/opt/memo/memo_test.go b/pkg/sql/opt/memo/memo_test.go index 3600d1069d56..6c3f8ab54214 100644 --- a/pkg/sql/opt/memo/memo_test.go +++ b/pkg/sql/opt/memo/memo_test.go @@ -34,32 +34,32 @@ import ( func TestMemo(t *testing.T) { flags := memo.ExprFmtHideCost | memo.ExprFmtHideRuleProps | memo.ExprFmtHideQualifications | memo.ExprFmtHideStats - runDataDrivenTest(t, "testdata/memo", flags) + runDataDrivenTest(t, testutils.TestDataPath(t, "memo"), flags) } func TestFormat(t *testing.T) { - runDataDrivenTest(t, "testdata/format", memo.ExprFmtShowAll) + runDataDrivenTest(t, testutils.TestDataPath(t, "format"), memo.ExprFmtShowAll) } func TestLogicalProps(t *testing.T) { flags := memo.ExprFmtHideCost | memo.ExprFmtHideQualifications | memo.ExprFmtHideStats - runDataDrivenTest(t, "testdata/logprops/", flags) + runDataDrivenTest(t, testutils.TestDataPath(t, "logprops"), flags) } func TestStats(t *testing.T) { flags := memo.ExprFmtHideCost | memo.ExprFmtHideRuleProps | memo.ExprFmtHideQualifications | memo.ExprFmtHideScalars - runDataDrivenTest(t, "testdata/stats/", flags) + runDataDrivenTest(t, testutils.TestDataPath(t, "stats"), flags) } func TestStatsQuality(t *testing.T) { flags := memo.ExprFmtHideCost | memo.ExprFmtHideRuleProps | memo.ExprFmtHideQualifications | memo.ExprFmtHideScalars - runDataDrivenTest(t, "testdata/stats_quality/", flags) + runDataDrivenTest(t, testutils.TestDataPath(t, "stats_quality"), flags) } func TestCompositeSensitive(t *testing.T) { - datadriven.RunTest(t, "testdata/composite_sensitive", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "composite_sensitive"), func(t *testing.T, d *datadriven.TestData) string { semaCtx := tree.MakeSemaContext() evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) diff --git a/pkg/sql/opt/memo/typing_test.go b/pkg/sql/opt/memo/typing_test.go index e7d728a2a637..eff37cba4ba7 100644 --- a/pkg/sql/opt/memo/typing_test.go +++ b/pkg/sql/opt/memo/typing_test.go @@ -18,10 +18,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + tu "github.com/cockroachdb/cockroach/pkg/testutils" ) func TestTyping(t *testing.T) { - runDataDrivenTest(t, "testdata/typing", + runDataDrivenTest(t, tu.TestDataPath(t, "typing"), memo.ExprFmtHideMiscProps| memo.ExprFmtHideConstraints| memo.ExprFmtHideFuncDeps| diff --git a/pkg/sql/opt/norm/BUILD.bazel b/pkg/sql/opt/norm/BUILD.bazel index 02249b60748a..869b29464333 100644 --- a/pkg/sql/opt/norm/BUILD.bazel +++ b/pkg/sql/opt/norm/BUILD.bazel @@ -80,6 +80,7 @@ go_test( "//pkg/sql/opt/xform", "//pkg/sql/sem/tree", "//pkg/sql/types", + "//pkg/testutils", "//pkg/util/leaktest", "//pkg/util/log", "@com_github_cockroachdb_datadriven//:datadriven", diff --git a/pkg/sql/opt/norm/norm_test.go b/pkg/sql/opt/norm/norm_test.go index a1706f443ffc..a3274650a7a7 100644 --- a/pkg/sql/opt/norm/norm_test.go +++ b/pkg/sql/opt/norm/norm_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/opttester" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/testcat" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/datadriven" @@ -42,7 +43,7 @@ func TestNormRules(t *testing.T) { const fmtFlags = memo.ExprFmtHideStats | memo.ExprFmtHideCost | memo.ExprFmtHideRuleProps | memo.ExprFmtHideQualifications | memo.ExprFmtHideScalars | memo.ExprFmtHideTypes - datadriven.Walk(t, "testdata/rules", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "rules"), func(t *testing.T, path string) { catalog := testcat.New() datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { tester := opttester.New(catalog, d.Input) @@ -61,7 +62,7 @@ func TestNormRuleProps(t *testing.T) { const fmtFlags = memo.ExprFmtHideStats | memo.ExprFmtHideCost | memo.ExprFmtHideQualifications | memo.ExprFmtHideScalars | memo.ExprFmtHideTypes - datadriven.Walk(t, "testdata/ruleprops", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "ruleprops"), func(t *testing.T, path string) { catalog := testcat.New() datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { tester := opttester.New(catalog, d.Input) diff --git a/pkg/sql/opt/opbench/BUILD.bazel b/pkg/sql/opt/opbench/BUILD.bazel index 5bf8f01f2505..16d1bb75b089 100644 --- a/pkg/sql/opt/opbench/BUILD.bazel +++ b/pkg/sql/opt/opbench/BUILD.bazel @@ -24,6 +24,7 @@ go_test( deps = [ "//pkg/sql/opt/memo", "//pkg/sql/opt/testutils/opttester", + "//pkg/testutils", "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", ], diff --git a/pkg/sql/opt/opbench/opbench_test.go b/pkg/sql/opt/opbench/opbench_test.go index dabeb9664b82..887662f7d38a 100644 --- a/pkg/sql/opt/opbench/opbench_test.go +++ b/pkg/sql/opt/opbench/opbench_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" "github.com/cockroachdb/cockroach/pkg/sql/opt/opbench" "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/opttester" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" ) @@ -78,7 +79,7 @@ func TestOpBench(t *testing.T) { } for _, spec := range Benches { t.Run(spec.Name, func(t *testing.T) { - runBench(t, spec, fmt.Sprintf("testdata/%s.csv", spec.Name), rm) + runBench(t, spec, fmt.Sprintf(testutils.TestDataPath(t, "%s.csv"), spec.Name), rm) }) } } diff --git a/pkg/sql/opt/optbuilder/BUILD.bazel b/pkg/sql/opt/optbuilder/BUILD.bazel index b76c8761005d..c88eee0e8839 100644 --- a/pkg/sql/opt/optbuilder/BUILD.bazel +++ b/pkg/sql/opt/optbuilder/BUILD.bazel @@ -106,6 +106,7 @@ go_test( "//pkg/sql/sem/builtins", "//pkg/sql/sem/tree", "//pkg/sql/types", + "//pkg/testutils", "//pkg/util/leaktest", "@com_github_cockroachdb_datadriven//:datadriven", ], diff --git a/pkg/sql/opt/optbuilder/builder_test.go b/pkg/sql/opt/optbuilder/builder_test.go index 13d127c810bf..5072003754a4 100644 --- a/pkg/sql/opt/optbuilder/builder_test.go +++ b/pkg/sql/opt/optbuilder/builder_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" _ "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + tu "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/datadriven" ) @@ -53,7 +54,7 @@ import ( func TestBuilder(t *testing.T) { defer leaktest.AfterTest(t)() - datadriven.Walk(t, testutils.TestDataPath(t), func(t *testing.T, path string) { + datadriven.Walk(t, tu.TestDataPath(t), func(t *testing.T, path string) { catalog := testcat.New() datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { diff --git a/pkg/sql/opt/optgen/lang/BUILD.bazel b/pkg/sql/opt/optgen/lang/BUILD.bazel index ffb943227760..41b4c4beefc6 100644 --- a/pkg/sql/opt/optgen/lang/BUILD.bazel +++ b/pkg/sql/opt/optgen/lang/BUILD.bazel @@ -55,6 +55,7 @@ go_test( data = glob(["testdata/**"]), embed = [":lang"], deps = [ + "//pkg/testutils", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_cockroachdb_errors//:errors", ], diff --git a/pkg/sql/opt/optgen/lang/compiler_test.go b/pkg/sql/opt/optgen/lang/compiler_test.go index 68de97c85bb8..d17d37cf1ebc 100644 --- a/pkg/sql/opt/optgen/lang/compiler_test.go +++ b/pkg/sql/opt/optgen/lang/compiler_test.go @@ -16,12 +16,13 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/datadriven" "github.com/cockroachdb/errors" ) func TestCompiler(t *testing.T) { - datadriven.RunTest(t, "testdata/compiler", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "compiler"), func(t *testing.T, d *datadriven.TestData) string { // Only compile command supported. if d.Cmd != "compile" { t.FailNow() diff --git a/pkg/sql/opt/optgen/lang/parser_test.go b/pkg/sql/opt/optgen/lang/parser_test.go index ffb1cc3e7e91..53a7d76ad09c 100644 --- a/pkg/sql/opt/optgen/lang/parser_test.go +++ b/pkg/sql/opt/optgen/lang/parser_test.go @@ -16,11 +16,12 @@ import ( "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/datadriven" ) func TestParser(t *testing.T) { - datadriven.RunTest(t, "testdata/parser", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "parser"), func(t *testing.T, d *datadriven.TestData) string { // Only parse command supported. if d.Cmd != "parse" { t.FailNow() diff --git a/pkg/sql/opt/optgen/lang/scanner_test.go b/pkg/sql/opt/optgen/lang/scanner_test.go index 408c47705c46..6159e7ce6e17 100644 --- a/pkg/sql/opt/optgen/lang/scanner_test.go +++ b/pkg/sql/opt/optgen/lang/scanner_test.go @@ -19,11 +19,12 @@ import ( "testing" "testing/iotest" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/datadriven" ) func TestScanner(t *testing.T) { - datadriven.RunTest(t, "testdata/scanner", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "scanner"), func(t *testing.T, d *datadriven.TestData) string { // Only scan command supported. if d.Cmd != "scan" { t.FailNow() diff --git a/pkg/sql/opt/partialidx/BUILD.bazel b/pkg/sql/opt/partialidx/BUILD.bazel index 88a04482a21e..d341e09194a1 100644 --- a/pkg/sql/opt/partialidx/BUILD.bazel +++ b/pkg/sql/opt/partialidx/BUILD.bazel @@ -33,6 +33,7 @@ go_test( "//pkg/sql/opt/testutils", "//pkg/sql/parser", "//pkg/sql/sem/tree", + "//pkg/testutils", "//pkg/util/leaktest", "@com_github_cockroachdb_datadriven//:datadriven", ], diff --git a/pkg/sql/opt/partialidx/implicator_test.go b/pkg/sql/opt/partialidx/implicator_test.go index dc56b67c20bf..2e9b38f124de 100644 --- a/pkg/sql/opt/partialidx/implicator_test.go +++ b/pkg/sql/opt/partialidx/implicator_test.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/testutils" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + tu "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/datadriven" ) @@ -51,7 +52,7 @@ import ( func TestImplicator(t *testing.T) { defer leaktest.AfterTest(t)() - datadriven.Walk(t, "testdata/implicator", func(t *testing.T, path string) { + datadriven.Walk(t, tu.TestDataPath(t, "implicator"), func(t *testing.T, path string) { semaCtx := tree.MakeSemaContext() evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) diff --git a/pkg/sql/opt/xform/BUILD.bazel b/pkg/sql/opt/xform/BUILD.bazel index e46751d061a1..332a5c035269 100644 --- a/pkg/sql/opt/xform/BUILD.bazel +++ b/pkg/sql/opt/xform/BUILD.bazel @@ -86,6 +86,7 @@ go_test( "//pkg/sql/opt/testutils/testcat", "//pkg/sql/sem/tree", "//pkg/sql/types", + "//pkg/testutils", "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/randutil", diff --git a/pkg/sql/opt/xform/optimizer_test.go b/pkg/sql/opt/xform/optimizer_test.go index 030b2dae9437..a8242531eab3 100644 --- a/pkg/sql/opt/xform/optimizer_test.go +++ b/pkg/sql/opt/xform/optimizer_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/xform" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + tu "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/datadriven" @@ -138,7 +139,7 @@ func TestCoster(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) runDataDrivenTest( - t, "testdata/coster/", + t, tu.TestDataPath(t, "coster", ""), memo.ExprFmtHideRuleProps|memo.ExprFmtHideQualifications|memo.ExprFmtHideScalars| memo.ExprFmtHideTypes, ) @@ -152,7 +153,7 @@ func TestPhysicalProps(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) runDataDrivenTest( - t, "testdata/physprops/", + t, tu.TestDataPath(t, "physprops", ""), memo.ExprFmtHideConstraints| memo.ExprFmtHideRuleProps| memo.ExprFmtHideStats| @@ -171,7 +172,7 @@ func TestRuleProps(t *testing.T) { defer log.Scope(t).Close(t) runDataDrivenTest( t, - "testdata/ruleprops/", + tu.TestDataPath(t, "ruleprops"), memo.ExprFmtHideStats|memo.ExprFmtHideCost|memo.ExprFmtHideQualifications| memo.ExprFmtHideScalars|memo.ExprFmtHideTypes, ) @@ -186,14 +187,14 @@ func TestRules(t *testing.T) { defer log.Scope(t).Close(t) runDataDrivenTest( t, - "testdata/rules/", + tu.TestDataPath(t, "rules"), memo.ExprFmtHideStats|memo.ExprFmtHideCost|memo.ExprFmtHideRuleProps| memo.ExprFmtHideQualifications|memo.ExprFmtHideScalars|memo.ExprFmtHideTypes, ) } var externalTestData = flag.String( - "d", "testdata/external/", "test files directory for TestExternal", + "d", "testdata/external", "test files directory for TestExternal", ) // TestExternal contains test cases from external customers and external @@ -223,7 +224,7 @@ func TestPlaceholderFastPath(t *testing.T) { defer log.Scope(t).Close(t) runDataDrivenTest( t, - "testdata/placeholder-fast-path", + tu.TestDataPath(t, "placeholder-fast-path"), memo.ExprFmtHideCost|memo.ExprFmtHideRuleProps| memo.ExprFmtHideQualifications|memo.ExprFmtHideScalars|memo.ExprFmtHideTypes, ) diff --git a/pkg/sql/pgwire/auth_test.go b/pkg/sql/pgwire/auth_test.go index 66a9c0823981..e6d2efd108e5 100644 --- a/pkg/sql/pgwire/auth_test.go +++ b/pkg/sql/pgwire/auth_test.go @@ -165,7 +165,7 @@ func hbaRunTest(t *testing.T, insecure bool) { httpScheme = "https://" } - datadriven.Walk(t, "testdata/auth", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "auth"), func(t *testing.T, path string) { defer leaktest.AfterTest(t)() maybeSocketDir, maybeSocketFile, cleanup := makeSocketFile(t) diff --git a/pkg/sql/pgwire/hba/scanner_test.go b/pkg/sql/pgwire/hba/scanner_test.go index 712467830105..6afa89db6131 100644 --- a/pkg/sql/pgwire/hba/scanner_test.go +++ b/pkg/sql/pgwire/hba/scanner_test.go @@ -14,6 +14,7 @@ import ( "fmt" "testing" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/datadriven" "github.com/kr/pretty" ) @@ -41,7 +42,7 @@ func TestSpecialCharacters(t *testing.T) { } func TestScanner(t *testing.T) { - datadriven.RunTest(t, "testdata/scan", func(t *testing.T, td *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "scan"), func(t *testing.T, td *datadriven.TestData) string { switch td.Cmd { case "token": remaining, tok, trailingComma, err := nextToken(td.Input) diff --git a/pkg/sql/pgwire/pgtest_test.go b/pkg/sql/pgwire/pgtest_test.go index 2fb96d59b1f4..b37d4a22c509 100644 --- a/pkg/sql/pgwire/pgtest_test.go +++ b/pkg/sql/pgwire/pgtest_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" _ "github.com/cockroachdb/cockroach/pkg/cloud/impl" // register cloud storage providers "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/pgtest" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -50,8 +51,8 @@ func TestPGTest(t *testing.T) { _, _ = db.ExecContext(ctx, "SET CLUSTER SETTING sql.conn.max_read_buffer_message_size = '32 KiB'") return addr, user, cleanup } - pgtest.WalkWithNewServer(t, "testdata/pgtest", newServer) + pgtest.WalkWithNewServer(t, testutils.TestDataPath(t, "pgtest"), newServer) } else { - pgtest.WalkWithRunningServer(t, "testdata/pgtest", *flagAddr, *flagUser) + pgtest.WalkWithRunningServer(t, testutils.TestDataPath(t, "pgtest"), *flagAddr, *flagUser) } } diff --git a/pkg/sql/sem/tree/pretty_test.go b/pkg/sql/sem/tree/pretty_test.go index 1db633dacafc..169b81766c0c 100644 --- a/pkg/sql/sem/tree/pretty_test.go +++ b/pkg/sql/sem/tree/pretty_test.go @@ -187,7 +187,7 @@ func TestPrettyVerify(t *testing.T) { } func BenchmarkPrettyData(b *testing.B) { - matches, err := filepath.Glob(testutils.TestDataPath(t, "pretty", "*.sql")) + matches, err := filepath.Glob(testutils.TestDataPath(b, "pretty", "*.sql")) if err != nil { b.Fatal(err) } diff --git a/pkg/sql/session_migration_test.go b/pkg/sql/session_migration_test.go index b9eeee6644d1..f3dcf9e349f2 100644 --- a/pkg/sql/session_migration_test.go +++ b/pkg/sql/session_migration_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" @@ -39,7 +40,7 @@ func TestSessionMigration(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - datadriven.Walk(t, "testdata/session_migration", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "session_migration"), func(t *testing.T, path string) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) defer tc.Stopper().Stop(ctx) diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index d4f5da4aa083..cf97db21d78c 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -104,7 +104,7 @@ func TestInitialKeys(t *testing.T) { func TestInitialKeysAndSplits(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - datadriven.RunTest(t, "testdata/initial_keys", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "initial_keys"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "initial-keys": var tenant string diff --git a/pkg/storage/disk_map_test.go b/pkg/storage/disk_map_test.go index 05d9915ef923..6979e0db8ef3 100644 --- a/pkg/storage/disk_map_test.go +++ b/pkg/storage/disk_map_test.go @@ -180,7 +180,7 @@ func TestPebbleMap(t *testing.T) { } defer e.Close() - runTestForEngine(ctx, t, "testdata/diskmap", e) + runTestForEngine(ctx, t, testutils.TestDataPath(t, "diskmap"), e) } @@ -197,7 +197,7 @@ func TestPebbleMultiMap(t *testing.T) { } defer e.Close() - runTestForEngine(ctx, t, "testdata/diskmap_duplicates_pebble", e) + runTestForEngine(ctx, t, testutils.TestDataPath(t, "diskmap_duplicates_pebble"), e) } diff --git a/pkg/storage/intent_interleaving_iter_test.go b/pkg/storage/intent_interleaving_iter_test.go index 33478f299d37..50116d9d3225 100644 --- a/pkg/storage/intent_interleaving_iter_test.go +++ b/pkg/storage/intent_interleaving_iter_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -204,7 +205,7 @@ func TestIntentInterleavingIter(t *testing.T) { } }() - datadriven.Walk(t, "testdata/intent_interleaving_iter", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "intent_interleaving_iter"), func(t *testing.T, path string) { if (util.RaceEnabled && strings.HasSuffix(path, "race_off")) || (!util.RaceEnabled && strings.HasSuffix(path, "race")) { return diff --git a/pkg/storage/intent_reader_writer_test.go b/pkg/storage/intent_reader_writer_test.go index 2a0b9d965fbc..baab535d195a 100644 --- a/pkg/storage/intent_reader_writer_test.go +++ b/pkg/storage/intent_reader_writer_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -192,7 +193,7 @@ func TestIntentDemuxWriter(t *testing.T) { var w intentDemuxWriter var scratch []byte var err error - datadriven.RunTest(t, "testdata/intent_demux_writer", + datadriven.RunTest(t, testutils.TestDataPath(t, "intent_demux_writer"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "new-writer": diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 306ffd8b75c7..4ec9b513863f 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -91,7 +92,7 @@ func TestMVCCHistories(t *testing.T) { // Everything reads/writes under the same prefix. span := roachpb.Span{Key: keys.LocalMax, EndKey: roachpb.KeyMax} - datadriven.Walk(t, "testdata/mvcc_histories", func(t *testing.T, path string) { + datadriven.Walk(t, testutils.TestDataPath(t, "mvcc_histories"), func(t *testing.T, path string) { // We start from a clean slate in every test file. engine, err := Open(ctx, InMemory(), CacheSize(1<<20 /* 1 MiB */), func(cfg *engineConfig) error { diff --git a/pkg/storage/pebble_file_registry_test.go b/pkg/storage/pebble_file_registry_test.go index 8ec1ffea331e..3465245ac741 100644 --- a/pkg/storage/pebble_file_registry_test.go +++ b/pkg/storage/pebble_file_registry_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/datadriven" @@ -329,7 +330,7 @@ func TestFileRegistry(t *testing.T) { fs := loggingFS{FS: vfs.NewMem(), w: &buf} var registry *PebbleFileRegistry - datadriven.RunTest(t, "testdata/file_registry", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "file_registry"), func(t *testing.T, d *datadriven.TestData) string { buf.Reset() switch d.Cmd { diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index e96f3c75cf7e..bf7307d00323 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -47,7 +48,7 @@ func TestPebbleTimeBoundPropCollector(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - datadriven.RunTest(t, "testdata/time_bound_props", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "time_bound_props"), func(t *testing.T, d *datadriven.TestData) string { c := &pebbleTimeBoundPropCollector{} switch d.Cmd { case "build": diff --git a/pkg/util/admission/BUILD.bazel b/pkg/util/admission/BUILD.bazel index b3e536d9f3f2..26ab1ced09f5 100644 --- a/pkg/util/admission/BUILD.bazel +++ b/pkg/util/admission/BUILD.bazel @@ -35,6 +35,7 @@ go_test( deps = [ "//pkg/roachpb:with-mocks", "//pkg/settings/cluster", + "//pkg/testutils", "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/syncutil", diff --git a/pkg/util/admission/granter_test.go b/pkg/util/admission/granter_test.go index f364f0b9c106..e445618dd2e2 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -104,7 +105,7 @@ func TestGranterBasic(t *testing.T) { } settings := cluster.MakeTestingClusterSettings() KVSlotAdjusterOverloadThreshold.Override(context.Background(), &settings.SV, 1) - datadriven.RunTest(t, "testdata/granter", func(t *testing.T, d *datadriven.TestData) string { + datadriven.RunTest(t, testutils.TestDataPath(t, "granter"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "init-grant-coordinator": var opts Options @@ -336,7 +337,7 @@ func TestIOLoadListener(t *testing.T) { var ioll *ioLoadListener ctx := context.Background() st := cluster.MakeTestingClusterSettings() - datadriven.RunTest(t, "testdata/io_load_listener", + datadriven.RunTest(t, testutils.TestDataPath(t, "io_load_listener"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "set-state": diff --git a/pkg/util/admission/work_queue_test.go b/pkg/util/admission/work_queue_test.go index 28c0c465a8d5..ab1062c62bf7 100644 --- a/pkg/util/admission/work_queue_test.go +++ b/pkg/util/admission/work_queue_test.go @@ -18,6 +18,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -138,7 +139,7 @@ func TestWorkQueueBasic(t *testing.T) { var tg *testGranter var wrkMap workMap var buf builderWithMu - datadriven.RunTest(t, "testdata/work_queue", + datadriven.RunTest(t, testutils.TestDataPath(t, "work_queue"), func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "init":