Skip to content

Commit

Permalink
Merge pull request #54370 from arulajmani/backport20.2-53968-54368
Browse files Browse the repository at this point in the history
sql: ensure auto-retrying transactions respect the statement_timeout
  • Loading branch information
arulajmani committed Sep 17, 2020
2 parents e698a64 + 4451817 commit 48e1123
Show file tree
Hide file tree
Showing 4 changed files with 99 additions and 13 deletions.
53 changes: 42 additions & 11 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -280,11 +280,34 @@ func (ex *connExecutor) execStmtInOpenState(
// in that jungle, we just overwrite them all here with an error that's
// nicer to look at for the client.
if res != nil && ctx.Err() != nil && res.Err() != nil {
if queryTimedOut {
res.SetError(sqlerrors.QueryTimeoutError)
} else {
res.SetError(cancelchecker.QueryCanceledError)
// Even in the cases where the error is a retryable error, we want to
// intercept the event and payload returned here to ensure that the query
// is not retried.
retEv = eventNonRetriableErr{
IsCommit: fsm.FromBool(isCommit(stmt.AST)),
}
res.SetError(cancelchecker.QueryCanceledError)
retPayload = eventNonRetriableErrPayload{err: cancelchecker.QueryCanceledError}
}

// If the query timed out, we intercept the error, payload, and event here
// for the same reasons we intercept them for canceled queries above.
// Overriding queries with a QueryTimedOut error needs to happen after
// we've checked for canceled queries as some queries may be canceled
// because of a timeout, in which case the appropriate error to return to
// the client is one that indicates the timeout, rather than the more general
// query canceled error. It's important to note that a timed out query may
// not have been canceled (eg. We never even start executing a query
// because the timeout has already expired), and therefore this check needs
// to happen outside the canceled query check above.
if queryTimedOut {
// A timed out query should never produce retryable errors/events/payloads
// so we intercept and overwrite them all here.
retEv = eventNonRetriableErr{
IsCommit: fsm.FromBool(isCommit(stmt.AST)),
}
res.SetError(sqlerrors.QueryTimeoutError)
retPayload = eventNonRetriableErrPayload{err: sqlerrors.QueryTimeoutError}
}
}
// Generally we want to unregister after the auto-commit below. However, in
Expand Down Expand Up @@ -376,9 +399,22 @@ func (ex *connExecutor) execStmtInOpenState(
}()
}

if ex.sessionData.StmtTimeout > 0 {
makeErrEvent := func(err error) (fsm.Event, fsm.EventPayload, error) {
ev, payload := ex.makeErrEvent(err, stmt.AST)
return ev, payload, nil
}

// We exempt `SET` statements from the statement timeout, particularly so as
// not to block the `SET statement_timeout` command itself.
if ex.sessionData.StmtTimeout > 0 && stmt.AST.StatementTag() != "SET" {
timerDuration := ex.sessionData.StmtTimeout - timeutil.Since(ex.phaseTimes[sessionQueryReceived])
// There's no need to proceed with execution if the timer has already expired.
if timerDuration < 0 {
queryTimedOut = true
return makeErrEvent(sqlerrors.QueryTimeoutError)
}
timeoutTicker = time.AfterFunc(
ex.sessionData.StmtTimeout-timeutil.Since(ex.phaseTimes[sessionQueryReceived]),
timerDuration,
func() {
ex.cancelQuery(stmt.queryID)
queryTimedOut = true
Expand All @@ -405,11 +441,6 @@ func (ex *connExecutor) execStmtInOpenState(
}
}()

makeErrEvent := func(err error) (fsm.Event, fsm.EventPayload, error) {
ev, payload := ex.makeErrEvent(err, stmt.AST)
return ev, payload, nil
}

switch s := stmt.AST.(type) {
case *tree.BeginTransaction:
// BEGIN is always an error when in the Open state. It's legitimate only in
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/sequences
Original file line number Diff line number Diff line change
Expand Up @@ -890,7 +890,7 @@ DROP SEQUENCE sv
subtest generator_timeout

statement ok
SET statement_timeout = 1
SET statement_timeout = 10

statement error pq: query execution canceled due to statement timeout
select * from generate_series(1,10000000) where generate_series = 0;
Expand Down
19 changes: 19 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/set
Original file line number Diff line number Diff line change
Expand Up @@ -290,6 +290,25 @@ SHOW statement_timeout
----
100

# Set the statement timeout to something absurdly small, so that no query would
# presumably be able to go through. It should still be possible to get out of
# this "bad state" by resetting the statement timeout.
subtest impossible_statement_timeout_recovery

statement ok
SET statement_timeout = '1us'

statement error query execution canceled due to statement timeout
SHOW statement_timeout

statement ok
SET statement_timeout = 0

query T
SHOW statement_timeout
----
0

# Test that composite variable names get rejected properly, especially
# when "tracing" is used as prefix.

Expand Down
38 changes: 37 additions & 1 deletion pkg/sql/run_control_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/lib/pq"
"github.com/stretchr/testify/require"
)

func TestCancelSelectQuery(t *testing.T) {
Expand Down Expand Up @@ -609,7 +610,6 @@ func TestIdleInSessionTimeout(t *testing.T) {
})
defer tc.Stopper().Stop(ctx)

var err error
conn, err := tc.ServerConn(0).Conn(ctx)
if err != nil {
t.Fatal(err)
Expand Down Expand Up @@ -850,6 +850,42 @@ func TestIdleInTransactionSessionTimeoutCommitWaitState(t *testing.T) {
}
}

func TestStatementTimeoutRetryableErrors(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()

numNodes := 1
tc := serverutils.StartNewTestCluster(t, numNodes,
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
})
defer tc.Stopper().Stop(ctx)

conn, err := tc.ServerConn(0).Conn(ctx)
require.NoError(t, err)

_, err = conn.QueryContext(ctx,
`SET statement_timeout = '0.1s'`)
require.NoError(t, err)

testutils.RunTrueAndFalse(t, "test statement timeout with explicit txn",
func(t *testing.T, explicitTxn bool) {
query := `SELECT crdb_internal.force_retry('2s');`
if explicitTxn {
query = `BEGIN; ` + query + ` COMMIT;`
}
startTime := timeutil.Now()
_, err = conn.QueryContext(ctx, query)
require.Regexp(t, "pq: query execution canceled due to statement timeout", err)

// The query timeout should be triggered and therefore the force retry
// should not last for 2 seconds as specified.
if timeutil.Since(startTime) >= 2*time.Second {
t.Fatal("expected the query to error out due to the statement_timeout.")
}
})
}

func getUserConn(t *testing.T, username string, server serverutils.TestServerInterface) *gosql.DB {
pgURL := url.URL{
Scheme: "postgres",
Expand Down

0 comments on commit 48e1123

Please sign in to comment.