Skip to content

Commit

Permalink
schemachanger: improve panic- and error handling
Browse files Browse the repository at this point in the history
Previously, the declarative schema changer would only recover from
runtime errors in certain subsystems like building the targets or
planning the execution of operations. Consequently an implementation bug
leading to a runtime error in the execution layer would trigger a panic
which would not be recovered and which would cause the whole process to
crash.

This commit fixes this by introducing a common error handler in the form
of scerrors.HandleErrorOrPanic, which recovers from panics, wraps
errors, and prints informative log messages in a uniform way, to be used
at the top of (or near the top of) the declarative schema changer call
stack.

Fixes #91400.

Release note (bug fix): fixed a bug in which panics triggered by certain
DDL statements were not properly recovered, leading to the cluster node
crashing.
  • Loading branch information
Marius Posta committed Nov 7, 2022
1 parent d9cc084 commit af2bf18
Show file tree
Hide file tree
Showing 10 changed files with 85 additions and 55 deletions.
2 changes: 0 additions & 2 deletions pkg/sql/schemachanger/scbuild/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -47,8 +47,6 @@ go_library(
"//pkg/sql/sessiondatapb",
"//pkg/sql/sqlerrors",
"//pkg/sql/types",
"//pkg/util/log",
"//pkg/util/timeutil",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
Expand Down
30 changes: 4 additions & 26 deletions pkg/sql/schemachanger/scbuild/build.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ package scbuild

import (
"context"
"runtime"

"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand All @@ -27,9 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

// Build constructs a new state from an initial state and a statement.
Expand All @@ -39,13 +36,9 @@ import (
func Build(
ctx context.Context, dependencies Dependencies, initial scpb.CurrentState, n tree.Statement,
) (_ scpb.CurrentState, err error) {
start := timeutil.Now()
defer func() {
if err != nil || !log.ExpensiveLogEnabled(ctx, 2) {
return
}
log.Infof(ctx, "build for %s took %v", n.StatementTag(), timeutil.Since(start))
}()
defer scerrors.HandleErrorOrPanic(ctx, &err,
"building declarative schema change targets for %s", redact.Safe(n.StatementTag()),
)()
initial = initial.DeepCopy()
bs := newBuilderState(ctx, dependencies, initial)
els := newEventLogState(dependencies, initial, n)
Expand All @@ -64,21 +57,6 @@ func Build(
TreeAnnotator: an,
SchemaFeatureChecker: dependencies.FeatureChecker(),
}
defer func() {
switch recErr := recover().(type) {
case nil:
// No error.
case runtime.Error:
err = errors.WithAssertionFailure(recErr)
case error:
err = recErr
default:
err = errors.AssertionFailedf(
"unexpected error encountered while building schema change plan %s",
recErr,
)
}
}()
scbuildstmt.Process(b, an.GetStatement())
an.ValidateAnnotations()
els.statements[len(els.statements)-1].RedactedStatement =
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/schemachanger/scerrors/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@ go_library(
"//pkg/sql/catalog",
"//pkg/sql/catalog/descpb",
"//pkg/sql/sem/tree",
"//pkg/util/log",
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
],
)
Expand Down
46 changes: 46 additions & 0 deletions pkg/sql/schemachanger/scerrors/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,15 +11,61 @@
package scerrors

import (
"context"
"fmt"
"runtime"
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
)

// HandleErrorOrPanic generates a closure, intended to be deferred, to handle
// panics which reach all the way up to the declarative schema changer's entry
// points.
// Due to it being called at the entry points, this function also handles
// logging and wrapping errors, for convenience.
func HandleErrorOrPanic(
ctx context.Context, err *error, wrapMsgFmt string, wrapMsgArgs ...interface{},
) func() {
isExpensive := log.ExpensiveLogEnabled(ctx, 2)
var start time.Time
if isExpensive {
start = timeutil.Now()
}
const logDepth = 1
log.InfofDepth(ctx, logDepth, wrapMsgFmt, wrapMsgArgs...)
return func() {
switch recErr := recover().(type) {
case nil:
// No panicked error.
case runtime.Error:
*err = errors.WithAssertionFailure(recErr)
case error:
*err = recErr
default:
*err = errors.AssertionFailedf("recovered from uncategorizable panic: %v", recErr)
}
wrapMsgFmt = "done " + wrapMsgFmt
if *err != nil {
wrapMsgFmt = wrapMsgFmt + " with error: %v"
wrapMsgArgs = append(wrapMsgArgs, *err)
} else if isExpensive {
wrapMsgFmt = wrapMsgFmt + "in %s"
wrapMsgArgs = append(wrapMsgArgs, timeutil.Since(start))
}
log.InfofDepth(ctx, logDepth, wrapMsgFmt, wrapMsgArgs...)
if *err != nil && errors.HasAssertionFailure(*err) {
*err = errors.Wrapf(*err, wrapMsgFmt, wrapMsgArgs...)
}
}
}

type notImplementedError struct {
n tree.NodeFormatter
detail string
Expand Down
13 changes: 13 additions & 0 deletions pkg/sql/schemachanger/scpb/state.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package scpb

import (
"sort"
"strings"

"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -69,6 +70,18 @@ func (s *CurrentState) Rollback() {
s.InRollback = true
}

// StatementTags returns the concatenated statement tags in the current state.
func (s CurrentState) StatementTags() string {
var sb strings.Builder
for i, stmt := range s.Statements {
if i > 0 {
sb.WriteString("; ")
}
sb.WriteString(stmt.StatementTag)
}
return sb.String()
}

// NumStatus is the number of values which Status may take on.
var NumStatus = len(Status_name)

Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/schemachanger/scplan/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ go_library(
deps = [
"//pkg/clusterversion",
"//pkg/jobs/jobspb",
"//pkg/sql/schemachanger/scerrors",
"//pkg/sql/schemachanger/scop",
"//pkg/sql/schemachanger/scpb",
"//pkg/sql/schemachanger/scplan/internal/opgen",
Expand All @@ -25,6 +26,7 @@ go_library(
"//pkg/util/timeutil",
"//pkg/util/treeprinter",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
"@in_gopkg_yaml_v2//:yaml_v2",
],
)
Expand Down
15 changes: 5 additions & 10 deletions pkg/sql/schemachanger/scplan/plan.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/opgen"
Expand All @@ -24,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

// Params holds the arguments for planning.
Expand Down Expand Up @@ -79,6 +81,9 @@ func (p Plan) StagesForCurrentPhase() []scstage.Stage {
// MakePlan generates a Plan for a particular phase of a schema change, given
// the initial state for a set of targets. Returns an error when planning fails.
func MakePlan(ctx context.Context, initial scpb.CurrentState, params Params) (p Plan, err error) {
defer scerrors.HandleErrorOrPanic(ctx, &err,
"building declarative schema changer plan for %s", redact.Safe(initial.StatementTags()),
)()
p = Plan{
CurrentState: initial,
Params: params,
Expand All @@ -91,16 +96,6 @@ func MakePlan(ctx context.Context, initial scpb.CurrentState, params Params) (p
}

func makePlan(ctx context.Context, p *Plan) (err error) {
defer func() {
if r := recover(); r != nil {
rAsErr, ok := r.(error)
if !ok {
rAsErr = errors.Errorf("panic during MakePlan: %v", r)
}
err = rAsErr
}
err = errors.WithAssertionFailure(err)
}()
{
start := timeutil.Now()
p.Graph = buildGraph(ctx, p.Params.ActiveVersion, p.CurrentState)
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/schemachanger/scrun/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,8 @@ go_library(
"//pkg/sql/schemachanger/scop",
"//pkg/sql/schemachanger/scpb",
"//pkg/sql/schemachanger/scplan",
"//pkg/util/log",
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
],
)

Expand Down
25 changes: 10 additions & 15 deletions pkg/sql/schemachanger/scrun/scrun.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

// RunStatementPhase executes in-transaction schema changes for the targeted
Expand Down Expand Up @@ -158,22 +157,15 @@ func executeStage(
stageIdx int,
stage scplan.Stage,
) (err error) {

log.Infof(ctx, "executing %s (rollback=%v)", stage, p.InRollback)

defer scerrors.HandleErrorOrPanic(ctx, &err,
"executing declarative schema change %s (rollback=%v) for %s",
redact.Safe(stage), redact.Safe(p.InRollback), redact.Safe(p.StatementTags()),
)()
if knobs != nil && knobs.BeforeStage != nil {
if err := knobs.BeforeStage(p, stageIdx); err != nil {
return err
}
}

start := timeutil.Now()
defer func() {
if log.ExpensiveLogEnabled(ctx, 2) {
log.Infof(ctx, "executing %s (rollback=%v) took %v: err = %v",
stage, p.InRollback, timeutil.Since(start), err)
}
}()
if err := scexec.ExecuteStage(ctx, deps, stage.Ops()); err != nil {
// Don't go through the effort to wrap the error if it's a retry or it's a
// cancelation.
Expand Down Expand Up @@ -209,7 +201,10 @@ func makeState(
descriptorIDs []descpb.ID,
rollback bool,
withCatalog withCatalogFunc,
) (scpb.CurrentState, error) {
) (state scpb.CurrentState, err error) {
defer scerrors.HandleErrorOrPanic(ctx, &err,
"rebuilding declarative schema change state from descriptors %v", redact.Safe(descriptorIDs),
)()
descError := func(desc catalog.Descriptor, err error) error {
return errors.Wrapf(err, "descriptor %q (%d)", desc.GetName(), desc.GetID())
}
Expand Down Expand Up @@ -273,7 +268,7 @@ func makeState(
}); err != nil {
return scpb.CurrentState{}, err
}
state, err := scpb.MakeCurrentStateFromDescriptors(descriptorStates)
state, err = scpb.MakeCurrentStateFromDescriptors(descriptorStates)
if err != nil {
return scpb.CurrentState{}, err
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/testutils/lint/passes/fmtsafe/functions.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,6 +138,8 @@ var requireConstFmt = map[string]bool{
"(*github.com/cockroachdb/cockroach/pkg/sql/logictest.logicTest).Errorf": true,
"(*github.com/cockroachdb/cockroach/pkg/sql/logictest.logicTest).Fatalf": true,

"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors.HandleErrorOrPanic": true,

"github.com/cockroachdb/cockroach/pkg/server.serverErrorf": true,
"github.com/cockroachdb/cockroach/pkg/server.guaranteedExitFatal": true,

Expand Down

0 comments on commit af2bf18

Please sign in to comment.