Skip to content

Commit

Permalink
Merge #92476
Browse files Browse the repository at this point in the history
92476: teamcity-trigger: remove make-related code and pass custom timeouts r=rickystewart a=healthy-pod

This code change removes make-related code from teamcity-trigger. It
also allows us to customize the timeout used per test target by updating
one source which is the `customTimeouts` map. The timeout will then be
propagated to bazel's `test_timeout` and go's `-test.timeout` as needed.

Release note: None
Epic: None

Co-authored-by: healthy-pod <[email protected]>
  • Loading branch information
craig[bot] and healthy-pod committed Nov 29, 2022
2 parents 1af116f + 412bb24 commit c595495
Show file tree
Hide file tree
Showing 5 changed files with 152 additions and 201 deletions.
5 changes: 0 additions & 5 deletions build/bazelutil/check.sh
Original file line number Diff line number Diff line change
Expand Up @@ -51,12 +51,7 @@ pkg/util/timeutil/zoneinfo.go://go:generate go run gen/main.go
"

EXISTING_BROKEN_TESTS_IN_BAZEL="
pkg/acceptance/BUILD.bazel
pkg/cmd/cockroach-oss/BUILD.bazel
pkg/cmd/bazci/githubpost/BUILD.bazel
pkg/cmd/prereqs/BUILD.bazel
pkg/cmd/roachtest/BUILD.bazel
pkg/cmd/teamcity-trigger/BUILD.bazel
"

EXISTING_CRDB_TEST_BUILD_CONSTRAINTS="
Expand Down
52 changes: 18 additions & 34 deletions build/teamcity/cockroach/nightlies/stress_impl.sh
Original file line number Diff line number Diff line change
Expand Up @@ -15,40 +15,24 @@ bazel build //pkg/cmd/bazci --config=ci
BAZEL_BIN=$(bazel info bazel-bin --config=ci)
ARTIFACTS_DIR=/artifacts

# Query to list all affected tests.
PKG=${PKG#"./"}
if [[ $(basename $PKG) != ... ]]
if [[ ! -z $(bazel query "attr(tags, \"broken_in_bazel\", $TARGET)") ]]
then
PKG="$PKG:all"
echo "Skipping test $TARGET as it is broken in bazel"
exit 0
fi
if [[ ! -z $(bazel query "attr(tags, \"integration\", $TARGET)") ]]
then
echo "Skipping test $TARGET as it is an integration test"
exit 0
fi
tests=$(bazel query "kind(go_test, $PKG)" --output=label)

# Run affected tests.
for test in $tests
do
if [[ ! -z $(bazel query "attr(tags, \"broken_in_bazel\", $test)") ]]
then
echo "Skipping test $test as it is broken in bazel"
continue
fi
if [[ ! -z $(bazel query "attr(tags, \"integration\", $test)") ]]
then
echo "Skipping test $test as it is an integration test"
continue
fi
exit_status=0
$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci -- test --config=ci "$test" \
--test_env=COCKROACH_NIGHTLY_STRESS=true \
--test_timeout="$TESTTIMEOUTSECS" \
--run_under "@com_github_cockroachdb_stress//:stress -bazel -shardable-artifacts 'XML_OUTPUT_FILE=$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci merge-test-xmls' $STRESSFLAGS" \
--define "gotags=$TAGS" \
--nocache_test_results \
--test_output streamed \
${EXTRA_BAZEL_FLAGS} \
|| exit_status=$?
if [ $exit_status -ne 0 ]
then
exit $exit_status
fi
done

GOTESTTIMEOUTSECS=$(($TESTTIMEOUTSECS - 5))
$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci -- test --config=ci "$TARGET" \
--test_env=COCKROACH_NIGHTLY_STRESS=true \
--test_timeout="$TESTTIMEOUTSECS" \
--test_arg=-test.timeout="${GOTESTTIMEOUTSECS}s"
--run_under "@com_github_cockroachdb_stress//:stress -bazel -shardable-artifacts 'XML_OUTPUT_FILE=$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci merge-test-xmls' $STRESSFLAGS" \
--define "gotags=$TAGS" \
--nocache_test_results \
--test_output streamed \
${EXTRA_BAZEL_FLAGS} \
4 changes: 1 addition & 3 deletions pkg/cmd/teamcity-trigger/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -7,10 +7,8 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/cmd/teamcity-trigger",
visibility = ["//visibility:private"],
deps = [
"//pkg/build/bazel",
"//pkg/cmd/cmdutil",
"@com_github_abourget_teamcity//:teamcity",
"@com_github_kisielk_gotool//:gotool",
],
)

Expand All @@ -26,7 +24,7 @@ go_test(
srcs = ["main_test.go"],
args = ["-test.timeout=55s"],
embed = [":teamcity-trigger_lib"],
tags = ["broken_in_bazel"],
deps = ["@com_github_stretchr_testify//require"],
)

