Skip to content

Commit

Permalink
sql: simplifiy tracking of injected txn retry errors
Browse files Browse the repository at this point in the history
Rather than using the txn epoch, we can just track how many errors were
injected. This lets us have a bit more control over how many errors to
inject, without having to rely on how the KV layer handles different
types of transaction retries.

Release note: None
  • Loading branch information
rafiss committed Aug 16, 2023
1 parent 177e8d9 commit 6ecbbc0
Show file tree
Hide file tree
Showing 2 changed files with 9 additions and 10 deletions.
10 changes: 4 additions & 6 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -1305,12 +1305,11 @@ func (ex *connExecutor) commitSQLTransaction(
ex.extraTxnState.idleLatency += ex.statsCollector.PhaseTimes().
GetIdleLatency(ex.statsCollector.PreviousPhaseTimes())
if ex.sessionData().InjectRetryErrorsOnCommitEnabled && ast.StatementTag() == "COMMIT" {
if ex.planner.Txn().Epoch() < ex.state.lastEpoch+numTxnRetryErrors {
if ex.state.injectedTxnRetryCounter < numTxnRetryErrors {
retryErr := ex.state.mu.txn.GenerateForcedRetryableErr(
ctx, "injected by `inject_retry_errors_on_commit_enabled` session variable")
ex.state.injectedTxnRetryCounter++
return ex.makeErrEvent(retryErr, ast)
} else {
ex.state.lastEpoch = ex.planner.Txn().Epoch()
}
}
ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionStartTransactionCommit, timeutil.Now())
Expand Down Expand Up @@ -1740,12 +1739,11 @@ func (ex *connExecutor) dispatchToExecutionEngine(
isSetOrShow := stmt.AST.StatementTag() == "SET" || stmt.AST.StatementTag() == "SHOW"
if ex.sessionData().InjectRetryErrorsEnabled && !isSetOrShow &&
planner.Txn().Sender().TxnStatus() == roachpb.PENDING {
if planner.Txn().Epoch() < ex.state.lastEpoch+numTxnRetryErrors {
if ex.state.injectedTxnRetryCounter < numTxnRetryErrors {
retryErr := planner.Txn().GenerateForcedRetryableErr(
ctx, "injected by `inject_retry_errors_enabled` session variable")
res.SetError(retryErr)
} else {
ex.state.lastEpoch = planner.Txn().Epoch()
ex.state.injectedTxnRetryCounter++
}
}
}
Expand Down
9 changes: 5 additions & 4 deletions pkg/sql/txn_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/mon"
Expand Down Expand Up @@ -108,8 +107,10 @@ type txnState struct {
// through the use of AS OF SYSTEM TIME.
isHistorical bool

// lastEpoch is the last observed epoch in the current txn.
lastEpoch enginepb.TxnEpoch
// injectedTxnRetryCounter keeps track of how many errors have been
// injected in this transaction with the inject_retry_errors_enabled
// flag.
injectedTxnRetryCounter int

// mon tracks txn-bound objects like the running state of
// planNode in the midst of performing a computation.
Expand Down Expand Up @@ -191,7 +192,7 @@ func (ts *txnState) resetForNewSQLTxn(
// Reset state vars to defaults.
ts.sqlTimestamp = sqlTimestamp
ts.isHistorical = false
ts.lastEpoch = 0
ts.injectedTxnRetryCounter = 0

// Create a context for this transaction. It will include a root span that
// will contain everything executed as part of the upcoming SQL txn, including
Expand Down

0 comments on commit 6ecbbc0

Please sign in to comment.