From b9691c4badc2597f4ff98a1f84b0e58d8a7bc7a4 Mon Sep 17 00:00:00 2001 From: Ricky Stewart Date: Mon, 9 Aug 2021 16:48:53 -0500 Subject: [PATCH] dev: implement `dev generate docs` Do some refactoring so that we don't have to repeat code in a bunch of places. The implementation is pretty straightforward and just involves building the docs then copying them over to the workspace. Closes #68563. Release note: None --- .../cockroach/ci/builds/build_impl.sh | 2 +- docs/generated/bazel_targets.txt | 11 ++ pkg/BUILD.bazel | 1 + pkg/build/util/BUILD.bazel | 15 +++ pkg/build/util/util.go | 75 +++++++++++++ pkg/build/util/util_test.go | 62 +++++++++++ pkg/cmd/bazci/BUILD.bazel | 1 + pkg/cmd/bazci/watch.go | 17 +-- pkg/cmd/dev/BUILD.bazel | 1 + pkg/cmd/dev/build.go | 32 +----- pkg/cmd/dev/generate.go | 55 +++++++++ pkg/cmd/dev/io/os/os.go | 24 ++++ pkg/cmd/dev/testdata/generate.txt | 18 +++ pkg/cmd/dev/testdata/recording/generate.txt | 105 ++++++++++++++++++ pkg/cmd/dev/util.go | 13 ++- 15 files changed, 387 insertions(+), 45 deletions(-) create mode 100644 docs/generated/bazel_targets.txt create mode 100644 pkg/build/util/BUILD.bazel create mode 100644 pkg/build/util/util.go create mode 100644 pkg/build/util/util_test.go diff --git a/build/teamcity/cockroach/ci/builds/build_impl.sh b/build/teamcity/cockroach/ci/builds/build_impl.sh index a45a5ccc3691..4a799caadea9 100755 --- a/build/teamcity/cockroach/ci/builds/build_impl.sh +++ b/build/teamcity/cockroach/ci/builds/build_impl.sh @@ -14,7 +14,7 @@ CONFIG="$1" DOC_TARGETS= if [ "$CONFIG" == "crosslinux" ] then - DOC_TARGETS="//docs/generated:gen-logging-md //docs/generated:gen-logsinks-md //docs/generated:gen-eventlog-md //docs/generated:gen-logformats-md //docs/generated/settings:settings //docs/generated/settings:settings_for_tenants //docs/generated/sql //docs/generated/sql/bnf" + DOC_TARGETS=$(grep '^//' docs/generated/bazel_targets.txt) fi bazel build //pkg/cmd/bazci --config=ci diff --git a/docs/generated/bazel_targets.txt b/docs/generated/bazel_targets.txt new file mode 100644 index 000000000000..ccd53fafbd25 --- /dev/null +++ b/docs/generated/bazel_targets.txt @@ -0,0 +1,11 @@ +This file lists all the targets you need to build to build all the generated +documentation. Lines not beginning with // should be ignored. + +//docs/generated:gen-logging-md +//docs/generated:gen-logsinks-md +//docs/generated:gen-eventlog-md +//docs/generated:gen-logformats-md +//docs/generated/settings +//docs/generated/settings:settings_for_tenants +//docs/generated/sql +//docs/generated/sql/bnf diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 8dfad485037d..d94ff766a658 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -7,6 +7,7 @@ ALL_TESTS = [ "//pkg/bench/rttanalysis:rttanalysis_test", "//pkg/bench:bench_test", "//pkg/blobs:blobs_test", + "//pkg/build/util:util_test", "//pkg/ccl/backupccl/backupresolver:backupresolver_test", "//pkg/ccl/backupccl:backupccl_test", "//pkg/ccl/baseccl:baseccl_test", diff --git a/pkg/build/util/BUILD.bazel b/pkg/build/util/BUILD.bazel new file mode 100644 index 000000000000..66fdff32211b --- /dev/null +++ b/pkg/build/util/BUILD.bazel @@ -0,0 +1,15 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "util", + srcs = ["util.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/build/util", + visibility = ["//visibility:public"], +) + +go_test( + name = "util_test", + srcs = ["util_test.go"], + embed = [":util"], + deps = ["@com_github_stretchr_testify//require"], +) diff --git a/pkg/build/util/util.go b/pkg/build/util/util.go new file mode 100644 index 000000000000..e63342cdbe36 --- /dev/null +++ b/pkg/build/util/util.go @@ -0,0 +1,75 @@ +// 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. +// +// This file contains assorted utilities for working with Bazel internals. + +package util + +import ( + "fmt" + "path/filepath" + "regexp" + "strings" +) + +// OutputOfBinaryRule returns the path of the binary produced by the +// given build target, relative to bazel-bin. That is, +// filepath.Join(bazelBin, OutputOfBinaryRule(target)) is the absolute +// path to the build binary for the target. +func OutputOfBinaryRule(target string) string { + colon := strings.Index(target, ":") + var bin string + if colon >= 0 { + bin = target[colon+1:] + } else { + bin = target[strings.LastIndex(target, "/")+1:] + } + var head string + if strings.HasPrefix(target, "@") { + doubleSlash := strings.Index(target, "//") + head = filepath.Join("external", target[1:doubleSlash]) + } else if colon >= 0 { + head = strings.TrimPrefix(target[:colon], "//") + } else { + head = strings.TrimPrefix(target, "//") + } + return filepath.Join(head, bin+"_", bin) +} + +// OutputsOfGenrule lists the outputs of a genrule. The first argument +// is the name of the target (e.g. //docs/generated/sql), and the second +// should be the output of `bazel query --output=xml $TARGET`. The +// returned slice is the list of outputs, all of which are relative +// paths atop `bazel-bin` as in `OutputOfBinaryRule`. +func OutputsOfGenrule(target, xmlQueryOutput string) ([]string, error) { + // XML parsing is a bit heavyweight here, and encoding/xml + // refuses to parse the query output since it's XML 1.1 instead + // of 1.0. Have fun with regexes instead. + colon := strings.LastIndex(target, ":") + if colon < 0 { + colon = len(target) + } + regexStr := fmt.Sprintf("^.*)\"/>$", regexp.QuoteMeta(target[:colon])) + re, err := regexp.Compile(regexStr) + if err != nil { + return nil, err + } + var ret []string + for _, line := range strings.Split(xmlQueryOutput, "\n") { + line = strings.TrimSpace(line) + submatch := re.FindStringSubmatch(line) + if submatch == nil { + continue + } + relBinPath := filepath.Join(strings.TrimPrefix(target[:colon], "//"), submatch[1]) + ret = append(ret, relBinPath) + } + return ret, nil +} diff --git a/pkg/build/util/util_test.go b/pkg/build/util/util_test.go new file mode 100644 index 000000000000..52cbfca20baa --- /dev/null +++ b/pkg/build/util/util_test.go @@ -0,0 +1,62 @@ +// Copyright 2021 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 util + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestOutputOfBinaryRule(t *testing.T) { + require.Equal(t, OutputOfBinaryRule("//pkg/cmd/cockroach-short"), "pkg/cmd/cockroach-short/cockroach-short_/cockroach-short") + require.Equal(t, OutputOfBinaryRule("//pkg/cmd/cockroach-short:cockroach-short"), "pkg/cmd/cockroach-short/cockroach-short_/cockroach-short") + require.Equal(t, OutputOfBinaryRule("pkg/cmd/cockroach-short"), "pkg/cmd/cockroach-short/cockroach-short_/cockroach-short") + + require.Equal(t, OutputOfBinaryRule("@com_github_cockroachdb_stress//:stress"), "external/com_github_cockroachdb_stress/stress_/stress") +} + +func TestOutputsOfGenrule(t *testing.T) { + xmlQueryOutput := ` + + + + + + + + + + + + + + + + + + + +` + expected := []string{ + "docs/generated/sql/aggregates.md", + "docs/generated/sql/functions.md", + "docs/generated/sql/operators.md", + "docs/generated/sql/window_functions.md", + } + out, err := OutputsOfGenrule("//docs/generated/sql:sql", xmlQueryOutput) + require.NoError(t, err) + require.Equal(t, out, expected) + out, err = OutputsOfGenrule("//docs/generated/sql", xmlQueryOutput) + require.NoError(t, err) + require.Equal(t, out, expected) +} diff --git a/pkg/cmd/bazci/BUILD.bazel b/pkg/cmd/bazci/BUILD.bazel index 37fdce79a07a..283ae685a03c 100644 --- a/pkg/cmd/bazci/BUILD.bazel +++ b/pkg/cmd/bazci/BUILD.bazel @@ -10,6 +10,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/cmd/bazci", visibility = ["//visibility:private"], deps = [ + "//pkg/build/util", "@com_github_cockroachdb_errors//:errors", "@com_github_spf13_cobra//:cobra", ], diff --git a/pkg/cmd/bazci/watch.go b/pkg/cmd/bazci/watch.go index 22dadeb9e5f9..5a411543419d 100644 --- a/pkg/cmd/bazci/watch.go +++ b/pkg/cmd/bazci/watch.go @@ -19,10 +19,10 @@ import ( "os" "path" "path/filepath" - "regexp" "strings" "time" + bazelutil "github.com/cockroachdb/cockroach/pkg/build/util" "github.com/cockroachdb/errors" ) @@ -216,22 +216,11 @@ func (w watcher) stageBinaryArtifacts() error { if err != nil { return err } - // XML parsing is a bit heavyweight here, and encoding/xml - // refuses to parse the query output since it's XML 1.1 instead - // of 1.0. Have fun with regexes instead. - colon := strings.LastIndex(bin, ":") - regexStr := fmt.Sprintf("^.*)\"/>$", regexp.QuoteMeta(bin[:colon])) - re, err := regexp.Compile(regexStr) + outs, err := bazelutil.OutputsOfGenrule(bin, query) if err != nil { return err } - for _, line := range strings.Split(query, "\n") { - line = strings.TrimSpace(line) - submatch := re.FindStringSubmatch(line) - if submatch == nil { - continue - } - relBinPath := filepath.Join(strings.TrimPrefix(bin[:colon], "//"), submatch[1]) + for _, relBinPath := range outs { err := w.maybeStageArtifact(binSourceDir, relBinPath, 0666, finalizePhase, copyContentTo) if err != nil { return err diff --git a/pkg/cmd/dev/BUILD.bazel b/pkg/cmd/dev/BUILD.bazel index 877edfc6a5c1..dd90a2404fd6 100644 --- a/pkg/cmd/dev/BUILD.bazel +++ b/pkg/cmd/dev/BUILD.bazel @@ -16,6 +16,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/cmd/dev", visibility = ["//visibility:private"], deps = [ + "//pkg/build/util", "//pkg/cmd/dev/io/exec", "//pkg/cmd/dev/io/os", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/cmd/dev/build.go b/pkg/cmd/dev/build.go index df729c7809ba..04b1aaa2b540 100644 --- a/pkg/cmd/dev/build.go +++ b/pkg/cmd/dev/build.go @@ -18,6 +18,7 @@ import ( "path/filepath" "strings" + bazelutil "github.com/cockroachdb/cockroach/pkg/build/util" "github.com/cockroachdb/errors" "github.com/cockroachdb/errors/oserror" "github.com/spf13/cobra" @@ -103,7 +104,7 @@ func (d *dev) build(cmd *cobra.Command, targets []string) error { script.WriteString(fmt.Sprintf("bazel %s\n", strings.Join(args, " "))) script.WriteString(fmt.Sprintf("BAZELBIN=`bazel info bazel-bin --color=no --config=%s`\n", cross)) for _, target := range fullTargets { - script.WriteString(fmt.Sprintf("cp $BAZELBIN/%s /artifacts\n", targetToRelativeBinPath(target))) + script.WriteString(fmt.Sprintf("cp $BAZELBIN/%s /artifacts\n", bazelutil.OutputOfBinaryRule(target))) script.WriteString(fmt.Sprintf("chmod +w /artifacts/%s\n", targetToBinBasename(target))) } _, err = d.exec.CommandContextWithInput(ctx, script.String(), "docker", dockerArgs...) @@ -158,28 +159,6 @@ func (d *dev) symlinkBinaries(ctx context.Context, targets []string) error { return nil } -// targetToRelativeBinPath returns the path of the binary produced by this build -// target relative to bazel-bin. That is, -// filepath.Join(bazelBin, targetToRelativeBinPath(target)) is the absolute -// path to the build binary for the target. -func targetToRelativeBinPath(target string) string { - var head string - if strings.HasPrefix(target, "@") { - doubleSlash := strings.Index(target, "//") - head = filepath.Join("external", target[1:doubleSlash]) - } else { - head = strings.TrimPrefix(target, "//") - } - var bin string - colon := strings.Index(target, ":") - if colon >= 0 { - bin = target[colon+1:] - } else { - bin = target[strings.LastIndex(target, "/")+1:] - } - return filepath.Join(head, bin+"_", bin) -} - func targetToBinBasename(target string) string { base := filepath.Base(strings.TrimPrefix(target, "//")) // If there's a colon, the actual name of the executable is @@ -192,14 +171,11 @@ func targetToBinBasename(target string) string { } func (d *dev) getPathToBin(ctx context.Context, target string) (string, error) { - args := []string{"info", "bazel-bin", "--color=no"} - args = append(args, getConfigFlags()...) - out, err := d.exec.CommandContextSilent(ctx, "bazel", args...) + bazelBin, err := d.getBazelBin(ctx) if err != nil { return "", err } - bazelBin := strings.TrimSpace(string(out)) - rel := targetToRelativeBinPath(target) + rel := bazelutil.OutputOfBinaryRule(target) return filepath.Join(bazelBin, rel), nil } diff --git a/pkg/cmd/dev/generate.go b/pkg/cmd/dev/generate.go index ab320f56be14..1b7d877264a7 100644 --- a/pkg/cmd/dev/generate.go +++ b/pkg/cmd/dev/generate.go @@ -12,7 +12,9 @@ package main import ( "path/filepath" + "strings" + bazelutil "github.com/cockroachdb/cockroach/pkg/build/util" "github.com/cockroachdb/errors" "github.com/spf13/cobra" ) @@ -42,6 +44,7 @@ func (d *dev) generate(cmd *cobra.Command, targets []string) error { // TODO(irfansharif): Flesh out the remaining targets. var generatorTargetMapping = map[string]func(cmd *cobra.Command) error{ "bazel": d.generateBazel, + "docs": d.generateDocs, } if len(targets) == 0 { @@ -74,3 +77,55 @@ func (d *dev) generateBazel(cmd *cobra.Command) error { _, err = d.exec.CommandContext(ctx, filepath.Join(workspace, "build", "bazelutil", "bazel-generate.sh")) return err } + +func (d *dev) generateDocs(cmd *cobra.Command) error { + ctx := cmd.Context() + workspace, err := d.getWorkspace(ctx) + if err != nil { + return err + } + // List targets we need to build. + targetsFile, err := d.os.ReadFile(filepath.Join(workspace, "docs/generated/bazel_targets.txt")) + if err != nil { + return err + } + var targets []string + for _, line := range strings.Split(targetsFile, "\n") { + line = strings.TrimSpace(line) + if strings.HasPrefix(line, "//") { + targets = append(targets, line) + } + } + // Build targets. + var args []string + args = append(args, "build", "--color=yes", "--experimental_convenience_symlinks=ignore") + args = append(args, mustGetRemoteCacheArgs(remoteCacheAddr)...) + args = append(args, getConfigFlags()...) + args = append(args, targets...) + err = d.exec.CommandContextInheritingStdStreams(ctx, "bazel", args...) + if err != nil { + return err + } + // Copy docs from bazel-bin to workspace. + bazelBin, err := d.getBazelBin(ctx) + if err != nil { + return err + } + for _, target := range targets { + query, err := d.exec.CommandContextSilent(ctx, "bazel", "query", "--output=xml", target) + if err != nil { + return err + } + outputs, err := bazelutil.OutputsOfGenrule(target, string(query)) + if err != nil { + return err + } + for _, output := range outputs { + err = d.os.CopyFile(filepath.Join(bazelBin, output), filepath.Join(workspace, output)) + if err != nil { + return err + } + } + } + return nil +} diff --git a/pkg/cmd/dev/io/os/os.go b/pkg/cmd/dev/io/os/os.go index 853fb93ae969..8202bb3e2539 100644 --- a/pkg/cmd/dev/io/os/os.go +++ b/pkg/cmd/dev/io/os/os.go @@ -12,6 +12,7 @@ package os import ( "fmt" + "io" "io/ioutil" "log" "os" @@ -182,6 +183,29 @@ func (o *OS) ReadFile(filename string) (string, error) { return ret, err } +// CopyFile copies a file from one location to another. +func (o *OS) CopyFile(src, dst string) error { + command := fmt.Sprintf("cp %s %s", src, dst) + o.logger.Print(command) + + if o.Recording == nil { + // Do the real thing. + srcFile, err := os.Open(src) + if err != nil { + return err + } + dstFile, err := os.Create(dst) + if err != nil { + return err + } + _, err = io.Copy(dstFile, srcFile) + return err + } + + _, err := o.replay(command) + return err +} + // replay replays the specified command, erroring out if it's mismatched with // what the recording plays back next. It returns the recorded output. func (o *OS) replay(command string) (output string, err error) { diff --git a/pkg/cmd/dev/testdata/generate.txt b/pkg/cmd/dev/testdata/generate.txt index a916f11850a1..992ffca29471 100644 --- a/pkg/cmd/dev/testdata/generate.txt +++ b/pkg/cmd/dev/testdata/generate.txt @@ -6,3 +6,21 @@ readlink /usr/local/opt/ccache/libexec/cc export PATH=/usr/local/opt/make/libexec/gnubin:/usr/local/bin:/usr/bin:/bin:/usr/sbin:/sbin:/usr/local/go/bin:/Library/Apple/usr/bin bazel info workspace --color=no --config=dev go/src/github.com/cockroachdb/cockroach/build/bazelutil/bazel-generate.sh + +dev gen docs +---- +getenv PATH +which cc +readlink /usr/local/opt/ccache/libexec/cc +export PATH=/usr/local/opt/make/libexec/gnubin:/usr/local/bin:/usr/bin:/bin:/usr/sbin:/sbin:/usr/local/go/bin:/Library/Apple/usr/bin +bazel info workspace --color=no --config=dev +cat go/src/github.com/cockroachdb/cockroach/docs/generated/bazel_targets.txt +bazel build --color=yes --experimental_convenience_symlinks=ignore --config=dev //docs/generated:gen-logging-md //docs/generated/sql +bazel info bazel-bin --color=no --config=dev +bazel query --output=xml //docs/generated:gen-logging-md +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/logging.md go/src/github.com/cockroachdb/cockroach/docs/generated/logging.md +bazel query --output=xml //docs/generated/sql +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/aggregates.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/aggregates.md +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/functions.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/functions.md +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/operators.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/operators.md +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/window_functions.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/window_functions.md diff --git a/pkg/cmd/dev/testdata/recording/generate.txt b/pkg/cmd/dev/testdata/recording/generate.txt index 892d3708bc3f..48d5904815ab 100644 --- a/pkg/cmd/dev/testdata/recording/generate.txt +++ b/pkg/cmd/dev/testdata/recording/generate.txt @@ -19,3 +19,108 @@ go/src/github.com/cockroachdb/cockroach go/src/github.com/cockroachdb/cockroach/build/bazelutil/bazel-generate.sh ---- + +getenv PATH +---- +/usr/local/opt/ccache/libexec:/usr/local/opt/make/libexec/gnubin:/usr/local/bin:/usr/bin:/bin:/usr/sbin:/sbin:/usr/local/go/bin:/Library/Apple/usr/bin + +which cc +---- +/usr/local/opt/ccache/libexec/cc + +readlink /usr/local/opt/ccache/libexec/cc +---- +../bin/ccache + +export PATH=/usr/local/opt/make/libexec/gnubin:/usr/local/bin:/usr/bin:/bin:/usr/sbin:/sbin:/usr/local/go/bin:/Library/Apple/usr/bin +---- + +bazel info workspace --color=no --config=dev +---- +go/src/github.com/cockroachdb/cockroach + +cat go/src/github.com/cockroachdb/cockroach/docs/generated/bazel_targets.txt +---- +---- +This line is ignored. + +//docs/generated:gen-logging-md +//docs/generated/sql +---- +---- + +bazel build --color=yes --experimental_convenience_symlinks=ignore --config=dev //docs/generated:gen-logging-md //docs/generated/sql +---- + +bazel info bazel-bin --color=no --config=dev +---- +/private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin + +bazel query --output=xml //docs/generated:gen-logging-md +---- +---- + + + + + + + + + + + + + + + + + + +---- +---- + +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/logging.md go/src/github.com/cockroachdb/cockroach/docs/generated/logging.md +---- + +bazel query --output=xml //docs/generated/sql +---- +---- + + + + + + + + + + + + + + + + + + + + + +---- +---- + +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/aggregates.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/aggregates.md +---- + +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/functions.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/functions.md +---- + +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/operators.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/operators.md +---- + +cp /private/var/tmp/_bazel/99e666e4e674209ecdb66b46371278df/execroot/cockroach/bazel-out/darwin-fastbuild/bin/docs/generated/sql/window_functions.md go/src/github.com/cockroachdb/cockroach/docs/generated/sql/window_functions.md +---- diff --git a/pkg/cmd/dev/util.go b/pkg/cmd/dev/util.go index 3e87745c1a48..f274167b67d8 100644 --- a/pkg/cmd/dev/util.go +++ b/pkg/cmd/dev/util.go @@ -82,14 +82,23 @@ func parseAddr(addr string) (string, error) { return fmt.Sprintf("%s:%s", ip, port), nil } -func (d *dev) getWorkspace(ctx context.Context) (string, error) { - args := []string{"info", "workspace", "--color=no"} +func (d *dev) getBazelInfo(ctx context.Context, key string) (string, error) { + args := []string{"info", key, "--color=no"} args = append(args, getConfigFlags()...) out, err := d.exec.CommandContextSilent(ctx, "bazel", args...) if err != nil { return "", err } return strings.TrimSpace(string(out)), nil + +} + +func (d *dev) getWorkspace(ctx context.Context) (string, error) { + return d.getBazelInfo(ctx, "workspace") +} + +func (d *dev) getBazelBin(ctx context.Context) (string, error) { + return d.getBazelInfo(ctx, "bazel-bin") } func getConfigFlags() []string {