get_x_data(name = "get_x_data")
145 changes: 83 additions & 62 deletions pkg/cmd/teamcity-trigger/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,14 +21,64 @@ import (
"fmt"
"log"
"os"
"os/exec"
"strings"
"time"

"github.com/abourget/teamcity"
"github.com/cockroachdb/cockroach/pkg/build/bazel"
"github.com/cockroachdb/cockroach/pkg/cmd/cmdutil"
"github.com/kisielk/gotool"
)

type timeoutSpec struct {
// Set `recursive` to also apply this timeout to all subpackages. Remember that timeouts should
// be just over what is needed for the tests to pass to be alerted of any future performance regressions
// so take care when setting this option and only set it if you really need what it does.
recursive bool
// maxTime will be used for the following:
// 1) Passed as a stress flag.
// 2) Bazel timeout will be set to `maxTime` + 1 minute to give `stress` an extra minute to clean up.
// 3) Go test process timeout will be set to `maxTime` + 55 seconds to let it timeout before Bazel kills it and allow
// us to get stacktraces.
maxTime time.Duration
}

const (
buildID = "Cockroach_Nightlies_StressBazel"
)

var (
customTimeouts = map[string]timeoutSpec{
"//pkg/sql/logictest": {
maxTime: 3 * time.Hour,
},
"//pkg/kv/kvserver": {
maxTime: 3 * time.Hour,
},
"//pkg/ccl/backupccl": {
maxTime: 2 * time.Hour,
},
"//pkg/ccl/logictestccl/tests/3node-tenant": {
maxTime: 2 * time.Hour,
},
}
)

func getMaxTime(testTarget string) time.Duration {
pathWithoutTarget := strings.Split(testTarget, ":")[0]
// Case 1: importPath is explicitly covered by customTimeouts.
if spec, ok := customTimeouts[pathWithoutTarget]; ok {
return spec.maxTime
}
// Case 2: importPath is implicitly covered by customTimeouts.
for prefixPath, spec := range customTimeouts {
if spec.recursive && strings.HasPrefix(pathWithoutTarget, prefixPath) {
return spec.maxTime
}
}
// Case 3: default timeout is 1 hour.
return 1 * time.Hour
}

