Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
72838: bazel: don't use different `gotags` for test and non-test scenarios r=RaduBerinde a=rickystewart


In a couple different places in-tree we use the build tags `crdb_test`
and `crdb_test_off` to support conditional compilation. Unfortunately,
this results in low cache utilization when swapping between builds and
tests, since recompiling with different `gotags` causes Bazel to
recompile EVERYTHING, including the entire standard library, with these
different tags. This means that you can't build `cockroach` and then run
a quick test using cached build artifacts, for example.

Fix this by adding a new configuration, `//build/toolchains:crdb_test`.
We use `select()` in `pkg/util/buildutil/BUILD.bazel` to choose between
`crdb_test_on.go` and `crdb_test_off.go` accordingly.

Also add logic to `build/bazelutil/check.sh` to detect new uses of these
build constraints and warn people that the logic also needs to be
Bazelfied.

Closes #71857.

Release note: None

72882: ui: save sort on cache for Transaction page r=maryliag a=maryliag

Previously, a sort selection was not maintained when
the page change (e.g. coming back from Transaction details).
This commits saves the selected value to be used.

Partially adresses #68199

Release note: None

72915: build/README: add more info about how to authenticate for mirroring r=rail a=rickystewart

Release note: None

Co-authored-by: Ricky Stewart <[email protected]>
Co-authored-by: Marylia Gutierrez <[email protected]>
  • Loading branch information
3 people committed Nov 18, 2021
4 parents 4d82c5f + f606f15 + 73bae4b + b2bf43c commit b6aeffc
Show file tree
Hide file tree
Showing 61 changed files with 368 additions and 135 deletions.
4 changes: 2 additions & 2 deletions .bazelrc
Original file line number Diff line number Diff line change
@@ -1,9 +1,9 @@
try-import %workspace%/.bazelrc.user

build --symlink_prefix=_bazel/ --ui_event_filters=-DEBUG --define gotags=bazel,crdb_test_off,gss --experimental_proto_descriptor_sets_include_source_info --incompatible_strict_action_env
build --symlink_prefix=_bazel/ --ui_event_filters=-DEBUG --define gotags=bazel,gss --experimental_proto_descriptor_sets_include_source_info --incompatible_strict_action_env
test --config=test
build:with_ui --define cockroach_with_ui=y
build:test --define gotags=bazel,crdb_test,gss
build:test --define crdb_test=y
build:race --@io_bazel_rules_go//go/config:race --test_env=GORACE=halt_on_error=1 --test_sharding_strategy=disabled
test:test --test_env=TZ=
query --ui_event_filters=-DEBUG
Expand Down
5 changes: 4 additions & 1 deletion BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ load("@bazel_gazelle//:def.bzl", "gazelle")
#
# gazelle:prefix github.com/cockroachdb/cockroach
# gazelle:build_file_name BUILD.bazel
# gazelle:build_tags bazel,crdb_test,crdb_test_off,fast_int_set_small,fast_int_set_large,gss
# gazelle:build_tags bazel,fast_int_set_small,fast_int_set_large,gss

