Skip to content

Commit

Permalink
Merge #67325 #68134
Browse files Browse the repository at this point in the history
67325: build: add TeamCity build script to stress roachtests r=tbg a=erikgrinaker

### roachtest: handle multiple runs of same test on TeamCity

The TeamCity markers output via `--teamcity` currently do not handle
multiple runs of the same test via `--count`, since they use the same
flow ID for all runs, making TeamCity unable to distinguish between
them. A similar problem exists with artifacts, where each run would
publish the root `artifacts/` directory rather than the `run_`
subdirectories.

This patch fixes the above issues by including run identifiers where
appropriate.

Release note: None

### roachtest: add roachtest/noop test

This adds a `roachtest/noop` test that instantly succeeds without
spinning up any servers, and a `roachtest/noop-maybefail` test that
randomly fails with 20% probability. These can be useful when testing
e.g. CI builds or other test infrastructure. They are disabled by
default and require the `roachtest` tag.

Release note: None

### roachtest: add --disable-issue flag to disable GitHub issues

Release note: None

### build: add TeamCity build script to stress roachtests

This adds a build script that can be used to stress roachtests on
TeamCity, in order to reproduce failures. The corresponding TeamCity
build is `Cockroach_Nightlies_RoachtestStress`.

Release note: None

### build: explicitly build binary for TeamCity nightly tests

The TeamCity nightly tests depend on the "publish bleeding edge" build,
which pulls in a ton of additional builds (e.g. for multiple OSes) that
we don't really need to run the nightlies. This can cause unrelated
build failures to fail the nightly tests.

This patch explicitly builds a CRDB binary in
`teamcity-nightly-roachtest.sh` such that the build can be decoupled
from "publish bleeding edge". This is a temporary mitigation until the
TeamCity builds and dependencies are cleaned up.

Touches https://cockroachlabs.atlassian.net/browse/DEVINF-16.
Touches #64263.

Release note: None

---

This is still a work-in-progress, but I suggest we merge the script itself
which makes it easier to generalize the TeamCity build and work on e.g.
branch specification.

There's a bit of work remaining on the actual build, which will be addressed
separately:

* Notifications: TeamCity does not expose the email address of the person
  who started the build, so for now we'll send notifications to the Slack
  channel `#roachtest-stress-ops`.

* Running clusters: We need to let the user know about any test clusters
  that are still running, so that these can be inspected and shut down as
  appropriate. We'll probably also need to change how roachtest extends
  cluster lifetimes, since we want to reuse clusters for tests but make sure
  every test invocation will extend the lifetime to 36 hours from the start
  of the test (not the cluster creation, which is currently the case).

* Trigger link: We should include a link on test failures to a TeamCity form
  for starting a stress build (if possible), or at least instructions for how
  to start one.

* Branch/commit selection: We need to make sure it's easy for users to pick
  which branch or commit to build.

68134: tenantcostserver: add TenantID to TokenBucketRequest r=RaduBerinde a=RaduBerinde

This should address the "error issuing TokenBucket RPC" loop observed in #65830.

Informs #65830.

#### tenantcostserver: add Error field

This change adds an Error field to TokenBucketResponse, allowing the
Connector to differentiate between RPC errors and logical errors
(similar to RangeLookup).

This fixes an infinite retry loop caused by a logical issue.

Release note: None

#### tenantcostserver: add TenantID to TokenBucketRequest

We are using TenantFromContext to identify the tenant from which a
TenantBucketRequest is coming from (similarly to other connector
APIs). Unfortunately, this does not work when the connections are
insecure (as is the case in some tests and possibly when testing
manually during development).

This commit adds a TenantID field. It is cross-checked against the
tenant when running in secure mode.

Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
3 people committed Jul 29, 2021
3 parents 3ea1633 + 471f6da + 62bf97a commit 4938d16
Show file tree
Hide file tree
Showing 23 changed files with 821 additions and 599 deletions.
36 changes: 13 additions & 23 deletions build/teamcity-nightly-roachtest.sh
Original file line number Diff line number Diff line change
Expand Up @@ -8,11 +8,6 @@ source "$(dirname "${0}")/teamcity-support.sh"
# apologies, you're going to have to dig around for them below or even better
# yet, look at the job).

