From b6ff7d90466a01ec6c8a205aea557b957cd1f551 Mon Sep 17 00:00:00 2001 From: Herko Lategan Date: Tue, 28 Jan 2025 13:57:22 +0000 Subject: [PATCH] wip --- .../actions/microbenchmark-build/action.yml | 65 ++++++ .github/actions/microbenchmark-run/action.yml | 41 ++++ .github/workflows/code-cover-gen.yml | 1 + .../workflows/github-actions-essential-ci.yml | 14 ++ .github/workflows/microbenchmarks-ci.yaml | 103 +++++++++ build/github/microbenchmarks-build.sh | 35 +++ build/github/microbenchmarks-compare.sh | 45 ++++ build/github/microbenchmarks-run.sh | 44 ++++ pkg/BUILD.bazel | 4 + pkg/cmd/dev/build.go | 1 + pkg/cmd/microbench-ci/BUILD.bazel | 35 +++ pkg/cmd/microbench-ci/benchmark.go | 131 +++++++++++ pkg/cmd/microbench-ci/compare.go | 108 +++++++++ .../config/pull-request-suite.yml | 26 +++ pkg/cmd/microbench-ci/github.go | 72 ++++++ pkg/cmd/microbench-ci/main.go | 206 ++++++++++++++++++ pkg/cmd/microbench-ci/report.go | 197 +++++++++++++++++ pkg/cmd/microbench-ci/run.go | 132 +++++++++++ .../microbench-ci/template/github_summary.md | 44 ++++ pkg/cmd/roachprod-microbench/BUILD.bazel | 2 + pkg/cmd/roachprod-microbench/clean.go | 9 +- pkg/cmd/roachprod-microbench/executor.go | 67 +----- pkg/cmd/roachprod-microbench/executor_test.go | 3 +- .../roachprod-microbench/parser/BUILD.bazel | 8 + pkg/cmd/roachprod-microbench/parser/parser.go | 57 +++++ pkg/sql/tests/BUILD.bazel | 1 + pkg/sql/tests/main_test.go | 2 + pkg/sql/tests/sysbench_test.go | 19 +- pkg/testutils/benchmark/BUILD.bazel | 9 + pkg/testutils/benchmark/scatterplot.go | 90 ++++++++ 30 files changed, 1506 insertions(+), 65 deletions(-) create mode 100644 .github/actions/microbenchmark-build/action.yml create mode 100644 .github/actions/microbenchmark-run/action.yml create mode 100644 .github/workflows/microbenchmarks-ci.yaml create mode 100755 build/github/microbenchmarks-build.sh create mode 100755 build/github/microbenchmarks-compare.sh create mode 100755 build/github/microbenchmarks-run.sh create mode 100644 pkg/cmd/microbench-ci/BUILD.bazel create mode 100644 pkg/cmd/microbench-ci/benchmark.go create mode 100644 pkg/cmd/microbench-ci/compare.go create mode 100644 pkg/cmd/microbench-ci/config/pull-request-suite.yml create mode 100644 pkg/cmd/microbench-ci/github.go create mode 100644 pkg/cmd/microbench-ci/main.go create mode 100644 pkg/cmd/microbench-ci/report.go create mode 100644 pkg/cmd/microbench-ci/run.go create mode 100644 pkg/cmd/microbench-ci/template/github_summary.md create mode 100644 pkg/cmd/roachprod-microbench/parser/BUILD.bazel create mode 100644 pkg/cmd/roachprod-microbench/parser/parser.go create mode 100644 pkg/testutils/benchmark/BUILD.bazel create mode 100644 pkg/testutils/benchmark/scatterplot.go diff --git a/.github/actions/microbenchmark-build/action.yml b/.github/actions/microbenchmark-build/action.yml new file mode 100644 index 000000000000..285a37b4598c --- /dev/null +++ b/.github/actions/microbenchmark-build/action.yml @@ -0,0 +1,65 @@ +name: "Microbenchmark Build" +description: "Builds microbenchmarks for a given package and revision." +inputs: + ref: + description: "head or base" + required: true + pkg: + description: "test package to build" + required: true +outputs: + merge_base: + description: "merge base" + value: ${{ steps.determine-merge-base.outputs.merge_base }} + +runs: + using: "composite" + steps: + - name: Copy build script + run: cp build/github/microbenchmarks-build.sh ${RUNNER_TEMP}/ + shell: bash + + - run: ./build/github/get-engflow-keys.sh + shell: bash + + - name: Checkout code with limited depth + if: inputs.ref == 'base' + uses: actions/checkout@v4 + with: + fetch-depth: 15 + + - name: Determine merge base + id: determine-merge-base + if: inputs.ref == 'base' + run: | + set -e + MERGE_BASE=$(git merge-base ${{ github.event.pull_request.base.sha }} ${{ github.event.pull_request.head.sha }}) + if [ -z "$MERGE_BASE" ]; then + echo "Error: Merge base could not be found within the last 15 commits." >&2 + exit 1 + fi + echo "merge_base=$MERGE_BASE" >> "$GITHUB_OUTPUT" + shell: bash + + - name: Checkout build commit + if: inputs.ref == 'base' + uses: actions/checkout@v4 + with: + ref: ${{ steps.determine-merge-base.outputs.merge_base }} + + - name: Build tests + run: ${RUNNER_TEMP}/microbenchmarks-build.sh + shell: bash + env: + TEST_PKG: ${{ inputs.pkg }} + + - name: Checkout Head # required for post job cleanup (if still on the base ref) + uses: actions/checkout@v4 + with: + ref: ${{ github.event.pull_request.head.sha || github.ref }} + if: always() && inputs.ref == 'base' + + - name: Clean up + run: ./build/github/cleanup-engflow-keys.sh + shell: bash + if: always() diff --git a/.github/actions/microbenchmark-run/action.yml b/.github/actions/microbenchmark-run/action.yml new file mode 100644 index 000000000000..32eba7d2360c --- /dev/null +++ b/.github/actions/microbenchmark-run/action.yml @@ -0,0 +1,41 @@ +name: "Microbenchmark Run" +description: "Runs microbenchmarks." +inputs: + base: + description: "Base SHA" + required: true + pkg: + description: "Test packages required for the run" + required: true + group: + description: "Runner group" + required: true + +runs: + using: "composite" + steps: + - name: Checkout Head + uses: actions/checkout@v4 + with: + ref: ${{ github.event.pull_request.head.sha || github.ref }} + + - run: ./build/github/get-engflow-keys.sh + shell: bash + + - name: Unique Build ID + run: echo "BUILD_ID=${{ github.run_id }}-${{ github.run_attempt }}" >> $GITHUB_ENV + shell: bash + + - name: Run benchmarks + run: build/github/microbenchmarks-run.sh + shell: bash + env: + BASE_SHA: ${{ inputs.base }} + HEAD_SHA: ${{ github.event.pull_request.head.sha || github.ref }} + TEST_PACKAGES: ${{ inputs.pkg }} + GROUP: ${{ inputs.group }} + + - name: Clean up + run: ./build/github/cleanup-engflow-keys.sh + shell: bash + if: always() diff --git a/.github/workflows/code-cover-gen.yml b/.github/workflows/code-cover-gen.yml index f078ca56f985..68767b5f815f 100644 --- a/.github/workflows/code-cover-gen.yml +++ b/.github/workflows/code-cover-gen.yml @@ -6,6 +6,7 @@ on: jobs: code-cover-gen: + if: false # This will always skip this job runs-on: ubuntu-latest env: PR: ${{ github.event.pull_request.number }} diff --git a/.github/workflows/github-actions-essential-ci.yml b/.github/workflows/github-actions-essential-ci.yml index ab18508a8c9b..705754a10e66 100644 --- a/.github/workflows/github-actions-essential-ci.yml +++ b/.github/workflows/github-actions-essential-ci.yml @@ -53,6 +53,7 @@ concurrency: cancel-in-progress: true jobs: acceptance: + if: false # This will always skip this job runs-on: [self-hosted, basic_big_runner_group] timeout-minutes: 30 steps: @@ -74,6 +75,7 @@ jobs: run: ./build/github/cleanup-engflow-keys.sh if: always() check_generated_code: + if: false # This will always skip this job runs-on: [self-hosted, basic_runner_group] timeout-minutes: 30 steps: @@ -89,6 +91,7 @@ jobs: run: ./build/github/cleanup-engflow-keys.sh if: always() docker_image_amd64: + if: false # This will always skip this job runs-on: [self-hosted, basic_runner_group] timeout-minutes: 30 steps: @@ -110,6 +113,7 @@ jobs: run: ./build/github/cleanup-engflow-keys.sh if: always() examples_orms: + if: false # This will always skip this job runs-on: [self-hosted, basic_big_runner_group] timeout-minutes: 40 steps: @@ -131,6 +135,7 @@ jobs: run: ./cockroach/build/github/cleanup-engflow-keys.sh if: always() lint: + if: false # This will always skip this job runs-on: [self-hosted, basic_big_runner_group] timeout-minutes: 60 steps: @@ -155,6 +160,7 @@ jobs: run: ./build/github/cleanup-engflow-keys.sh if: always() local_roachtest: + if: false # This will always skip this job runs-on: [self-hosted, basic_big_runner_group] timeout-minutes: 60 steps: @@ -176,6 +182,7 @@ jobs: run: ./build/github/cleanup-engflow-keys.sh if: always() local_roachtest_fips: + if: false # This will always skip this job runs-on: [self-hosted, basic_runner_group_fips] timeout-minutes: 60 steps: @@ -197,6 +204,7 @@ jobs: run: ./build/github/cleanup-engflow-keys.sh if: always() linux_amd64_build: + if: false # This will always skip this job runs-on: [self-hosted, basic_runner_group] timeout-minutes: 30 steps: @@ -218,6 +226,7 @@ jobs: run: ./build/github/cleanup-engflow-keys.sh if: always() linux_amd64_fips_build: + if: false # This will always skip this job runs-on: [self-hosted, basic_runner_group] timeout-minutes: 30 steps: @@ -239,6 +248,7 @@ jobs: run: ./build/github/cleanup-engflow-keys.sh if: always() linux_arm64_build: + if: false # This will always skip this job runs-on: [self-hosted, basic_runner_group] timeout-minutes: 30 steps: @@ -260,6 +270,7 @@ jobs: run: ./build/github/cleanup-engflow-keys.sh if: always() macos_amd64_build: + if: false # This will always skip this job runs-on: [self-hosted, basic_runner_group] timeout-minutes: 30 steps: @@ -281,6 +292,7 @@ jobs: run: ./build/github/cleanup-engflow-keys.sh if: always() macos_arm64_build: + if: false # This will always skip this job runs-on: [self-hosted, basic_runner_group] timeout-minutes: 30 steps: @@ -302,6 +314,7 @@ jobs: run: ./build/github/cleanup-engflow-keys.sh if: always() unit_tests: + if: false # This will always skip this job runs-on: [self-hosted, basic_runner_group] timeout-minutes: 60 steps: @@ -325,6 +338,7 @@ jobs: run: ./build/github/cleanup-engflow-keys.sh if: always() windows_build: + if: false # This will always skip this job runs-on: [self-hosted, basic_runner_group] timeout-minutes: 30 steps: diff --git a/.github/workflows/microbenchmarks-ci.yaml b/.github/workflows/microbenchmarks-ci.yaml new file mode 100644 index 000000000000..b01269dd0a04 --- /dev/null +++ b/.github/workflows/microbenchmarks-ci.yaml @@ -0,0 +1,103 @@ +name: Microbenchmarks CI +on: + pull_request: + types: [ opened, reopened, synchronize ] + branches: [ master ] + +env: + HEAD: ${{ github.event.pull_request.head.sha || github.ref }} + BUCKET: "cockroach-microbench-ci" + PACKAGE: "pkg/sql/tests" + +jobs: + base: + name: build base + runs-on: [self-hosted, basic_runner_group] + timeout-minutes: 30 + outputs: + merge_base: ${{ steps.build.outputs.merge_base }} + steps: + - name: Checkout Head + uses: actions/checkout@v4 + with: + ref: ${{ env.HEAD }} + - name: Run Microbenchmark Build + id: build + uses: ./.github/actions/microbenchmark-build + with: + ref: base + pkg: ${{ env.PACKAGE }} + + head: + name: build head + runs-on: [self-hosted, basic_runner_group] + timeout-minutes: 30 + outputs: + merge_base: ${{ steps.build.outputs.merge_base }} + steps: + - name: Checkout Head + uses: actions/checkout@v4 + with: + ref: ${{ env.HEAD }} + - name: Run Microbenchmark Build + id: build + uses: ./.github/actions/microbenchmark-build + with: + ref: head + pkg: ${{ env.PACKAGE }} + + run-group-1: + runs-on: [self-hosted, basic_runner_group] + timeout-minutes: 30 + needs: [base, head] + steps: + - name: Checkout Head + uses: actions/checkout@v4 + with: + ref: ${{ env.HEAD }} + - name: Run Microbenchmarks + uses: ./.github/actions/microbenchmark-run + with: + base: ${{ needs.base.outputs.merge_base }} + pkg: ${{ env.PACKAGE }} + group: 1 + + run-group-2: + runs-on: [self-hosted, basic_runner_group] + timeout-minutes: 30 + needs: [base, head] + steps: + - name: Checkout Head + uses: actions/checkout@v4 + with: + ref: ${{ env.HEAD }} + - name: Run Microbenchmarks + uses: ./.github/actions/microbenchmark-run + with: + base: ${{ needs.base.outputs.merge_base }} + pkg: ${{ env.PACKAGE }} + group: 2 + compare: + runs-on: [self-hosted, basic_runner_group] + timeout-minutes: 30 + needs: [base, run-group-1, run-group-2] + steps: + - name: Checkout Head + uses: actions/checkout@v4 + with: + ref: ${{ github.event.pull_request.head.sha || github.ref }} + + - run: ./build/github/get-engflow-keys.sh + + - name: Unique Build ID + run: echo "BUILD_ID=${{ github.run_id }}-${{ github.run_attempt }}" >> $GITHUB_ENV + + - name: Compare benchmarks + run: build/github/microbenchmarks-compare.sh + env: + BASE_SHA: ${{ needs.base.outputs.merge_base }} + HEAD_SHA: ${{ env.HEAD }} + + - name: Clean up + run: ./build/github/cleanup-engflow-keys.sh + if: always() diff --git a/build/github/microbenchmarks-build.sh b/build/github/microbenchmarks-build.sh new file mode 100755 index 000000000000..fd6564e52257 --- /dev/null +++ b/build/github/microbenchmarks-build.sh @@ -0,0 +1,35 @@ +#!/bin/bash + +# Copyright 2025 The Cockroach Authors. +# +# Use of this software is governed by the CockroachDB Software License +# included in the /LICENSE file. + +set -euxo pipefail + +pkg_last=$(basename "${TEST_PKG}") +pkg_bin=$(echo "${TEST_PKG}" | tr '/' '_') + +build_sha=$(git rev-parse HEAD) +storage_bucket="$BUCKET" +output_url="gs://${storage_bucket}/builds/${build_sha}/bin/${pkg_bin}" + +# Disable parallel uploads, as it requires specific permissions +gcloud config set storage/parallel_composite_upload_enabled False + +if gcloud storage ls "${output_url}" &>/dev/null; then + echo "Build for $build_sha already exists. Skipping..." + exit 0 +fi + +bazel build "//${TEST_PKG}:tests_test" \ + --jobs 100 \ + --crdb_test_off \ + --bes_keywords integration-test-artifact-build \ + --config=crosslinux \ + --remote_download_minimal \ + $(./build/github/engflow-args.sh) + +# Copy the binary to a GCS cache +bazel_bin=$(bazel info bazel-bin --config=crosslinux) +gcloud storage cp "${bazel_bin}/pkg/sql/tests/${pkg_last}_test_/${pkg_last}_test" "${output_url}" diff --git a/build/github/microbenchmarks-compare.sh b/build/github/microbenchmarks-compare.sh new file mode 100755 index 000000000000..0ba7e77eee3c --- /dev/null +++ b/build/github/microbenchmarks-compare.sh @@ -0,0 +1,45 @@ +#!/bin/bash + +# Copyright 2025 The Cockroach Authors. +# +# Use of this software is governed by the CockroachDB Software License +# included in the /LICENSE file. + +set -euxo pipefail + +working_dir=$(mktemp -d) +output_dir=$(mktemp -d) +storage_bucket="$BUCKET" +shas=("$BASE_SHA" "$HEAD_SHA") + +# Disable parallel uploads, as it requires specific permissions +gcloud config set storage/parallel_composite_upload_enabled False + +# Retrieve outputs from the runs +for sha in "${shas[@]}"; do + mkdir -p "${working_dir}/${sha}/artifacts" + gsutil cp -r "gs://${storage_bucket}/artifacts/${sha}/${BUILD_ID}/*" "${working_dir}/${sha}/artifacts/" +done + +# Build microbenchmark CI utility +bazel build --config=crosslinux $(./build/github/engflow-args.sh) \ + --jobs 100 \ + --bes_keywords integration-test-artifact-build \ + //pkg/cmd/microbench-ci \ + +bazel_bin=$(bazel info bazel-bin --config=crosslinux) +microbench_ci_bin=$bazel_bin/pkg/cmd/microbench-ci/microbench-ci_ + +# Compare the microbenchmarks +$microbench_ci_bin/microbench-ci compare \ + --working-dir="$working_dir" \ + --summary="$output_dir/summary.json" \ + --github-summary="$output_dir/summary.md" \ + --build-id="$BUILD_ID" \ + --old="$BASE_SHA" \ + --new="$HEAD_SHA" + +cat "$output_dir/summary.md" > "$GITHUB_STEP_SUMMARY" + +# Copy comparison results to GCS +gsutil cp -r "$output_dir/*" "gs://${storage_bucket}/results/${HEAD_SHA}/${BUILD_ID}" diff --git a/build/github/microbenchmarks-run.sh b/build/github/microbenchmarks-run.sh new file mode 100755 index 000000000000..4339f9bb5389 --- /dev/null +++ b/build/github/microbenchmarks-run.sh @@ -0,0 +1,44 @@ +#!/bin/bash + +# Copyright 2025 The Cockroach Authors. +# +# Use of this software is governed by the CockroachDB Software License +# included in the /LICENSE file. + +set -euxo pipefail + +working_dir=$(mktemp -d) +storage_bucket="$BUCKET" +shas=("$BASE_SHA" "$HEAD_SHA") + +# Disable parallel uploads, as it requires specific permissions +gcloud config set storage/parallel_composite_upload_enabled False + +# Retrieve required binaries from the base and head builds +for pkg in "${TEST_PACKAGES[@]}"; do + for sha in "${shas[@]}"; do + pkg_bin=$(echo "${pkg}" | tr '/' '_') + url="gs://${storage_bucket}/builds/${sha}/bin/${pkg_bin}" + dest="$working_dir/$sha/bin/" + mkdir -p "$dest" + gcloud storage cp "${url}" "$dest/${pkg_bin}" + chmod +x "$dest/${pkg_bin}" + done +done + +# Build microbenchmark CI utility +bazel build --config=crosslinux $(./build/github/engflow-args.sh) \ + --jobs 100 \ + --bes_keywords integration-test-artifact-build \ + //pkg/cmd/microbench-ci \ + +bazel_bin=$(bazel info bazel-bin --config=crosslinux) +microbench_ci_bin=$bazel_bin/pkg/cmd/microbench-ci/microbench-ci_ + +# Run the microbenchmarks +$microbench_ci_bin/microbench-ci run --group="$GROUP" --working-dir="$working_dir" --old="$BASE_SHA" --new="$HEAD_SHA" + +# Copy benchmark results to GCS +for sha in "${shas[@]}"; do + gsutil cp -r "${working_dir}/${sha}/artifacts/*" "gs://${storage_bucket}/artifacts/${sha}/${BUILD_ID}/" +done diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 3169ffcc1f59..59c43ac45e72 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -1151,6 +1151,8 @@ GO_TARGETS = [ "//pkg/cmd/label-merged-pr:label-merged-pr", "//pkg/cmd/label-merged-pr:label-merged-pr_lib", "//pkg/cmd/label-merged-pr:label-merged-pr_test", + "//pkg/cmd/microbench-ci:microbench-ci", + "//pkg/cmd/microbench-ci:microbench-ci_lib", "//pkg/cmd/mirror/go:go_lib", "//pkg/cmd/mirror/go:go_test", "//pkg/cmd/mirror/go:mirror", @@ -1182,6 +1184,7 @@ GO_TARGETS = [ "//pkg/cmd/roachprod-microbench/cluster:cluster_test", "//pkg/cmd/roachprod-microbench/google:google", "//pkg/cmd/roachprod-microbench/model:model", + "//pkg/cmd/roachprod-microbench/parser:parser", "//pkg/cmd/roachprod-microbench/util:util", "//pkg/cmd/roachprod-microbench/util:util_test", "//pkg/cmd/roachprod-microbench:roachprod-microbench", @@ -2321,6 +2324,7 @@ GO_TARGETS = [ "//pkg/storage:storage", "//pkg/storage:storage_test", "//pkg/testutils/bazelcodecover:bazelcodecover", + "//pkg/testutils/benchmark:benchmark", "//pkg/testutils/colcontainerutils:colcontainerutils", "//pkg/testutils/datapathutils:datapathutils", "//pkg/testutils/diagutils:diagutils", diff --git a/pkg/cmd/dev/build.go b/pkg/cmd/dev/build.go index 1ee15fbd44a4..edd97045a4a4 100644 --- a/pkg/cmd/dev/build.go +++ b/pkg/cmd/dev/build.go @@ -92,6 +92,7 @@ var buildTargetMapping = map[string]string{ "geos": geosTarget, "langgen": "//pkg/sql/opt/optgen/cmd/langgen:langgen", "libgeos": geosTarget, + "microbench-ci": "//pkg/cmd/microbench-ci:microbench-ci", "optgen": "//pkg/sql/opt/optgen/cmd/optgen:optgen", "optfmt": "//pkg/sql/opt/optgen/cmd/optfmt:optfmt", "reduce": "//pkg/cmd/reduce:reduce", diff --git a/pkg/cmd/microbench-ci/BUILD.bazel b/pkg/cmd/microbench-ci/BUILD.bazel new file mode 100644 index 000000000000..f56aa8ee422d --- /dev/null +++ b/pkg/cmd/microbench-ci/BUILD.bazel @@ -0,0 +1,35 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library") + +go_library( + name = "microbench-ci_lib", + srcs = [ + "benchmark.go", + "compare.go", + "github.go", + "main.go", + "report.go", + "run.go", + ], + embedsrcs = ["template/github_summary.md"], + importpath = "github.com/cockroachdb/cockroach/pkg/cmd/microbench-ci", + visibility = ["//visibility:private"], + deps = [ + "//pkg/cmd/roachprod-microbench/model", + "//pkg/cmd/roachprod-microbench/parser", + "@com_github_cockroachdb_errors//:errors", + "@com_github_google_go_github_v61//github", + "@com_github_spf13_cobra//:cobra", + "@in_gopkg_yaml_v2//:yaml_v2", + "@org_golang_x_exp//maps", + "@org_golang_x_oauth2//:oauth2", + "@org_golang_x_perf//benchfmt", + "@org_golang_x_perf//benchmath", + "@org_golang_x_perf//benchunit", + ], +) + +go_binary( + name = "microbench-ci", + embed = [":microbench-ci_lib"], + visibility = ["//visibility:public"], +) diff --git a/pkg/cmd/microbench-ci/benchmark.go b/pkg/cmd/microbench-ci/benchmark.go new file mode 100644 index 000000000000..621d75f3ecbf --- /dev/null +++ b/pkg/cmd/microbench-ci/benchmark.go @@ -0,0 +1,131 @@ +package main + +import ( + "fmt" + "hash/fnv" + "os" + "regexp" + "strconv" + "strings" + + "github.com/cockroachdb/errors" + yaml "gopkg.in/yaml.v2" +) + +type ( + Benchmark struct { + DisplayName string `yaml:"display_name"` + Package string `yaml:"package"` + Labels []string `yaml:"labels"` + Name string + RunnerGroup int `yaml:"runner_group"` + MeasureCount int `yaml:"measure_count"` + Measure Sample `yaml:"measure"` + CPUProfile Sample `yaml:"cpu_profile"` + MemProfile Sample `yaml:"mem_profile"` + MutexProfile Sample `yaml:"mutex_profile"` + Thresholds map[string]float64 `yaml:"thresholds"` + } + Sample struct { + Iterations int `yaml:"iterations"` + } + Benchmarks []Benchmark + BenchmarkIterationType uint8 +) + +const ( + Measure BenchmarkIterationType = iota + ProfileCPU + ProfileMemory + ProfileMutex +) + +var sanitizeRe = regexp.MustCompile(`\W+`) + +func (b *Benchmark) sanitizedPackageName() string { + return strings.ReplaceAll(b.Package, "/", "_") +} + +func (b *Benchmark) sanitizedName() string { + return sanitizeRe.ReplaceAllString(strings.TrimPrefix(b.Name, "Benchmark"), "_") +} + +func (b *Benchmark) binaryName() string { + return b.sanitizedPackageName() +} + +func (b *Benchmark) rawLog() string { + return fmt.Sprintf("raw_%s.log", b.sanitizedName()) +} + +func (b *Benchmark) cleanLog() string { + return fmt.Sprintf("cleaned_%s.log", b.sanitizedName()) +} + +func (b *Benchmark) cpuProfile() string { + return fmt.Sprintf("cpu_%s.out", b.sanitizedName()) +} + +func (b *Benchmark) memProfile() string { + return fmt.Sprintf("mem_%s.out", b.sanitizedName()) +} + +func (b *Benchmark) mutexProfile() string { + return fmt.Sprintf("mutex_%s.out", b.sanitizedName()) +} + +func (b *Benchmark) packageHash() string { + h := fnv.New32a() + _, _ = h.Write([]byte("./" + b.Package)) + u := h.Sum32() + return strconv.Itoa(int(u)) +} + +func (b *Benchmark) args(outputDir string, iterType BenchmarkIterationType) []string { + args := []string{ + "-test.run", "^$", + "-test.bench", b.Name, + "-test.benchmem", + "-test.count", "1", + "-test.outputdir", outputDir, + "-test.benchtime", fmt.Sprintf("%dx", b.Measure.Iterations), + } + + var sample Sample + switch iterType { + case Measure: + sample = b.Measure + case ProfileCPU: + sample = b.CPUProfile + args = append(args, "-test.cpuprofile", b.cpuProfile()) + case ProfileMemory: + sample = b.MemProfile + args = append(args, "-test.memprofile", b.memProfile()) + case ProfileMutex: + sample = b.MemProfile + args = append(args, "-test.mutexprofile", b.mutexProfile()) + default: + panic("unknown iteration type") + } + // If the specified sample has no iterations, use the default iterations. + if sample.Iterations == 0 { + sample.Iterations = b.Measure.Iterations + } + args = append(args, "-test.benchtime", fmt.Sprintf("%dx", sample.Iterations)) + return args +} + +func loadBenchmarkConfig(path string) ([]Benchmark, error) { + var c = struct { + Benchmarks []Benchmark `yaml:"benchmarks"` + }{} + fileContent, err := os.ReadFile(path) + if err != nil { + return nil, errors.Wrapf(err, "failed to read config file %s", path) + } + err = yaml.UnmarshalStrict(fileContent, &c) + if err != nil { + return nil, errors.Wrapf(err, "failed to unmarshal config file %s", path) + } + return c.Benchmarks, nil +} diff --git a/pkg/cmd/microbench-ci/compare.go b/pkg/cmd/microbench-ci/compare.go new file mode 100644 index 000000000000..a39c75930e6d --- /dev/null +++ b/pkg/cmd/microbench-ci/compare.go @@ -0,0 +1,108 @@ +package main + +import ( + "bytes" + "fmt" + "math" + "os" + "path" + + "github.com/cockroachdb/cockroach/pkg/cmd/roachprod-microbench/model" + "github.com/cockroachdb/errors" + "golang.org/x/exp/maps" + "golang.org/x/perf/benchfmt" +) + +type ( + CompareResult struct { + Benchmark *Benchmark + MetricMap model.MetricMap + EntryName string + } + CompareResults []*CompareResult + Status int +) + +const ( + NoChange Status = iota + Better + Worse + Regression +) + +func (c *CompareResult) status(metricName string) Status { + entry := c.MetricMap[metricName] + if entry == nil { + return NoChange + } + cc := entry.ComputeComparison(c.EntryName, string(Old), string(New)) + b := c.Benchmark + status := NoChange + threshold := b.Thresholds[metricName] * 100.0 + if cc.Delta*float64(entry.Better) > 0 { + status = Better + } else if cc.Delta*float64(entry.Better) < 0 { + status = Worse + if math.Abs(cc.Delta) >= threshold { + status = Regression + } + } + return status +} + +func (c *CompareResult) regressed() bool { + for metric := range c.MetricMap { + status := c.status(metric) + if status == Regression { + return true + } + } + return false +} + +func (b *Benchmark) compare() (*CompareResult, error) { + builder := model.NewBuilder() + compareResult := CompareResult{Benchmark: b} + for _, revision := range []Revision{Old, New} { + data, err := os.ReadFile(path.Join(suite.artifactsDir(revision), b.cleanLog())) + if err != nil { + return nil, err + } + reader := benchfmt.NewReader(bytes.NewReader(data), b.cleanLog()) + err = builder.AddMetrics(string(revision), "", reader) + if err != nil { + return nil, err + } + } + + metricMap := builder.ComputeMetricMap() + if len(metricMap) == 0 { + return nil, errors.Newf("no metrics found for benchmark %s", b.Name) + } + entry := metricMap[maps.Keys(metricMap)[0]] + if len(entry.BenchmarkEntries) == 0 { + return nil, errors.Newf("no benchmark entries found for benchmark %s", b.Name) + } + entries := len(maps.Keys(entry.BenchmarkEntries)) + if entries != 1 { + hint := fmt.Sprintf("please ensure the benchmark name %q atches a single benchmark", b.Name) + return nil, errors.Newf("unexpected multiple benchmark entries (%s): %v", + hint, maps.Keys(entry.BenchmarkEntries)) + } + + compareResult.EntryName = maps.Keys(entry.BenchmarkEntries)[0] + compareResult.MetricMap = metricMap + return &compareResult, nil +} + +func (b Benchmarks) compareBenchmarks() (CompareResults, error) { + compareResults := make(CompareResults, 0, len(b)) + for _, benchmark := range b { + compareResult, err := benchmark.compare() + if err != nil { + return nil, err + } + compareResults = append(compareResults, compareResult) + } + return compareResults, nil +} diff --git a/pkg/cmd/microbench-ci/config/pull-request-suite.yml b/pkg/cmd/microbench-ci/config/pull-request-suite.yml new file mode 100644 index 000000000000..7c9c6f5167af --- /dev/null +++ b/pkg/cmd/microbench-ci/config/pull-request-suite.yml @@ -0,0 +1,26 @@ +benchmarks: + - display_name: Sysbench + labels: ["SQL", "3node", "oltp_read_write"] + name: "BenchmarkSysbench/SQL/3node/oltp_read_write" + package: "pkg/sql/tests" + runner_group: 1 + measure_count: 10 + measure: + iterations: 2000 + thresholds: + "sec/op": .05 + "B/op": .03 + "allocs/op": .03 + + - display_name: Sysbench + labels: ["KV", "1node", "local", "oltp_write_only"] + name: "BenchmarkSysbench/KV/1node_local/oltp_write_only" + package: "pkg/sql/tests" + runner_group: 2 + measure_count: 10 + measure: + iterations: 4000 + thresholds: + "sec/op": .03 + "B/op": .02 + "allocs/op": .02 diff --git a/pkg/cmd/microbench-ci/github.go b/pkg/cmd/microbench-ci/github.go new file mode 100644 index 000000000000..04fc4b1d2f9c --- /dev/null +++ b/pkg/cmd/microbench-ci/github.go @@ -0,0 +1,72 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package main + +import ( + "context" + "fmt" + "os" + "strings" + + "github.com/cockroachdb/errors" + "github.com/google/go-github/v61/github" + "golang.org/x/oauth2" +) + +// CommentTag is used to identify an existing comment. +const CommentTag = "" + +func post(summaryText, githubRepository string, prNumber int) error { + token := os.Getenv("GITHUB_TOKEN") + if token == "" { + return errors.New("GITHUB_TOKEN is not set, this command is meant to be run in a GitHub Action") + } + + repoInfo := strings.Split(githubRepository, "/") + if len(repoInfo) != 2 { + return errors.New("invalid GitHub repository flag") + } + owner := repoInfo[0] + repo := repoInfo[1] + + ctx := context.Background() + ts := oauth2.StaticTokenSource(&oauth2.Token{AccessToken: token}) + tc := oauth2.NewClient(ctx, ts) + client := github.NewClient(tc) + + // Check for an existing comment + comments, _, err := client.Issues.ListComments(ctx, owner, repo, prNumber, nil) + if err != nil { + return err + } + var existingComment *github.IssueComment + for _, comment := range comments { + if strings.Contains(comment.GetBody(), CommentTag) { + existingComment = comment + break + } + } + + commentBody := github.String(fmt.Sprintf("%s\n%s", summaryText, CommentTag)) + if existingComment != nil { + // Update the existing comment + existingComment.Body = commentBody + _, _, err = client.Issues.EditComment(ctx, owner, repo, existingComment.GetID(), existingComment) + if err != nil { + return err + } + return nil + } + + // Create a new comment + _, _, err = client.Issues.CreateComment(ctx, owner, repo, prNumber, &github.IssueComment{ + Body: commentBody, + }) + if err != nil { + return err + } + return nil +} diff --git a/pkg/cmd/microbench-ci/main.go b/pkg/cmd/microbench-ci/main.go new file mode 100644 index 000000000000..8a071958f788 --- /dev/null +++ b/pkg/cmd/microbench-ci/main.go @@ -0,0 +1,206 @@ +package main + +import ( + "fmt" + "log" + "os" + "path" + + "github.com/spf13/cobra" +) + +type ( + Config struct { + WorkingDir string + BenchmarkConfigPath string + SummaryPath string + GitHubSummaryPath string + BuildID string + Old string + New string + Group int + } + Suite struct { + Benchmarks Benchmarks + Revisions Revisions + } + Revisions map[Revision]string + Revision string +) + +const ( + Old Revision = "old" + New Revision = "new" +) + +// Bucket is the GCS bucket where artifacts are stored. +const Bucket = "cockroach-microbench-ci" + +var ( + config *Config + suite *Suite +) + +func defaultConfig() *Config { + return &Config{ + WorkingDir: "microbench-ci-artifacts", + BenchmarkConfigPath: "pkg/cmd/microbench-ci/config/pull-request-suite.yml", + SummaryPath: "summary.json", + GitHubSummaryPath: "github-summary.md", + } +} + +func (c *Config) loadSuite() error { + suite = &Suite{ + Revisions: Revisions{ + Old: config.Old, + New: config.New, + }, + } + benchmarks, err := loadBenchmarkConfig(c.BenchmarkConfigPath) + if err != nil { + return err + } + suite.Benchmarks = benchmarks + return nil +} + +func (s *Suite) revisionDir(revision Revision) string { + return path.Join(config.WorkingDir, s.revisionSHA(revision)) +} + +func (s *Suite) revisionSHA(revision Revision) string { + return s.Revisions[revision] +} + +func (s *Suite) binDir(revision Revision) string { + return path.Join(s.revisionDir(revision), "bin") +} + +func (s *Suite) artifactsDir(revision Revision) string { + return path.Join(s.revisionDir(revision), "artifacts") +} + +func (s *Suite) artifactsURL(revision Revision) string { + return fmt.Sprintf("gs://%s/artifacts/%s/%s/artifacts.zip", + Bucket, s.revisionSHA(revision), config.BuildID) +} + +func (s *Suite) binURL(revision Revision, benchmark *Benchmark) string { + return fmt.Sprintf("gs://%s/builds/%s/bin/%s", + Bucket, s.revisionSHA(revision), benchmark.binaryName()) +} + +func makeRunCommand() *cobra.Command { + cmdFunc := func(cmd *cobra.Command, commandLine []string) error { + if err := config.loadSuite(); err != nil { + return err + } + return suite.Benchmarks.run() + } + cmd := &cobra.Command{ + Use: "run", + Short: "run benchmarks and output artifacts", + Args: cobra.ExactArgs(0), + RunE: cmdFunc, + } + cmd.Flags().IntVar(&config.Group, "group", config.Group, "run only the benchmarks in the group (0 runs all groups)") + return cmd +} + +func makeCompareCommand() *cobra.Command { + cmdFunc := func(cmd *cobra.Command, commandLine []string) error { + if err := config.loadSuite(); err != nil { + return err + } + results, err := suite.Benchmarks.compareBenchmarks() + if err != nil { + return err + } + if err = results.writeJSONSummary(config.SummaryPath); err != nil { + return err + } + return results.writeGitHubSummary(config.GitHubSummaryPath) + } + cmd := &cobra.Command{ + Use: "compare", + Short: "compare artifacts for the given revisions and output an HTML report", + Args: cobra.ExactArgs(0), + RunE: cmdFunc, + } + cmd.Flags().StringVar(&config.SummaryPath, "summary", config.SummaryPath, "path to write comparison results to (JSON)") + return cmd +} + +func makePostCommand() (*cobra.Command, error) { + repo := "cockroachdb/cockroach" + var prNumber int + cmdFunc := func(cmd *cobra.Command, commandLine []string) error { + summaryText, err := os.ReadFile(config.GitHubSummaryPath) + if err != nil { + return err + } + return post(string(summaryText), repo, prNumber) + } + cmd := &cobra.Command{ + Use: "post", + Short: "post creates or updates a microbench-ci summary comment on a GitHub PR", + Args: cobra.ExactArgs(0), + RunE: cmdFunc, + } + cmd.Flags().StringVar(&repo, "repo", repo, "repository") + cmd.Flags().IntVar(&prNumber, "pr-number", 0, "PR number") + if err := cmd.MarkFlagRequired("pr-number"); err != nil { + return nil, err + } + return cmd, nil +} + +func run() error { + cmd := &cobra.Command{ + Use: "microbench-ci [command] (flags)", + Short: "microbench-ci is used by the Microbenchmark CI pipeline to run benchmarks.", + SilenceUsage: true, + SilenceErrors: true, + } + + runCmd := makeRunCommand() + compareCmd := makeCompareCommand() + postCmd, err := makePostCommand() + if err != nil { + return err + } + + for _, c := range []*cobra.Command{runCmd, compareCmd} { + c.Flags().StringVar(&config.WorkingDir, "working-dir", config.WorkingDir, "directory to store or load artifacts from") + c.Flags().StringVar(&config.BenchmarkConfigPath, "config", config.BenchmarkConfigPath, "suite configuration file") + c.Flags().StringVar(&config.Old, "old", "", "old commit") + c.Flags().StringVar(&config.New, "new", "", "new commit") + if err = c.MarkFlagRequired("old"); err != nil { + return err + } + if err = c.MarkFlagRequired("new"); err != nil { + return err + } + } + + for _, c := range []*cobra.Command{postCmd, compareCmd} { + c.Flags().StringVar(&config.GitHubSummaryPath, "github-summary", config.GitHubSummaryPath, "path to write comparison results to (GitHub Markdown)") + } + + compareCmd.Flags().StringVar(&config.BuildID, "build-id", config.BuildID, "GitHub build ID to identify this run") + + cmd.AddCommand(runCmd) + cmd.AddCommand(makeCompareCommand()) + cmd.AddCommand(postCmd) + + return cmd.Execute() +} + +func main() { + config = defaultConfig() + if err := run(); err != nil { + log.Printf("ERROR: %+v", err) + os.Exit(1) + } +} diff --git a/pkg/cmd/microbench-ci/report.go b/pkg/cmd/microbench-ci/report.go new file mode 100644 index 000000000000..5b1f7f994243 --- /dev/null +++ b/pkg/cmd/microbench-ci/report.go @@ -0,0 +1,197 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package main + +import ( + _ "embed" + "encoding/json" + "fmt" + "math" + "os" + "sort" + "strings" + "text/template" + + "golang.org/x/perf/benchmath" + "golang.org/x/perf/benchunit" +) + +//go:embed template/github_summary.md +var githubSummary string + +type SummaryData struct { + Metric string + OldCenter string + NewCenter string + Delta string + Note string + Threshold string + Status string +} + +type GitHubData struct { + BenchmarkStatus string + DisplayName string + Labels string + Summaries []SummaryData + Benchdiff BenchdiffData +} + +type BenchdiffData struct { + Package string + Run string + Dir map[Revision]string + BinURL map[Revision]string + BinDest map[Revision]string + TrimmedSHA map[Revision]string + Old Revision + New Revision +} + +func (c *CompareResult) generateSummaryData( + statusTemplateFunc func(status Status) string, +) []SummaryData { + summaryData := make([]SummaryData, 0, len(c.MetricMap)) + for metricName, entry := range c.MetricMap { + benchmark := entry.BenchmarkEntries[c.EntryName] + cc := entry.ComputeComparison(c.EntryName, string(Old), string(New)) + threshold := c.Benchmark.Thresholds[metricName] * 100.0 + status := statusTemplateFunc(c.status(metricName)) + oldSum := benchmark.Summaries[string(Old)] + newSum := benchmark.Summaries[string(New)] + summaryData = append(summaryData, SummaryData{ + Metric: metricName, + OldCenter: fmt.Sprintf("%s ±%s", formatValue(oldSum.Center, metricName), oldSum.PctRangeString()), + NewCenter: fmt.Sprintf("%s ±%s", formatValue(newSum.Center, metricName), newSum.PctRangeString()), + Delta: cc.FormattedDelta, + Note: cc.Distribution.String(), + Threshold: fmt.Sprintf("%.1f%%", threshold), + Status: status, + }) + } + sort.Slice(summaryData, func(i, j int) bool { + return summaryData[i].Metric > summaryData[j].Metric + }) + return summaryData +} + +func (c *CompareResult) benchdiffData() BenchdiffData { + d := BenchdiffData{ + Package: c.Benchmark.Package, + Run: c.Benchmark.Name, + Dir: make(map[Revision]string), + BinURL: make(map[Revision]string), + BinDest: make(map[Revision]string), + TrimmedSHA: make(map[Revision]string), + Old: Old, + New: New, + } + for _, revision := range []Revision{Old, New} { + sha := suite.revisionSHA(revision) + shortSHA := sha[:int(math.Min(float64(len(sha)), 7))] + d.Dir[revision] = fmt.Sprintf("benchdiff/%s/bin/%s", + shortSHA, c.Benchmark.packageHash()) + d.BinURL[revision] = suite.binURL(revision, c.Benchmark) + d.BinDest[revision] = fmt.Sprintf("%s/%s", d.Dir[revision], + "cockroachdb_cockroach_"+c.Benchmark.sanitizedPackageName()) + d.TrimmedSHA[revision] = shortSHA + } + + return d +} + +func (c CompareResults) writeJSONSummary(path string) error { + file, err := os.Create(path) + if err != nil { + return err + } + defer file.Close() + encoder := json.NewEncoder(file) + + type Data struct { + Metric string + Summary benchmath.Summary + } + data := make(map[string][]Data) + for _, cr := range c { + for name, m := range cr.MetricMap { + for _, r := range []Revision{Old, New} { + data[string(r)] = append(data[string(r)], Data{ + Metric: name, + Summary: *m.BenchmarkEntries[cr.EntryName].Summaries[string(r)], + }) + } + } + } + return encoder.Encode(struct { + Summaries map[string][]Data + Revisions Revisions + }{ + Summaries: data, + Revisions: suite.Revisions, + }) +} + +func (c CompareResults) writeGitHubSummary(path string) error { + file, err := os.Create(path) + if err != nil { + return err + } + defer file.Close() + + regressionDetected := false + summaries := make([]GitHubData, 0, len(c)) + for _, cr := range c { + finalStatus := NoChange + data := GitHubData{ + DisplayName: cr.Benchmark.DisplayName, + Labels: strings.Join(cr.Benchmark.Labels, ", "), + Benchdiff: cr.benchdiffData(), + } + summaryData := cr.generateSummaryData(func(status Status) string { + if status > finalStatus { + finalStatus = status + } + if status == Regression { + regressionDetected = true + } + return statusToDot(status) + }) + data.BenchmarkStatus = statusToDot(finalStatus) + data.Summaries = summaryData + summaries = append(summaries, data) + } + + tmpl, err := template.New("github").Parse(githubSummary) + if err != nil { + return err + } + description := "No regressions detected!" + if regressionDetected { + description = "A regression has been detected, please investigate further!" + } + return tmpl.Execute(file, struct { + GitHubSummaryData []GitHubData + Artifacts map[Revision]string + Description string + }{ + GitHubSummaryData: summaries, + Description: description, + Artifacts: map[Revision]string{ + Old: suite.artifactsURL(Old), + New: suite.artifactsURL(New), + }, + }) +} + +func statusToDot(status Status) string { + return string([]rune("⚪🟢🟡🔴")[status]) +} + +func formatValue(val float64, metric string) string { + cls := benchunit.ClassOf(metric) + return benchunit.Scale(val, cls) +} diff --git a/pkg/cmd/microbench-ci/run.go b/pkg/cmd/microbench-ci/run.go new file mode 100644 index 000000000000..5f5b5b73ff81 --- /dev/null +++ b/pkg/cmd/microbench-ci/run.go @@ -0,0 +1,132 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package main + +import ( + "log" + "os" + "os/exec" + "path" + "strings" + + "github.com/cockroachdb/cockroach/pkg/cmd/roachprod-microbench/parser" + "github.com/cockroachdb/errors" +) + +func (b *Benchmark) command(revision Revision, iterType BenchmarkIterationType) *exec.Cmd { + cmd := exec.Command( + path.Join(suite.binDir(revision), b.binaryName()), + b.args(suite.artifactsDir(revision), iterType)..., + ) + cmd.Env = append(os.Environ(), "COCKROACH_RANDOM_SEED=1") + return cmd +} + +func (b *Benchmark) runIteration(revision Revision) error { + cmd := b.command(revision, Measure) + output, err := cmd.CombinedOutput() + if err != nil { + return errors.Wrapf(err, "benchmark %q, command %q failed to run:\n%s", + b.DisplayName, cmd.String(), string(output)) + } + + results := parser.ExtractBenchmarkResults(string(output)) + if results.Errors { + return errors.Newf("benchmark results contained errors:\n%s", string(output)) + } + if results.Skipped { + return errors.Newf("benchmark invalid due to being skipped:\n%s", string(output)) + } + + var cleanedOutput strings.Builder + for _, result := range results.Results { + cleanedOutput.WriteString(strings.Join(result, " ")) + cleanedOutput.WriteString("\n") + } + err = appendToFile(path.Join(suite.artifactsDir(revision), b.cleanLog()), cleanedOutput.String()) + if err != nil { + return err + } + err = appendToFile(path.Join(suite.artifactsDir(revision), b.rawLog()), string(output)) + if err != nil { + return err + } + return nil +} + +func (b *Benchmark) runProfiles(revision Revision) error { + for _, profile := range []BenchmarkIterationType{ProfileCPU, ProfileMemory, ProfileMutex} { + cmd := b.command(revision, profile) + output, err := cmd.CombinedOutput() + if err != nil { + return errors.Wrapf(err, "profile benchmark %q, command %q failed to run:\n%s", + b.DisplayName, cmd.String(), string(output)) + } + } + return nil +} + +func (b *Benchmark) run() error { + for _, revision := range []Revision{New, Old} { + err := os.MkdirAll(suite.artifactsDir(revision), 0755) + if err != nil { + return err + } + } + + log.Printf("Running benchmark %q for %d iterations", b.Name, b.MeasureCount) + for i := 0; i < b.MeasureCount; i++ { + for _, revision := range []Revision{New, Old} { + log.Printf("%s binary iteration (%d out of %d)", + revision, i+1, b.MeasureCount, + ) + err := b.runIteration(revision) + if err != nil { + return err + } + } + } + + // Only collect profiles if there was a regression. + compareResult, err := b.compare() + if err != nil { + return err + } + if compareResult.regressed() { + for _, revision := range []Revision{New, Old} { + log.Printf("collecting profiles for revision %s", revision) + err := b.runProfiles(revision) + if err != nil { + return err + } + } + } + return nil +} + +func appendToFile(filename, data string) error { + f, err := os.OpenFile(filename, os.O_APPEND|os.O_WRONLY|os.O_CREATE, 0644) + if err != nil { + return err + } + defer f.Close() + if _, err = f.WriteString(data); err != nil { + return err + } + return nil +} + +func (b Benchmarks) run() error { + for _, benchmark := range b { + if config.Group != 0 && benchmark.RunnerGroup != config.Group { + continue + } + if err := benchmark.run(); err != nil { + return err + } + } + return nil +} diff --git a/pkg/cmd/microbench-ci/template/github_summary.md b/pkg/cmd/microbench-ci/template/github_summary.md new file mode 100644 index 000000000000..e2301e0e4d27 --- /dev/null +++ b/pkg/cmd/microbench-ci/template/github_summary.md @@ -0,0 +1,44 @@ +{{- range .GitHubSummaryData}} +
{{.BenchmarkStatus}} {{.DisplayName}} [{{.Labels}}] + +| Metric | Old Commit | New Commit | Delta | Note | Threshold | +|-----------------------------|----------------|----------------|------------|--------------|----------------| +{{- range .Summaries}} +| {{.Status}} **{{.Metric}}** | {{.OldCenter}} | {{.NewCenter}} | {{.Delta}} | {{.Note}} | {{.Threshold}} | +{{- end}} + +
Reproduce + +**benchdiff binaries**: +```shell +{{- $benchdiff := .Benchdiff}} +{{- range $rev, $dir := $benchdiff.Dir }} +mkdir -p {{$dir}} +gcloud storage cp {{index $benchdiff.BinURL $rev}} {{index $benchdiff.BinDest $rev}} +chmod +x {{index $benchdiff.BinDest $rev}} +{{- end}} +``` +**benchdiff command**: +```shell +benchdiff --run=^{{$benchdiff.Run}}$ --old={{index $benchdiff.TrimmedSHA $benchdiff.Old}} --new={{index $benchdiff.TrimmedSHA $benchdiff.New}} ./{{$benchdiff.Package}} +``` + +
+ +
+ +{{end}} + +
Artifacts + +**download**: +```shell +{{- range $rev, $url := $.Artifacts }} +mkdir -p {{$rev}} +gcloud storage cp {{$url}} {{$rev}} +{{- end}} +``` + +
+ +{{.Description}} diff --git a/pkg/cmd/roachprod-microbench/BUILD.bazel b/pkg/cmd/roachprod-microbench/BUILD.bazel index 2055ccab6e09..56b86e0988d7 100644 --- a/pkg/cmd/roachprod-microbench/BUILD.bazel +++ b/pkg/cmd/roachprod-microbench/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "//pkg/cmd/roachprod-microbench/cluster", "//pkg/cmd/roachprod-microbench/google", "//pkg/cmd/roachprod-microbench/model", + "//pkg/cmd/roachprod-microbench/parser", "//pkg/cmd/roachprod-microbench/util", "//pkg/roachprod", "//pkg/roachprod/config", @@ -58,6 +59,7 @@ go_test( embed = [":roachprod-microbench_lib"], deps = [ "//pkg/cmd/roachprod-microbench/model", + "//pkg/cmd/roachprod-microbench/parser", "//pkg/testutils/datapathutils", "//pkg/util/timeutil", "@com_github_cockroachdb_datadriven//:datadriven", diff --git a/pkg/cmd/roachprod-microbench/clean.go b/pkg/cmd/roachprod-microbench/clean.go index 0ae1350f2568..fa1cdbd4cb93 100644 --- a/pkg/cmd/roachprod-microbench/clean.go +++ b/pkg/cmd/roachprod-microbench/clean.go @@ -12,6 +12,7 @@ import ( "path/filepath" "strings" + "github.com/cockroachdb/cockroach/pkg/cmd/roachprod-microbench/parser" "github.com/cockroachdb/errors" ) @@ -34,7 +35,9 @@ func newClean(config cleanConfig) (*clean, error) { return &clean{cleanConfig: config, inputFile: file}, nil } -func (c *clean) writeCleanOutputToFile(cleanedBenchmarkOutputLog benchmarkExtractionResult) error { +func (c *clean) writeCleanOutputToFile( + cleanedBenchmarkOutputLog parser.BenchmarkParseResults, +) error { if err := os.MkdirAll(filepath.Dir(c.outputFilePath), os.ModePerm); err != nil { return err @@ -46,7 +49,7 @@ func (c *clean) writeCleanOutputToFile(cleanedBenchmarkOutputLog benchmarkExtrac } defer outputFile.Close() - for _, benchmarkResult := range cleanedBenchmarkOutputLog.results { + for _, benchmarkResult := range cleanedBenchmarkOutputLog.Results { if _, writeErr := outputFile.WriteString( fmt.Sprintf("%s\n", strings.Join(benchmarkResult, " "))); writeErr != nil { return errors.Wrap(writeErr, "failed to write benchmark result to file") @@ -64,7 +67,7 @@ func (c *clean) cleanBenchmarkOutputLog() error { return err } - cleanedBenchmarkOutputLog := extractBenchmarkResults(string(rawBenchmarkLogs)) + cleanedBenchmarkOutputLog := parser.ExtractBenchmarkResults(string(rawBenchmarkLogs)) if err = c.writeCleanOutputToFile(cleanedBenchmarkOutputLog); err != nil { return err } diff --git a/pkg/cmd/roachprod-microbench/executor.go b/pkg/cmd/roachprod-microbench/executor.go index bba409d31116..cebd71c45f99 100644 --- a/pkg/cmd/roachprod-microbench/executor.go +++ b/pkg/cmd/roachprod-microbench/executor.go @@ -17,6 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/cmd/roachprod-microbench/cluster" + "github.com/cockroachdb/cockroach/pkg/cmd/roachprod-microbench/parser" "github.com/cockroachdb/cockroach/pkg/cmd/roachprod-microbench/util" "github.com/cockroachdb/cockroach/pkg/roachprod" roachprodConfig "github.com/cockroachdb/cockroach/pkg/roachprod/config" @@ -62,12 +63,6 @@ type benchmarkKey struct { key string } -type benchmarkExtractionResult struct { - results [][]string - errors bool - skipped bool -} - func newExecutor(config executorConfig) (*executor, error) { // Exclude packages that should not to be probed. This is useful for excluding // packages that have known issues and unable to list its benchmarks, or are @@ -376,16 +371,16 @@ func (e *executor) executeBenchmarks() error { if !e.quiet { fmt.Print(".") } - extractResults := extractBenchmarkResults(response.Stdout) + extractResults := parser.ExtractBenchmarkResults(response.Stdout) benchmarkResponse := response.Metadata.(benchmarkKey) report := reporters[benchmarkResponse.key] - for _, benchmarkResult := range extractResults.results { + for _, benchmarkResult := range extractResults.Results { if _, writeErr := report.benchmarkOutput[benchmarkResponse.pkg].WriteString( fmt.Sprintf("%s\n", strings.Join(benchmarkResult, " "))); writeErr != nil { e.log.Errorf("Failed to write benchmark result to file - %v", writeErr) } } - if extractResults.errors || response.Err != nil { + if extractResults.Errors || response.Err != nil { if !e.quiet { fmt.Println() } @@ -407,11 +402,11 @@ func (e *executor) executeBenchmarks() error { } // If we didn't find any results, increment the appropriate counter. - if len(extractResults.results) == 0 { + if len(extractResults.Results) == 0 { switch { - case extractResults.errors || response.Err != nil || response.ExitStatus != 0: + case extractResults.Errors || response.Err != nil || response.ExitStatus != 0: failedBenchmarks[benchmarkResponse.benchmark]++ - case extractResults.skipped: + case extractResults.Skipped: skippedBenchmarks[benchmarkResponse.benchmark]++ default: missingBenchmarks[benchmarkResponse.benchmark]++ @@ -443,51 +438,3 @@ func (e *executor) executeBenchmarks() error { } return nil } - -// extractBenchmarkResults extracts the microbenchmark results generated by a -// test binary and reports if any failures or skips were found in the output. -// This method makes specific assumptions regarding the format of the output, -// and attempts to ignore any spurious output that the test binary may have -// logged. The returned list of string arrays each represent a row of metrics as -// outputted by the test binary. -func extractBenchmarkResults(benchmarkOutput string) benchmarkExtractionResult { - keywords := map[string]struct{}{ - "ns/op": {}, - "B/op": {}, - "allocs/op": {}, - } - results := make([][]string, 0) - buf := make([]string, 0) - containsErrors := false - skipped := false - var benchName string - for _, line := range strings.Split(benchmarkOutput, "\n") { - elems := strings.Fields(line) - for _, s := range elems { - if !containsErrors { - containsErrors = strings.Contains(s, "FAIL") || strings.Contains(s, "panic:") - } - if !skipped { - skipped = strings.Contains(s, "SKIP") - } - if strings.HasPrefix(s, "Benchmark") && len(s) > 9 { - benchName = s - } - if _, ok := keywords[s]; ok { - row := elems - if elems[0] == benchName { - row = elems[1:] - } - - buf = append(buf, row...) - if benchName != "" { - buf = append([]string{benchName}, buf...) - results = append(results, buf) - } - buf = make([]string, 0) - benchName = "" - } - } - } - return benchmarkExtractionResult{results, containsErrors, skipped} -} diff --git a/pkg/cmd/roachprod-microbench/executor_test.go b/pkg/cmd/roachprod-microbench/executor_test.go index 6be1ef4da321..c0c4bbe33e5d 100644 --- a/pkg/cmd/roachprod-microbench/executor_test.go +++ b/pkg/cmd/roachprod-microbench/executor_test.go @@ -10,6 +10,7 @@ import ( "path" "testing" + "github.com/cockroachdb/cockroach/pkg/cmd/roachprod-microbench/parser" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" "github.com/cockroachdb/datadriven" ) @@ -20,7 +21,7 @@ func TestExtractBenchmarkResultsDataDriven(t *testing.T) { if d.Cmd != "benchmark" { d.Fatalf(t, "unknown command %s", d.Cmd) } - result := extractBenchmarkResults(d.Input) + result := parser.ExtractBenchmarkResults(d.Input) output := fmt.Sprintf("%v", result) return output }) diff --git a/pkg/cmd/roachprod-microbench/parser/BUILD.bazel b/pkg/cmd/roachprod-microbench/parser/BUILD.bazel new file mode 100644 index 000000000000..43738991d794 --- /dev/null +++ b/pkg/cmd/roachprod-microbench/parser/BUILD.bazel @@ -0,0 +1,8 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "parser", + srcs = ["parser.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/cmd/roachprod-microbench/parser", + visibility = ["//visibility:public"], +) diff --git a/pkg/cmd/roachprod-microbench/parser/parser.go b/pkg/cmd/roachprod-microbench/parser/parser.go new file mode 100644 index 000000000000..c051f682c4cf --- /dev/null +++ b/pkg/cmd/roachprod-microbench/parser/parser.go @@ -0,0 +1,57 @@ +package parser + +import "strings" + +type BenchmarkParseResults struct { + Results [][]string + Errors bool + Skipped bool +} + +// ExtractBenchmarkResults extracts the microbenchmark results generated by a +// test binary and reports if any failures or skips were found in the output. +// This method makes specific assumptions regarding the format of the output, +// and attempts to ignore any spurious output that the test binary may have +// logged. The returned list of string arrays each represent a row of metrics as +// outputted by the test binary. +func ExtractBenchmarkResults(benchmarkOutput string) BenchmarkParseResults { + keywords := map[string]struct{}{ + "ns/op": {}, + "B/op": {}, + "allocs/op": {}, + } + results := make([][]string, 0) + buf := make([]string, 0) + containsErrors := false + skipped := false + var benchName string + for _, line := range strings.Split(benchmarkOutput, "\n") { + elems := strings.Fields(line) + for _, s := range elems { + if !containsErrors { + containsErrors = strings.Contains(s, "FAIL") || strings.Contains(s, "panic:") + } + if !skipped { + skipped = strings.Contains(s, "SKIP") + } + if strings.HasPrefix(s, "Benchmark") && len(s) > 9 { + benchName = s + } + if _, ok := keywords[s]; ok { + row := elems + if elems[0] == benchName { + row = elems[1:] + } + + buf = append(buf, row...) + if benchName != "" { + buf = append([]string{benchName}, buf...) + results = append(results, buf) + } + buf = make([]string, 0) + benchName = "" + } + } + } + return BenchmarkParseResults{results, containsErrors, skipped} +} diff --git a/pkg/sql/tests/BUILD.bazel b/pkg/sql/tests/BUILD.bazel index 7596fda0b9af..6899bce1e3d8 100644 --- a/pkg/sql/tests/BUILD.bazel +++ b/pkg/sql/tests/BUILD.bazel @@ -118,6 +118,7 @@ go_test( "//pkg/sql/types", "//pkg/storage", "//pkg/testutils", + "//pkg/testutils/benchmark", "//pkg/testutils/datapathutils", "//pkg/testutils/serverutils", "//pkg/testutils/skip", diff --git a/pkg/sql/tests/main_test.go b/pkg/sql/tests/main_test.go index 93e4365a6b8f..3a9bf5b07372 100644 --- a/pkg/sql/tests/main_test.go +++ b/pkg/sql/tests/main_test.go @@ -12,6 +12,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/securityassets" "github.com/cockroachdb/cockroach/pkg/security/securitytest" "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/benchmark" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -20,6 +21,7 @@ import ( //go:generate ../../util/leaktest/add-leaktest.sh *_test.go func TestMain(m *testing.M) { + benchmark.ScatterPlotFlags() securityassets.SetLoader(securitytest.EmbeddedAssets) randutil.SeedForTests() serverutils.InitTestServerFactory(server.TestServerFactory) diff --git a/pkg/sql/tests/sysbench_test.go b/pkg/sql/tests/sysbench_test.go index 449700af2b1b..95b4c8b592f7 100644 --- a/pkg/sql/tests/sysbench_test.go +++ b/pkg/sql/tests/sysbench_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils/benchmark" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -687,6 +688,11 @@ func BenchmarkSysbench(b *testing.B) { b.Run(driver.name, func(b *testing.B) { for _, workload := range workloads { b.Run(workload.name, func(b *testing.B) { + // Skip the check run (it's only useful for time vs iterations!) + if b.N <= 1 { + return + } + defer func() { if r := recover(); r != nil { b.Fatalf("%+v", r) @@ -697,14 +703,25 @@ func BenchmarkSysbench(b *testing.B) { sys, cleanup := driver.constructorFn(ctx, b) defer cleanup() - rng := rand.New(rand.NewSource(0)) + //rng := rand.New(rand.NewSource(0)) + rng, _ := randutil.NewTestRand() sys.prep(rng) + // Warmup + for i := 0; i < 250; i++ { + workload.opFn(sys, rng) + } + + scatterPlot := benchmark.NewScatterPlot(b) + defer scatterPlot.Stop(b) + defer startAllocsProfile(b).Stop(b) + scatterPlot.Start() defer b.StopTimer() b.ResetTimer() for i := 0; i < b.N; i++ { workload.opFn(sys, rng) + scatterPlot.Measure(i) } }) } diff --git a/pkg/testutils/benchmark/BUILD.bazel b/pkg/testutils/benchmark/BUILD.bazel new file mode 100644 index 000000000000..2e3ded970b70 --- /dev/null +++ b/pkg/testutils/benchmark/BUILD.bazel @@ -0,0 +1,9 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "benchmark", + srcs = ["scatterplot.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/testutils/benchmark", + visibility = ["//visibility:public"], + deps = ["@com_github_stretchr_testify//require"], +) diff --git a/pkg/testutils/benchmark/scatterplot.go b/pkg/testutils/benchmark/scatterplot.go new file mode 100644 index 000000000000..78133b4c9e8d --- /dev/null +++ b/pkg/testutils/benchmark/scatterplot.go @@ -0,0 +1,90 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package benchmark + +import ( + "flag" + "fmt" + "os" + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +type ScatterPlot struct { + file *os.File + timeOffsets []time.Time + bucketSize int + bucketMod int + index int + startTime time.Time + configured bool +} + +var scatterPlotConfig struct { + path *string + bucketSize *int +} + +func ScatterPlotFlags() { + scatterPlotConfig.path = flag.String("scatterplot-file", "", "outputs a scatter plot file") + scatterPlotConfig.bucketSize = flag.Int("scatterplot-bucket-size", 10, "number of iterations to bucket per point") +} + +func NewScatterPlot(b *testing.B) *ScatterPlot { + if b.N <= 1 || *scatterPlotConfig.path == "" { + return &ScatterPlot{ + configured: false, + } + } + if *scatterPlotConfig.bucketSize <= 0 { + b.Fatalf("bucket size must be greater than 0") + } + // The output is opened in append mode to allow for multiple runs to be appended to the same file. + file, err := os.OpenFile(*scatterPlotConfig.path, os.O_CREATE|os.O_WRONLY|os.O_APPEND, 0644) + require.NoError(b, err) + return &ScatterPlot{ + file: file, + timeOffsets: make([]time.Time, b.N/(*scatterPlotConfig.bucketSize)), + bucketSize: *scatterPlotConfig.bucketSize, + bucketMod: *scatterPlotConfig.bucketSize - 1, + configured: true, + } +} + +func (sp *ScatterPlot) Start() { + if !sp.configured { + return + } + sp.startTime = time.Now() +} + +func (sp *ScatterPlot) Stop(b *testing.B) { + if !sp.configured || sp.index == 0 { + return + } + for i := 0; i < len(sp.timeOffsets); i++ { + subTime := sp.startTime + if i > 0 { + subTime = sp.timeOffsets[i-1] + } + _, err := fmt.Fprintf(sp.file, "%d %d\n", i, sp.timeOffsets[i].Sub(subTime).Nanoseconds()) + require.NoError(b, err) + } + err := sp.file.Close() + require.NoError(b, err) +} + +func (sp *ScatterPlot) Measure(i int) { + if !sp.configured { + return + } + if i%sp.bucketSize == sp.bucketMod { + sp.timeOffsets[sp.index] = time.Now() + sp.index++ + } +}