Skip to content

Commit

Permalink
cmd: replacing release-20.1 version of roachprod, roachtest, workload…
Browse files Browse the repository at this point in the history
…(ccl)

This change does the following:

- Deletes release-20.1 roachprod, roachtest, workload and workloadccl
  packages.

- Adds the above packages from master.

- Make all the above packages compatible with 20.1. I have highlighted
  some of the major changes in each package and will inline links to why
  they were changed on master. I took some decisions on what required a
  backport and what looked too invasive, but am open to suggestions
  here.

roachtest:
- Deleted roachtests with min version 20.2.0 namely:
	- backup/KMS
	- join-init/mixed
	- disk_full
	- multitenant acceptance
	- tpchvec/bench
	- tpchvec/perf_no_stats
- Backported `tpcdsvec` roachtest

workload:
- Backported `schemachange` workload
- deleted geospatial.go
- switched all `rowenc` pkg references back to `sqlbase`
- type representation in 20.2 differed from what was supported in 20.1.
  All workloads had to be switched back to using `colTypes`

workloadccl:
- More type representation related changes
- Set --deprecated-fk-indexes to be always true for tpcc workload

Release note: None
  • Loading branch information
adityamaru committed Oct 30, 2020
1 parent 4ea9b3b commit 6e90d0b
Show file tree
Hide file tree
Showing 226 changed files with 12,549 additions and 6,060 deletions.
19 changes: 19 additions & 0 deletions build/teamcity-nightly-roachtest-invoke.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
#!/usr/bin/env bash
set -euo pipefail

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}" \
--roachprod="${PWD}/bin/roachprod" \
--workload="${PWD}/bin/workload" \
--teamcity=true \
--slack-token="${SLACK_TOKEN}" \
--cluster-id="${TC_BUILD_ID}" \
"${TESTS}"
117 changes: 117 additions & 0 deletions build/teamcity-nightly-roachtest.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
#!/usr/bin/env bash
set -euxo pipefail

# Entry point for the nightly roachtests. These are run from CI and require
# appropriate secrets for the ${CLOUD} parameter (along with other things,
# 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

# The artifacts dir should match up with that supplied by TC.
artifacts=$PWD/artifacts
mkdir -p "${artifacts}"
chmod o+rwx "${artifacts}"

# Disable global -json flag.
export PATH=$PATH:$(GOFLAGS=; go env GOPATH)/bin

make bin/workload bin/roachtest bin/roachprod > "${artifacts}/build.txt" 2>&1 || cat "${artifacts}/build.txt"

# Set up Google credentials. Note that we need this for all clouds since we upload
# perf artifacts to Google Storage at the end.
if [[ "$GOOGLE_EPHEMERAL_CREDENTIALS" ]]; then
echo "$GOOGLE_EPHEMERAL_CREDENTIALS" > creds.json
gcloud auth activate-service-account --key-file=creds.json
export ROACHPROD_USER=teamcity
else
echo 'warning: GOOGLE_EPHEMERAL_CREDENTIALS not set' >&2
echo "Assuming that you've run \`gcloud auth login\` from inside the builder." >&2
fi

# Early bind the stats dir. Roachtest invocations can take ages, and we want the
# date at the time of the start of the run (which identifies the version of the
# code run best).
stats_dir="$(date +"%Y%m%d")-${TC_BUILD_ID}"

# Set up a function we'll invoke at the end.
function upload_stats {
if [[ "${TC_BUILD_BRANCH}" == "master" ]]; then
bucket="cockroach-nightly-${CLOUD}"
if [[ "${CLOUD}" == "gce" ]]; then
# GCE, having been there first, gets an exemption.
bucket="cockroach-nightly"
fi
# The stats.json files need some path translation:
# ${artifacts}/path/to/test/stats.json
# to
# gs://${bucket}/artifacts/${stats_dir}/path/to/test/stats.json
#
# `find` below will expand "{}" as ./path/to/test/stats.json. We need
# to bend over backwards to remove the `./` prefix or gsutil will have
# a `.` folder in ${stats_dir}, which we don't want.
(cd "${artifacts}" && \
while IFS= read -r f; do
if [[ -n "${f}" ]]; then
gsutil cp "${f}" "gs://${bucket}/artifacts/${stats_dir}/${f}"
fi
done <<< "$(find . -name stats.json | sed 's/^\.\///')")
fi
}