# Enable protobuf generation.
#
Expand Down Expand Up @@ -126,6 +126,9 @@ load("@bazel_gazelle//:def.bzl", "gazelle")
# gazelle:exclude pkg/**/*_string.go
# gazelle:exclude pkg/ui/distccl/distccl_no_bazel.go
# gazelle:exclude pkg/ui/distoss/distoss_no_bazel.go
# gazelle:exclude pkg/util/buildutil/crdb_test_dyn.go
# gazelle:exclude pkg/util/buildutil/crdb_test_off.go
# gazelle:exclude pkg/util/buildutil/crdb_test_on.go
#
# Generally useful references:
#
Expand Down
3 changes: 2 additions & 1 deletion build/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -169,7 +169,8 @@ file on your local branch, 2) push a commit containing this import to the `vendo
8. Run `./dev generate bazel --mirror` to regenerate DEPS.bzl with the updated Go dependency information.
Note that you need engineer permissions to mirror dependencies; if you want to get the Bazel build
working locally without mirroring, `./dev generate bazel` will work, but you won't be able to check
your changes in.
your changes in. (Assuming that you do have engineer permissions, you can run
`gcloud auth application-default login` to authenticate if you get a credentials error.)
9. Follow instructions for [pushing the dependency to the `vendored` submodule](#pushing-the-dependency-to-the-vendored-submodule)

### Updating a Dependency
Expand Down
21 changes: 20 additions & 1 deletion build/bazelutil/check.sh
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,12 @@ pkg/cmd/roachtest/BUILD.bazel
pkg/cmd/teamcity-trigger/BUILD.bazel
"

EXISTING_CRDB_TEST_BUILD_CONSTRAINTS="
pkg/util/buildutil/crdb_test_dyn.go://go:build bazel
pkg/util/buildutil/crdb_test_off.go://go:build !crdb_test || crdb_test_off
pkg/util/buildutil/crdb_test_on.go://go:build crdb_test && !crdb_test_off
"

git grep 'go:generate stringer' pkg | while read LINE; do
dir=$(dirname $(echo $LINE | cut -d: -f1))
type=$(echo $LINE | grep -o -- '-type[= ][^ ]*' | sed 's/-type[= ]//g' | awk '{print tolower($0)}')
Expand Down Expand Up @@ -85,4 +91,17 @@ git grep 'broken_in_bazel' pkg | grep BUILD.bazel: | grep -v pkg/BUILD.bazel | g
echo "A new broken test in Bazel was added in $LINE"
echo 'Ensure the test runs with Bazel, then remove the broken_in_bazel tag.'
exit 1
done
done

git grep '//go:build' pkg | grep crdb_test | while read LINE; do
if [[ "$EXISTING_CRDB_TEST_BUILD_CONSTRAINTS" == *"$LINE"* ]]; then
# Grandfathered.
continue
fi
echo "A new crdb_test/crdb_test_off build constraint was added in $LINE"
echo 'Make sure you port the conditional compilation logic to the Bazel build,'
echo 'which does not use the build tags in the same way.'
echo "Once you've done so, you can add the line to "
echo 'EXISTING_CRDB_TEST_BUILD_CONSTRAINTS in build/bazelutil/check.sh.'
exit 1
done
37 changes: 37 additions & 0 deletions build/toolchains/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,26 @@ platform(
],
)

config_setting(
name = "dbg_crdb_test",
define_values = {
"crdb_test": "y",
},
values = {
"compilation_mode": "dbg",
},
)

config_setting(
name = "fastbuild_crdb_test",
define_values = {
"crdb_test": "y",
},
values = {
"compilation_mode": "fastbuild",
},
)

config_setting(
name = "dev",
define_values = {
Expand All @@ -112,3 +132,20 @@ config_setting(
"cockroach_cross": "y",
},
)

config_setting(
name = "opt_crdb_test",
define_values = {
"crdb_test": "y",
},
values = {
"compilation_mode": "opt",
},
)

config_setting(
name = "opt",
values = {
"compilation_mode": "opt",
},
)
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -348,6 +348,7 @@ ALL_TESTS = [
"//pkg/util/admission:admission_test",
"//pkg/util/binfetcher:binfetcher_test",
"//pkg/util/bitarray:bitarray_test",
"//pkg/util/buildutil:buildutil_test",
"//pkg/util/cache:cache_test",
"//pkg/util/caller:caller_test",
"//pkg/util/cgroups:cgroups_test",
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/multitenantccl/tenantcostserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ go_library(
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"//pkg/util",
"//pkg/util/buildutil",
"//pkg/util/log",
"//pkg/util/metric",
"//pkg/util/metric/aggmetric",
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/multitenantccl/tenantcostserver/system_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -439,7 +439,7 @@ func (h *sysTableHelper) maybeCleanupStaleInstances(
// maybeCheckInvariants checks the invariants for the system table with a random
// probability and only if this is a test build.
func (h *sysTableHelper) maybeCheckInvariants() error {
if util.CrdbTestBuild && rand.Intn(10) == 0 {
if buildutil.CrdbTestBuild && rand.Intn(10) == 0 {
return h.checkInvariants()
}
return nil
Expand Down
1 change: 1 addition & 0 deletions pkg/col/coldata/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ go_library(
"//pkg/sql/memsize",
"//pkg/sql/types",
"//pkg/util",
"//pkg/util/buildutil",
"//pkg/util/duration",
"//pkg/util/json",
"//pkg/util/uuid",
Expand Down
4 changes: 2 additions & 2 deletions pkg/col/coldata/bytes.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/memsize"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -135,7 +135,7 @@ func (b *Bytes) Get(i int) []byte {
// This method will panic if i is less than maximum previously Set index.
//gcassert:inline
func (b *Bytes) getAppendTo(i int) []byte {
if util.CrdbTestBuild {
if buildutil.CrdbTestBuild {
if b.isWindow {
panic("getAppendTo is called on a window into Bytes")
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -382,6 +382,7 @@ go_library(
"//pkg/util",
"//pkg/util/admission",
"//pkg/util/bitarray",
"//pkg/util/buildutil",
"//pkg/util/cancelchecker",
"//pkg/util/contextutil",
"//pkg/util/ctxgroup",
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/colexec/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ go_library(
"//pkg/sql/sqltelemetry", # keep
"//pkg/sql/types",
"//pkg/util",
"//pkg/util/buildutil",
"//pkg/util/duration", # keep
"//pkg/util/encoding", # keep
"//pkg/util/humanizeutil",
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/colexec/colbuilder/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ go_library(
"//pkg/sql/sessiondatapb",
"//pkg/sql/types",
"//pkg/util",
"//pkg/util/buildutil",
"//pkg/util/log",
"//pkg/util/mon",
"@com_github_cockroachdb_errors//:errors",
Expand Down
9 changes: 5 additions & 4 deletions pkg/sql/colexec/colbuilder/execplan.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -589,7 +590,7 @@ func (r opResult) createAndWrapRowSource(
}
r.Root = c
r.Columnarizer = c
if util.CrdbTestBuild {
if buildutil.CrdbTestBuild {
r.Root = colexec.NewInvariantsChecker(r.Root)
}
takeOverMetaInfo(&r.OpWithMetaInfo, inputs)
Expand All @@ -606,7 +607,7 @@ func (r opResult) createAndWrapRowSource(
// nil is returned.
func MaybeRemoveRootColumnarizer(r colexecargs.OpWithMetaInfo) execinfra.RowSource {
root := r.Root
if util.CrdbTestBuild {
if buildutil.CrdbTestBuild {
// We might have an invariants checker as the root right now, we gotta
// peek inside of it if so.
root = colexec.MaybeUnwrapInvariantsChecker(root)
Expand Down Expand Up @@ -1515,7 +1516,7 @@ func NewColOperator(
}

takeOverMetaInfo(&result.OpWithMetaInfo, inputs)
if util.CrdbTestBuild {
if buildutil.CrdbTestBuild {
// Plan an invariants checker if it isn't already the root of the
// tree.
if i := colexec.MaybeUnwrapInvariantsChecker(r.Root); i == r.Root {
Expand Down Expand Up @@ -1674,7 +1675,7 @@ func (r opResult) finishBufferedWindowerArgs(

func (r opResult) finishScanPlanning(op colfetcher.ScanOperator, resultTypes []*types.T) {
r.Root = op
if util.CrdbTestBuild {
if buildutil.CrdbTestBuild {
r.Root = colexec.NewInvariantsChecker(r.Root)
}
r.KVReader = op
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/colexec/columnarizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -126,7 +126,7 @@ func newColumnarizer(
// Close will call InternalClose(). Note that we don't return
// any trailing metadata here because the columnarizers
// propagate it in DrainMeta.
if err := c.Close(); util.CrdbTestBuild && err != nil {
if err := c.Close(); buildutil.CrdbTestBuild && err != nil {
// Close never returns an error.
colexecerror.InternalError(errors.AssertionFailedf("unexpected error %v from Columnarizer.Close", err))
}
Expand Down Expand Up @@ -167,7 +167,7 @@ func (c *Columnarizer) Init(ctx context.Context) {

// GetStats is part of the colexecop.VectorizedStatsCollector interface.
func (c *Columnarizer) GetStats() *execinfrapb.ComponentStats {
if c.removedFromFlow && util.CrdbTestBuild {
if c.removedFromFlow && buildutil.CrdbTestBuild {
colexecerror.InternalError(errors.AssertionFailedf(
"unexpectedly the columnarizer was removed from the flow when stats are being collected",
))
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/colexec/invariants_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/colexecop"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/errors"
)

Expand All @@ -37,7 +37,7 @@ var _ colexecop.ClosableOperator = &invariantsChecker{}

// NewInvariantsChecker creates a new invariantsChecker.
func NewInvariantsChecker(input colexecop.Operator) colexecop.DrainableOperator {
if !util.CrdbTestBuild {
if !buildutil.CrdbTestBuild {
colexecerror.InternalError(errors.AssertionFailedf(
"an invariantsChecker is attempted to be created in non-test build",
))
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/colflow/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ go_library(
"//pkg/sql/types",
"//pkg/util",
"//pkg/util/admission",
"//pkg/util/buildutil",
"//pkg/util/log",
"//pkg/util/metric",
"//pkg/util/mon",
Expand Down Expand Up @@ -112,8 +113,8 @@ go_test(
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util",
"//pkg/util/admission",
"//pkg/util/buildutil",
"//pkg/util/hlc",
"//pkg/util/humanizeutil",
"//pkg/util/leaktest",
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/colflow/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/colflow/colrpc"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
Expand Down Expand Up @@ -287,7 +287,7 @@ func (nvsc *networkVectorizedStatsCollectorImpl) GetStats() *execinfrapb.Compone
// specified. See comment on statsInvariantChecker for the kind of invariants
// checked.
func maybeAddStatsInvariantChecker(op *colexecargs.OpWithMetaInfo) {
if util.CrdbTestBuild {
if buildutil.CrdbTestBuild {
c := &statsInvariantChecker{}
op.StatsCollectors = append(op.StatsCollectors, c)
op.MetadataSources = append(op.MetadataSources, c)
Expand Down
13 changes: 7 additions & 6 deletions pkg/sql/colflow/vectorized_flow.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/admission"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/mon"
Expand Down Expand Up @@ -309,7 +310,7 @@ func (f *vectorizedFlow) Cleanup(ctx context.Context) {
// This cleans up all the memory and disk monitoring of the vectorized flow.
f.creator.cleanup(ctx)

if util.CrdbTestBuild && f.FlowBase.Started() {
if buildutil.CrdbTestBuild && f.FlowBase.Started() {
// Check that all closers have been closed. Note that we don't check
// this in case the flow was never started in the first place (it is ok
// to not check this since closers haven't allocated any resources in
Expand Down Expand Up @@ -746,7 +747,7 @@ func (s *vectorizedFlowCreator) setupRouter(

foundLocalOutput := false
for i, op := range outputs {
if util.CrdbTestBuild {
if buildutil.CrdbTestBuild {
op = colexec.NewInvariantsChecker(op)
}
stream := &output.Streams[i]
Expand Down Expand Up @@ -855,7 +856,7 @@ func (s *vectorizedFlowCreator) setupInput(
s.addStreamEndpoint(inputStream.StreamID, inbox, s.waitGroup)
op := colexecop.Operator(inbox)
ms := colexecop.MetadataSource(inbox)
if util.CrdbTestBuild {
if buildutil.CrdbTestBuild {
op = colexec.NewInvariantsChecker(op)
ms = op.(colexecop.MetadataSource)
}
Expand Down Expand Up @@ -914,7 +915,7 @@ func (s *vectorizedFlowCreator) setupInput(
// instead.
statsInputs = nil
}
if util.CrdbTestBuild {
if buildutil.CrdbTestBuild {
opWithMetaInfo.Root = colexec.NewInvariantsChecker(opWithMetaInfo.Root)
opWithMetaInfo.MetadataSources[0] = opWithMetaInfo.Root.(colexecop.MetadataSource)
}
Expand Down Expand Up @@ -1000,7 +1001,7 @@ func (s *vectorizedFlowCreator) setupOutput(
// We need to use the row receiving output.
if input != nil {
// We successfully removed the columnarizer.
if util.CrdbTestBuild {
if buildutil.CrdbTestBuild {
// That columnarizer was added as a closer, so we need to
// decrement the number of expected closers.
s.numClosers--
Expand Down Expand Up @@ -1127,7 +1128,7 @@ func (s *vectorizedFlowCreator) setupFlow(
if flowCtx.EvalCtx.SessionData().TestingVectorizeInjectPanics {
result.Root = newPanicInjector(result.Root)
}
if util.CrdbTestBuild {
if buildutil.CrdbTestBuild {
toCloseCopy := append(colexecop.Closers{}, result.ToClose...)
for i := range toCloseCopy {
func(idx int) {
Expand Down
Loading

0 comments on commit b6aeffc

Please sign in to comment.