Skip to content

Commit

Permalink
sql: don't rewind to BEGIN when retrying txn
Browse files Browse the repository at this point in the history
Release note (bug fix): Fixed a bug where CockroachDB would
sometimes automatically retry the BEGIN statement of an explicit
transaction.
  • Loading branch information
rafiss committed Jun 8, 2022
1 parent 175eebd commit 2ab7292
Show file tree
Hide file tree
Showing 5 changed files with 65 additions and 5 deletions.
6 changes: 4 additions & 2 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -2181,7 +2181,9 @@ func (ex *connExecutor) updateTxnRewindPosMaybe(
if _, ok := ex.machine.CurState().(stateOpen); !ok {
return nil
}
if advInfo.txnEvent.eventType == txnStart || advInfo.txnEvent.eventType == txnRestart {
if advInfo.txnEvent.eventType == txnStart ||
advInfo.txnEvent.eventType == txnRestart ||
advInfo.txnEvent.eventType == txnUpgradeToExplicit {
var nextPos CmdPos
switch advInfo.code {
case stayInPlace:
Expand Down Expand Up @@ -2859,7 +2861,7 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper(

// Handle transaction events which cause updates to txnState.
switch advInfo.txnEvent.eventType {
case noEvent:
case noEvent, txnUpgradeToExplicit:
_, nextStateIsAborted := ex.machine.CurState().(stateAborted)
// Update the deadline on the transaction based on the collections,
// if the transaction is currently open. If the next state is aborted
Expand Down
52 changes: 52 additions & 0 deletions pkg/sql/conn_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -727,6 +727,58 @@ func TestRetriableErrorDuringPrepare(t *testing.T) {
defer func() { _ = stmt.Close() }()
}

// TestRetriableErrorDuringUpgradedTransaction ensures that a retriable error
// that happens during a transaction that was upgraded from an implicit
// transaction into an explicit transaction does not cause the BEGIN to be
// re-executed.
func TestRetriableErrorDuringUpgradedTransaction(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
var failed int64
const numToFail = 2 // only fail on the first two attempts
filter := newDynamicRequestFilter()
s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
TestingRequestFilter: filter.filter,
},
},
})
defer s.Stopper().Stop(context.Background())

conn, err := sqlDB.Conn(context.Background())
require.NoError(t, err)
testDB := sqlutils.MakeSQLRunner(conn)

var fooTableId uint32
testDB.Exec(t, "SET enable_implicit_transaction_for_batch_statements = true")
testDB.Exec(t, "CREATE TABLE foo (a INT PRIMARY KEY)")
testDB.QueryRow(t, "SELECT 'foo'::regclass::oid").Scan(&fooTableId)

// Inject an error that will happen during execution.
filter.setFilter(func(ctx context.Context, ba roachpb.BatchRequest) *roachpb.Error {
if ba.Txn == nil {
return nil
}
if req, ok := ba.GetArg(roachpb.ConditionalPut); ok {
put := req.(*roachpb.ConditionalPutRequest)
_, tableID, err := keys.SystemSQLCodec.DecodeTablePrefix(put.Key)
if err != nil || tableID != fooTableId {
err = nil
return nil
}
if atomic.AddInt64(&failed, 1) <= numToFail {
return roachpb.NewErrorWithTxn(
roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN, "boom"), ba.Txn,
)
}
}
return nil
})

testDB.Exec(t, "SELECT 1; BEGIN; INSERT INTO foo VALUES(1); COMMIT;")
}

// This test ensures that when in an explicit transaction and statement
// preparation uses the user's transaction, errors during those planning queries
// are handled correctly.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/conn_fsm.go
Original file line number Diff line number Diff line change
Expand Up @@ -303,7 +303,7 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{
args.Extended.(*txnState).setAdvanceInfo(
advanceOne,
noRewind,
txnEvent{eventType: noEvent},
txnEvent{eventType: txnUpgradeToExplicit},
)
return nil
},
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/txn_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -402,6 +402,11 @@ const (
// the transaction. This allows such savepoints to reset more state than other
// savepoints.
txnRestart
// txnUpgradeToExplicit means that the current implicit transaction was
// upgraded to an explicit one. This happens when BEGIN is executed during the
// extended protocol or as part of a batch of statements. It's used to
// indicate that the transaction rewind position should be updated.
txnUpgradeToExplicit
)

// advanceInfo represents instructions for the connExecutor about what statement
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/txneventtype_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

0 comments on commit 2ab7292

Please sign in to comment.