Skip to content

Commit

Permalink
*: modify 'session transaction isolation level' to take effect once (p…
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao authored and coocood committed Apr 4, 2018
1 parent 0c58e9f commit 6287cfb
Show file tree
Hide file tree
Showing 8 changed files with 93 additions and 25 deletions.
8 changes: 7 additions & 1 deletion distsql/request_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,13 @@ func (builder *RequestBuilder) SetKeepOrder(order bool) *RequestBuilder {
}

func (builder *RequestBuilder) getIsolationLevel(sv *variable.SessionVars) kv.IsoLevel {
isoLevel, _ := sv.GetSystemVar(variable.TxnIsolation)
var isoLevel string
if sv.TxnIsolationLevelOneShot.State == 2 {
isoLevel = sv.TxnIsolationLevelOneShot.Value
}
if isoLevel == "" {
isoLevel, _ = sv.GetSystemVar(variable.TxnIsolation)
}
if isoLevel == ast.ReadCommitted {
return kv.RC
}
Expand Down
49 changes: 26 additions & 23 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,29 +66,31 @@ var (

// Error instances.
var (
ErrUnknownPlan = terror.ClassExecutor.New(codeUnknownPlan, "Unknown plan")
ErrPrepareMulti = terror.ClassExecutor.New(codePrepareMulti, "Can not prepare multiple statements")
ErrPrepareDDL = terror.ClassExecutor.New(codePrepareDDL, "Can not prepare DDL statements")
ErrPasswordNoMatch = terror.ClassExecutor.New(CodePasswordNoMatch, "Can't find any matching row in the user table")
ErrResultIsEmpty = terror.ClassExecutor.New(codeResultIsEmpty, "result is empty")
ErrBuildExecutor = terror.ClassExecutor.New(codeErrBuildExec, "Failed to build executor")
ErrBatchInsertFail = terror.ClassExecutor.New(codeBatchInsertFail, "Batch insert failed, please clean the table and try again.")
ErrWrongValueCountOnRow = terror.ClassExecutor.New(codeWrongValueCountOnRow, "Column count doesn't match value count at row %d")
ErrPasswordFormat = terror.ClassExecutor.New(codePasswordFormat, "The password hash doesn't have the expected format. Check if the correct password algorithm is being used with the PASSWORD() function.")
ErrUnknownPlan = terror.ClassExecutor.New(codeUnknownPlan, "Unknown plan")
ErrPrepareMulti = terror.ClassExecutor.New(codePrepareMulti, "Can not prepare multiple statements")
ErrPrepareDDL = terror.ClassExecutor.New(codePrepareDDL, "Can not prepare DDL statements")
ErrPasswordNoMatch = terror.ClassExecutor.New(CodePasswordNoMatch, "Can't find any matching row in the user table")
ErrResultIsEmpty = terror.ClassExecutor.New(codeResultIsEmpty, "result is empty")
ErrBuildExecutor = terror.ClassExecutor.New(codeErrBuildExec, "Failed to build executor")
ErrBatchInsertFail = terror.ClassExecutor.New(codeBatchInsertFail, "Batch insert failed, please clean the table and try again.")
ErrWrongValueCountOnRow = terror.ClassExecutor.New(codeWrongValueCountOnRow, "Column count doesn't match value count at row %d")
ErrPasswordFormat = terror.ClassExecutor.New(codePasswordFormat, "The password hash doesn't have the expected format. Check if the correct password algorithm is being used with the PASSWORD() function.")
ErrCantChangeTxCharacteristics = terror.ClassExecutor.New(codeErrCantChangeTxCharacteristics, "Transaction characteristics can't be changed while a transaction is in progress")
)

// Error codes.
const (
codeUnknownPlan terror.ErrCode = 1
codePrepareMulti terror.ErrCode = 2
codePrepareDDL terror.ErrCode = 7
codeResultIsEmpty terror.ErrCode = 8
codeErrBuildExec terror.ErrCode = 9
codeBatchInsertFail terror.ErrCode = 10
CodePasswordNoMatch terror.ErrCode = 1133 // MySQL error code
CodeCannotUser terror.ErrCode = 1396 // MySQL error code
codeWrongValueCountOnRow terror.ErrCode = 1136 // MySQL error code
codePasswordFormat terror.ErrCode = 1827 // MySQL error code
codeUnknownPlan terror.ErrCode = 1
codePrepareMulti terror.ErrCode = 2
codePrepareDDL terror.ErrCode = 7
codeResultIsEmpty terror.ErrCode = 8
codeErrBuildExec terror.ErrCode = 9
codeBatchInsertFail terror.ErrCode = 10
CodePasswordNoMatch terror.ErrCode = 1133 // MySQL error code
CodeCannotUser terror.ErrCode = 1396 // MySQL error code
codeWrongValueCountOnRow terror.ErrCode = 1136 // MySQL error code
codePasswordFormat terror.ErrCode = 1827 // MySQL error code
codeErrCantChangeTxCharacteristics terror.ErrCode = 1568
)

// Row represents a result set row, it may be returned from a table, a join, or a projection.
Expand Down Expand Up @@ -604,10 +606,11 @@ func init() {
}
}
tableMySQLErrCodes := map[terror.ErrCode]uint16{
CodeCannotUser: mysql.ErrCannotUser,
CodePasswordNoMatch: mysql.ErrPasswordNoMatch,
codeWrongValueCountOnRow: mysql.ErrWrongValueCountOnRow,
codePasswordFormat: mysql.ErrPasswordFormat,
CodeCannotUser: mysql.ErrCannotUser,
CodePasswordNoMatch: mysql.ErrPasswordNoMatch,
codeWrongValueCountOnRow: mysql.ErrWrongValueCountOnRow,
codePasswordFormat: mysql.ErrPasswordFormat,
codeErrCantChangeTxCharacteristics: mysql.ErrCantChangeTxCharacteristics,
}
terror.ErrClassToMySQLCodes[terror.ClassExecutor] = tableMySQLErrCodes
}
Expand Down
3 changes: 3 additions & 0 deletions executor/set.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,6 +150,9 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e
return errors.Trace(err)
}
oldSnapshotTS := sessionVars.SnapshotTS
if name == variable.TxnIsolationOneShot && sessionVars.InTxn() {
return errors.Trace(ErrCantChangeTxCharacteristics)
}
err = variable.SetSessionSystemVar(sessionVars, name, value)
if err != nil {
return errors.Trace(err)
Expand Down
9 changes: 8 additions & 1 deletion parser/parser.y
Original file line number Diff line number Diff line change
Expand Up @@ -4560,7 +4560,14 @@ SetStmt:
}
| "SET" "TRANSACTION" TransactionChars
{
$$ = &ast.SetStmt{Variables: $3.([]*ast.VariableAssignment)}
assigns := $3.([]*ast.VariableAssignment)
for i:=0; i<len(assigns); i++ {
if assigns[i].Name == "tx_isolation" {
// A special session variable that make setting tx_isolation take effect one time.
assigns[i].Name = "tx_isolation_one_shot"
}
}
$$ = &ast.SetStmt{Variables: assigns}
}

