Skip to content

Commit

Permalink
Merge #76274 #76320
Browse files Browse the repository at this point in the history
76274: backup: redact incremental storage URIs r=dt a=dt

Release note: none.

76320: dev: various improvements to `--cross` compilation r=rail a=rickystewart

1. We bind-mount an empty file over `.bazelrc.user` so that user
   configuration doesn't interfere with or break cross builds.
2. Add a new `whereis` executable. It's basically just `realpath` except
   portable.
3. Update how `dev cache` finds the location of the `bazel-remote`
   binary: now we use `bazel run --run_under=whereis`, which doesn't
   require doing any funny stuff with `cquery`.
4. Allow cross-building `cmake` and `go_test` targets, using `realpath`
   to find where test and binary targets are placed.

Closes #76094.
Closes #76260.

Release note: None

Co-authored-by: David Taylor <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
  • Loading branch information
3 people committed Feb 11, 2022
3 parents daf7e7b + fce9bea + 6d18cb8 commit 657c0e1
Show file tree
Hide file tree
Showing 11 changed files with 122 additions and 80 deletions.
1 change: 1 addition & 0 deletions build/bazelutil/empty.bazelrc
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
# intentionally empty (this file is mounted as .bazelrc.user for cross builds)
14 changes: 14 additions & 0 deletions build/bazelutil/whereis/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library")

go_library(
name = "whereis_lib",
srcs = ["main.go"],
importpath = "github.com/cockroachdb/cockroach/build/bazelutil/whereis",
visibility = ["//visibility:private"],
)

go_binary(
name = "whereis",
embed = [":whereis_lib"],
visibility = ["//visibility:public"],
)
33 changes: 33 additions & 0 deletions build/bazelutil/whereis/main.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
// Copyright 2022 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 main

import (
"fmt"
"os"
"path/filepath"
)

// whereis is a helper executable that is basically just `realpath`. It's meant
// to be used like:
// bazel run ... --run_under //build/bazelutil/whereis
// ... which will print the location of the binary you're running. Useful
// because Bazel can be a little unclear about where exactly to find any given
// executable.
func main() {
if len(os.Args) != 2 {
panic("expected a single argument")
}
abs, err := filepath.Abs(os.Args[1])
if err != nil {
panic(err)
}
fmt.Printf("%s\n", abs)
}
12 changes: 12 additions & 0 deletions pkg/ccl/backupccl/create_scheduled_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -1002,6 +1002,18 @@ func (m ScheduledBackupExecutionArgs) MarshalJSONPB(marshaller *jsonpb.Marshaler
backup.IncrementalFrom[i] = tree.NewDString(clean)
}

for i := range backup.Options.IncrementalStorage {
raw, ok := backup.Options.IncrementalStorage[i].(*tree.StrVal)
if !ok {
return nil, errors.Errorf("unexpected %T arg in backup schedule: %v", raw, raw)
}
clean, err := cloud.SanitizeExternalStorageURI(raw.RawString(), nil /* extraParams */)
if err != nil {
return nil, err
}
backup.Options.IncrementalStorage[i] = tree.NewDString(clean)
}

