Skip to content

Commit

Permalink
util/assertion: add runtime assertion API
Browse files Browse the repository at this point in the history
This patch adds a canonical API for runtime assertions. It is intended
to encourage liberal use of runtime assertions in a safe and performant
manner. It does not attempt to reinvent the wheel, but instead builds on
existing infrastructure.

This PR follows the `must` API initially introduced in 3250477,
which was later abandoned for performance reasons (when passing format
args, these unconditionally incurred interface boxing allocations, even
in the happy case). The new `assertion` API is significantly simpler to
avoid this cost:

```go
if foo != bar {
  return assertion.Failed(ctx, "oh no: %v != %v", foo, bar)
}
```

Assertion failures are fatal in all non-release builds, including
roachprod clusters and roachtests, to ensure they will be noticed. In
release builds, they instead log the failure and report it to Sentry (if
enabled), and return an assertion error to the caller for propagation.
This avoids excessive disruption in production environments, where an
assertion failure is often scoped to an individual RPC request,
transaction, or range, and crashing the node can turn a minor problem
into a full-blown outage. It is still possible to kill the node when
appropriate, but this should be the exception rather than the norm.

It also supports expensive assertions that must be compiled out of
normal dev/test/release builds for performance reasons. These are
instead enabled in special test builds.

This is intended to be used instead of other existing assertion
mechanisms, which have various shortcomings:

* `log.Fatalf`: kills the node even in release builds, which can cause
  severe disruption over often minor issues.

* `errors.AssertionFailedf`: only suitable when we have an error return
  path, does not fatal in non-release builds, and are not always
  notified in release builds.

* `logcrash.ReportOrPanic`: panics rather than fatals, which can leave
  the node limping along. Requires the caller to implement separate
  assertion handling in release builds, which is easy to forget. Also
  requires propagating cluster settings, which aren't always available.

* `buildutil.CrdbTestBuild`: only enabled in Go tests, not roachtests,
  roachprod clusters, or production clusters.

* `util.RaceEnabled`: only enabled in race builds. Expensive assertions
  should be possible to run without the additional overhead of the race
  detector.

For more details and examples, see the `assertion` package documentation.

Epic: none
Release note: None
  • Loading branch information
