Skip to content

Commit

Permalink
sql: rationalize the txnAborted event
Browse files Browse the repository at this point in the history
The connection state machine returns an event to the connExecutor -
txnAborted - telling it to clean up some state. This event was
confusingly returned both when entering the Aborted state and sometimes
when leaving it.
This patch replaces txnAborted with a txnRollback event which is
returned strictly on rollback. Entering the Aborted state no longer
triggers this event; it triggers no event. This is in preparation for
broader support for savepoints, which want to enter the Aborted state
without much fuss.

Release note: None
  • Loading branch information
andreimatei committed Feb 28, 2020
1 parent 88eed60 commit 483c170
Show file tree
Hide file tree
Showing 5 changed files with 24 additions and 28 deletions.
6 changes: 3 additions & 3 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -863,7 +863,7 @@ func (ex *connExecutor) close(ctx context.Context, closeType closeType) {

ev := noEvent
if _, noTxn := ex.machine.CurState().(stateNoTxn); !noTxn {
ev = txnAborted
ev = txnRollback
}

if closeType == normalClose {
Expand Down Expand Up @@ -1195,7 +1195,7 @@ func (ex *connExecutor) resetExtraTxnState(
}

switch ev {
case txnCommit, txnAborted:
case txnCommit, txnRollback:
// After txn is finished, we need to call onTxnFinish (if it's non-nil).
if ex.extraTxnState.onTxnFinish != nil {
ex.extraTxnState.onTxnFinish(ev)
Expand Down Expand Up @@ -2201,7 +2201,7 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper(
ex.extraTxnState.tables.waitForCacheToDropDatabases(ex.Ctx())

fallthrough
case txnRestart, txnAborted:
case txnRestart, txnRollback:
if err := ex.resetExtraTxnState(ex.Ctx(), ex.server.dbCache, advInfo.txnEvent); err != nil {
return advanceInfo{}, err
}
Expand Down
16 changes: 8 additions & 8 deletions pkg/sql/conn_fsm.go
Original file line number Diff line number Diff line change
Expand Up @@ -158,7 +158,7 @@ func (e eventTxnFinishPayload) toEvent() txnEvent {
if e.commit {
return txnCommit
}
return txnAborted
return txnRollback
}

type eventTxnRestart struct{}
Expand Down Expand Up @@ -323,7 +323,7 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{
Action: func(args fsm.Args) error {
ts := args.Extended.(*txnState)
ts.mu.txn.CleanupOnError(ts.Ctx, args.Payload.(payloadWithError).errorCause())
ts.setAdvanceInfo(skipBatch, noRewind, txnAborted)
ts.setAdvanceInfo(skipBatch, noRewind, noEvent)
ts.txnAbortCount.Inc(1)
return nil
},
Expand All @@ -350,7 +350,7 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{
Action: func(args fsm.Args) error {
ts := args.Extended.(*txnState)
ts.mu.txn.CleanupOnError(ts.Ctx, args.Payload.(payloadWithError).errorCause())
ts.setAdvanceInfo(skipBatch, noRewind, txnAborted)
ts.setAdvanceInfo(skipBatch, noRewind, noEvent)
ts.txnAbortCount.Inc(1)
return nil
},
Expand Down Expand Up @@ -471,7 +471,7 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{
Action: func(args fsm.Args) error {
ts := args.Extended.(*txnState)
ts.mu.txn.CleanupOnError(ts.Ctx, args.Payload.(eventNonRetriableErrPayload).err)
ts.setAdvanceInfo(skipBatch, noRewind, txnAborted)
ts.setAdvanceInfo(skipBatch, noRewind, noEvent)
ts.txnAbortCount.Inc(1)
return nil
},
Expand Down Expand Up @@ -508,7 +508,7 @@ func cleanupAndFinish(args fsm.Args) error {
ts := args.Extended.(*txnState)
ts.mu.txn.CleanupOnError(ts.Ctx, args.Payload.(payloadWithError).errorCause())
ts.finishSQLTxn()
ts.setAdvanceInfo(skipBatch, noRewind, txnAborted)
ts.setAdvanceInfo(skipBatch, noRewind, txnRollback)
return nil
}

Expand Down Expand Up @@ -553,14 +553,14 @@ func (ts *txnState) finishTxn(payload eventTxnFinishPayload) error {
var BoundTxnStateTransitions = fsm.Compile(fsm.Pattern{
stateOpen{ImplicitTxn: fsm.False, RetryIntent: fsm.False}: {
// We accept eventNonRetriableErr with both IsCommit={True, fsm.False}, even
// those this state machine does not support COMMIT statements because
// though this state machine does not support COMMIT statements because
// connExecutor.close() sends an eventNonRetriableErr{IsCommit: fsm.True} event.
eventNonRetriableErr{IsCommit: fsm.Any}: {
Next: stateInternalError{},
Action: func(args fsm.Args) error {
ts := args.Extended.(*txnState)
ts.finishSQLTxn()
ts.setAdvanceInfo(skipBatch, noRewind, txnAborted)
ts.setAdvanceInfo(skipBatch, noRewind, txnRollback)
return nil
},
},
Expand All @@ -569,7 +569,7 @@ var BoundTxnStateTransitions = fsm.Compile(fsm.Pattern{
Action: func(args fsm.Args) error {
ts := args.Extended.(*txnState)
ts.finishSQLTxn()
ts.setAdvanceInfo(skipBatch, noRewind, txnAborted)
ts.setAdvanceInfo(skipBatch, noRewind, txnRollback)
return nil
},
},
Expand Down
8 changes: 2 additions & 6 deletions pkg/sql/txn_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -370,12 +370,8 @@ const (
// This event is produced both when entering the CommitWait state and also
// when leaving it.
txnCommit
// txnAborted means that the transaction will not commit. This doesn't mean
// that the SQL txn is necessarily "finished" - the connection might be in the
// Aborted state.
// This event is produced both when entering the Aborted state and sometimes
// when leaving it.
txnAborted
// txnRollback means that the transaction has rolled back.
txnRollback
// txnRestart means that the transaction is restarting. The iteration
// of the txn just finished will not commit.
txnRestart
Expand Down
16 changes: 8 additions & 8 deletions pkg/sql/txn_state_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -484,7 +484,7 @@ func TestTransitions(t *testing.T) {
expState: stateNoTxn{},
expAdv: expAdvance{
expCode: skipBatch,
expEv: txnAborted,
expEv: txnRollback,
},
// Expect nil txn.
expTxn: nil,
Expand All @@ -501,7 +501,7 @@ func TestTransitions(t *testing.T) {
expState: stateNoTxn{},
expAdv: expAdvance{
expCode: skipBatch,
expEv: txnAborted,
expEv: txnRollback,
},
// Expect nil txn.
expTxn: nil,
Expand All @@ -524,7 +524,7 @@ func TestTransitions(t *testing.T) {
expState: stateAborted{RetryIntent: fsm.False},
expAdv: expAdvance{
expCode: skipBatch,
expEv: txnAborted,
expEv: noEvent,
},
expTxn: &expKVTxn{},
},
Expand All @@ -546,7 +546,7 @@ func TestTransitions(t *testing.T) {
expState: stateNoTxn{},
expAdv: expAdvance{
expCode: skipBatch,
expEv: txnAborted,
expEv: txnRollback,
},
// Expect the txn to have been cleared.
expTxn: nil,
Expand All @@ -563,7 +563,7 @@ func TestTransitions(t *testing.T) {
expState: stateAborted{RetryIntent: fsm.False},
expAdv: expAdvance{
expCode: skipBatch,
expEv: txnAborted,
expEv: noEvent,
},
expTxn: &expKVTxn{},
},
Expand Down Expand Up @@ -616,7 +616,7 @@ func TestTransitions(t *testing.T) {
expState: stateNoTxn{},
expAdv: expAdvance{
expCode: advanceOne,
expEv: txnAborted,
expEv: txnRollback,
},
expTxn: nil,
},
Expand Down Expand Up @@ -674,7 +674,7 @@ func TestTransitions(t *testing.T) {
expState: stateNoTxn{},
expAdv: expAdvance{
expCode: advanceOne,
expEv: txnAborted,
expEv: txnRollback,
},
expTxn: nil,
},
Expand Down Expand Up @@ -704,7 +704,7 @@ func TestTransitions(t *testing.T) {
expState: stateAborted{RetryIntent: fsm.True},
expAdv: expAdvance{
expCode: skipBatch,
expEv: txnAborted,
expEv: noEvent,
},
expTxn: &expKVTxn{},
},
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/txnevent_string.go

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

0 comments on commit 483c170

Please sign in to comment.