TransactionChars:
Expand Down
11 changes: 11 additions & 0 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -350,6 +350,17 @@ func (s *session) doCommitWithRetry(ctx context.Context) error {
metrics.StatementPerTransaction.WithLabelValues(metrics.RetLabel(err)).Observe(float64(counter))
metrics.TransactionDuration.WithLabelValues(metrics.RetLabel(err)).Observe(float64(duration))
s.cleanRetryInfo()

if isoLevelOneShot := &s.sessionVars.TxnIsolationLevelOneShot; isoLevelOneShot.State != 0 {
switch isoLevelOneShot.State {
case 1:
isoLevelOneShot.State = 2
case 2:
isoLevelOneShot.State = 0
isoLevelOneShot.Value = ""
}
}

if err != nil {
log.Warnf("[%d] finished txn:%v, %v", s.sessionVars.ConnectionID, s.txn, err)
return errors.Trace(err)
Expand Down
24 changes: 24 additions & 0 deletions session/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1977,6 +1977,30 @@ func (s *testSessionSuite) TestRollbackOnCompileError(c *C) {
c.Assert(recoverErr, IsTrue)
}

func (s *testSessionSuite) TestSetTransactionIsolationOneShot(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("create table t (k int, v int)")
tk.MustExec("insert t values (1, 42)")
tk.MustExec("set transaction isolation level read committed")

// Check isolation level is set to read committed.
ctx := context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) {
c.Assert(req.IsolationLevel, Equals, kv.RC)
})
tk.Se.Execute(ctx, "select * from t where k = 1")

// Check it just take effect for one time.
ctx = context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) {
c.Assert(req.IsolationLevel, Equals, kv.SI)
})
tk.Se.Execute(ctx, "select * from t where k = 1")

// Can't change isolation level when it's inside a transaction.
tk.MustExec("begin")
_, err := tk.Se.Execute(ctx, "set transaction isolation level read committed")
c.Assert(err, NotNil)
}

func (s *testSessionSuite) TestDBUserNameLength(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("create table if not exists t (a int)")
Expand Down
13 changes: 13 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -162,6 +162,15 @@ type SessionVars struct {
// Should be reset on transaction finished.
TxnCtx *TransactionContext

// TxnIsolationLevelOneShot is used to implements "set transaction isolation level ..."
TxnIsolationLevelOneShot struct {
// state 0 means default
// state 1 means it's set in current transaction.
// state 2 means it should be used in current transaction.
State int
Value string
}

// Following variables are special for current session.

Status uint16
Expand Down Expand Up @@ -448,6 +457,9 @@ func (s *SessionVars) deleteSystemVar(name string) error {
// SetSystemVar sets the value of a system variable.
func (s *SessionVars) SetSystemVar(name string, val string) error {
switch name {
case TxnIsolationOneShot:
s.TxnIsolationLevelOneShot.State = 1
s.TxnIsolationLevelOneShot.Value = val
case TimeZone:
tz, err := parseTimeZone(val)
if err != nil {
Expand Down Expand Up @@ -536,6 +548,7 @@ const (
MaxAllowedPacket = "max_allowed_packet"
TimeZone = "time_zone"
TxnIsolation = "tx_isolation"
TxnIsolationOneShot = "tx_isolation_one_shot"
)

// TableDelta stands for the changed count for one table.
Expand Down
1 change: 1 addition & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -631,6 +631,7 @@ var defaultSysVars = []*SysVar{
{ScopeSession, TIDBMemQuotaIndexLookupJoin, strconv.FormatInt(DefTiDBMemQuotaIndexLookupJoin, 10)},
{ScopeSession, TIDBMemQuotaNestedLoopApply, strconv.FormatInt(DefTiDBMemQuotaNestedLoopApply, 10)},
{ScopeSession, TiDBEnableStreaming, "0"},
{ScopeSession, TxnIsolationOneShot, ""},
/* The following variable is defined as session scope but is actually server scope. */
{ScopeSession, TiDBGeneralLog, strconv.Itoa(DefTiDBGeneralLog)},
{ScopeSession, TiDBConfig, ""},
Expand Down

0 comments on commit 6287cfb

Please sign in to comment.