diff --git a/.bazelrc b/.bazelrc index c36f50953967..e4965413370a 100644 --- a/.bazelrc +++ b/.bazelrc @@ -1,6 +1,7 @@ # TODO(irfansharif): We should fold this into `dev` instead (#56965). -build --ui_event_filters=-DEBUG +build --ui_event_filters=-DEBUG --define gotags=bazel +test --define gotags=bazel query --ui_event_filters=-DEBUG try-import %workspace%/.bazelrc.user diff --git a/BUILD.bazel b/BUILD.bazel index a00e9e60a953..fb311389dbd6 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -12,6 +12,7 @@ load("@bazel_gazelle//:def.bzl", "gazelle") # # gazelle:prefix github.com/cockroachdb/cockroach # gazelle:build_file_name BUILD.bazel +# gazelle:build_tags bazel # Enable protobuf generation. # diff --git a/DEPS.bzl b/DEPS.bzl index d3aba3f665bd..4cd9ed5e3de8 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -287,6 +287,13 @@ def go_deps(): sum = "h1:TYi4+3m5t6K48TGI9AUdb+IzbnSxvnvUMfuitfgcfuo=", version = "v0.6.0", ) + go_repository( + name = "com_github_bazelbuild_rules_go", + build_file_proto_mode = "disable_global", + importpath = "github.com/bazelbuild/rules_go", + sum = "h1:2F449QezDZcVW6Jt+kSs8Htd/YI3EXMcvd0aNfVNCI4=", + version = "v0.26.0", + ) go_repository( name = "com_github_benesch_cgosymbolizer", diff --git a/go.mod b/go.mod index d324142fe444..21139e8a3fa8 100644 --- a/go.mod +++ b/go.mod @@ -26,6 +26,7 @@ require ( github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e github.com/aws/aws-sdk-go v1.36.33 github.com/axiomhq/hyperloglog v0.0.0-20181223111420-4b99d0c2c99e + github.com/bazelbuild/rules_go v0.26.0 github.com/benesch/cgosymbolizer v0.0.0-20180702220239-70e1ee2b39d3 github.com/biogo/store v0.0.0-20160505134755-913427a1d5e8 github.com/cenkalti/backoff v2.1.1+incompatible diff --git a/go.sum b/go.sum index 077fad8833df..decefc6a6ab7 100644 --- a/go.sum +++ b/go.sum @@ -151,6 +151,8 @@ github.com/aws/aws-sdk-go v1.36.33/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2z github.com/axiomhq/hyperloglog v0.0.0-20181223111420-4b99d0c2c99e h1:190ugM9MsyFauTkR/UqcHG/mn5nmFe6SvHJqEHIrtrA= github.com/axiomhq/hyperloglog v0.0.0-20181223111420-4b99d0c2c99e/go.mod h1:IOXAcuKIFq/mDyuQ4wyJuJ79XLMsmLM+5RdQ+vWrL7o= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= +github.com/bazelbuild/rules_go v0.26.0 h1:2F449QezDZcVW6Jt+kSs8Htd/YI3EXMcvd0aNfVNCI4= +github.com/bazelbuild/rules_go v0.26.0/go.mod h1:MC23Dc/wkXEyk3Wpq6lCqz0ZAYOZDw2DR5y3N1q2i7M= github.com/benesch/cgosymbolizer v0.0.0-20180702220239-70e1ee2b39d3 h1:Llg88pHOiUbcFOFhr009G8fOBQL6gaVDnxUUeWZuLog= github.com/benesch/cgosymbolizer v0.0.0-20180702220239-70e1ee2b39d3/go.mod h1:eMD2XUcPsHYbakFEocKrWZp47G0MRJYoC60qFblGjpA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= diff --git a/pkg/bench/ddl_analysis/validate_benchmark_data_test.go b/pkg/bench/ddl_analysis/validate_benchmark_data_test.go index d9734a236795..7e97e2076a5f 100644 --- a/pkg/bench/ddl_analysis/validate_benchmark_data_test.go +++ b/pkg/bench/ddl_analysis/validate_benchmark_data_test.go @@ -207,7 +207,7 @@ func resultsToExpectations(results []benchmarkResult) benchmarkExpectations { } func writeExpectationsFile(t *testing.T, expectations benchmarkExpectations) { - f, err := os.Create(testutils.TestDataPath("testdata", expectationsFilename)) + f, err := os.Create(testutils.TestDataPath(t, expectationsFilename)) require.NoError(t, err) defer func() { require.NoError(t, f.Close()) }() w := csv.NewWriter(f) @@ -225,7 +225,7 @@ func writeExpectationsFile(t *testing.T, expectations benchmarkExpectations) { } func readExpectationsFile(t *testing.T) benchmarkExpectations { - f, err := os.Open(testutils.TestDataPath("testdata", expectationsFilename)) + f, err := os.Open(testutils.TestDataPath(t, expectationsFilename)) require.NoError(t, err) defer func() { _ = f.Close() }() diff --git a/pkg/build/bazel/BUILD.bazel b/pkg/build/bazel/BUILD.bazel new file mode 100644 index 000000000000..ba8d76a03fb6 --- /dev/null +++ b/pkg/build/bazel/BUILD.bazel @@ -0,0 +1,9 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "bazel", + srcs = ["bazel.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/build/bazel", + visibility = ["//visibility:public"], + deps = ["@io_bazel_rules_go//go/tools/bazel:go_default_library"], +) diff --git a/pkg/build/bazel/bazel.go b/pkg/build/bazel/bazel.go new file mode 100644 index 000000000000..920a678a6939 --- /dev/null +++ b/pkg/build/bazel/bazel.go @@ -0,0 +1,91 @@ +// Copyright 2015 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// +build bazel + +package bazel + +import ( + "fmt" + "os" + "path" + "path/filepath" + "strings" + + inner "github.com/bazelbuild/rules_go/go/tools/bazel" +) + +// Return true iff this library was built with Bazel. +func BuiltWithBazel() bool { + return true +} + +// FindBinary is a convenience wrapper around the rules_go variant. +func FindBinary(pkg, name string) (string, bool) { + return inner.FindBinary(pkg, name) +} + +// Runfile is a convenience wrapper around the rules_go variant. +func Runfile(path string) (string, error) { + return inner.Runfile(path) +} + +// RunfilePath is a convenience wrapper around the rules_go variant. +func RunfilesPath() (string, error) { + return inner.RunfilesPath() +} + +// TestTmpDir is a convenience wrapper around the rules_go variant. +func TestTmpDir() string { + return inner.TestTmpDir() +} + +// Updates the current environment to use the Go toolchain that Bazel built this +// binary/test with (updates the `PATH`/`GOROOT`/`GOCACHE` environment +// variables). +// If you want to use this function, your binary/test target MUST have +// `@go_sdk//:files` in its `data` -- this will make sure the whole toolchain +// gets pulled into the sandbox as well. Generally, this function should only +// be called in init(). +func SetGoEnv() { + gobin, err := Runfile("bin/go") + if err != nil { + panic(err) + } + + if err := os.Setenv("PATH", fmt.Sprintf("%s%c%s", filepath.Dir(gobin), os.PathListSeparator, os.Getenv("PATH"))); err != nil { + panic(err) + } + if err := os.Setenv("GOROOT", filepath.Dir(filepath.Dir(gobin))); err != nil { + panic(err) + } + if err := os.Setenv("GOCACHE", path.Join(inner.TestTmpDir(), ".gocache")); err != nil { + panic(err) + } +} + +// Name of the environment variable containing the bazel target path +// (//pkg/cmd/foo:bar). +const testTargetEnv = "TEST_TARGET" + +// RelativeTestTargetPath returns relative path to the package +// of the current test. +func RelativeTestTargetPath() string { + target := os.Getenv(testTargetEnv) + if target == "" { + return "" + } + + // Drop target name. + if last := strings.LastIndex(target, ":"); last > 0 { + target = target[:last] + } + return strings.TrimPrefix(target, "//") +} diff --git a/pkg/build/bazel/non_bazel.go b/pkg/build/bazel/non_bazel.go new file mode 100644 index 000000000000..10ef5c93af24 --- /dev/null +++ b/pkg/build/bazel/non_bazel.go @@ -0,0 +1,46 @@ +// Copyright 2015 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// +build !bazel + +package bazel + +// This file contains stub implementations for non-bazel builds. +// See bazel.go for full documentation on the contracts of these functions. + +// BuiltWithBazel returns true iff this library was built with Bazel. +func BuiltWithBazel() bool { + return false +} + +// Runfile is not implemented. +func Runfile(string) (string, error) { + panic("not built with Bazel") +} + +// RunfilesPath is not implemented. +func RunfilesPath() (string, error) { + panic("not built with Bazel") +} + +// TestTmpDir is not implemented. +func TestTmpDir() string { + panic("not built with Bazel") +} + +// RelativeTestTargetPath is not implemented. +func RelativeTestTargetPath() string { + panic("not built with Bazel") +} + +// SetGoEnv is not implemented. +func SetGoEnv() { + panic("not built with Bazel") +} diff --git a/pkg/ccl/importccl/import_stmt_test.go b/pkg/ccl/importccl/import_stmt_test.go index 122c88195e9b..05f37af146c0 100644 --- a/pkg/ccl/importccl/import_stmt_test.go +++ b/pkg/ccl/importccl/import_stmt_test.go @@ -1712,7 +1712,7 @@ func TestImportCSVStmt(t *testing.T) { blockGC := make(chan struct{}) ctx := context.Background() - baseDir := testutils.TestDataPath("testdata", "csv") + baseDir := testutils.TestDataPath(t, "csv") tc := testcluster.StartTestCluster(t, nodes, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ SQLMemoryPoolSize: 256 << 20, ExternalIODir: baseDir, @@ -2611,7 +2611,7 @@ func TestImportIntoCSV(t *testing.T) { rowsPerRaceFile := 16 ctx := context.Background() - baseDir := testutils.TestDataPath("testdata", "csv") + baseDir := testutils.TestDataPath(t, "csv") tc := testcluster.StartTestCluster(t, nodes, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ExternalIODir: baseDir}}) defer tc.Stopper().Stop(ctx) conn := tc.Conns[0] diff --git a/pkg/sql/catalog/catalogkv/unwrap_validation_test.go b/pkg/sql/catalog/catalogkv/unwrap_validation_test.go index 1cde4832356b..d5fee0df659c 100644 --- a/pkg/sql/catalog/catalogkv/unwrap_validation_test.go +++ b/pkg/sql/catalog/catalogkv/unwrap_validation_test.go @@ -34,7 +34,7 @@ import ( // will hold a file "descriptors.csv" which is a csv of id,descriptor where // descriptor is hex encoded. func TestUnwrapValidation(t *testing.T) { - testdata := testutils.TestDataPath("testdata", "unwrap_validation") + testdata := testutils.TestDataPath(t, "unwrap_validation") const descriptorsCSVFilename = "descriptors.csv" dirs, err := ioutil.ReadDir(testdata) require.NoError(t, err) diff --git a/pkg/testutils/BUILD.bazel b/pkg/testutils/BUILD.bazel index b9053aa3be0d..837feab57857 100644 --- a/pkg/testutils/BUILD.bazel +++ b/pkg/testutils/BUILD.bazel @@ -4,7 +4,7 @@ go_library( name = "testutils", srcs = [ "base.go", - "bazel.go", + "data_path.go", "dir.go", "error.go", "files.go", @@ -20,6 +20,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/build/bazel", "//pkg/roachpb", "//pkg/security", "//pkg/sql/pgwire/pgerror", @@ -30,6 +31,7 @@ go_library( "//pkg/util/timeutil", "//pkg/util/tracing", "@com_github_cockroachdb_errors//:errors", + "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/testutils/bazel.go b/pkg/testutils/bazel.go deleted file mode 100644 index b5af8f44c7cf..000000000000 --- a/pkg/testutils/bazel.go +++ /dev/null @@ -1,110 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package testutils - -import ( - "os" - "path" - "strings" - - "github.com/cockroachdb/errors" -) - -// -// Utilities in this file are intended to provide helpers for tests -// to access system resources in a correct way, when executing under bazel. -// -// See https://docs.bazel.build/versions/master/test-encyclopedia.html for more details -// on the directory layout. -// -// When test is compiled with bazel, bazel will create a directory, called RUNFILES directory, -// containing all of the resources required to execute such test (libraries, final binary, -// data resources, etc). -// -// Bazel also sets up various environmental variables that point to the location(s) of -// those resources. -// - -// Name of the environment variable pointing to the absolute path to the base of the RUNFILES tree. -const testSrcDirEnv = "TEST_SRCDIR" - -// Name of the environment variable containing the name of the "workspace". -const testWorkspaceEnv = "TEST_WORKSPACE" - -// Name of the environment variable pointing to the absolute path of the -// temporary directory created for the execution of the test. -const testTmpDirEnv = "TEST_TMPDIR" - -// Name of the environment variable containing the bazel target path (//pkg/cmd/foo:bar). -const testTargetEnv = "TEST_TARGET" - -// runningUnderBazel returns true if the test is executed by bazel. -func runningUnderBazel() bool { - return os.Getenv(testSrcDirEnv) != "" -} - -func requireEnv(env string) string { - if v := os.Getenv(env); v != "" { - return v - } - panic(errors.AssertionFailedf("expected value for env: %s", env)) -} - -// TestSrcDir returns the path to the "source" tree. -// -// If running under bazel, this will point to a private, *readonly* -// directory containing symlinks (or copies) of the test data dependencies. -// This directory must be treated readonly. It's an error to try to modify -// anything under this directory: though the operation may succeed, the test -// would not be hermetic, and may fail under other environments. -func TestSrcDir() string { - // If testSrcDirEnv is not set, it means we are not running under bazel, - // and so we can use "" as our directory which should point to the - // src root. - if srcDir := os.Getenv(testSrcDirEnv); srcDir != "" { - return srcDir - } - return "" -} - -// bazeRelativeTargetPath returns relative path to the package -// of the current test. -func bazelRelativeTargetPath() string { - target := os.Getenv(testTargetEnv) - if target == "" { - return "" - } - - // Drop target name. - if last := strings.LastIndex(target, ":"); last > 0 { - target = target[:last] - } - return strings.TrimPrefix(target, "//") -} - -// TestDataPath returns a path to the directory containing test data files. -// -// Test files are usually checked into the repository under "testdata" directory. -// If we are not using bazel, then the test executes in the directory of -// the actual test, so the files can be referenced via "testdata/subdir/file" relative path. -// -// However, if we are running under bazel, the data files are specified -// via go_test "data" attribute. These files, in turn, are available under RUNFILES directory. -// This helper attempts to construct appropriate path to the RUNFILES directory -// containing test data files, given the relative (to the test) path components. -// -func TestDataPath(relative ...string) string { - if runningUnderBazel() { - return path.Join(TestSrcDir(), requireEnv(testWorkspaceEnv), bazelRelativeTargetPath(), - path.Join(relative...)) - } - return path.Join(relative...) -} diff --git a/pkg/testutils/data_path.go b/pkg/testutils/data_path.go new file mode 100644 index 000000000000..53456a867b89 --- /dev/null +++ b/pkg/testutils/data_path.go @@ -0,0 +1,41 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package testutils + +import ( + "path" + "path/filepath" + "testing" + + "github.com/cockroachdb/cockroach/pkg/build/bazel" + "github.com/stretchr/testify/require" +) + +// TestDataPath returns a path to an asset in the testdata directory. It knows +// to access accesses the right path when executing under bazel. +// +// For example, if there is a file testdata/a.txt, you can get a path to that +// file using TestDataPath(t, "a.txt"). +func TestDataPath(t *testing.T, relative ...string) string { + relative = append([]string{"testdata"}, relative...) + if bazel.BuiltWithBazel() { + runfiles, err := bazel.RunfilesPath() + require.NoError(t, err) + return path.Join(runfiles, bazel.RelativeTestTargetPath(), path.Join(relative...)) + } + + // If we're not running in Bazel, we're in the package directory and can + // just return a relative path. + ret := path.Join(relative...) + ret, err := filepath.Abs(ret) + require.NoError(t, err) + return ret +} diff --git a/pkg/testutils/dir.go b/pkg/testutils/dir.go index aeeb23ef2a58..8fe3a44229d7 100644 --- a/pkg/testutils/dir.go +++ b/pkg/testutils/dir.go @@ -15,6 +15,7 @@ import ( "os" "testing" + "github.com/cockroachdb/cockroach/pkg/build/bazel" "github.com/cockroachdb/cockroach/pkg/util/fileutil" ) @@ -22,14 +23,14 @@ import ( // test. func TempDir(t testing.TB) (string, func()) { tmpDir := "" - if runningUnderBazel() { + if bazel.BuiltWithBazel() { // Bazel sets up private temp directories for each test. // Normally, this private temp directory will be cleaned up automatically. // However, we do use external tools (such as stress) which re-execute the // same test multiple times. Bazel, on the other hand, does not know about // this, and only creates this temporary directory once. So, ensure we create // a unique temporary directory underneath bazel TEST_TMPDIR. - tmpDir = requireEnv(testTmpDirEnv) + tmpDir = bazel.TestTmpDir() } dir, err := ioutil.TempDir(tmpDir, fileutil.EscapeFilename(t.Name())) diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 8e77384acebc..c966875c9ccd 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -421,6 +421,7 @@ func TestLint(t *testing.T) { re: `\bos\.(Getenv|LookupEnv)\(`, excludes: []string{ ":!acceptance", + ":!build/bazel", ":!ccl/acceptanceccl/backup_test.go", ":!ccl/backupccl/backup_cloud_test.go", // KMS requires AWS credentials from environment variables. @@ -433,7 +434,7 @@ func TestLint(t *testing.T) { ":!nightly", ":!testutils/lint", ":!util/envutil/env.go", - ":!testutils/bazel.go", + ":!testutils/data_path.go", ":!util/log/tracebacks.go", ":!util/sdnotify/sdnotify_unix.go", ":!util/grpcutil", // GRPC_GO_* variables @@ -1627,6 +1628,9 @@ func TestLint(t *testing.T) { filter, // Skip .pb.go and .pb.gw.go generated files. stream.GrepNot(`pkg/.*\.pb(\.gw|)\.go:`), + // This file is a conditionally-compiled stub implementation that + // will produce fake "func is unused" errors. + stream.GrepNot(`pkg/build/bazel/non_bazel.go`), // Skip generated file. stream.GrepNot(`pkg/ui/distoss/bindata.go`), stream.GrepNot(`pkg/ui/distccl/bindata.go`), diff --git a/pkg/testutils/lint/passes/fmtsafe/BUILD.bazel b/pkg/testutils/lint/passes/fmtsafe/BUILD.bazel index 25513095a7f0..6e633b128234 100644 --- a/pkg/testutils/lint/passes/fmtsafe/BUILD.bazel +++ b/pkg/testutils/lint/passes/fmtsafe/BUILD.bazel @@ -22,10 +22,13 @@ go_test( name = "fmtsafe_test", size = "small", srcs = ["fmtsafe_test.go"], - data = glob(["testdata/**"]), - tags = ["broken_in_bazel"], + data = glob(["testdata/**"]) + [ + "@go_sdk//:files", + ], deps = [ ":fmtsafe", + "//pkg/build/bazel", + "//pkg/testutils", "//pkg/testutils/skip", "@org_golang_x_tools//go/analysis/analysistest", ], diff --git a/pkg/testutils/lint/passes/fmtsafe/fmtsafe_test.go b/pkg/testutils/lint/passes/fmtsafe/fmtsafe_test.go index f15da16a1af9..b03d6c406460 100644 --- a/pkg/testutils/lint/passes/fmtsafe/fmtsafe_test.go +++ b/pkg/testutils/lint/passes/fmtsafe/fmtsafe_test.go @@ -13,15 +13,24 @@ package fmtsafe_test import ( "testing" + "github.com/cockroachdb/cockroach/pkg/build/bazel" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/fmtsafe" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "golang.org/x/tools/go/analysis/analysistest" ) +func init() { + if bazel.BuiltWithBazel() { + bazel.SetGoEnv() + } +} + func Test(t *testing.T) { skip.UnderStress(t) fmtsafe.Tip = "" - testdata := analysistest.TestData() + testdata := testutils.TestDataPath(t) + analysistest.TestData = func() string { return testdata } results := analysistest.Run(t, testdata, fmtsafe.Analyzer, "a") for _, r := range results { for _, d := range r.Diagnostics { diff --git a/pkg/testutils/lint/passes/forbiddenmethod/BUILD.bazel b/pkg/testutils/lint/passes/forbiddenmethod/BUILD.bazel index 9e4cfe35c85a..9521801f7788 100644 --- a/pkg/testutils/lint/passes/forbiddenmethod/BUILD.bazel +++ b/pkg/testutils/lint/passes/forbiddenmethod/BUILD.bazel @@ -1,5 +1,11 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +filegroup( + name = "testdata", + srcs = glob(["testdata/**"]), + visibility = ["//pkg/testutils/lint/passes:__subpackages__"], +) + go_library( name = "forbiddenmethod", srcs = [ @@ -20,10 +26,14 @@ go_test( name = "forbiddenmethod_test", size = "small", srcs = ["descriptormarshal_test.go"], - data = glob(["testdata/**"]), - tags = ["broken_in_bazel"], + data = [ + ":testdata", + "@go_sdk//:files", + ], deps = [ ":forbiddenmethod", + "//pkg/build/bazel", + "//pkg/testutils", "//pkg/testutils/skip", "@org_golang_x_tools//go/analysis/analysistest", ], diff --git a/pkg/testutils/lint/passes/forbiddenmethod/descriptormarshal_test.go b/pkg/testutils/lint/passes/forbiddenmethod/descriptormarshal_test.go index 352233fdc317..7dae7686f824 100644 --- a/pkg/testutils/lint/passes/forbiddenmethod/descriptormarshal_test.go +++ b/pkg/testutils/lint/passes/forbiddenmethod/descriptormarshal_test.go @@ -13,14 +13,23 @@ package forbiddenmethod_test import ( "testing" + "github.com/cockroachdb/cockroach/pkg/build/bazel" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "golang.org/x/tools/go/analysis/analysistest" ) +func init() { + if bazel.BuiltWithBazel() { + bazel.SetGoEnv() + } +} + func Test(t *testing.T) { skip.UnderStress(t) - testdata := analysistest.TestData() + testdata := testutils.TestDataPath(t) + analysistest.TestData = func() string { return testdata } analysistest.Run(t, testdata, forbiddenmethod.DescriptorMarshalAnalyzer, "descmarshaltest") analysistest.Run(t, testdata, forbiddenmethod.GRPCClientConnCloseAnalyzer, "grpcconnclosetest") analysistest.Run(t, testdata, forbiddenmethod.GRPCStatusWithDetailsAnalyzer, "grpcstatuswithdetailstest") diff --git a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/descmarshaltest/BUILD.bazel b/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/descmarshaltest/BUILD.bazel deleted file mode 100644 index 6a9c5413ba7b..000000000000 --- a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/descmarshaltest/BUILD.bazel +++ /dev/null @@ -1,17 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "a", - srcs = ["a.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/a", - visibility = ["//visibility:public"], - deps = ["//pkg/sql/catalog/descpb"], -) - -go_library( - name = "descmarshaltest", - srcs = ["foo.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/descmarshaltest", - visibility = ["//visibility:public"], - deps = ["//pkg/sql/catalog/descpb"], -) diff --git a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb/BUILD.bazel b/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb/BUILD.bazel deleted file mode 100644 index f3004629ba56..000000000000 --- a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb/BUILD.bazel +++ /dev/null @@ -1,8 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "descpb", - srcs = ["test_descriptor_lint.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb", - visibility = ["//visibility:public"], -) diff --git a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/google.golang.org/grpc/BUILD.bazel b/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/google.golang.org/grpc/BUILD.bazel deleted file mode 100644 index 833d2d53441e..000000000000 --- a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/google.golang.org/grpc/BUILD.bazel +++ /dev/null @@ -1,8 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "grpc", - srcs = ["foo.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/google.golang.org/grpc", - visibility = ["//visibility:public"], -) diff --git a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/google.golang.org/grpc/internal/status/BUILD.bazel b/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/google.golang.org/grpc/internal/status/BUILD.bazel deleted file mode 100644 index ce20ca4084ff..000000000000 --- a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/google.golang.org/grpc/internal/status/BUILD.bazel +++ /dev/null @@ -1,8 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "status", - srcs = ["status.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/google.golang.org/grpc/internal/status", - visibility = ["//visibility:public"], -) diff --git a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/google.golang.org/grpc/status/BUILD.bazel b/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/google.golang.org/grpc/status/BUILD.bazel deleted file mode 100644 index ea7f9e020c3b..000000000000 --- a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/google.golang.org/grpc/status/BUILD.bazel +++ /dev/null @@ -1,8 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "status", - srcs = ["status.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/google.golang.org/grpc/status", - visibility = ["//visibility:public"], -) diff --git a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/grpcconnclosetest/BUILD.bazel b/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/grpcconnclosetest/BUILD.bazel deleted file mode 100644 index ef8b84fa6d60..000000000000 --- a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/grpcconnclosetest/BUILD.bazel +++ /dev/null @@ -1,25 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "a", - srcs = ["a.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/a", - visibility = ["//visibility:public"], - deps = ["//pkg/sql/catalog/descpb"], -) - -go_library( - name = "descmarshaltest", - srcs = ["foo.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/descmarshaltest", - visibility = ["//visibility:public"], - deps = ["//pkg/sql/catalog/descpb"], -) - -go_library( - name = "grpcconnclosetest", - srcs = ["foo.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/grpcconnclosetest", - visibility = ["//visibility:public"], - deps = ["@org_golang_google_grpc//:go_default_library"], -) diff --git a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/grpcstatuswithdetailstest/BUILD.bazel b/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/grpcstatuswithdetailstest/BUILD.bazel deleted file mode 100644 index 0c2000c889e6..000000000000 --- a/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/grpcstatuswithdetailstest/BUILD.bazel +++ /dev/null @@ -1,9 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "grpstatuswithdetailstest", - srcs = ["foo.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod/testdata/src/grpcstatuswithdetailstest", - visibility = ["//visibility:public"], - deps = ["@org_golang_google_grpc//:go_default_library"], -) diff --git a/pkg/testutils/lint/passes/hash/BUILD.bazel b/pkg/testutils/lint/passes/hash/BUILD.bazel index d5f2a1140b07..29daf17c23b5 100644 --- a/pkg/testutils/lint/passes/hash/BUILD.bazel +++ b/pkg/testutils/lint/passes/hash/BUILD.bazel @@ -15,10 +15,13 @@ go_test( name = "hash_test", size = "small", srcs = ["hash_test.go"], - data = glob(["testdata/**"]), - tags = ["broken_in_bazel"], + data = glob(["testdata/**"]) + [ + "@go_sdk//:files", + ], deps = [ ":hash", + "//pkg/build/bazel", + "//pkg/testutils", "//pkg/testutils/skip", "@org_golang_x_tools//go/analysis/analysistest", ], diff --git a/pkg/testutils/lint/passes/hash/hash_test.go b/pkg/testutils/lint/passes/hash/hash_test.go index 67ce3a8afb45..874baa3e330e 100644 --- a/pkg/testutils/lint/passes/hash/hash_test.go +++ b/pkg/testutils/lint/passes/hash/hash_test.go @@ -13,13 +13,22 @@ package hash_test import ( "testing" + "github.com/cockroachdb/cockroach/pkg/build/bazel" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/hash" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "golang.org/x/tools/go/analysis/analysistest" ) +func init() { + if bazel.BuiltWithBazel() { + bazel.SetGoEnv() + } +} + func Test(t *testing.T) { skip.UnderStress(t) - testdata := analysistest.TestData() + testdata := testutils.TestDataPath(t) + analysistest.TestData = func() string { return testdata } analysistest.Run(t, testdata, hash.Analyzer, "a") } diff --git a/pkg/testutils/lint/passes/hash/testdata/src/a/BUILD.bazel b/pkg/testutils/lint/passes/hash/testdata/src/a/BUILD.bazel deleted file mode 100644 index 89b0c70d11b2..000000000000 --- a/pkg/testutils/lint/passes/hash/testdata/src/a/BUILD.bazel +++ /dev/null @@ -1,8 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "a", - srcs = ["a.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/hash/testdata/src/a", - visibility = ["//visibility:public"], -) diff --git a/pkg/testutils/lint/passes/leaktestcall/BUILD.bazel b/pkg/testutils/lint/passes/leaktestcall/BUILD.bazel index ab76db1e6c5c..019e58beb6e9 100644 --- a/pkg/testutils/lint/passes/leaktestcall/BUILD.bazel +++ b/pkg/testutils/lint/passes/leaktestcall/BUILD.bazel @@ -15,10 +15,13 @@ go_library( go_test( name = "leaktestcall_test", srcs = ["leaktestcall_test.go"], - data = glob(["testdata/**"]), - tags = ["broken_in_bazel"], + data = glob(["testdata/**"]) + [ + "@go_sdk//:files", + ], deps = [ ":leaktestcall", + "//pkg/build/bazel", + "//pkg/testutils", "//pkg/testutils/skip", "@org_golang_x_tools//go/analysis/analysistest", ], diff --git a/pkg/testutils/lint/passes/leaktestcall/leaktestcall_test.go b/pkg/testutils/lint/passes/leaktestcall/leaktestcall_test.go index 4b869a144880..506094d613ec 100644 --- a/pkg/testutils/lint/passes/leaktestcall/leaktestcall_test.go +++ b/pkg/testutils/lint/passes/leaktestcall/leaktestcall_test.go @@ -13,13 +13,22 @@ package leaktestcall_test import ( "testing" + "github.com/cockroachdb/cockroach/pkg/build/bazel" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/leaktestcall" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "golang.org/x/tools/go/analysis/analysistest" ) +func init() { + if bazel.BuiltWithBazel() { + bazel.SetGoEnv() + } +} + func Test(t *testing.T) { skip.UnderStress(t) - testdata := analysistest.TestData() + testdata := testutils.TestDataPath(t) + analysistest.TestData = func() string { return testdata } analysistest.Run(t, testdata, leaktestcall.Analyzer, "a") } diff --git a/pkg/testutils/lint/passes/nocopy/BUILD.bazel b/pkg/testutils/lint/passes/nocopy/BUILD.bazel index 07efda15b70c..e25f33a58341 100644 --- a/pkg/testutils/lint/passes/nocopy/BUILD.bazel +++ b/pkg/testutils/lint/passes/nocopy/BUILD.bazel @@ -16,10 +16,13 @@ go_test( name = "nocopy_test", size = "small", srcs = ["nocopy_test.go"], - data = glob(["testdata/**"]), - tags = ["broken_in_bazel"], + data = glob(["testdata/**"]) + [ + "@go_sdk//:files", + ], deps = [ ":nocopy", + "//pkg/build/bazel", + "//pkg/testutils", "//pkg/testutils/skip", "@org_golang_x_tools//go/analysis/analysistest", ], diff --git a/pkg/testutils/lint/passes/nocopy/nocopy_test.go b/pkg/testutils/lint/passes/nocopy/nocopy_test.go index 92d9ce4ed349..73b822a83ae7 100644 --- a/pkg/testutils/lint/passes/nocopy/nocopy_test.go +++ b/pkg/testutils/lint/passes/nocopy/nocopy_test.go @@ -14,13 +14,22 @@ package nocopy_test import ( "testing" + "github.com/cockroachdb/cockroach/pkg/build/bazel" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/nocopy" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "golang.org/x/tools/go/analysis/analysistest" ) +func init() { + if bazel.BuiltWithBazel() { + bazel.SetGoEnv() + } +} + func Test(t *testing.T) { skip.UnderStress(t) - testdata := analysistest.TestData() + testdata := testutils.TestDataPath(t) + analysistest.TestData = func() string { return testdata } analysistest.Run(t, testdata, nocopy.Analyzer, "a") } diff --git a/pkg/testutils/lint/passes/nocopy/testdata/src/a/BUILD.bazel b/pkg/testutils/lint/passes/nocopy/testdata/src/a/BUILD.bazel deleted file mode 100644 index 56fdf6fe08c5..000000000000 --- a/pkg/testutils/lint/passes/nocopy/testdata/src/a/BUILD.bazel +++ /dev/null @@ -1,9 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "a", - srcs = ["a.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/nocopy/testdata/src/a", - visibility = ["//visibility:public"], - deps = ["//pkg/util"], -) diff --git a/pkg/testutils/lint/passes/nocopy/testdata/src/github.com/cockroachdb/cockroach/pkg/util/BUILD.bazel b/pkg/testutils/lint/passes/nocopy/testdata/src/github.com/cockroachdb/cockroach/pkg/util/BUILD.bazel deleted file mode 100644 index e77ea0d1e52b..000000000000 --- a/pkg/testutils/lint/passes/nocopy/testdata/src/github.com/cockroachdb/cockroach/pkg/util/BUILD.bazel +++ /dev/null @@ -1,8 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "util", - srcs = ["nocopy.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/nocopy/testdata/src/github.com/cockroachdb/cockroach/pkg/util", - visibility = ["//visibility:public"], -) diff --git a/pkg/testutils/lint/passes/passesutil/BUILD.bazel b/pkg/testutils/lint/passes/passesutil/BUILD.bazel index be7ede8e9826..1d3419bbffe6 100644 --- a/pkg/testutils/lint/passes/passesutil/BUILD.bazel +++ b/pkg/testutils/lint/passes/passesutil/BUILD.bazel @@ -15,8 +15,13 @@ go_test( name = "passesutil_test", size = "small", srcs = ["passes_util_test.go"], - tags = ["broken_in_bazel"], + data = [ + "//pkg/testutils/lint/passes/forbiddenmethod:testdata", + "//pkg/testutils/lint/passes/unconvert:testdata", + "@go_sdk//:files", + ], deps = [ + "//pkg/build/bazel", "//pkg/testutils/lint/passes/forbiddenmethod", "//pkg/testutils/lint/passes/unconvert", "//pkg/testutils/skip", diff --git a/pkg/testutils/lint/passes/passesutil/passes_util_test.go b/pkg/testutils/lint/passes/passesutil/passes_util_test.go index 93169c8f9ddc..8cfcfca031a0 100644 --- a/pkg/testutils/lint/passes/passesutil/passes_util_test.go +++ b/pkg/testutils/lint/passes/passesutil/passes_util_test.go @@ -14,6 +14,7 @@ import ( "path/filepath" "testing" + "github.com/cockroachdb/cockroach/pkg/build/bazel" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/forbiddenmethod" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/unconvert" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -21,6 +22,12 @@ import ( "golang.org/x/tools/go/analysis/analysistest" ) +func init() { + if bazel.BuiltWithBazel() { + bazel.SetGoEnv() + } +} + // Use tests from other packages to also test this package. This ensures // that if that code changes, somebody will look here. Also it allows for // coverage checking here. @@ -32,19 +39,29 @@ func requireNotEmpty(t *testing.T, path string) { require.NotEmpty(t, files) } +func getTestdataForPackage(t *testing.T, pkg string) string { + if bazel.BuiltWithBazel() { + runfiles, err := bazel.RunfilesPath() + require.NoError(t, err) + return filepath.Join(runfiles, "pkg", "testutils", "lint", "passes", pkg, "testdata") + } + return filepath.Join("..", pkg, "testdata") +} + func TestDescriptorMarshal(t *testing.T) { skip.UnderStress(t) - testdata, err := filepath.Abs(filepath.Join("..", "forbiddenmethod", "testdata")) + testdata, err := filepath.Abs(getTestdataForPackage(t, "forbiddenmethod")) require.NoError(t, err) requireNotEmpty(t, testdata) - + analysistest.TestData = func() string { return testdata } analysistest.Run(t, testdata, forbiddenmethod.DescriptorMarshalAnalyzer, "descmarshaltest") } func TestUnconvert(t *testing.T) { skip.UnderStress(t) - testdata, err := filepath.Abs(filepath.Join("..", "unconvert", "testdata")) + testdata, err := filepath.Abs(getTestdataForPackage(t, "unconvert")) require.NoError(t, err) requireNotEmpty(t, testdata) + analysistest.TestData = func() string { return testdata } analysistest.Run(t, testdata, unconvert.Analyzer, "a") } diff --git a/pkg/testutils/lint/passes/returnerrcheck/BUILD.bazel b/pkg/testutils/lint/passes/returnerrcheck/BUILD.bazel index d74e0dd00140..f2c2f5aa4fc2 100644 --- a/pkg/testutils/lint/passes/returnerrcheck/BUILD.bazel +++ b/pkg/testutils/lint/passes/returnerrcheck/BUILD.bazel @@ -18,10 +18,13 @@ go_test( name = "returnerrcheck_test", size = "small", srcs = ["returnerrcheck_test.go"], - data = glob(["testdata/**"]), - tags = ["broken_in_bazel"], + data = glob(["testdata/**"]) + [ + "@go_sdk//:files", + ], deps = [ ":returnerrcheck", + "//pkg/build/bazel", + "//pkg/testutils", "//pkg/testutils/skip", "@org_golang_x_tools//go/analysis/analysistest", ], diff --git a/pkg/testutils/lint/passes/returnerrcheck/returnerrcheck_test.go b/pkg/testutils/lint/passes/returnerrcheck/returnerrcheck_test.go index 53e8db4591e3..e3920b779133 100644 --- a/pkg/testutils/lint/passes/returnerrcheck/returnerrcheck_test.go +++ b/pkg/testutils/lint/passes/returnerrcheck/returnerrcheck_test.go @@ -13,13 +13,22 @@ package returnerrcheck_test import ( "testing" + "github.com/cockroachdb/cockroach/pkg/build/bazel" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/returnerrcheck" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "golang.org/x/tools/go/analysis/analysistest" ) +func init() { + if bazel.BuiltWithBazel() { + bazel.SetGoEnv() + } +} + func Test(t *testing.T) { skip.UnderStress(t, "Go cache files don't work under stress") - testdata := analysistest.TestData() + testdata := testutils.TestDataPath(t) + analysistest.TestData = func() string { return testdata } analysistest.Run(t, testdata, returnerrcheck.Analyzer, "a") } diff --git a/pkg/testutils/lint/passes/returnerrcheck/testdata/src/a/BUILD.bazel b/pkg/testutils/lint/passes/returnerrcheck/testdata/src/a/BUILD.bazel deleted file mode 100644 index 963887b6cbc5..000000000000 --- a/pkg/testutils/lint/passes/returnerrcheck/testdata/src/a/BUILD.bazel +++ /dev/null @@ -1,8 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "a", - srcs = ["a.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/returnerrcheck/testdata/src/a", - visibility = ["//visibility:public"], -) diff --git a/pkg/testutils/lint/passes/timer/BUILD.bazel b/pkg/testutils/lint/passes/timer/BUILD.bazel index d3207b1f101f..c9e3d755d731 100644 --- a/pkg/testutils/lint/passes/timer/BUILD.bazel +++ b/pkg/testutils/lint/passes/timer/BUILD.bazel @@ -16,10 +16,13 @@ go_test( name = "timer_test", size = "small", srcs = ["timer_test.go"], - data = glob(["testdata/**"]), - tags = ["broken_in_bazel"], + data = glob(["testdata/**"]) + [ + "@go_sdk//:files", + ], deps = [ ":timer", + "//pkg/build/bazel", + "//pkg/testutils", "//pkg/testutils/skip", "@org_golang_x_tools//go/analysis/analysistest", ], diff --git a/pkg/testutils/lint/passes/timer/testdata/src/a/BUILD.bazel b/pkg/testutils/lint/passes/timer/testdata/src/a/BUILD.bazel deleted file mode 100644 index 6d2923600fcf..000000000000 --- a/pkg/testutils/lint/passes/timer/testdata/src/a/BUILD.bazel +++ /dev/null @@ -1,9 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "a", - srcs = ["a.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/timer/testdata/src/a", - visibility = ["//visibility:public"], - deps = ["//pkg/util/timeutil"], -) diff --git a/pkg/testutils/lint/passes/timer/testdata/src/github.com/cockroachdb/cockroach/pkg/util/timeutil/BUILD.bazel b/pkg/testutils/lint/passes/timer/testdata/src/github.com/cockroachdb/cockroach/pkg/util/timeutil/BUILD.bazel deleted file mode 100644 index 2be817ec24b5..000000000000 --- a/pkg/testutils/lint/passes/timer/testdata/src/github.com/cockroachdb/cockroach/pkg/util/timeutil/BUILD.bazel +++ /dev/null @@ -1,8 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "timeutil", - srcs = ["timer.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/timer/testdata/src/github.com/cockroachdb/cockroach/pkg/util/timeutil", - visibility = ["//visibility:public"], -) diff --git a/pkg/testutils/lint/passes/timer/timer_test.go b/pkg/testutils/lint/passes/timer/timer_test.go index 5e815ea0875b..8abfb89cd724 100644 --- a/pkg/testutils/lint/passes/timer/timer_test.go +++ b/pkg/testutils/lint/passes/timer/timer_test.go @@ -13,13 +13,22 @@ package timer_test import ( "testing" + "github.com/cockroachdb/cockroach/pkg/build/bazel" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/timer" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "golang.org/x/tools/go/analysis/analysistest" ) +func init() { + if bazel.BuiltWithBazel() { + bazel.SetGoEnv() + } +} + func Test(t *testing.T) { skip.UnderStress(t, "Go cache files don't work under stress") - testdata := analysistest.TestData() + testdata := testutils.TestDataPath(t) + analysistest.TestData = func() string { return testdata } analysistest.Run(t, testdata, timer.Analyzer, "a") } diff --git a/pkg/testutils/lint/passes/unconvert/BUILD.bazel b/pkg/testutils/lint/passes/unconvert/BUILD.bazel index 447d565e567d..1ab26ea554fb 100644 --- a/pkg/testutils/lint/passes/unconvert/BUILD.bazel +++ b/pkg/testutils/lint/passes/unconvert/BUILD.bazel @@ -1,5 +1,11 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +filegroup( + name = "testdata", + srcs = glob(["testdata/**"]), + visibility = ["//pkg/testutils/lint/passes:__subpackages__"], +) + go_library( name = "unconvert", srcs = ["unconvert.go"], @@ -17,10 +23,14 @@ go_test( name = "unconvert_test", size = "small", srcs = ["unconvert_test.go"], - data = glob(["testdata/**"]), - tags = ["broken_in_bazel"], + data = [ + ":testdata", + "@go_sdk//:files", + ], deps = [ ":unconvert", + "//pkg/build/bazel", + "//pkg/testutils", "//pkg/testutils/skip", "@org_golang_x_tools//go/analysis/analysistest", ], diff --git a/pkg/testutils/lint/passes/unconvert/testdata/src/a/BUILD.bazel b/pkg/testutils/lint/passes/unconvert/testdata/src/a/BUILD.bazel deleted file mode 100644 index 7b21506e4b96..000000000000 --- a/pkg/testutils/lint/passes/unconvert/testdata/src/a/BUILD.bazel +++ /dev/null @@ -1,8 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "a", - srcs = ["a.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/unconvert/testdata/src/a", - visibility = ["//visibility:public"], -) diff --git a/pkg/testutils/lint/passes/unconvert/unconvert_test.go b/pkg/testutils/lint/passes/unconvert/unconvert_test.go index f5f8aa99c58f..c49c8425bbfd 100644 --- a/pkg/testutils/lint/passes/unconvert/unconvert_test.go +++ b/pkg/testutils/lint/passes/unconvert/unconvert_test.go @@ -13,13 +13,22 @@ package unconvert_test import ( "testing" + "github.com/cockroachdb/cockroach/pkg/build/bazel" + "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/unconvert" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "golang.org/x/tools/go/analysis/analysistest" ) +func init() { + if bazel.BuiltWithBazel() { + bazel.SetGoEnv() + } +} + func Test(t *testing.T) { skip.UnderStress(t, "Go cache files don't work under stress") - testdata := analysistest.TestData() + testdata := testutils.TestDataPath(t) + analysistest.TestData = func() string { return testdata } analysistest.Run(t, testdata, unconvert.Analyzer, "a") } diff --git a/vendor b/vendor index 4e25bac8042f..b083859ac534 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit 4e25bac8042ff94808987a3724e254c53e351314 +Subproject commit b083859ac534539060900381d4a164a38754c105