# Note that when this script is called, the cockroach binary to be tested
# already exists in the current directory.
COCKROACH_BINARY="${PWD}/cockroach.linux-2.6.32-gnu-amd64"
chmod +x "${COCKROACH_BINARY}"

if [[ ! -f ~/.ssh/id_rsa.pub ]]; then
ssh-keygen -q -C "roachtest-nightly $(date)" -N "" -f ~/.ssh/id_rsa
fi
Expand All @@ -25,7 +20,8 @@ chmod o+rwx "${artifacts}"
# Disable global -json flag.
export PATH=$PATH:$(GOFLAGS=; go env GOPATH)/bin

build/builder/mkrelease.sh amd64-linux-gnu bin/workload bin/roachtest bin/roachprod > "${artifacts}/build.txt" 2>&1 || cat "${artifacts}/build.txt"
build/builder/mkrelease.sh amd64-linux-gnu build bin/workload bin/roachtest bin/roachprod \
> "${artifacts}/build.txt" 2>&1 || (cat "${artifacts}/build.txt"; false)

# Set up Google credentials. Note that we need this for all clouds since we upload
# perf artifacts to Google Storage at the end.
Expand Down Expand Up @@ -80,8 +76,6 @@ function upload_stats {
trap upload_stats EXIT

# Set up the parameters for the roachtest invocation.

ARTIFACTS="${artifacts}"
PARALLELISM=16
CPUQUOTA=1024
ZONES=""
Expand All @@ -108,22 +102,18 @@ case "${CLOUD}" in
;;
esac

export \
CLOUD="${CLOUD}" \
ARTIFACTS="${ARTIFACTS}" \
PARALLELISM="${PARALLELISM}" \
CPUQUOTA="${CPUQUOTA}" \
ZONES="${ZONES}" \
COUNT="${COUNT-1}" \
DEBUG="${DEBUG-false}" \
BUILD_TAG="${BUILD_TAG}" \
COCKROACH_BINARY="${COCKROACH_BINARY}" \
SLACK_TOKEN="${SLACK_TOKEN}" \
TC_BUILD_ID="${TC_BUILD_ID}" \
TESTS="${TESTS}"

# Teamcity has a 1300 minute timeout that, when reached, kills the process
# without a stack trace (probably SIGKILL). We'd love to see a stack trace
# though, so after 1200 minutes, kill with SIGINT which will allow roachtest to
# fail tests and cleanup.
timeout -s INT $((1200*60)) "build/teamcity-nightly-roachtest-invoke.sh"
timeout -s INT $((1200*60)) "build/teamcity-roachtest-invoke.sh" \
--cloud="${CLOUD}" \
--count="${COUNT-1}" \
--parallelism="${PARALLELISM}" \
--cpu-quota="${CPUQUOTA}" \
--cluster-id="${TC_BUILD_ID}" \
--build-tag="${BUILD_TAG}" \
--cockroach="${PWD}/cockroach-linux-2.6.32-gnu-amd64" \
--artifacts="${artifacts}" \
--slack-token="${SLACK_TOKEN}" \
"${TESTS}"
Original file line number Diff line number Diff line change
Expand Up @@ -2,22 +2,13 @@
set -euo pipefail

set +e
# Append any given command-line parameters. If a switch listed below is also
# passed by the caller, the passed one takes precedence.
bin/roachtest run \
--cloud="${CLOUD}" \
--artifacts="${ARTIFACTS}" \
--parallelism="${PARALLELISM}" \
--cpu-quota="${CPUQUOTA}" \
--zones="${ZONES}" \
--count="${COUNT-1}" \
--debug="${DEBUG-false}" \
--build-tag="${BUILD_TAG}" \
--cockroach="${COCKROACH_BINARY}" \
--teamcity \
--roachprod="${PWD}/bin/roachprod" \
--workload="${PWD}/bin/workload" \
--teamcity=true \
--slack-token="${SLACK_TOKEN}" \
--cluster-id="${TC_BUILD_ID}" \
"${TESTS}"
"$@"
code=$?
set -e

Expand Down
38 changes: 38 additions & 0 deletions build/teamcity-roachtest-stress.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
#!/usr/bin/env bash
set -euo pipefail

source "$(dirname "${0}")/teamcity-support.sh"

google_credentials="$GOOGLE_EPHEMERAL_CREDENTIALS"
generate_ssh_key
log_into_gcloud

set -x

export ROACHPROD_USER=teamcity

# TODO(erikgrinaker): We should use a dedicated or more appropriate project
# here, but use andrei-jepsen for now. We don't want to use the regular test
# project here to avoid disturbing test runs due to e.g. quota limits. See:
# https://cockroachlabs.atlassian.net/browse/DEVINF-140
export GCE_PROJECT=${GCE_PROJECT-andrei-jepsen}

mkdir -p artifacts

build/builder/mkrelease.sh amd64-linux-gnu build bin/workload bin/roachtest bin/roachprod \
> artifacts/build.txt 2>&1 || (cat artifacts/build.txt; false)

build/teamcity-roachtest-invoke.sh \
--cloud=gce \
--zones=us-central1-b,us-west1-b,europe-west2-b \
--debug="${DEBUG-false}" \
--count="${COUNT-16}" \
--parallelism="${PARALLELISM-16}" \
--cpu-quota="${CPUQUOTA-1024}" \
--cluster-id="${TC_BUILD_ID}" \
--build-tag="${BUILD_TAG}" \
--create-args="--lifetime=36h" \
--cockroach="${PWD}/cockroach-linux-2.6.32-gnu-amd64" \
--artifacts="${PWD}/artifacts" \
--disable-issue \
"${TESTS}"
1 change: 1 addition & 0 deletions pkg/ccl/kvccl/kvtenantccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ go_library(
"//pkg/util/syncutil/singleflight",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//errorspb",
"@org_golang_google_grpc//:go_default_library",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//status",
Expand Down
5 changes: 5 additions & 0 deletions pkg/ccl/kvccl/kvtenantccl/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/syncutil/singleflight"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/errors/errorspb"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
Expand Down Expand Up @@ -405,6 +406,10 @@ func (c *Connector) TokenBucket(
c.tryForgetClient(ctx, client)
continue
}
if resp.Error != (errorspb.EncodedError{}) {
// Hard logical error. Propagate.
return nil, errors.DecodeError(ctx, resp.Error)
}
return resp, nil
}
return nil, ctx.Err()
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ func (c *tenantSideCostController) mainLoop(ctx context.Context, stopper *stop.S
select {
case <-ticker.C:
req := roachpb.TokenBucketRequest{
TenantID: c.tenantID.ToUint64(),
ConsumptionSinceLastRequest: roachpb.TokenBucketRequest_Consumption{
// Report a dummy 1 RU consumption each time.
RU: 1,
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/multitenantccl/tenantcostserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ go_test(
"//pkg/util/metric",
"//pkg/util/randutil",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_cockroachdb_errors//:errors",
"@in_gopkg_yaml_v2//:yaml_v2",
],
)
8 changes: 5 additions & 3 deletions pkg/ccl/multitenantccl/tenantcostserver/server_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/datadriven"
"github.com/cockroachdb/errors"
"gopkg.in/yaml.v2"
)

Expand Down Expand Up @@ -77,6 +78,7 @@ func TestDataDriven(t *testing.T) {
d.Fatalf(t, "failed to parse request yaml: %v", err)
}
req := roachpb.TokenBucketRequest{
TenantID: uint64(tenantID),
ConsumptionSinceLastRequest: roachpb.TokenBucketRequest_Consumption{
RU: args.Consumption.RU,
ReadRequests: args.Consumption.ReadReq,
Expand All @@ -86,9 +88,9 @@ func TestDataDriven(t *testing.T) {
SQLPodCPUSeconds: args.Consumption.SQLPodsCPUUsage,
},
}
_, err = tenantUsage.TokenBucketRequest(ctx, roachpb.MakeTenantID(uint64(tenantID)), &req)
if err != nil {
return fmt.Sprintf("error: %v", err)
res := tenantUsage.TokenBucketRequest(ctx, roachpb.MakeTenantID(uint64(tenantID)), &req)
if res.Error != (errors.EncodedError{}) {
return fmt.Sprintf("error: %v", errors.DecodeError(context.Background(), res.Error))
}
return ""

Expand Down
12 changes: 8 additions & 4 deletions pkg/ccl/multitenantccl/tenantcostserver/token_bucket.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,11 @@ import (
// kvtenant.Connector).
func (s *instance) TokenBucketRequest(
ctx context.Context, tenantID roachpb.TenantID, in *roachpb.TokenBucketRequest,
) (*roachpb.TokenBucketResponse, error) {
) *roachpb.TokenBucketResponse {
if tenantID == roachpb.SystemTenantID {
return nil, errors.New("token bucket request for system tenant")
return &roachpb.TokenBucketResponse{
Error: errors.EncodeError(ctx, errors.New("token bucket request for system tenant")),
}
}

result := &roachpb.TokenBucketResponse{}
Expand Down Expand Up @@ -63,9 +65,11 @@ func (s *instance) TokenBucketRequest(

return updateTenantUsageState(ctx, s.executor, txn, tenantID, state)
}); err != nil {
return nil, err
*result = roachpb.TokenBucketResponse{
Error: errors.EncodeError(ctx, err),
}
}
return result, nil
return result
}

type tenantUsageState struct {
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ var (
clusterWipe bool
zonesF string
teamCity bool
disableIssue bool
)

type encryptValue string
Expand Down
2 changes: 2 additions & 0 deletions pkg/cmd/roachtest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -187,6 +187,8 @@ runner itself.
&slackToken, "slack-token", "", "Slack bot token")
runCmd.Flags().BoolVar(
&teamCity, "teamcity", false, "include teamcity-specific markers in output")
runCmd.Flags().BoolVar(
&disableIssue, "disable-issue", false, "disable posting GitHub issue for failures")

var benchCmd = &cobra.Command{
// Don't display usage when tests fail.
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/registry/owners.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,4 +25,5 @@ const (
OwnerSQLQueries Owner = `sql-queries`
OwnerSQLSchema Owner = `sql-schema`
OwnerStorage Owner = `storage`
OwnerTestEng Owner = `test-eng`
)
35 changes: 23 additions & 12 deletions pkg/cmd/roachtest/test_runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,8 @@ type testRunner struct {
// the registry uses for running tests. It implies skipClusterWipeOnAttach.
skipClusterStopOnAttach bool
skipClusterWipeOnAttach bool
// disableIssue disables posting GitHub issues for test failures.
disableIssue bool
}

status struct {
Expand Down Expand Up @@ -105,6 +107,7 @@ func newTestRunner(cr *clusterRegistry, buildVersion version.Version) *testRunne
buildVersion: buildVersion,
}
r.config.skipClusterWipeOnAttach = !clusterWipe
r.config.disableIssue = disableIssue
r.workersMu.workers = make(map[string]*workerStatus)
return r
}
Expand Down Expand Up @@ -468,10 +471,10 @@ func (r *testRunner) runWorker(
artifactsDir = filepath.Join(base, runSuffix)
logPath = filepath.Join(artifactsDir, "test.log")

// Map artifacts/TestFoo/** => TestFoo/**, i.e. collect the artifacts
// Map artifacts/TestFoo/run_?/** => TestFoo/run_?/**, i.e. collect the artifacts
// for this test exactly as they are laid out on disk (when the time
// comes).
artifactsSpec = fmt.Sprintf("%s/** => %s", base, escapedTestName)
artifactsSpec = fmt.Sprintf("%s/%s/** => %s/%s", base, runSuffix, escapedTestName, runSuffix)
}
testL, err := logger.RootLogger(logPath, teeOpt)
if err != nil {
Expand All @@ -497,7 +500,7 @@ func (r *testRunner) runWorker(
// Now run the test.
l.PrintfCtx(ctx, "starting test: %s:%d", testToRun.spec.Name, testToRun.runNum)
var success bool
success, err = r.runTest(ctx, t, testToRun.runNum, c, testRunnerLogPath, stdout, testL)
success, err = r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, testRunnerLogPath, stdout, testL)
if err != nil {
shout(ctx, l, stdout, "test returned error: %s: %s", t.Name(), err)
// Mark the test as failed if it isn't already.
Expand Down Expand Up @@ -607,6 +610,7 @@ func (r *testRunner) runTest(
ctx context.Context,
t *testImpl,
runNum int,
runCount int,
c *clusterImpl,
testRunnerLogPath string,
stdout io.Writer,
Expand All @@ -616,10 +620,14 @@ func (r *testRunner) runTest(
return false, fmt.Errorf("can't run skipped test: %s: %s", t.Name(), t.Spec().(*registry.TestSpec).Skip)
}

runID := t.Name()
if runCount > 1 {
runID += fmt.Sprintf("#%d", runNum)
}
if teamCity {
shout(ctx, l, stdout, "##teamcity[testStarted name='%s' flowId='%s']", t.Name(), t.Name())
shout(ctx, l, stdout, "##teamcity[testStarted name='%s' flowId='%s']", t.Name(), runID)
} else {
shout(ctx, l, stdout, "=== RUN %s", t.Name())
shout(ctx, l, stdout, "=== RUN %s", runID)
}

r.status.Lock()
Expand Down Expand Up @@ -653,8 +661,7 @@ func (r *testRunner) runTest(

if teamCity {
shout(ctx, l, stdout, "##teamcity[testFailed name='%s' details='%s' flowId='%s']",
t.Name(), teamCityEscape(output), t.Name(),
)
t.Name(), teamCityEscape(output), runID)

// Copy a snapshot of the testrunner's log to the test's artifacts dir
// so that we collect it below.
Expand All @@ -664,21 +671,21 @@ func (r *testRunner) runTest(
}
}

shout(ctx, l, stdout, "--- FAIL: %s (%s)\n%s", t.Name(), durationStr, output)
shout(ctx, l, stdout, "--- FAIL: %s (%s)\n%s", runID, durationStr, output)

issueOutput := output
if t.timedOut() {
issueOutput = "test timed out (see artifacts for details)"
}
r.maybePostGithubIssue(ctx, l, t, stdout, issueOutput)
} else {
shout(ctx, l, stdout, "--- PASS: %s (%s)", t.Name(), durationStr)
shout(ctx, l, stdout, "--- PASS: %s (%s)", runID, durationStr)
// If `##teamcity[testFailed ...]` is not present before `##teamCity[testFinished ...]`,
// TeamCity regards the test as successful.
}

if teamCity {
shout(ctx, l, stdout, "##teamcity[testFinished name='%s' flowId='%s']", t.Name(), t.Name())
shout(ctx, l, stdout, "##teamcity[testFinished name='%s' flowId='%s']", t.Name(), runID)

// Zip the artifacts. This improves the TeamCity UX where we can navigate
// through zip files just fine, but we can't download subtrees of the
Expand Down Expand Up @@ -882,9 +889,13 @@ func (r *testRunner) runTest(
}

func (r *testRunner) shouldPostGithubIssue(t test.Test) bool {
// NB: check NodeCount > 0 to avoid posting issues from this pkg's unit tests.
opts := issues.DefaultOptionsFromEnv()
return opts.CanPost() && opts.IsReleaseBranch() && t.Spec().(*registry.TestSpec).Run != nil && t.Spec().(*registry.TestSpec).Cluster.NodeCount > 0
return !r.config.disableIssue &&
opts.CanPost() &&
opts.IsReleaseBranch() &&
t.Spec().(*registry.TestSpec).Run != nil &&
// NB: check NodeCount > 0 to avoid posting issues from this pkg's unit tests.
t.Spec().(*registry.TestSpec).Cluster.NodeCount > 0
}

func (r *testRunner) maybePostGithubIssue(
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,7 @@ go_library(
"restart.go",
"restore.go",
"roachmart.go",
"roachtest.go",
"ruby_pg.go",
"ruby_pg_blocklist.go",
"schema_change_database_version_upgrade.go",
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/tests/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,7 @@ func RegisterTests(r registry.Registry) {
registerRestoreNodeShutdown(r)
registerRestore(r)
registerRoachmart(r)
registerRoachtest(r)
registerRubyPG(r)
registerSchemaChangeBulkIngest(r)
registerSchemaChangeDatabaseVersionUpgrade(r)
Expand Down
Loading

0 comments on commit 4938d16

Please sign in to comment.