for i := range backup.Options.EncryptionKMSURI {
raw, ok := backup.Options.EncryptionKMSURI[i].(*tree.StrVal)
if !ok {
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/dev/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ go_test(
data = glob(["testdata/**"]),
embed = [":dev_lib"],
deps = [
"//pkg/build/bazel",
"//pkg/cmd/dev/io/exec",
"//pkg/cmd/dev/io/os",
"//pkg/testutils",
Expand Down
77 changes: 40 additions & 37 deletions pkg/cmd/dev/build.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,8 @@ const (
)

type buildTarget struct {
fullName string
isGoBinary bool
fullName string
kind string
}

// makeBuildCmd constructs the subcommand used to build the specified binaries.
Expand Down Expand Up @@ -128,14 +128,6 @@ func (d *dev) build(cmd *cobra.Command, commandLine []string) error {
}
return d.stageArtifacts(ctx, buildTargets)
}
// Cross-compilation case.
for _, target := range buildTargets {
if !target.isGoBinary {
// We can't cross-compile these targets because we can't be sure where
// Bazel is going to stage their output files.
return fmt.Errorf("cannot cross-compile target %s because it is not a go binary", target.fullName)
}
}
volume := mustGetFlagString(cmd, volumeFlag)
cross = "cross" + cross
return d.crossBuild(ctx, args, buildTargets, cross, volume)
Expand All @@ -145,6 +137,7 @@ func (d *dev) crossBuild(
ctx context.Context, bazelArgs []string, targets []buildTarget, crossConfig string, volume string,
) error {
bazelArgs = append(bazelArgs, fmt.Sprintf("--config=%s", crossConfig), "--config=ci")
configArgs := getConfigArgs(bazelArgs)
dockerArgs, err := d.getDockerRunArgs(ctx, volume, false)
if err != nil {
return err
Expand All @@ -153,29 +146,39 @@ func (d *dev) crossBuild(
// to the appropriate location in /artifacts.
var script strings.Builder
script.WriteString("set -euxo pipefail\n")
// TODO(ricky): Actually, we need to shell-quote the arguments,
// but that's hard and I don't think it's necessary for now.
script.WriteString(fmt.Sprintf("bazel %s\n", strings.Join(bazelArgs, " ")))
script.WriteString(fmt.Sprintf("BAZELBIN=`bazel info bazel-bin --color=no --config=%s --config=ci`\n", crossConfig))
script.WriteString(fmt.Sprintf("bazel %s\n", shellescape.QuoteCommand(bazelArgs)))
for _, arg := range bazelArgs {
if arg == "--config=with_ui" {
script.WriteString("bazel run @nodejs//:yarn -- --check-files --cwd pkg/ui --offline\n")
break
}
}
var bazelBinSet bool
script.WriteString("set +x\n")
for _, target := range targets {
output := bazelutil.OutputOfBinaryRule(target.fullName, strings.Contains(crossConfig, "windows"))
script.WriteString(fmt.Sprintf("cp $BAZELBIN/%s /artifacts\n", output))
script.WriteString(fmt.Sprintf("chmod a+w /artifacts/%s", filepath.Base(output)))
if target.kind == "cmake" {
if !bazelBinSet {
script.WriteString(fmt.Sprintf("BAZELBIN=$(bazel info bazel-bin %s)\n", shellescape.QuoteCommand(configArgs)))
bazelBinSet = true
}
targetComponents := strings.Split(target.fullName, ":")
pkgname := strings.TrimPrefix(targetComponents[0], "//")
dirname := targetComponents[1]
script.WriteString(fmt.Sprintf("cp -R $BAZELBIN/%s/%s /artifacts/%s\n", pkgname, dirname, dirname))
script.WriteString(fmt.Sprintf("chmod a+w -R /artifacts/%s\n", dirname))
script.WriteString(fmt.Sprintf("echo \"Successfully built target %s at artifacts/%s\"\n", target.fullName, dirname))
continue
}
// NB: For test targets, the `stdout` output from `bazel run` is
// going to have some extra garbage. We grep ^/ to select out
// only the filename we're looking for.
script.WriteString(fmt.Sprintf("BIN=$(bazel run %s %s --run_under=realpath | grep ^/ | tail -n1)\n", target.fullName, shellescape.QuoteCommand(configArgs)))
script.WriteString("cp $BIN /artifacts\n")
script.WriteString("chmod a+w /artifacts/$(basename $BIN)\n")
script.WriteString(fmt.Sprintf("echo \"Successfully built binary for target %s at artifacts/$(basename $BIN)\"\n", target.fullName))
}
_, err = d.exec.CommandContextWithInput(ctx, script.String(), "docker", dockerArgs...)
if err != nil {
return err
}
for _, target := range targets {
logSuccessfulBuild(target.fullName, filepath.Join("artifacts", targetToBinBasename(target.fullName)))
}
return nil
return err
}

func (d *dev) stageArtifacts(ctx context.Context, targets []buildTarget) error {
Expand All @@ -193,7 +196,7 @@ func (d *dev) stageArtifacts(ctx context.Context, targets []buildTarget) error {
}

for _, target := range targets {
if !target.isGoBinary {
if target.kind != "go_binary" {
// Skip staging for these.
continue
}
Expand Down Expand Up @@ -235,7 +238,7 @@ func (d *dev) stageArtifacts(ctx context.Context, targets []buildTarget) error {
if err != nil {
rel = symlinkPath
}
logSuccessfulBuild(target.fullName, rel)
log.Printf("Successfully built binary for target %s at %s", target.fullName, rel)
}

shouldHoist := false
Expand Down Expand Up @@ -299,15 +302,8 @@ func (d *dev) getBasicBuildArgs(
fields := strings.Fields(line)
fullTargetName := fields[len(fields)-1]
typ := fields[0]
if typ != "go_binary" && typ != "go_library" && typ != "go_test" {
// Skip all targets besides go_binary targets, go_library
// targets, and go_test targets. Notably this does not
// include go_proto_library targets which at this point
// cannot be built standalone.
continue
}
args = append(args, fullTargetName)
buildTargets = append(buildTargets, buildTarget{fullName: fullTargetName, isGoBinary: typ == "go_binary"})
buildTargets = append(buildTargets, buildTarget{fullName: fullTargetName, kind: typ})
if typ == "go_test" {
shouldBuildWithTestConfig = true
}
Expand All @@ -321,7 +317,7 @@ func (d *dev) getBasicBuildArgs(
}

args = append(args, aliased)
buildTargets = append(buildTargets, buildTarget{fullName: aliased, isGoBinary: true})
buildTargets = append(buildTargets, buildTarget{fullName: aliased, kind: "go_binary"})
}

// Add --config=with_ui iff we're building a target that needs it.
Expand Down Expand Up @@ -423,6 +419,13 @@ func (d *dev) hoistGeneratedCode(ctx context.Context, workspace string, bazelBin
return nil
}

func logSuccessfulBuild(target, rel string) {
log.Printf("Successfully built binary for target %s at %s", target, rel)
// Given a list of Bazel arguments, find the ones starting with --config= and
// return them.
func getConfigArgs(args []string) (ret []string) {
for _, arg := range args {
if strings.HasPrefix(arg, "--config=") {
ret = append(ret, arg)
}
}
return
}
1 change: 1 addition & 0 deletions pkg/cmd/dev/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -134,6 +134,7 @@ func (d *dev) getDockerRunArgs(
}
args = append(args, "-v", workspace+":/cockroach")
args = append(args, "--workdir=/cockroach")
args = append(args, "-v", filepath.Join(workspace, "build", "bazelutil", "empty.bazelrc")+":/cockroach/.bazelrc.user")
// Create the artifacts directory.
artifacts := filepath.Join(workspace, "artifacts")
err = d.os.MkdirAll(artifacts)
Expand Down
34 changes: 2 additions & 32 deletions pkg/cmd/dev/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"strings"
"syscall"

bazelutil "github.com/cockroachdb/cockroach/pkg/build/util"
"github.com/spf13/cobra"
)

Expand Down Expand Up @@ -135,13 +134,11 @@ func (d *dev) setUpCache(ctx context.Context) (string, error) {

log.Printf("Configuring cache...\n")

if _, err := d.exec.CommandContextSilent(ctx, "bazel", "build", bazelRemoteTarget); err != nil {
return "", err
}
bazelBin, err := d.getBazelBin(ctx)
bazelRemoteLoc, err := d.exec.CommandContextSilent(ctx, "bazel", "run", bazelRemoteTarget, "--run_under=//build/bazelutil/whereis")
if err != nil {
return "", err
}
bazelRemoteBinary := strings.TrimSpace(string(bazelRemoteLoc))

// write config file unless already exists
cacheDir, err := bazelRemoteCacheDir()
Expand Down Expand Up @@ -173,33 +170,6 @@ port: 9867
}
log.Printf("Using cache configuration file at %s\n", configFile)

// Unfortunately bazel-remote is a `go_transition_binary` so I have to
// do this whole song and dance to find where the binary is. This logic
// is mostly copied from `bazci`.
output, err := d.exec.CommandContextSilent(ctx, "bazel", "cquery", bazelRemoteTarget, "--output=label_kind")
if err != nil {
return "", err
}
configHash := strings.Fields(string(output))[3]
configHash = strings.TrimPrefix(configHash, "(")
configHash = strings.TrimSuffix(configHash, ")")
output, err = d.exec.CommandContextSilent(ctx, "bazel", "config", configHash)
if err != nil {
return "", err
}
var binDirForBazelRemote string
for _, line := range strings.Split(string(output), "\n") {
if strings.Contains(line, "transition directory name fragment") {
fragmentLine := strings.Split(line, ":")
fragment := strings.TrimSpace(fragmentLine[1])
binDirForBazelRemote = filepath.Join(filepath.Dir(bazelBin)+"-"+fragment, filepath.Base(bazelBin))
break
}
}
if binDirForBazelRemote == "" {
return "", fmt.Errorf("could not find bazel-remote binary; this is a bug")
}
bazelRemoteBinary := filepath.Join(binDirForBazelRemote, bazelutil.OutputOfBinaryRule(bazelRemoteTarget, false))
cmd := exec.Command(bazelRemoteBinary, "--config_file", configFile)
stdout, err := os.Create(filepath.Join(cacheDir, "stdout.log"))
if err != nil {
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/dev/recorderdriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/build/bazel"
"github.com/cockroachdb/cockroach/pkg/cmd/dev/io/exec"
"github.com/cockroachdb/cockroach/pkg/cmd/dev/io/os"
"github.com/cockroachdb/cockroach/pkg/testutils"
Expand Down Expand Up @@ -74,7 +75,7 @@ func TestRecorderDriven(t *testing.T) {
if f := flag.Lookup("rewrite"); f != nil && f.Value.String() == "true" {
rewriting = true
}
if rewriting {
if rewriting && bazel.BuiltWithBazel() {
t.Fatalf("not supported under bazel") // needs to shell out to bazel itself
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/dev/testdata/recorderdriven/builder
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@ bazel info workspace --color=no
cat crdb-checkout/build/teamcity-bazel-support.sh
docker volume inspect bzlhome
mkdir crdb-checkout/artifacts
docker run --rm -it -v crdb-checkout:/cockroach --workdir=/cockroach -v crdb-checkout/artifacts:/artifacts -v bzlhome:/home/roach:delegated -u 503:503 cockroachdb/bazel:20220121-121551
docker run --rm -it -v crdb-checkout:/cockroach --workdir=/cockroach -v crdb-checkout/build/bazelutil/empty.bazelrc:/cockroach/.bazelrc.user -v crdb-checkout/artifacts:/artifacts -v bzlhome:/home/roach:delegated -u 502:502 cockroachdb/bazel:20220121-121551

dev builder echo hi
----
Expand All @@ -16,4 +16,4 @@ bazel info workspace --color=no
cat crdb-checkout/build/teamcity-bazel-support.sh
docker volume inspect bzlhome
mkdir crdb-checkout/artifacts
docker run --rm -i -v crdb-checkout:/cockroach --workdir=/cockroach -v crdb-checkout/artifacts:/artifacts -v bzlhome:/home/roach:delegated -u 503:503 cockroachdb/bazel:20220121-121551 echo hi
docker run --rm -i -v crdb-checkout:/cockroach --workdir=/cockroach -v crdb-checkout/build/bazelutil/empty.bazelrc:/cockroach/.bazelrc.user -v crdb-checkout/artifacts:/artifacts -v bzlhome:/home/roach:delegated -u 502:502 cockroachdb/bazel:20220121-121551 echo hi
Loading

0 comments on commit 657c0e1

Please sign in to comment.