erikgrinaker committed Oct 7, 2023
1 parent 5251b37 commit c7ff5ee
Show file tree
Hide file tree
Showing 7 changed files with 375 additions and 0 deletions.
3 changes: 3 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -629,6 +629,7 @@ ALL_TESTS = [
"//pkg/upgrade/upgrades:upgrades_test",
"//pkg/util/admission/admissionpb:admissionpb_test",
"//pkg/util/admission:admission_test",
"//pkg/util/assertion:assertion_test",
"//pkg/util/binfetcher:binfetcher_test",
"//pkg/util/bitarray:bitarray_test",
"//pkg/util/bitmap:bitmap_test",
Expand Down Expand Up @@ -2274,6 +2275,8 @@ GO_TARGETS = [
"//pkg/util/allstacks:allstacks",
"//pkg/util/arith:arith",
"//pkg/util/asciitsdb:asciitsdb",
"//pkg/util/assertion:assertion",
"//pkg/util/assertion:assertion_test",
"//pkg/util/binfetcher:binfetcher",
"//pkg/util/binfetcher:binfetcher_test",
"//pkg/util/bitarray:bitarray",
Expand Down
5 changes: 5 additions & 0 deletions pkg/testutils/lint/passes/fmtsafe/functions.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,11 @@ var requireConstFmt = map[string]bool{

"github.com/cockroachdb/cockroach/pkg/util/log/logcrash.ReportOrPanic": true,

"github.com/cockroachdb/cockroach/pkg/util/assertion.Failed": true,
"github.com/cockroachdb/cockroach/pkg/util/assertion.failedDepth": true,
"github.com/cockroachdb/cockroach/pkg/util/assertion.Fatal": true,
"github.com/cockroachdb/cockroach/pkg/util/assertion.Panic": true,

"github.com/cockroachdb/cockroach/pkg/kv/kvpb.NewAmbiguousResultErrorf": true,
"github.com/cockroachdb/cockroach/pkg/kv/kvpb.NewDecommissionedStatusErrorf": true,

Expand Down
30 changes: 30 additions & 0 deletions pkg/util/assertion/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "assertion",
srcs = ["assertion.go"],
importpath = "github.com/cockroachdb/cockroach/pkg/util/assertion",
visibility = ["//visibility:public"],
deps = [
"//pkg/build",
"//pkg/util",
"//pkg/util/envutil",
"//pkg/util/log",
"@com_github_cockroachdb_errors//:errors",
],
)

go_test(
name = "assertion_test",
srcs = ["assertion_test.go"],
args = ["-test.timeout=295s"],
embed = [":assertion"],
deps = [
"//pkg/build",
"//pkg/util",
"//pkg/util/leaktest",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
"@com_github_stretchr_testify//require",
],
)
219 changes: 219 additions & 0 deletions pkg/util/assertion/assertion.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,219 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

// Package assertion provides a canonical API for runtime assertions, intended
// to encourage liberal use of assertions in a way that's safe and performant.
// These help build confidence in invariants and assumptions, plug gaps in test
// coverage, and run across all CRDB environments and workloads where they can
// detect bugs even in exceedingly rare corner cases.
//
// Typical usage:
//
// if foo != bar {
// return assertion.Failed(ctx, "oh no: %v != %v", foo, bar)
// }
//
// (the API is not e.g. Assert(foo == bar, "oh no: %v != %v", foo, bar) since
// this incurs unconditional runtime overhead for the format interface boxing)
//
// In non-release builds (including roachprod and roachtest clusters), assertion
// failures kill the process with a stack trace to ensure failures are noticed.
//
// In release builds, assertion failures are not fatal, since this is often
// excessive and causes unnecessary disruption and outages. Instead, an error is
// logged (with stack trace), reported to Sentry, and returned to the caller.
// The caller must explicitly handle the error (enforced by the errcheck
// linter), and should typically propagate it, but can also choose to either
// ignore or recover from the error (when safe), or unconditionally kill the
// process if necessary even in release builds.
//
// This can also be controlled explicitly via COCKROACH_FATAL_ASSERTIONS.
//
// Assertions are used to check and enforce internal invariants, not for general
// error handling or fatal errors. As a rule of thumb, only use an assertion for
// something we expect will never ever fail. Some examples:
//
// - Good assertion: a Raft range's start key must be before its end key. This
// must always be true, so this should be an assertion.
//
// - Bad assertion: writing to a file must succeed. We expect this to fail
// sometimes, e.g. with faulty disks, so this should be a (possibly fatal)
// error, not an assertion.
//
// - Worse assertion: users must enter a valid SQL query. We expect users to
// get this wrong all the time, so this should return an error. In general,
// assertions should never be used to validate external inputs.
//
// Sometimes, it may be appropriate to panic or fatal on assertion failures even
// in release builds. Panics are often used to propagate errors in SQL code.
// Fatals may be necessary when it is unsafe to keep the node running. The
// helpers Panic() and Fatal() can be used for these cases, but only sparingly.
//
// Some assertions may be too expensive to always check. These can be gated on
// ExpensiveEnabled, which is set to true in special test builds (currently race
// builds), and will otherwise compile the assertion out entirely.
//
// USAGE EXAMPLES
// ==============
//
// Example: ignoring an assertion failure. Here, RPC batch processing should
// never return both a Go error and a response error. In release builds, we're
// ok with just ignoring this and using the Go error.
//
// func (n *Node) Batch(ctx context.Context, req *kvpb.BatchRequest) *kvpb.BatchResponse {
// br, err := n.batchInternal(ctx, req)
// if err != nil {
// if br.Error != nil {
// _ = assertion.Failed(ctx, "returned both br.Err=%v and err=%v", br.Err, err)
// }
// br.Error = err // runs in release builds
// }
// return br
// }
//
// Example: returning early on assertion failure. Here, we guard against
// double-stopping a processor. In release builds, we can ignore the error and
// return early, since the processor has already been stopped.
//
// func (p *Processor) Stop(ctx context.Context) {
// if p.stopped {
// _ = assertion.Failed(ctx, "already stopped")
// return // runs in release builds
// }
// p.stopped = true
// }
//
// Example: recovering from an assertion failure. Here, we assert that a byte
// budget isn't closed with outstanding allocations. In release builds, we
// simply release the outstanding bytes and continue closing.
//
// func (m *BytesMonitor) Close(ctx context.Context) {
// m.mu.Lock()
// defer m.mu.Unlock()
// if m.mu.curAllocated != 0 {
// _ = assertion.Failed(ctx, "unexpected %d leftover bytes", m.mu.curAllocated)
// m.releaseBytesLocked(ctx, m.mu.curAllocated) // runs in release builds
// }
// mm.releaseBudget(ctx)
// }
//
// Example: returning an error on assertion failure. Here, we guard against
// unknown transaction statuses during EndTxn. In release builds, we return an
// error to the RPC client.
//
// func EndTxn(
// ctx context.Context, rw storage.ReadWriter, cArgs CommandArgs, resp kvpb.Response,
// ) (result.Result, error) {
// switch reply.Txn.Status {
// case roachpb.COMMITTED: // ...
// case roachpb.ABORTED: // ...
// case roachpb.PENDING: // ...
// case roachpb.STAGING: // ...
// default:
// err := assertion.Failed(ctx, "invalid txn status %s", reply.Txn.Status)
// return result.Result{}, err // runs in release builds
// }
// }
//
// Example: expensive assertions. Here, we assert that a range's MVCC stats are
// accurate after every write, verifying that incremental updates done by writes
// are correct. This is too expensive to do even in non-release builds, so we
// gate the assertion on ExpensiveEnabled which is only enabled in special test
// builds.
//
// func (b *replicaAppBatch) ApplyToStateMachine(ctx context.Context) error {
// // ...
// if assertion.ExpensiveEnabled {
// // runs in certain test builds
// stats, err := storage.ComputeStats(b.r.store.Engine, desc.StartKey, desc.EndKey, now)
// if err != nil {
// return err
// }
// if stats != *b.state.Stats {
// return assertion.Failed(ctx, "incorrect MVCC stats, %v != %v", stats, b.state.Stats)
// }
// }
// // ...
// }
//
// Example: fatal errors, which should not be assertions. Here, Pebble detects
// file corruption. This is not an assertion, but rather normal error handling,
// so we fatal via log.Fatalf() even in release builds.
//
// if err := iter.Close(); err == pebble.ErrCorruption {
// log.Fatalf(ctx, "%v", err)
// }
package assertion

import (
"context"

"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)

var (
// shouldFatal exits the process with a fatal error on assertion failures. It
// defaults to true in non-release builds.
shouldFatal = envutil.EnvOrDefaultBool("COCKROACH_FATAL_ASSERTIONS", !build.IsRelease())

// MaybeSendReport is injected by package logcrash, for Sentry reporting.
MaybeSendReport func(ctx context.Context, err error)
)

// ExpensiveEnabled is true if expensive assertions are enabled.
//
// TODO(erikgrinaker): This should use a separate/different build tag, but
// for now we gate them on race which has de-facto been used for this. See:
// https://github.com/cockroachdb/cockroach/issues/107425
const ExpensiveEnabled = util.RaceEnabled

// Failed reports an assertion failure. In non-release builds, it fatals with a
// stack trace. In release builds, if returns an assertion error, logs it with a
// stack trace, and reports it to Sentry (if enabled).
func Failed(ctx context.Context, format string, args ...interface{}) error {
return failedDepth(ctx, 1, format, args...)
}

// failedDepth is like Failed, but removes the given number of stack frames in
// the stack trace and log message source code location.
func failedDepth(ctx context.Context, depth int, format string, args ...interface{}) error {
depth += 1
err := errors.AssertionFailedWithDepthf(depth, format, args...)
if shouldFatal {
log.FatalfDepth(ctx, depth, "%+v", err)
} else {
log.ErrorfDepth(ctx, depth, "%+v", err)
if MaybeSendReport != nil { // can be nil in tests
MaybeSendReport(ctx, err)
}
}
return err
}

// Fatal unconditionally fatals the process, even in release builds or when
// fatal assertions are disabled. This should only be used when it is unsafe to
// keep the process running. Prefer Failed() when possible.
func Fatal(ctx context.Context, format string, args ...interface{}) {
// We don't call through to failedDepth(), to avoid logging and notifying the
// error twice.
err := errors.AssertionFailedWithDepthf(1, format, args...)
log.FatalfDepth(ctx, 1, "%+v", err)
}

// Panic unconditionally panics with an assertion error, even in release builds.
// This is primarily for use in SQL code where errors are sometimes propagated
// as panics. Prefer Failed() when possible.
func Panic(ctx context.Context, format string, args ...interface{}) {
panic(failedDepth(ctx, 1, format, args...))
}
113 changes: 113 additions & 0 deletions pkg/util/assertion/assertion_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,113 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package assertion

import (
"context"
"fmt"
"testing"

"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
"github.com/stretchr/testify/require"
)

// disableFatal sets shouldFatal to false, resetting it when the test completes.
func disableFatal(t *testing.T) {
old := shouldFatal
t.Cleanup(func() {
shouldFatal = old
})
shouldFatal = false
}

// onReport sets up a MaybeSendReport handler, resetting it when the test completes.
func onReport(t *testing.T, fn func(error)) {
old := MaybeSendReport
t.Cleanup(func() {
MaybeSendReport = old
})
MaybeSendReport = func(ctx context.Context, err error) {
fn(err)
}
}

// TestFail tests that assertion failures behave correctly.
func TestFail(t *testing.T) {
defer leaktest.AfterTest(t)()
disableFatal(t)

// Set up a custom MaybeSendReport handler for the duration of the test.
var reported error
onReport(t, func(err error) {
reported = err
})

// Failed returns an assertion error, and notifies Sentry.
ctx := context.Background()
err := Failed(ctx, "foo: %s", "bar")
require.Error(t, err)
require.True(t, errors.HasAssertionFailure(err))
require.Error(t, reported)
require.Same(t, reported, err)

// Format args are redacted.
require.Equal(t, err.Error(), "foo: bar")
require.EqualValues(t, redact.Sprint(err), "foo: ‹bar›")

// The error includes a stack trace, but strips the Failed frame.
require.Contains(t, fmt.Sprintf("%+v", err), ".TestFail")
require.NotContains(t, fmt.Sprintf("%+v", err), "assertion.Failed")

// We don't test the fatal handling, since it can't easily be tested, and the
// logic is trivial.
}

// TestSettings tests that various settings are initialized correctly.
func TestSettings(t *testing.T) {
defer leaktest.AfterTest(t)()

require.Equal(t, shouldFatal, !build.IsRelease())
require.Equal(t, ExpensiveEnabled, util.RaceEnabled)
}

// TestPanic tests that Panic() panics when fatal assertions are disabled.
func TestPanic(t *testing.T) {
defer leaktest.AfterTest(t)()
disableFatal(t)

// Set up a custom MaybeSendReport handler for the duration of the test.
var reported error
onReport(t, func(err error) {
reported = err
})

// Catch and inspect the panic error.
var panicErr error
func() {
defer func() {
if r := recover(); r != nil {
if e, ok := r.(error); ok {
panicErr = e
}
}
}()
Panic(context.Background(), "foo: %s", "bar")
}()

require.Error(t, panicErr)
require.True(t, errors.HasAssertionFailure(panicErr))
require.Error(t, reported)
require.Equal(t, panicErr, reported)
}
Loading

0 comments on commit c7ff5ee

Please sign in to comment.