Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
47053: sql: add telemetry for statement diagnostics r=RaduBerinde a=RaduBerinde

Add two telemetry counters for statement diagnostics - one when triggered via
the UI, one for EXPLAIN ANALYZE (DEBUG).

Release note: None

47056: workload/schemachange: create new table 90% of the time r=spaskob a=spaskob

Release note (bug fix): we were using an existing table name 100% of
the time when creating a new table which resulted in no tables created.

47063: retry: fix retry.WithMaxAttempt to deal with opt.Closer properly r=knz a=otan

In `beac4a53e0e2e2236eb5957f67abc1bf476ad1b6`, we introduced
stopper.ShouldQuiesce() to the retry.Closer so that server shutdowns
also shut down in-process retries to the temp schema cleaner.

However, when stopper.ShouldQuiesce() is called, the error that gets
wrapped in `errors.Wrap` is nil (as ctx.Err() is nil), and as such we
return with no error set. This causes potentially bugs afterwards as
users of the functions expected errors when this happens and not to
continue silently.

This PR bridges that gap by always wrapping an error around cases where
WithMaxAttempt is aborted by a context attempt.

Resolves #47057.

Release note: None.

Co-authored-by: Radu Berinde <[email protected]>
Co-authored-by: Spas Bojanov <[email protected]>
Co-authored-by: Oliver Tan <[email protected]>
  • Loading branch information
