Skip to content

Commit

Permalink
stmtdiagnostics: save the bundle on a statement timeout
Browse files Browse the repository at this point in the history
Previously if the traced statement is canceled due to a statement
timeout, the statement bundle would be created but would fail on the
insertion into the system table. This is suboptimal because we already
did all the work to collect the bundle as well as it might be desired to
see the partial trace, so this commit makes it so that the bundle is
saved correctly.

Release note (bug fix): Previously, when a statement bundle was
collected for a query that results in an error due to
a `statement_timeout`, the bundle would not be saved, and this is now
fixed.
  • Loading branch information
yuzefovich committed Sep 28, 2022
1 parent e783f14 commit 532274b
Show file tree
Hide file tree
Showing 3 changed files with 41 additions and 0 deletions.
1 change: 1 addition & 0 deletions pkg/sql/stmtdiagnostics/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ go_test(
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/sql/catalog/systemschema",
"//pkg/sql/sqlerrors",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/testcluster",
Expand Down
17 changes: 17 additions & 0 deletions pkg/sql/stmtdiagnostics/statement_diagnostics.go
Original file line number Diff line number Diff line change
Expand Up @@ -511,6 +511,23 @@ func (r *Registry) InsertStatementDiagnostics(
collectionErr error,
) (CollectedInstanceID, error) {
var diagID CollectedInstanceID
if ctx.Err() != nil {
// The only two possible errors on the context are the context
// cancellation or the context deadline being exceeded. The former seems
// more likely, and the cancellation is most likely to have occurred due
// to a statement timeout, so we still want to proceed with saving the
// statement bundle. Thus, we override the canceled context, but first
// we'll log the error as a warning.
log.Warningf(
ctx, "context has an error when saving the bundle, proceeding "+
"with the background one (with deadline of 10 seconds): %v", ctx.Err(),
)
// We want to be conservative, so we add a deadline of 10 seconds on top
// of the background context.
var cancel context.CancelFunc
ctx, cancel = context.WithTimeout(context.Background(), 10*time.Second) // nolint:context
defer cancel()
}
err := r.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
if requestID != 0 {
row, err := r.ie.QueryRowEx(ctx, "stmt-diag-check-completed", txn,
Expand Down
23 changes: 23 additions & 0 deletions pkg/sql/stmtdiagnostics/statement_diagnostics_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"context"
gosql "database/sql"
"fmt"
"strings"
"sync"
"testing"
"time"
Expand All @@ -25,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
Expand Down Expand Up @@ -137,6 +139,27 @@ func TestDiagnosticsRequest(t *testing.T) {
checkCompleted(id)
})

// Verify that if the traced query times out, the bundle is still saved.
t.Run("timeout", func(t *testing.T) {
reqID, err := registry.InsertRequestInternal(ctx, "SELECT pg_sleep(_)", samplingProbability, minExecutionLatency, expiresAfter)
require.NoError(t, err)
checkNotCompleted(reqID)

// Set the statement timeout (as well as clean it up in a defer).
_, err = db.Exec("SET statement_timeout = '100ms';")
require.NoError(t, err)
defer func() {
_, err = db.Exec("RESET statement_timeout;")
require.NoError(t, err)
}()

// Run the query that times out.
_, err = db.Exec("SELECT pg_sleep(999999)")
require.Error(t, err)
require.True(t, strings.Contains(err.Error(), sqlerrors.QueryTimeoutError.Error()))
checkCompleted(reqID)
})

// Verify that the bundle for a conditional request is only created when the
// condition is satisfied.
t.Run("conditional", func(t *testing.T) {
Expand Down

0 comments on commit 532274b

Please sign in to comment.