# Upload any stats.json we can find, no matter what happens.
trap upload_stats EXIT

# Set up the parameters for the roachtest invocation.

ARTIFACTS="${artifacts}"
PARALLELISM=16
CPUQUOTA=1024
ZONES=""
TESTS=""
case "${CLOUD}" in
gce)
# We specify --zones below so that nodes are created in us-central1-b by
# default. This reserves us-east1-b (the roachprod default zone) for use by
# manually created clusters.
ZONES="us-central1-b,us-west1-b,europe-west2-b"
;;
aws)
PARALLELISM=3
CPUQUOTA=384
if [ -z "${TESTS}" ]; then
TESTS="kv(0|95)|ycsb|tpcc/(headroom/n4cpu16)|tpccbench/(nodes=3/cpu=16)|scbench/randomload/(nodes=3/ops=2000/conc=1)|backup/(KMS/n3cpu4)"
fi
;;
*)
echo "unknown cloud ${CLOUD}"
exit 1
;;
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"
2 changes: 2 additions & 0 deletions pkg/ccl/workloadccl/allccl/all.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
_ "github.com/cockroachdb/cockroach/pkg/workload/debug"
_ "github.com/cockroachdb/cockroach/pkg/workload/examples"
_ "github.com/cockroachdb/cockroach/pkg/workload/indexes"
_ "github.com/cockroachdb/cockroach/pkg/workload/interleavebench"
_ "github.com/cockroachdb/cockroach/pkg/workload/interleavedpartitioned"
_ "github.com/cockroachdb/cockroach/pkg/workload/jsonload"
_ "github.com/cockroachdb/cockroach/pkg/workload/kv"
Expand All @@ -28,6 +29,7 @@ import (
_ "github.com/cockroachdb/cockroach/pkg/workload/querylog"
_ "github.com/cockroachdb/cockroach/pkg/workload/queue"
_ "github.com/cockroachdb/cockroach/pkg/workload/rand"
_ "github.com/cockroachdb/cockroach/pkg/workload/schemachange"
_ "github.com/cockroachdb/cockroach/pkg/workload/sqlsmith"
_ "github.com/cockroachdb/cockroach/pkg/workload/tpcc"
_ "github.com/cockroachdb/cockroach/pkg/workload/tpccchecks"
Expand Down
27 changes: 16 additions & 11 deletions pkg/ccl/workloadccl/allccl/all_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ package allccl
import (
"context"
"encoding/binary"
"fmt"
"hash"
"hash/fnv"
"math"
Expand All @@ -22,13 +23,15 @@ import (
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/col/coltypes"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/bufalloc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/workload"
"github.com/cockroachdb/cockroach/pkg/workload/workloadsql"
"github.com/pkg/errors"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -78,9 +81,10 @@ func TestAllRegisteredImportFixture(t *testing.T) {
}

t.Run(meta.Name, func(t *testing.T) {
if bigInitialData(meta) && testing.Short() {
t.Skipf(`%s loads a lot of data`, meta.Name)
if bigInitialData(meta) {
skip.UnderShort(t, fmt.Sprintf(`%s loads a lot of data`, meta.Name))
}
defer log.Scope(t).Close(t)

ctx := context.Background()
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{
Expand Down Expand Up @@ -127,7 +131,8 @@ func TestAllRegisteredSetup(t *testing.T) {
case `roachmart`:
// TODO(dan): It'd be nice to test this with the default flags. For now,
// this is better than nothing.
if err := gen.(workload.Flagser).Flags().Parse([]string{
flags := gen.(workload.Flagser).Flags()
if err := flags.Parse([]string{
`--users=10`, `--orders=100`, `--partition=false`,
}); err != nil {
t.Fatal(err)
Expand All @@ -138,6 +143,7 @@ func TestAllRegisteredSetup(t *testing.T) {
}

t.Run(meta.Name, func(t *testing.T) {
defer log.Scope(t).Close(t)
ctx := context.Background()
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{
UseDatabase: "d",
Expand Down Expand Up @@ -165,6 +171,7 @@ func TestAllRegisteredSetup(t *testing.T) {

func TestConsistentSchema(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
// Test that the table schemas are consistent when the workload is created
// multiple times with the same seed.

Expand All @@ -188,7 +195,7 @@ func hashTableInitialData(
h hash.Hash, data workload.BatchedTuples, a *bufalloc.ByteAllocator,
) error {
var scratch [8]byte
b := coldata.NewMemBatchWithSize(nil, 0)
b := coldata.NewMemBatchWithSize(nil /* types */, 0 /* size */)
for batchIdx := 0; batchIdx < data.NumBatches; batchIdx++ {
*a = (*a)[:0]
data.FillBatch(batchIdx, b, a)
Expand Down Expand Up @@ -237,9 +244,7 @@ func TestDeterministicInitialData(t *testing.T) {

// There are other tests that run initial data generation under race, so we
// don't get anything from running this one under race as well.
if util.RaceEnabled {
t.Skip(`uninteresting under race`)
}
skip.UnderRace(t, "uninteresting under race")

// Hardcode goldens for the fingerprint of the initial data of generators with
// default flags. This lets us opt in generators known to be deterministic and
Expand All @@ -263,7 +268,7 @@ func TestDeterministicInitialData(t *testing.T) {
`startrek`: 0xa0249fbdf612734c,
`tpcc`: 0xab32e4f5e899eb2f,
`tpch`: 0xdd952207e22aa577,
`ycsb`: 0x85dd34d8c07fd808,
`ycsb`: 0x1244ea1c29ef67f6,
}

var a bufalloc.ByteAllocator
Expand All @@ -277,8 +282,8 @@ func TestDeterministicInitialData(t *testing.T) {
continue
}
t.Run(meta.Name, func(t *testing.T) {
if bigInitialData(meta) && testing.Short() {
t.Skipf(`%s involves a lot of data`, meta.Name)
if bigInitialData(meta) {
skip.UnderShort(t, fmt.Sprintf(`%s involves a lot of data`, meta.Name))
}

h := fnv.New64()
Expand Down
5 changes: 2 additions & 3 deletions pkg/ccl/workloadccl/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
_ "github.com/cockroachdb/cockroach/pkg/ccl"
"github.com/cockroachdb/cockroach/pkg/ccl/workloadccl"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/workload"
"github.com/cockroachdb/cockroach/pkg/workload/tpcc"
Expand Down Expand Up @@ -48,9 +49,7 @@ func benchmarkImportFixture(b *testing.B, gen workload.Generator) {
}

func BenchmarkImportFixture(b *testing.B) {
if testing.Short() {
b.Skip("skipping long benchmark")
}
skip.UnderShort(b, "skipping long benchmark")

b.Run(`tpcc/warehouses=1`, func(b *testing.B) {
benchmarkImportFixture(b, tpcc.FromWarehouses(1))
Expand Down
7 changes: 6 additions & 1 deletion pkg/ccl/workloadccl/cliccl/fixtures.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/workload"
workloadcli "github.com/cockroachdb/cockroach/pkg/workload/cli"
"github.com/cockroachdb/cockroach/pkg/workload/workloadsql"
"github.com/pkg/errors"
"github.com/cockroachdb/errors"
"github.com/spf13/cobra"
"github.com/spf13/pflag"
"google.golang.org/api/option"
Expand All @@ -47,6 +47,7 @@ func config() workloadccl.FixtureConfig {
config.BillingProject = *gcsBillingProjectOverride
}
config.CSVServerURL = *fixturesMakeImportCSVServerURL
config.TableStats = *fixturesMakeTableStats
return config
}

Expand Down Expand Up @@ -91,6 +92,10 @@ var fixturesMakeFilesPerNode = fixturesMakeCmd.PersistentFlags().Int(
`files-per-node`, 1,
`number of file URLs to generate per node when using csv-server`)

var fixturesMakeTableStats = fixturesMakeCmd.PersistentFlags().Bool(
`table-stats`, true,
`generate full table statistics for all tables`)

var fixturesImportFilesPerNode = fixturesImportCmd.PersistentFlags().Int(
`files-per-node`, 1,
`number of file URLs to generate per node`)
Expand Down
Loading

0 comments on commit 6e90d0b

Please sign in to comment.