func main() {
if len(os.Args) != 1 {
fmt.Fprintf(os.Stderr, "usage: %s\n", os.Args[0])
Expand All @@ -41,7 +91,9 @@ func main() {
password := cmdutil.RequireEnv("TC_API_PASSWORD")

tcClient := teamcity.New(serverURL, username, password)
count := 0
runTC(func(buildID string, opts map[string]string) {
count++
build, err := tcClient.QueueBuild(buildID, branch, opts)
if err != nil {
log.Fatalf("failed to create teamcity build (buildID=%s, branch=%s, opts=%+v): %s",
Expand All @@ -50,40 +102,25 @@ func main() {
log.Printf("created teamcity build (buildID=%s, branch=%s, opts=%+v): %s",
buildID, branch, opts, build)
})
}

func getBaseImportPath() string {
if bazel.BuiltWithBazel() {
return "./"
if count == 0 {
log.Fatal("no builds were created")
}
return "github.com/cockroachdb/cockroach/pkg/"
}

func runTC(queueBuild func(string, map[string]string)) {
buildID := "Cockroach_Nightlies_Stress"
if bazel.BuiltWithBazel() {
buildID = "Cockroach_Nightlies_StressBazel"
targets, err := exec.Command("bazel", "query", "kind(go_test, //pkg/...)", "--output=label").Output()
if err != nil {
log.Fatal(err)
}
baseImportPath := getBaseImportPath()
importPaths := gotool.ImportPaths([]string{baseImportPath + "..."})

// Queue stress builds. One per configuration per package.
for _, importPath := range importPaths {
// Queue stress builds. One per configuration per test target.
for _, testTarget := range strings.Split(string(targets), "\n") {
testTarget = strings.TrimSpace(testTarget)
// By default, run each package for up to 100 iterations.
maxRuns := 100

// By default, run each package for up to 1h.
maxTime := 1 * time.Hour

maxTime := getMaxTime(testTarget)
// By default, fail the stress run on the first test failure.
maxFails := 1

// By default, a single test times out after 40 minutes.
// NOTE: This is used only for the (now deprecated) non-Bazel
// stress job. Bazel test timeouts are handled at the test
// target level (i.e. in BUILD.bazel).
testTimeout := 40 * time.Minute

// The stress program by default runs as many instances in parallel as there
// are CPUs. Each instance itself can run tests in parallel. The amount of
// parallelism needs to be reduced, or we can run into OOM issues,
Expand All @@ -96,48 +133,34 @@ func runTC(queueBuild func(string, map[string]string)) {
parallelism := 4

opts := map[string]string{
"env.PKG": importPath,
"env.TARGET": testTarget,
}

// Conditionally override settings.
switch importPath {
case baseImportPath + "kv/kvnemesis":
if testTarget == "//pkg/kv/kvnemesis:kvnemesis_test" {
// Disable -maxruns for kvnemesis. Run for the full 1h.
maxRuns = 0
if bazel.BuiltWithBazel() {
opts["env.EXTRA_BAZEL_FLAGS"] = "--test_env COCKROACH_KVNEMESIS_STEPS=10000"
} else {
opts["env.COCKROACH_KVNEMESIS_STEPS"] = "10000"
}
case baseImportPath + "sql/logictest", baseImportPath + "kv/kvserver",
baseImportPath + "ccl/logictestccl":
opts["env.EXTRA_BAZEL_FLAGS"] = "--test_env COCKROACH_KVNEMESIS_STEPS=10000"
}

if testTarget == "//pkg/sql/logictest:logictest_test" || testTarget == "//pkg/kv/kvserver:kvserver_test" {
// Stress heavy with reduced parallelism (to avoid overloading the
// machine, see https://github.com/cockroachdb/cockroach/pull/10966).
parallelism /= 2
// Increase test timeout to compensate.
testTimeout = 2 * time.Hour
maxTime = 3 * time.Hour
}

if bazel.BuiltWithBazel() {
// NB: This is what will eventually be passed to Bazel as the --test_timeout.
// `stress` will run for maxTime, so we give it an extra minute to clean up.
opts["env.TESTTIMEOUTSECS"] = fmt.Sprintf("%.0f", (maxTime + time.Minute).Seconds())
} else {
opts["env.TESTTIMEOUT"] = testTimeout.String()
}
// NB: This is what will eventually be passed to Bazel as the --test_timeout.
// `stress` will run for maxTime, so we give it an extra minute to clean up.
opts["env.TESTTIMEOUTSECS"] = fmt.Sprintf("%.0f", (maxTime + time.Minute).Seconds())

// Run non-race build.
if bazel.BuiltWithBazel() {
bazelFlags, ok := opts["env.EXTRA_BAZEL_FLAGS"]
if ok {
opts["env.EXTRA_BAZEL_FLAGS"] = fmt.Sprintf("%s --test_sharding_strategy=disabled --jobs %d", bazelFlags, parallelism)
} else {
opts["env.EXTRA_BAZEL_FLAGS"] = fmt.Sprintf("--test_sharding_strategy=disabled --jobs %d", parallelism)
}
bazelFlags, ok := opts["env.EXTRA_BAZEL_FLAGS"]
if ok {
opts["env.EXTRA_BAZEL_FLAGS"] = fmt.Sprintf("%s --test_sharding_strategy=disabled --jobs %d", bazelFlags, parallelism)
} else {
opts["env.GOFLAGS"] = fmt.Sprintf("-parallel=%d", parallelism)
opts["env.EXTRA_BAZEL_FLAGS"] = fmt.Sprintf("--test_sharding_strategy=disabled --jobs %d", parallelism)
}

opts["env.STRESSFLAGS"] = fmt.Sprintf("-maxruns %d -maxtime %s -maxfails %d -p %d",
maxRuns, maxTime, maxFails, parallelism)
queueBuild(buildID, opts)
Expand All @@ -149,16 +172,14 @@ func runTC(queueBuild func(string, map[string]string)) {

// Run race build. With run with -p 1 to avoid overloading the machine.
noParallelism := 1
if bazel.BuiltWithBazel() {
extraBazelFlags := opts["env.EXTRA_BAZEL_FLAGS"]
// NB: Normally we'd just use `--config race`, but that implies a
// `--test_timeout` that overrides the one we manually specify.
opts["env.EXTRA_BAZEL_FLAGS"] = fmt.Sprintf("%s --@io_bazel_rules_go//go/config:race --test_env=GORACE=halt_on_error=1", extraBazelFlags)
} else {
opts["env.GOFLAGS"] = fmt.Sprintf("-race -parallel=%d", parallelism)
}
extraBazelFlags := opts["env.EXTRA_BAZEL_FLAGS"]
// NB: Normally we'd just use `--config race`, but that implies a
// `--test_timeout` that overrides the one we manually specify.
opts["env.EXTRA_BAZEL_FLAGS"] = fmt.Sprintf("%s --@io_bazel_rules_go//go/config:race --test_env=GORACE=halt_on_error=1", extraBazelFlags)
opts["env.STRESSFLAGS"] = fmt.Sprintf("-maxruns %d -maxtime %s -maxfails %d -p %d",
maxRuns, maxTime, maxFails, noParallelism)
opts["env.TAGS"] = "race"
queueBuild(buildID, opts)
delete(opts, "env.TAGS")
}
}
Loading

0 comments on commit c595495

Please sign in to comment.