4 people committed Apr 6, 2020
4 parents bf8e378 + f79144d + 854b9c0 + c1df412 commit 80da27b
Show file tree
Hide file tree
Showing 7 changed files with 177 additions and 18 deletions.
2 changes: 2 additions & 0 deletions pkg/server/updates_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -243,6 +243,7 @@ func TestCBOReportUsage(t *testing.T) {
sqlDB.Exec(t, `EXPLAIN SELECT * FROM x`)
sqlDB.Exec(t, `EXPLAIN (distsql) SELECT * FROM x`)
sqlDB.Exec(t, `EXPLAIN ANALYZE SELECT * FROM x`)
sqlDB.Exec(t, `EXPLAIN ANALYZE (DEBUG) SELECT * FROM x`)
sqlDB.Exec(t, `EXPLAIN (opt) SELECT * FROM x`)
sqlDB.Exec(t, `EXPLAIN (opt, verbose) SELECT * FROM x`)
// Do joins different numbers of times to disambiguate them in the expected,
Expand Down Expand Up @@ -312,6 +313,7 @@ func TestCBOReportUsage(t *testing.T) {
"sql.plan.stats.created": 1,
"sql.plan.explain": 1,
"sql.plan.explain-analyze": 1,
"sql.plan.explain-analyze-debug": 1,
"sql.plan.explain-opt": 1,
"sql.plan.explain-opt-verbose": 1,
"sql.plan.explain-distsql": 1,
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
Expand Down Expand Up @@ -192,6 +193,7 @@ func (ex *connExecutor) execStmtInOpenState(
var finishCollectionDiagnostics StmtDiagnosticsTraceFinishFunc

if explainBundle, ok := stmt.AST.(*tree.ExplainAnalyzeDebug); ok {
telemetry.Inc(sqltelemetry.ExplainAnalyzeDebugUseCounter)
// Always collect diagnostics for EXPLAIN ANALYZE (DEBUG).
shouldCollectDiagnostics = true
// Strip off the explain node to execute the inner statement.
Expand All @@ -211,6 +213,9 @@ func (ex *connExecutor) execStmtInOpenState(
p.discardRows = true
} else {
shouldCollectDiagnostics, finishCollectionDiagnostics = ex.stmtDiagnosticsRecorder.ShouldCollectDiagnostics(ctx, stmt.AST)
if shouldCollectDiagnostics {
telemetry.Inc(sqltelemetry.StatementDiagnosticsCollectedCounter)
}
}

if shouldCollectDiagnostics {
Expand Down
20 changes: 20 additions & 0 deletions pkg/sql/sqltelemetry/diagnostics.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
// Copyright 2019 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 sqltelemetry

import "github.com/cockroachdb/cockroach/pkg/server/telemetry"

// StatementDiagnosticsCollectedCounter is to be incremented whenever a query is
// run with diagnostic collection (as a result of a user request through the
// UI). This does not include diagnostics collected through
// EXPLAIN ANALYZE (DEBUG), which has a separate counter.
// distributed across multiple nodes.
var StatementDiagnosticsCollectedCounter = telemetry.GetCounterOnce("sql.diagnostics.collected")
4 changes: 4 additions & 0 deletions pkg/sql/sqltelemetry/planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,10 @@ var ExplainDistSQLUseCounter = telemetry.GetCounterOnce("sql.plan.explain-distsq
// ExplainAnalyzeUseCounter is to be incremented whenever EXPLAIN ANALYZE is run.
var ExplainAnalyzeUseCounter = telemetry.GetCounterOnce("sql.plan.explain-analyze")

// ExplainAnalyzeDebugUseCounter is to be incremented whenever
// EXPLAIN ANALYZE (DEBUG) is run.
var ExplainAnalyzeDebugUseCounter = telemetry.GetCounterOnce("sql.plan.explain-analyze-debug")

// ExplainOptUseCounter is to be incremented whenever EXPLAIN (OPT) is run.
var ExplainOptUseCounter = telemetry.GetCounterOnce("sql.plan.explain-opt")

Expand Down
6 changes: 5 additions & 1 deletion pkg/util/retry/retry.go
Original file line number Diff line number Diff line change
Expand Up @@ -162,7 +162,11 @@ func WithMaxAttempts(ctx context.Context, opts Options, n int, fn func() error)
}
}
if err == nil {
err = errors.Wrap(ctx.Err(), "did not run function")
if ctx.Err() != nil {
err = errors.Wrap(ctx.Err(), "did not run function due to context completion")
} else {
err = errors.New("did not run function due to closed opts.Closer")
}
}
return err
}
Expand Down
156 changes: 140 additions & 16 deletions pkg/util/retry/retry_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,9 +12,11 @@ package retry

import (
"context"
"errors"
"testing"
"time"

"github.com/pkg/errors"
"github.com/stretchr/testify/require"
)

func TestRetryExceedsMaxBackoff(t *testing.T) {
Expand Down Expand Up @@ -140,26 +142,148 @@ func TestRetryNextCh(t *testing.T) {
}

func TestRetryWithMaxAttempts(t *testing.T) {
opts := Options{
InitialBackoff: time.Microsecond * 10,
MaxBackoff: time.Second,
Multiplier: 2,
MaxRetries: 1,
}

attempts := 0
const maxAttempts = 3
expectedErr := errors.New("placeholder")
errFn := func() error {
attempts := 0
errWithAttemptsCounterFunc := func() error {
attempts++
return expectedErr
}
errorUntilAttemptNumFunc := func(until int) func() error {
return func() error {
attempts++
if attempts == until {
return nil
}
return expectedErr
}
}
cancelCtx, cancelCtxFunc := context.WithCancel(context.Background())
closeCh := make(chan struct{})

testCases := []struct {
desc string

ctx context.Context
opts Options
preWithMaxAttemptsFunc func()
retryFunc func() error
maxAttempts int

// Due to channel races with select, we can allow a range of number of attempts.
minNumAttempts int
maxNumAttempts int
expectedErrText string
}{
{
desc: "succeeds when no errors are ever given",
ctx: context.Background(),
opts: Options{
InitialBackoff: time.Microsecond * 10,
MaxBackoff: time.Microsecond * 20,
Multiplier: 2,
MaxRetries: 1,
},
retryFunc: func() error { return nil },
maxAttempts: 3,

minNumAttempts: 0,
maxNumAttempts: 0,
},
{
desc: "succeeds after one faked error",
ctx: context.Background(),
opts: Options{
InitialBackoff: time.Microsecond * 10,
MaxBackoff: time.Microsecond * 20,
Multiplier: 2,
MaxRetries: 1,
},
retryFunc: errorUntilAttemptNumFunc(1),
maxAttempts: 3,

minNumAttempts: 1,
maxNumAttempts: 1,
},
{
desc: "errors when max attempts is exhausted",
ctx: context.Background(),
opts: Options{
InitialBackoff: time.Microsecond * 10,
MaxBackoff: time.Microsecond * 20,
Multiplier: 2,
MaxRetries: 1,
},
retryFunc: errWithAttemptsCounterFunc,
maxAttempts: 3,

actualErr := WithMaxAttempts(context.TODO(), opts, maxAttempts, errFn)
if actualErr != expectedErr {
t.Fatalf("expected err %v, got %v", expectedErr, actualErr)
minNumAttempts: 3,
maxNumAttempts: 3,
expectedErrText: expectedErr.Error(),
},
{
desc: "errors with context that is canceled",
ctx: cancelCtx,
opts: Options{
InitialBackoff: time.Microsecond * 10,
MaxBackoff: time.Microsecond * 20,
Multiplier: 2,
MaxRetries: 1,
},
retryFunc: errWithAttemptsCounterFunc,
maxAttempts: 3,
preWithMaxAttemptsFunc: func() {
cancelCtxFunc()
},

minNumAttempts: 0,
maxNumAttempts: 3,
expectedErrText: "did not run function due to context completion: context canceled",
},
{
desc: "errors with opt.Closer that is closed",
ctx: context.Background(),
opts: Options{
InitialBackoff: time.Microsecond * 10,
MaxBackoff: time.Microsecond * 20,
Multiplier: 2,
MaxRetries: 1,
Closer: closeCh,
},
retryFunc: errWithAttemptsCounterFunc,
maxAttempts: 3,
preWithMaxAttemptsFunc: func() {
close(closeCh)
},

minNumAttempts: 0,
maxNumAttempts: 3,
expectedErrText: "did not run function due to closed opts.Closer",
},
}
if attempts != maxAttempts {
t.Errorf("expected %d attempts, got %d attempts", maxAttempts, attempts)

for _, tc := range testCases {
t.Run(tc.desc, func(t *testing.T) {
attempts = 0
if tc.preWithMaxAttemptsFunc != nil {
tc.preWithMaxAttemptsFunc()
}
err := WithMaxAttempts(tc.ctx, tc.opts, tc.maxAttempts, tc.retryFunc)
if tc.expectedErrText != "" {
// Error can be either the expected error or the error timeout, as
// channels can race.
require.Truef(
t,
err.Error() == tc.expectedErrText || err.Error() == expectedErr.Error(),
"expected %s or %s, got %s",
tc.expectedErrText,
expectedErr.Error(),
err.Error(),
)
} else {
require.NoError(t, err)
}
require.GreaterOrEqual(t, attempts, tc.minNumAttempts)
require.LessOrEqual(t, attempts, tc.maxNumAttempts)
})
}
}
2 changes: 1 addition & 1 deletion pkg/workload/schemachange/schemachange.go
Original file line number Diff line number Diff line change
Expand Up @@ -458,7 +458,7 @@ func (w *schemaChangeWorker) createSequence(tx *pgx.Tx) (string, error) {
}

func (w *schemaChangeWorker) createTable(tx *pgx.Tx) (string, error) {
tableName, err := w.randTable(tx, 100)
tableName, err := w.randTable(tx, 10)
if err != nil {
return "", err
}
Expand Down

0 comments on commit 80da27b

Please sign in to comment.