From 6ef6a15eb6f6b07c2b10f848460a11c941e6acf8 Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 26 Apr 2019 19:37:30 +0800 Subject: [PATCH 01/48] update go.mod for parser --- go.mod | 2 ++ go.sum | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 4392ba138def8..1ad5c2fd2e0c8 100644 --- a/go.mod +++ b/go.mod @@ -72,3 +72,5 @@ require ( sourcegraph.com/sourcegraph/appdash v0.0.0-20180531100431-4c381bd170b4 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) + +replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190429053203-8106f70e326f diff --git a/go.sum b/go.sum index 26440dba9819c..48ca69540ad29 100644 --- a/go.sum +++ b/go.sum @@ -30,6 +30,8 @@ github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142/go.mod h1:F5haX7 github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbpBpLoyyu8B6e44T7hJy6potg= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= +github.com/crazycs520/parser v0.0.0-20190429053203-8106f70e326f h1:bVWi7q/REj0dmZMautvgv/SjRPDZHwlSpkerXNIjGmg= +github.com/crazycs520/parser v0.0.0-20190429053203-8106f70e326f/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= github.com/cznic/sortutil v0.0.0-20150617083342-4c7342852e65 h1:hxuZop6tSoOi0sxFzoGGYdRqNrPubyaIf9KoBG9tPiE= @@ -160,8 +162,6 @@ github.com/pingcap/kvproto v0.0.0-20190327032727-3d8cb3a30d5d/go.mod h1:QMdbTAXC github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v0.0.0-20190424024541-e2cdb851bce2 h1:WZPcqMEpW1PFsusiCEiNlatZdI8dGURXneIUrFNmehI= -github.com/pingcap/parser v0.0.0-20190424024541-e2cdb851bce2/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQkc/YLUttUdQXoH77gtuWCvL4= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= From 5ccef6f2eca9f79c4411130a3d535b3f87c3582b Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 29 Apr 2019 19:14:47 +0800 Subject: [PATCH 02/48] add lock tables and test in table_test, todo: add locks info to session --- ddl/table_lock.go | 1 + 1 file changed, 1 insertion(+) create mode 100644 ddl/table_lock.go diff --git a/ddl/table_lock.go b/ddl/table_lock.go new file mode 100644 index 0000000000000..8d0dc0c32d4e7 --- /dev/null +++ b/ddl/table_lock.go @@ -0,0 +1 @@ +package ddl From 1cd6c41427d17a15a51242c222ed6a1baa16f3bb Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 29 Apr 2019 19:15:10 +0800 Subject: [PATCH 03/48] add lock tables and test in table_test, todo: add locks info to session --- ddl/ddl.go | 5 ++ ddl/ddl_api.go | 63 ++++++++++++++++++++ ddl/ddl_worker.go | 5 ++ ddl/table_lock.go | 111 ++++++++++++++++++++++++++++++++++++ ddl/table_test.go | 51 ++++++++++++++++- executor/ddl.go | 14 +++++ go.mod | 2 +- go.sum | 8 +++ infoschema/infoschema.go | 43 ++++++++------ planner/core/planbuilder.go | 8 +++ session/session.go | 2 + 11 files changed, 292 insertions(+), 20 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index 2e49ee75b4e2f..c49b933f5675a 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -155,6 +155,8 @@ var ( ErrInvalidIndexState = terror.ClassDDL.New(codeInvalidIndexState, "invalid index state") // ErrInvalidForeignKeyState returns for invalid foreign key state. ErrInvalidForeignKeyState = terror.ClassDDL.New(codeInvalidForeignKeyState, "invalid foreign key state") + // ErrInvalidTableLockState returns for invalid table state. + ErrInvalidTableLockState = terror.ClassDDL.New(codeInvalidTableLockState, "invalid table lock state") // ErrUnsupportedModifyPrimaryKey returns an error when add or drop the primary key. // It's exported for testing. ErrUnsupportedModifyPrimaryKey = terror.ClassDDL.New(codeUnsupportedModifyPrimaryKey, "unsupported %s primary key") @@ -253,6 +255,8 @@ type DDL interface { AlterTable(ctx sessionctx.Context, tableIdent ast.Ident, spec []*ast.AlterTableSpec) error TruncateTable(ctx sessionctx.Context, tableIdent ast.Ident) error RenameTable(ctx sessionctx.Context, oldTableIdent, newTableIdent ast.Ident, isAlterTable bool) error + LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error + UnlockTables(ctx sessionctx.Context, stmt *ast.UnlockTablesStmt) error // GetLease returns current schema lease time. GetLease() time.Duration @@ -654,6 +658,7 @@ const ( codeInvalidColumnState = 102 codeInvalidIndexState = 103 codeInvalidForeignKeyState = 104 + codeInvalidTableLockState = 105 codeCantDropColWithIndex = 201 codeUnsupportedAddColumn = 202 diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index d49b228c46677..cddc2604504ae 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3103,3 +3103,66 @@ func buildPartitionInfo(meta *model.TableInfo, d *ddl, spec *ast.AlterTableSpec) } return part, nil } + +func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error { + if len(stmt.TableLocks) > 1 { + return errors.New("Currently only support lock 1 table at the same time") + } + tb := stmt.TableLocks[0].Table + is := d.infoHandle.Get() + schema, ok := is.SchemaByName(tb.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(tb.Schema) + } + + t, err := is.TableByName(tb.Schema, tb.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(tb.Schema, tb.Name)) + } + + arg := &lockTablesArg{ + TableIDs: []int64{t.Meta().ID}, + LockTypes: []model.TableLockType{stmt.TableLocks[0].Type}, + ServerID: d.GetID(), + SessionID: ctx.GetSessionVars().ConnectionID, + } + + job := &model.Job{ + SchemaID: schema.ID, + TableID: t.Meta().ID, + Type: model.ActionLockTable, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{arg}, + } + + err = d.doDDLJob(ctx, job) + err = d.callHookOnChanged(err) + return errors.Trace(err) +} + +func (d *ddl) UnlockTables(ctx sessionctx.Context, stmt *ast.UnlockTablesStmt) error { + arg := &lockTablesArg{ + TableIDs: []int64{0}, + ServerID: d.GetID(), + SessionID: ctx.GetSessionVars().ConnectionID, + } + job := &model.Job{ + // todo: use real ID. + SchemaID: 0, + TableID: 0, + Type: model.ActionLockTable, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{arg}, + } + + err := d.doDDLJob(ctx, job) + err = d.callHookOnChanged(err) + return errors.Trace(err) +} + +type lockTablesArg struct { + TableIDs []int64 + LockTypes []model.TableLockType + ServerID string + SessionID uint64 +} diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 6a9ed39f5a01e..f5c23f365174e 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -534,6 +534,11 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, ver, err = onModifyTableCharsetAndCollate(t, job) case model.ActionRecoverTable: ver, err = w.onRecoverTable(d, t, job) + case model.ActionLockTable: + ver, err = onLockTables(t, job) + case model.ActionUnlockTable: + ver, err = onUnlockTables(t, job) + default: // Invalid job, cancel it. job.State = model.JobStateCancelled diff --git a/ddl/table_lock.go b/ddl/table_lock.go index 8d0dc0c32d4e7..7402095cd8910 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -1 +1,112 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package ddl + +import ( + "fmt" + "github.com/pingcap/errors" + "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/meta" +) + +func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { + arg := &lockTablesArg{} + if err := job.DecodeArgs(arg); err != nil { + // Invalid arguments, cancel this job. + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + fmt.Printf("on lock table: arg: %#v\n---------\n\n", arg) + + tbInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, err + } + if tbInfo.Lock == nil { + tbInfo.Lock = &model.TableLockInfo{} + } + + switch tbInfo.Lock.State { + case model.TableLockStateNone: + // none -> pre_lock + err = checkLockTable(tbInfo, 0, arg) + if err != nil { + job.State = model.JobStateCancelled + return ver, err + } + + tbInfo.Lock.Tp = arg.LockTypes[0] + tbInfo.Lock.ServerIDs = append(tbInfo.Lock.ServerIDs, arg.ServerID) + tbInfo.Lock.SessionIDs = append(tbInfo.Lock.SessionIDs, arg.SessionID) + tbInfo.Lock.State = model.TableLockStatePreLock + tbInfo.Lock.TS = t.StartTS + job.SchemaState = model.StateDeleteOnly + ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) + case model.TableLockStatePreLock: + tbInfo.Lock.State = model.TableLockStatePublic + tbInfo.Lock.TS = t.StartTS + ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + // Finish this job. + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) + default: + return ver, ErrInvalidTableLockState.GenWithStack("invalid table lock state %v", tbInfo.Lock.State) + + } + return ver, err +} + +func checkLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error { + if tbInfo.Lock == nil || len(tbInfo.Lock.ServerIDs) == 0 { + return nil + } + if tbInfo.Lock.Tp == model.TableLockRead && arg.LockTypes[idx] == model.TableLockRead { + return nil + } + + return infoschema.ErrTableLocked.GenWithStackByArgs(tbInfo.Name.L, tbInfo.Lock.Tp, tbInfo.Lock.ServerIDs, tbInfo.Lock.SessionIDs) +} + +func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { + lockTablesArg := &lockTablesArg{} + if err := job.DecodeArgs(lockTablesArg); err != nil { + // Invalid arguments, cancel this job. + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + fmt.Printf("on unlock table: arg: %#v\n---------\n\n", lockTablesArg) + + tbInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, err + } + // Nothing need to do. + if tbInfo.Lock == nil { + // should never run to this. + job.State = model.JobStateCancelled + return ver, errors.Errorf("the lock of table %v was released, this should never hapen", tbInfo.Name.L) + } + + tbInfo.Lock = nil + ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + // Finish this job. + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) + return ver, nil +} diff --git a/ddl/table_test.go b/ddl/table_test.go index f9a2cfbee3b56..bf0affe14c50e 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -174,12 +174,50 @@ func testRenameTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID, oldSchem c.Assert(err, IsNil) v := getSchemaVer(c, ctx) - tblInfo.State = model.StatePublic checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) - tblInfo.State = model.StateNone return job } +func testLockTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID int64, tblInfo *model.TableInfo, lockTp model.TableLockType) *model.Job { + arg := &lockTablesArg{ + TableIDs: []int64{tblInfo.ID}, + LockTypes: []model.TableLockType{lockTp}, + ServerID: d.GetID(), + SessionID: ctx.GetSessionVars().ConnectionID, + } + job := &model.Job{ + SchemaID: newSchemaID, + TableID: tblInfo.ID, + Type: model.ActionLockTable, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{arg}, + } + err := d.doDDLJob(ctx, job) + c.Assert(err, IsNil) + + v := getSchemaVer(c, ctx) + checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + return job +} + +func checkTableLocked(c *C, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, serverID string, sessionID uint64, lockTp model.TableLockType) { + err := kv.RunInNewTxn(d.store, false, func(txn kv.Transaction) error { + t := meta.NewMeta(txn) + info, err := t.GetTable(dbInfo.ID, tblInfo.ID) + c.Assert(err, IsNil) + + c.Assert(info, NotNil) + c.Assert(info.Lock, NotNil) + c.Assert(len(info.Lock.ServerIDs) == 1, IsTrue) + c.Assert(info.Lock.ServerIDs[0], Equals, serverID) + c.Assert(info.Lock.SessionIDs[0], Equals, sessionID) + c.Assert(info.Lock.Tp, Equals, lockTp) + c.Assert(info.Lock.State, Equals, model.TableLockStatePublic) + return nil + }) + c.Assert(err, IsNil) +} + func testDropTable(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo) *model.Job { job := &model.Job{ SchemaID: dbInfo.ID, @@ -215,7 +253,7 @@ func testTruncateTable(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInf } func testCheckTableState(c *C, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, state model.SchemaState) { - kv.RunInNewTxn(d.store, false, func(txn kv.Transaction) error { + err := kv.RunInNewTxn(d.store, false, func(txn kv.Transaction) error { t := meta.NewMeta(txn) info, err := t.GetTable(dbInfo.ID, tblInfo.ID) c.Assert(err, IsNil) @@ -229,6 +267,7 @@ func testCheckTableState(c *C, d *ddl, dbInfo *model.DBInfo, tblInfo *model.Tabl c.Assert(info.State, Equals, state) return nil }) + c.Assert(err, IsNil) } func testGetTable(c *C, d *ddl, schemaID int64, tableID int64) table.Table { @@ -315,6 +354,12 @@ func (s *testTableSuite) TestTable(c *C) { job = testRenameTable(c, ctx, d, dbInfo1.ID, s.dbInfo.ID, tblInfo) testCheckTableState(c, d, dbInfo1, tblInfo, model.StatePublic) testCheckJobDone(c, d, job, true) + + job = testLockTable(c, ctx, d, dbInfo1.ID, tblInfo, model.TableLockWrite) + testCheckTableState(c, d, dbInfo1, tblInfo, model.StatePublic) + testCheckJobDone(c, d, job, true) + + checkTableLocked(c, d, dbInfo1, tblInfo, d.GetID(), ctx.GetSessionVars().ConnectionID, model.TableLockWrite) } func (s *testTableSuite) TestTableResume(c *C) { diff --git a/executor/ddl.go b/executor/ddl.go index e157bce138775..afb8825448023 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -107,6 +107,10 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.RecordBatch) (err error) err = e.executeRenameTable(x) case *ast.RecoverTableStmt: err = e.executeRecoverTable(x) + case *ast.LockTablesStmt: + err = e.executeLockTables(x) + case *ast.UnlockTablesStmt: + err = e.executeUnlockTables(x) } if err != nil { return e.toErr(err) @@ -426,3 +430,13 @@ func (e *DDLExec) getRecoverTableByTableName(s *ast.RecoverTableStmt, t *meta.Me } return job, tblInfo, nil } + +func (e *DDLExec) executeLockTables(s *ast.LockTablesStmt) error { + err := domain.GetDomain(e.ctx).DDL().LockTables(e.ctx, s) + return err +} + +func (e *DDLExec) executeUnlockTables(s *ast.UnlockTablesStmt) error { + err := domain.GetDomain(e.ctx).DDL().UnlockTables(e.ctx, s) + return err +} diff --git a/go.mod b/go.mod index 1ad5c2fd2e0c8..6328db7c93e8f 100644 --- a/go.mod +++ b/go.mod @@ -73,4 +73,4 @@ require ( sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190429053203-8106f70e326f +replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190429094900-82615a39430f diff --git a/go.sum b/go.sum index 48ca69540ad29..3186f8377fdd7 100644 --- a/go.sum +++ b/go.sum @@ -32,6 +32,14 @@ github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbp github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/crazycs520/parser v0.0.0-20190429053203-8106f70e326f h1:bVWi7q/REj0dmZMautvgv/SjRPDZHwlSpkerXNIjGmg= github.com/crazycs520/parser v0.0.0-20190429053203-8106f70e326f/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= +github.com/crazycs520/parser v0.0.0-20190429063023-191de28158f7 h1:TE6vjepIgrvpjtUNRHODoBlNgSVUiKXSsJdrnTcBzGQ= +github.com/crazycs520/parser v0.0.0-20190429063023-191de28158f7/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= +github.com/crazycs520/parser v0.0.0-20190429080204-707399bd8341 h1:zvEf1pPdUA720KwxQ7zGScYFsOSNfcjAtRECxuWr9R4= +github.com/crazycs520/parser v0.0.0-20190429080204-707399bd8341/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= +github.com/crazycs520/parser v0.0.0-20190429091436-934a2b54c6e0 h1:nsSKc7e7bjda9aJZm+xwaEEZJbAF253H1dcd5QResoU= +github.com/crazycs520/parser v0.0.0-20190429091436-934a2b54c6e0/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= +github.com/crazycs520/parser v0.0.0-20190429094900-82615a39430f h1:qe5g2qkOnvcfYZ8y+AxgJs9S3ALkJzlp8Y6jaLyUkG0= +github.com/crazycs520/parser v0.0.0-20190429094900-82615a39430f/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= github.com/cznic/sortutil v0.0.0-20150617083342-4c7342852e65 h1:hxuZop6tSoOi0sxFzoGGYdRqNrPubyaIf9KoBG9tPiE= diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 6953768049fb6..8e34eead1a29e 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -58,6 +58,10 @@ var ( ErrMultiplePriKey = terror.ClassSchema.New(codeMultiplePriKey, "Multiple primary key defined") // ErrTooManyKeyParts returns for too many key parts. ErrTooManyKeyParts = terror.ClassSchema.New(codeTooManyKeyParts, "Too many key parts specified; max %d parts allowed") + + ErrTableNotLockedForWrite = terror.ClassOptimizer.New(codeErrTableNotLockedForWrite, mysql.MySQLErrName[mysql.ErrTableNotLockedForWrite]) + ErrTableNotLocked = terror.ClassOptimizer.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrTableNotLocked]) + ErrTableLocked = terror.ClassOptimizer.New(codeTableLocked, "Table '%s' was locked in %s by server %v session %v.") ) // InfoSchema is the interface used to retrieve the schema information. @@ -300,6 +304,8 @@ func (h *Handle) EmptyClone() *Handle { // Schema error codes. const ( + codeTableLocked terror.ErrCode = 1 + codeDBDropExists terror.ErrCode = 1008 codeDatabaseNotExists = 1049 codeTableNotExists = 1146 @@ -318,26 +324,31 @@ const ( codeTooManyKeyParts = 1070 codeKeyNameDuplicate = 1061 codeKeyNotExists = 1176 + + codeErrTableNotLockedForWrite = mysql.ErrTableNotLockedForWrite + codeErrTableNotLocked = mysql.ErrTableNotLocked ) func init() { schemaMySQLErrCodes := map[terror.ErrCode]uint16{ - codeDBDropExists: mysql.ErrDBDropExists, - codeDatabaseNotExists: mysql.ErrBadDB, - codeTableNotExists: mysql.ErrNoSuchTable, - codeColumnNotExists: mysql.ErrBadField, - codeCannotAddForeign: mysql.ErrCannotAddForeign, - codeWrongFkDef: mysql.ErrWrongFkDef, - codeForeignKeyNotExists: mysql.ErrCantDropFieldOrKey, - codeDatabaseExists: mysql.ErrDBCreateExists, - codeTableExists: mysql.ErrTableExists, - codeBadTable: mysql.ErrBadTable, - codeColumnExists: mysql.ErrDupFieldName, - codeIndexExists: mysql.ErrDupIndex, - codeMultiplePriKey: mysql.ErrMultiplePriKey, - codeTooManyKeyParts: mysql.ErrTooManyKeyParts, - codeKeyNameDuplicate: mysql.ErrDupKeyName, - codeKeyNotExists: mysql.ErrKeyDoesNotExist, + codeDBDropExists: mysql.ErrDBDropExists, + codeDatabaseNotExists: mysql.ErrBadDB, + codeTableNotExists: mysql.ErrNoSuchTable, + codeColumnNotExists: mysql.ErrBadField, + codeCannotAddForeign: mysql.ErrCannotAddForeign, + codeWrongFkDef: mysql.ErrWrongFkDef, + codeForeignKeyNotExists: mysql.ErrCantDropFieldOrKey, + codeDatabaseExists: mysql.ErrDBCreateExists, + codeTableExists: mysql.ErrTableExists, + codeBadTable: mysql.ErrBadTable, + codeColumnExists: mysql.ErrDupFieldName, + codeIndexExists: mysql.ErrDupIndex, + codeMultiplePriKey: mysql.ErrMultiplePriKey, + codeTooManyKeyParts: mysql.ErrTooManyKeyParts, + codeKeyNameDuplicate: mysql.ErrDupKeyName, + codeKeyNotExists: mysql.ErrKeyDoesNotExist, + codeErrTableNotLockedForWrite: mysql.ErrTableNotLockedForWrite, + codeErrTableNotLocked: mysql.ErrTableNotLocked, } terror.ErrClassToMySQLCodes[terror.ClassSchema] = schemaMySQLErrCodes initInfoSchemaDB() diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 64bef437ed633..01eecc1fe13fa 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1750,6 +1750,14 @@ func (b *PlanBuilder) buildDDL(node ast.DDLNode) (Plan, error) { case *ast.RecoverTableStmt: // Recover table command can only be executed by administrator. b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil) + case *ast.LockTablesStmt: + for _, tl := range v.TableLocks { + if tl.Type == model.TableLockRead { + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, tl.Table.Schema.L, tl.Table.Name.L, "", infoschema.ErrTableLocked) + } else { + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.UpdatePriv, tl.Table.Schema.L, tl.Table.Name.L, "", infoschema.ErrTableLocked) + } + } } p := &DDL{Statement: node} return p, nil diff --git a/session/session.go b/session/session.go index 9685a4aa9e5c5..a82c0a8e42c6c 100644 --- a/session/session.go +++ b/session/session.go @@ -185,6 +185,8 @@ type session struct { statsCollector *handle.SessionStatsCollector // ddlOwnerChecker is used in `select tidb_is_ddl_owner()` statement; ddlOwnerChecker owner.DDLOwnerChecker + // lockedTables use to record the table locks hold by the session. + lockedTables map[int64]model.TableLockType } // DDLOwnerChecker returns s.ddlOwnerChecker. From bbeef6c403a2ef0c42354fbb280b60e3169a1d0c Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 30 Apr 2019 09:19:03 +0800 Subject: [PATCH 04/48] release lock before add lock and refactor struct --- ddl/ddl.go | 2 +- ddl/ddl_api.go | 42 +++++++++++++++++------------ ddl/table_lock.go | 58 +++++++++++++++++++++++++++++++--------- ddl/table_test.go | 8 +++--- executor/ddl.go | 3 ++- go.mod | 2 +- go.sum | 4 +++ infoschema/infoschema.go | 2 +- session/session.go | 51 ++++++++++++++++++++++++++++++++++- sessionctx/context.go | 12 +++++++-- table/tables/tables.go | 22 +++++++++++++++ util/mock/context.go | 23 ++++++++++++++++ 12 files changed, 189 insertions(+), 40 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index c49b933f5675a..e625fefec2bd8 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -256,7 +256,7 @@ type DDL interface { TruncateTable(ctx sessionctx.Context, tableIdent ast.Ident) error RenameTable(ctx sessionctx.Context, oldTableIdent, newTableIdent ast.Ident, isAlterTable bool) error LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error - UnlockTables(ctx sessionctx.Context, stmt *ast.UnlockTablesStmt) error + UnlockTables(ctx sessionctx.Context, tbIDs, dbIDs []int64) error // GetLease returns current schema lease time. GetLease() time.Duration diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index cddc2604504ae..ed9bf5d245569 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3121,10 +3121,11 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error } arg := &lockTablesArg{ - TableIDs: []int64{t.Meta().ID}, - LockTypes: []model.TableLockType{stmt.TableLocks[0].Type}, - ServerID: d.GetID(), - SessionID: ctx.GetSessionVars().ConnectionID, + LockTableIDs: []int64{t.Meta().ID}, + LockSchemaIDs: []int64{schema.ID}, + LockTypes: []model.TableLockType{stmt.TableLocks[0].Type}, + ServerID: d.GetID(), + SessionID: ctx.GetSessionVars().ConnectionID, } job := &model.Job{ @@ -3134,35 +3135,42 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{arg}, } - + ctx.AddTableLock(t.Meta().ID, schema.ID, stmt.TableLocks[0].Type) err = d.doDDLJob(ctx, job) err = d.callHookOnChanged(err) return errors.Trace(err) } -func (d *ddl) UnlockTables(ctx sessionctx.Context, stmt *ast.UnlockTablesStmt) error { +func (d *ddl) UnlockTables(ctx sessionctx.Context, tbIDs, dbIDs []int64) error { + if len(tbIDs) == 0 { + return nil + } arg := &lockTablesArg{ - TableIDs: []int64{0}, - ServerID: d.GetID(), - SessionID: ctx.GetSessionVars().ConnectionID, + LockTableIDs: tbIDs, + LockSchemaIDs: dbIDs, + ServerID: d.GetID(), + SessionID: ctx.GetSessionVars().ConnectionID, } job := &model.Job{ - // todo: use real ID. - SchemaID: 0, - TableID: 0, - Type: model.ActionLockTable, + SchemaID: dbIDs[0], + TableID: tbIDs[0], + Type: model.ActionUnlockTable, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{arg}, } err := d.doDDLJob(ctx, job) + if err == nil { + ctx.ReleaseAllTableLocks() + } err = d.callHookOnChanged(err) return errors.Trace(err) } type lockTablesArg struct { - TableIDs []int64 - LockTypes []model.TableLockType - ServerID string - SessionID uint64 + LockTableIDs []int64 + LockSchemaIDs []int64 + LockTypes []model.TableLockType + ServerID string + SessionID uint64 } diff --git a/ddl/table_lock.go b/ddl/table_lock.go index 7402095cd8910..f144295a7de8d 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -38,23 +38,20 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { tbInfo.Lock = &model.TableLockInfo{} } + err = checkLockTable(tbInfo, 0, arg) + if err != nil { + job.State = model.JobStateCancelled + return ver, err + } + switch tbInfo.Lock.State { case model.TableLockStateNone: // none -> pre_lock - err = checkLockTable(tbInfo, 0, arg) - if err != nil { - job.State = model.JobStateCancelled - return ver, err - } - - tbInfo.Lock.Tp = arg.LockTypes[0] - tbInfo.Lock.ServerIDs = append(tbInfo.Lock.ServerIDs, arg.ServerID) - tbInfo.Lock.SessionIDs = append(tbInfo.Lock.SessionIDs, arg.SessionID) tbInfo.Lock.State = model.TableLockStatePreLock tbInfo.Lock.TS = t.StartTS job.SchemaState = model.StateDeleteOnly ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) - case model.TableLockStatePreLock: + case model.TableLockStatePreLock, model.TableLockStatePublic: tbInfo.Lock.State = model.TableLockStatePublic tbInfo.Lock.TS = t.StartTS ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) @@ -71,14 +68,42 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { } func checkLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error { - if tbInfo.Lock == nil || len(tbInfo.Lock.ServerIDs) == 0 { + if tbInfo.Lock == nil || len(tbInfo.Lock.Sessions) == 0 { + tbInfo.Lock = &model.TableLockInfo{ + Tp: arg.LockTypes[0], + } + tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, model.SessionInfo{ServerID: arg.ServerID, SessionID:arg.SessionID}) + return nil + } + if tbInfo.Lock.State == model.TableLockStatePreLock { return nil } if tbInfo.Lock.Tp == model.TableLockRead && arg.LockTypes[idx] == model.TableLockRead { + contain := hasServerAndSessionID(tbInfo.Lock.Sessions, arg.ServerID, arg.SessionID) + // repeat lock. + if contain { + return nil + } + tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, model.SessionInfo{ServerID: arg.ServerID, SessionID:arg.SessionID}) return nil } + contain := hasServerAndSessionID(tbInfo.Lock.Sessions, arg.ServerID, arg.SessionID) + // repeat lock. + if contain { + if tbInfo.Lock.Tp == arg.LockTypes[idx] { + return nil + } + if len(tbInfo.Lock.Sessions) == 1 { + // just change lock tp directly. + tbInfo.Lock.Tp = arg.LockTypes[idx] + return nil + } + + // todo: release lock. + return infoschema.ErrTableLocked.GenWithStackByArgs(tbInfo.Name.L, tbInfo.Lock.Tp, tbInfo.Lock.Sessions[0]) + } - return infoschema.ErrTableLocked.GenWithStackByArgs(tbInfo.Name.L, tbInfo.Lock.Tp, tbInfo.Lock.ServerIDs, tbInfo.Lock.SessionIDs) + return infoschema.ErrTableLocked.GenWithStackByArgs(tbInfo.Name.L, tbInfo.Lock.Tp, tbInfo.Lock.Sessions[0]) } func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { @@ -110,3 +135,12 @@ func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) return ver, nil } + +func hasServerAndSessionID(sessions []model.SessionInfo,serverID string, sessionID uint64) bool { + for i := range sessions { + if sessions[i].ServerID == serverID && sessions[i].SessionID == sessionID { + return true + } + } + return false +} diff --git a/ddl/table_test.go b/ddl/table_test.go index bf0affe14c50e..7fa3292e1e2fe 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -180,10 +180,10 @@ func testRenameTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID, oldSchem func testLockTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID int64, tblInfo *model.TableInfo, lockTp model.TableLockType) *model.Job { arg := &lockTablesArg{ - TableIDs: []int64{tblInfo.ID}, - LockTypes: []model.TableLockType{lockTp}, - ServerID: d.GetID(), - SessionID: ctx.GetSessionVars().ConnectionID, + LockTableIDs: []int64{tblInfo.ID}, + LockTypes: []model.TableLockType{lockTp}, + ServerID: d.GetID(), + SessionID: ctx.GetSessionVars().ConnectionID, } job := &model.Job{ SchemaID: newSchemaID, diff --git a/executor/ddl.go b/executor/ddl.go index afb8825448023..bafc34bfb8047 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -437,6 +437,7 @@ func (e *DDLExec) executeLockTables(s *ast.LockTablesStmt) error { } func (e *DDLExec) executeUnlockTables(s *ast.UnlockTablesStmt) error { - err := domain.GetDomain(e.ctx).DDL().UnlockTables(e.ctx, s) + tbIDs, dbIDs := e.ctx.GetAllTableLocks() + err := domain.GetDomain(e.ctx).DDL().UnlockTables(e.ctx, tbIDs, dbIDs) return err } diff --git a/go.mod b/go.mod index 6328db7c93e8f..55576740a1201 100644 --- a/go.mod +++ b/go.mod @@ -73,4 +73,4 @@ require ( sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190429094900-82615a39430f +replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190430032010-a4525fb1844c diff --git a/go.sum b/go.sum index 3186f8377fdd7..fe78e45b0f914 100644 --- a/go.sum +++ b/go.sum @@ -40,6 +40,10 @@ github.com/crazycs520/parser v0.0.0-20190429091436-934a2b54c6e0 h1:nsSKc7e7bjda9 github.com/crazycs520/parser v0.0.0-20190429091436-934a2b54c6e0/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= github.com/crazycs520/parser v0.0.0-20190429094900-82615a39430f h1:qe5g2qkOnvcfYZ8y+AxgJs9S3ALkJzlp8Y6jaLyUkG0= github.com/crazycs520/parser v0.0.0-20190429094900-82615a39430f/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= +github.com/crazycs520/parser v0.0.0-20190430020852-5cdecbe3e166 h1:4iHvnr5NAsbIpzoK6EGxUfURSQaNPwH6BoN9Q96mcvs= +github.com/crazycs520/parser v0.0.0-20190430020852-5cdecbe3e166/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= +github.com/crazycs520/parser v0.0.0-20190430032010-a4525fb1844c h1:7NscklE3FOuQyr9XeqVvroma+RZ/5RlRttEQyZ8O1JE= +github.com/crazycs520/parser v0.0.0-20190430032010-a4525fb1844c/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= github.com/cznic/sortutil v0.0.0-20150617083342-4c7342852e65 h1:hxuZop6tSoOi0sxFzoGGYdRqNrPubyaIf9KoBG9tPiE= diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 8e34eead1a29e..4a112df014c9b 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -61,7 +61,7 @@ var ( ErrTableNotLockedForWrite = terror.ClassOptimizer.New(codeErrTableNotLockedForWrite, mysql.MySQLErrName[mysql.ErrTableNotLockedForWrite]) ErrTableNotLocked = terror.ClassOptimizer.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrTableNotLocked]) - ErrTableLocked = terror.ClassOptimizer.New(codeTableLocked, "Table '%s' was locked in %s by server %v session %v.") + ErrTableLocked = terror.ClassOptimizer.New(codeTableLocked, "Table '%s' was locked in %s by %v") ) // InfoSchema is the interface used to retrieve the schema information. diff --git a/session/session.go b/session/session.go index a82c0a8e42c6c..333824b6f11c0 100644 --- a/session/session.go +++ b/session/session.go @@ -186,7 +186,46 @@ type session struct { // ddlOwnerChecker is used in `select tidb_is_ddl_owner()` statement; ddlOwnerChecker owner.DDLOwnerChecker // lockedTables use to record the table locks hold by the session. - lockedTables map[int64]model.TableLockType + lockedTables map[int64]tableLockInfo +} + +type tableLockInfo struct { + tp model.TableLockType + dbID int64 +} + +// AddTableLock adds table lock to the session lock map. +func (s *session) AddTableLock(tblID, dbID int64, tp model.TableLockType) { + s.lockedTables[tblID] = tableLockInfo{ + tp: tp, + dbID: dbID, + } +} + +// CheckTableLocked checks the table lock. +func (s *session) CheckTableLocked(tblID int64) (bool, model.TableLockType) { + lt, ok := s.lockedTables[tblID] + if !ok { + return false, model.TableLockNone + } + return true, lt.tp +} + +// GetAllTableLocks gets all table locks table id and db id hold by the session. +func (s *session) GetAllTableLocks() ([]int64, []int64) { + tbIDs := make([]int64, 0, len(s.lockedTables)) + dbIDs := make([]int64, 0, len(s.lockedTables)) + for id, tl := range s.lockedTables { + tbIDs = append(tbIDs, id) + dbIDs = append(dbIDs, tl.dbID) + } + fmt.Printf("tb: %v, db: %v\n--------\n\n", tbIDs, dbIDs) + return tbIDs, dbIDs +} + +// ReleaseAllTableLocks releases all table locks hold by the session. +func (s *session) ReleaseAllTableLocks() { + s.lockedTables = make(map[int64]tableLockInfo) } // DDLOwnerChecker returns s.ddlOwnerChecker. @@ -1219,7 +1258,15 @@ func (s *session) ClearValue(key fmt.Stringer) { } // Close function does some clean work when session end. +// Close should release the table locks which hold by the session. func (s *session) Close() { + if len(s.lockedTables) > 0 { + tbIDs, dbIDs := s.GetAllTableLocks() + err := domain.GetDomain(s).DDL().UnlockTables(s, tbIDs, dbIDs) + if err != nil { + logutil.Logger(context.Background()).Error("release table lock failed", zap.Uint64("conn", s.sessionVars.ConnectionID)) + } + } if s.statsCollector != nil { s.statsCollector.Delete() } @@ -1469,6 +1516,7 @@ func createSession(store kv.Storage) (*session, error) { plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory) } s.mu.values = make(map[fmt.Stringer]interface{}) + s.lockedTables = make(map[int64]tableLockInfo) domain.BindDomain(s, dom) // session implements variable.GlobalVarAccessor. Bind it to ctx. s.sessionVars.GlobalVarsAccessor = s @@ -1492,6 +1540,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory) } s.mu.values = make(map[fmt.Stringer]interface{}) + s.lockedTables = make(map[int64]tableLockInfo) domain.BindDomain(s, dom) // session implements variable.GlobalVarAccessor. Bind it to ctx. s.sessionVars.GlobalVarsAccessor = s diff --git a/sessionctx/context.go b/sessionctx/context.go index a8d71a8f4626c..9fc640a4b0f3f 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -16,14 +16,14 @@ package sessionctx import ( "context" "fmt" - + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/kvcache" - binlog "github.com/pingcap/tipb/go-binlog" + "github.com/pingcap/tipb/go-binlog" ) // Context is an interface for transaction and executive args environment. @@ -83,6 +83,14 @@ type Context interface { StmtAddDirtyTableOP(op int, physicalID int64, handle int64, row []types.Datum) // DDLOwnerChecker returns owner.DDLOwnerChecker. DDLOwnerChecker() owner.DDLOwnerChecker + // AddTableLock adds table lock to the session lock map. + AddTableLock(tblID, dbID int64, tp model.TableLockType) + // CheckTableLocked checks the table lock. + CheckTableLocked(tblID int64) (bool, model.TableLockType) + + GetAllTableLocks() (tbIDs []int64, dbIDs []int64) + + ReleaseAllTableLocks() } type basicCtxType int diff --git a/table/tables/tables.go b/table/tables/tables.go index dec149c6a3bcc..9a3861169fb34 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1204,3 +1204,25 @@ func (ctx *ctxForPartitionExpr) StmtAddDirtyTableOP(op int, physicalID int64, ha func (ctx *ctxForPartitionExpr) DDLOwnerChecker() owner.DDLOwnerChecker { panic("not support") } + +// AddTableLock implements the sessionctx.Context interface. +func (ctx *ctxForPartitionExpr) AddTableLock(_, _ int64, _ model.TableLockType) { +} + +// CheckTableLocked implements the sessionctx.Context interface. +func (ctx *ctxForPartitionExpr) CheckTableLocked(tblID int64) (bool, model.TableLockType) { + return false, model.TableLockNone +} + +// GetAllTableLocks implements the sessionctx.Context interface. +func (ctx *ctxForPartitionExpr) GetAllTableLocks() ([]int64, []int64) { + return nil, nil +} + +// ReleaseAllTableLocks implements the sessionctx.Context interface. +func (ctx *ctxForPartitionExpr) ReleaseAllTableLocks() { +} + +// Close implements the sessionctx.Context interface. +func (ctx *ctxForPartitionExpr) Close() { +} diff --git a/util/mock/context.go b/util/mock/context.go index c3419792ac857..5255572190f71 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -17,6 +17,7 @@ package mock import ( "context" "fmt" + "github.com/pingcap/parser/model" "sync" "time" @@ -215,6 +216,28 @@ func (c *Context) StmtGetMutation(tableID int64) *binlog.TableMutation { func (c *Context) StmtAddDirtyTableOP(op int, tid int64, handle int64, row []types.Datum) { } +// AddTableLock implements the sessionctx.Context interface. +func (s *Context) AddTableLock(_, _ int64, _ model.TableLockType) { +} + +// CheckTableLocked implements the sessionctx.Context interface. +func (s *Context) CheckTableLocked(tblID int64) (bool, model.TableLockType) { + return false, model.TableLockNone +} + +// GetAllTableLocks implements the sessionctx.Context interface. +func (s *Context) GetAllTableLocks() ([]int64, []int64) { + return nil, nil +} + +// ReleaseAllTableLocks implements the sessionctx.Context interface. +func (s *Context) ReleaseAllTableLocks() { +} + +// Close implements the sessionctx.Context interface. +func (s *Context) Close() { +} + // NewContext creates a new mocked sessionctx.Context. func NewContext() *Context { ctx, cancel := context.WithCancel(context.Background()) From b937cd2064a3aebbae87c65cbed6ac47c9a9a5c6 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 30 Apr 2019 14:44:45 +0800 Subject: [PATCH 05/48] todo: support lock/unlock mutitable and check table lock after privilege check --- ddl/db_test.go | 31 ++++++++++++++++++ ddl/ddl.go | 2 +- ddl/ddl_api.go | 69 +++++++++++++++++++++------------------- ddl/table_lock.go | 14 ++++---- ddl/table_test.go | 13 ++++---- executor/ddl.go | 4 +-- go.mod | 2 +- go.sum | 4 +++ infoschema/infoschema.go | 2 +- session/session.go | 40 ++++++++++------------- sessionctx/context.go | 4 +-- table/tables/tables.go | 6 ++-- util/mock/context.go | 8 ++--- 13 files changed, 114 insertions(+), 85 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index c5867b6648576..16722f1885edb 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2749,3 +2749,34 @@ func (s *testDBSuite2) TestAlterShardRowIDBits(c *C) { c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "[autoid:1467]Failed to read auto-increment value from storage engine") } + +func (s *testDBSuite2) TestLockTables(c *C) { + s.tk = testkit.NewTestKit(c, s.store) + tk := s.tk + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + defer tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1 (a int)") + tk.MustExec("lock tables t1 write") + checkTableLock(c, tk.Se, "test", "t1", model.TableLockWrite) + tk.MustExec("lock tables t1 read") + checkTableLock(c, tk.Se, "test", "t1", model.TableLockRead) + tk.MustExec("unlock tables") + checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone) +} + +func checkTableLock(c *C, se session.Session, dbName, tableName string, lockTp model.TableLockType) { + tb := testGetTableByName(c, se, dbName, tableName) + dom := domain.GetDomain(se) + if lockTp != model.TableLockNone { + c.Assert(tb.Meta().Lock, NotNil) + c.Assert(tb.Meta().Lock.Tp, Equals, lockTp) + c.Assert(tb.Meta().Lock.State, Equals, model.TableLockStatePublic) + c.Assert(len(tb.Meta().Lock.Sessions) == 1, IsTrue) + c.Assert(tb.Meta().Lock.Sessions[0].ServerID, Equals, dom.DDL().GetID()) + c.Assert(tb.Meta().Lock.Sessions[0].SessionID, Equals, se.GetSessionVars().ConnectionID) + } else { + c.Assert(tb.Meta().Lock, IsNil) + } +} diff --git a/ddl/ddl.go b/ddl/ddl.go index e625fefec2bd8..28abec2389cfa 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -256,7 +256,7 @@ type DDL interface { TruncateTable(ctx sessionctx.Context, tableIdent ast.Ident) error RenameTable(ctx sessionctx.Context, oldTableIdent, newTableIdent ast.Ident, isAlterTable bool) error LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error - UnlockTables(ctx sessionctx.Context, tbIDs, dbIDs []int64) error + UnlockTables(ctx sessionctx.Context, lockedTables []model.TableLockTpInfo) error // GetLease returns current schema lease time. GetLease() time.Duration diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index ed9bf5d245569..75b6ac20cc8fb 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3105,55 +3105,59 @@ func buildPartitionInfo(meta *model.TableInfo, d *ddl, spec *ast.AlterTableSpec) } func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error { - if len(stmt.TableLocks) > 1 { - return errors.New("Currently only support lock 1 table at the same time") - } - tb := stmt.TableLocks[0].Table + //if len(stmt.TableLocks) > 1 { + // return errors.New("Currently only support lock 1 table at the same time") + //} + + lockTables := make([]model.TableLockTpInfo, 0, len(stmt.TableLocks)) is := d.infoHandle.Get() - schema, ok := is.SchemaByName(tb.Schema) - if !ok { - return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(tb.Schema) + for _, tl := range stmt.TableLocks { + tb := tl.Table + schema, ok := is.SchemaByName(tb.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(tb.Schema) + } + t, err := is.TableByName(tb.Schema, tb.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(tb.Schema, tb.Name)) + } + lockTables = append(lockTables, model.TableLockTpInfo{SchemaID: schema.ID, TableID: t.Meta().ID, Tp: tl.Type}) } - t, err := is.TableByName(tb.Schema, tb.Name) - if err != nil { - return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(tb.Schema, tb.Name)) - } + unlockTables := ctx.GetAllTableLocks() arg := &lockTablesArg{ - LockTableIDs: []int64{t.Meta().ID}, - LockSchemaIDs: []int64{schema.ID}, - LockTypes: []model.TableLockType{stmt.TableLocks[0].Type}, - ServerID: d.GetID(), - SessionID: ctx.GetSessionVars().ConnectionID, + LockTables: lockTables, + UnlockTables: unlockTables, + ServerID: d.GetID(), + SessionID: ctx.GetSessionVars().ConnectionID, } job := &model.Job{ - SchemaID: schema.ID, - TableID: t.Meta().ID, + SchemaID: lockTables[0].SchemaID, + TableID: lockTables[0].TableID, Type: model.ActionLockTable, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{arg}, } - ctx.AddTableLock(t.Meta().ID, schema.ID, stmt.TableLocks[0].Type) - err = d.doDDLJob(ctx, job) + ctx.AddTableLock(lockTables) + err := d.doDDLJob(ctx, job) err = d.callHookOnChanged(err) return errors.Trace(err) } -func (d *ddl) UnlockTables(ctx sessionctx.Context, tbIDs, dbIDs []int64) error { - if len(tbIDs) == 0 { +func (d *ddl) UnlockTables(ctx sessionctx.Context, unlockTables []model.TableLockTpInfo) error { + if len(unlockTables) == 0 { return nil } arg := &lockTablesArg{ - LockTableIDs: tbIDs, - LockSchemaIDs: dbIDs, - ServerID: d.GetID(), - SessionID: ctx.GetSessionVars().ConnectionID, + UnlockTables: unlockTables, + ServerID: d.GetID(), + SessionID: ctx.GetSessionVars().ConnectionID, } job := &model.Job{ - SchemaID: dbIDs[0], - TableID: tbIDs[0], + SchemaID: unlockTables[0].SchemaID, + TableID: unlockTables[0].TableID, Type: model.ActionUnlockTable, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{arg}, @@ -3168,9 +3172,8 @@ func (d *ddl) UnlockTables(ctx sessionctx.Context, tbIDs, dbIDs []int64) error { } type lockTablesArg struct { - LockTableIDs []int64 - LockSchemaIDs []int64 - LockTypes []model.TableLockType - ServerID string - SessionID uint64 + LockTables []model.TableLockTpInfo + UnlockTables []model.TableLockTpInfo + ServerID string + SessionID uint64 } diff --git a/ddl/table_lock.go b/ddl/table_lock.go index f144295a7de8d..a629bd083ceb7 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -70,32 +70,32 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { func checkLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error { if tbInfo.Lock == nil || len(tbInfo.Lock.Sessions) == 0 { tbInfo.Lock = &model.TableLockInfo{ - Tp: arg.LockTypes[0], + Tp: arg.LockTables[idx].Tp, } - tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, model.SessionInfo{ServerID: arg.ServerID, SessionID:arg.SessionID}) + tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, model.SessionInfo{ServerID: arg.ServerID, SessionID: arg.SessionID}) return nil } if tbInfo.Lock.State == model.TableLockStatePreLock { return nil } - if tbInfo.Lock.Tp == model.TableLockRead && arg.LockTypes[idx] == model.TableLockRead { + if tbInfo.Lock.Tp == model.TableLockRead && arg.LockTables[idx].Tp == model.TableLockRead { contain := hasServerAndSessionID(tbInfo.Lock.Sessions, arg.ServerID, arg.SessionID) // repeat lock. if contain { return nil } - tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, model.SessionInfo{ServerID: arg.ServerID, SessionID:arg.SessionID}) + tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, model.SessionInfo{ServerID: arg.ServerID, SessionID: arg.SessionID}) return nil } contain := hasServerAndSessionID(tbInfo.Lock.Sessions, arg.ServerID, arg.SessionID) // repeat lock. if contain { - if tbInfo.Lock.Tp == arg.LockTypes[idx] { + if tbInfo.Lock.Tp == arg.LockTables[idx].Tp { return nil } if len(tbInfo.Lock.Sessions) == 1 { // just change lock tp directly. - tbInfo.Lock.Tp = arg.LockTypes[idx] + tbInfo.Lock.Tp = arg.LockTables[idx].Tp return nil } @@ -136,7 +136,7 @@ func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { return ver, nil } -func hasServerAndSessionID(sessions []model.SessionInfo,serverID string, sessionID uint64) bool { +func hasServerAndSessionID(sessions []model.SessionInfo, serverID string, sessionID uint64) bool { for i := range sessions { if sessions[i].ServerID == serverID && sessions[i].SessionID == sessionID { return true diff --git a/ddl/table_test.go b/ddl/table_test.go index 7fa3292e1e2fe..c3ca00adfa43c 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -180,10 +180,9 @@ func testRenameTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID, oldSchem func testLockTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID int64, tblInfo *model.TableInfo, lockTp model.TableLockType) *model.Job { arg := &lockTablesArg{ - LockTableIDs: []int64{tblInfo.ID}, - LockTypes: []model.TableLockType{lockTp}, - ServerID: d.GetID(), - SessionID: ctx.GetSessionVars().ConnectionID, + LockTables: []model.TableLockTpInfo{{SchemaID: newSchemaID, TableID: tblInfo.ID, Tp: lockTp}}, + ServerID: d.GetID(), + SessionID: ctx.GetSessionVars().ConnectionID, } job := &model.Job{ SchemaID: newSchemaID, @@ -208,9 +207,9 @@ func checkTableLocked(c *C, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableIn c.Assert(info, NotNil) c.Assert(info.Lock, NotNil) - c.Assert(len(info.Lock.ServerIDs) == 1, IsTrue) - c.Assert(info.Lock.ServerIDs[0], Equals, serverID) - c.Assert(info.Lock.SessionIDs[0], Equals, sessionID) + c.Assert(len(info.Lock.Sessions) == 1, IsTrue) + c.Assert(info.Lock.Sessions[0].ServerID, Equals, serverID) + c.Assert(info.Lock.Sessions[0].SessionID, Equals, sessionID) c.Assert(info.Lock.Tp, Equals, lockTp) c.Assert(info.Lock.State, Equals, model.TableLockStatePublic) return nil diff --git a/executor/ddl.go b/executor/ddl.go index bafc34bfb8047..552b776c55759 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -437,7 +437,7 @@ func (e *DDLExec) executeLockTables(s *ast.LockTablesStmt) error { } func (e *DDLExec) executeUnlockTables(s *ast.UnlockTablesStmt) error { - tbIDs, dbIDs := e.ctx.GetAllTableLocks() - err := domain.GetDomain(e.ctx).DDL().UnlockTables(e.ctx, tbIDs, dbIDs) + lockedTables := e.ctx.GetAllTableLocks() + err := domain.GetDomain(e.ctx).DDL().UnlockTables(e.ctx, lockedTables) return err } diff --git a/go.mod b/go.mod index 55576740a1201..bb51f348b43e8 100644 --- a/go.mod +++ b/go.mod @@ -73,4 +73,4 @@ require ( sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190430032010-a4525fb1844c +replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190430053142-0f4907461312 diff --git a/go.sum b/go.sum index fe78e45b0f914..23900f4336a0f 100644 --- a/go.sum +++ b/go.sum @@ -44,6 +44,10 @@ github.com/crazycs520/parser v0.0.0-20190430020852-5cdecbe3e166 h1:4iHvnr5NAsbIp github.com/crazycs520/parser v0.0.0-20190430020852-5cdecbe3e166/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= github.com/crazycs520/parser v0.0.0-20190430032010-a4525fb1844c h1:7NscklE3FOuQyr9XeqVvroma+RZ/5RlRttEQyZ8O1JE= github.com/crazycs520/parser v0.0.0-20190430032010-a4525fb1844c/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= +github.com/crazycs520/parser v0.0.0-20190430034757-798dc77c7d76 h1:1em8yDP3h8xnwCgiAxhY2jCpAbD50UEkMZYCM82L1iE= +github.com/crazycs520/parser v0.0.0-20190430034757-798dc77c7d76/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= +github.com/crazycs520/parser v0.0.0-20190430053142-0f4907461312 h1:W1OI+Z88CBsOVzIDl5bcxB9EOIPsRQS9AnJynldNBBw= +github.com/crazycs520/parser v0.0.0-20190430053142-0f4907461312/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= github.com/cznic/sortutil v0.0.0-20150617083342-4c7342852e65 h1:hxuZop6tSoOi0sxFzoGGYdRqNrPubyaIf9KoBG9tPiE= diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 4a112df014c9b..dc2755ab27c59 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -61,7 +61,7 @@ var ( ErrTableNotLockedForWrite = terror.ClassOptimizer.New(codeErrTableNotLockedForWrite, mysql.MySQLErrName[mysql.ErrTableNotLockedForWrite]) ErrTableNotLocked = terror.ClassOptimizer.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrTableNotLocked]) - ErrTableLocked = terror.ClassOptimizer.New(codeTableLocked, "Table '%s' was locked in %s by %v") + ErrTableLocked = terror.ClassOptimizer.New(codeTableLocked, "Table '%s' was locked in %s by %s") ) // InfoSchema is the interface used to retrieve the schema information. diff --git a/session/session.go b/session/session.go index 333824b6f11c0..bad1c55ee29ee 100644 --- a/session/session.go +++ b/session/session.go @@ -186,19 +186,13 @@ type session struct { // ddlOwnerChecker is used in `select tidb_is_ddl_owner()` statement; ddlOwnerChecker owner.DDLOwnerChecker // lockedTables use to record the table locks hold by the session. - lockedTables map[int64]tableLockInfo -} - -type tableLockInfo struct { - tp model.TableLockType - dbID int64 + lockedTables map[int64]model.TableLockTpInfo } // AddTableLock adds table lock to the session lock map. -func (s *session) AddTableLock(tblID, dbID int64, tp model.TableLockType) { - s.lockedTables[tblID] = tableLockInfo{ - tp: tp, - dbID: dbID, +func (s *session) AddTableLock(locks []model.TableLockTpInfo) { + for _, l := range locks { + s.lockedTables[l.TableID] = l } } @@ -208,24 +202,22 @@ func (s *session) CheckTableLocked(tblID int64) (bool, model.TableLockType) { if !ok { return false, model.TableLockNone } - return true, lt.tp + return true, lt.Tp } // GetAllTableLocks gets all table locks table id and db id hold by the session. -func (s *session) GetAllTableLocks() ([]int64, []int64) { - tbIDs := make([]int64, 0, len(s.lockedTables)) - dbIDs := make([]int64, 0, len(s.lockedTables)) - for id, tl := range s.lockedTables { - tbIDs = append(tbIDs, id) - dbIDs = append(dbIDs, tl.dbID) +func (s *session) GetAllTableLocks() []model.TableLockTpInfo { + lockTpInfo := make([]model.TableLockTpInfo, 0, len(s.lockedTables)) + for _, tl := range s.lockedTables { + lockTpInfo = append(lockTpInfo, tl) } - fmt.Printf("tb: %v, db: %v\n--------\n\n", tbIDs, dbIDs) - return tbIDs, dbIDs + fmt.Printf("dbTB: %v\n--------\n\n", lockTpInfo) + return lockTpInfo } // ReleaseAllTableLocks releases all table locks hold by the session. func (s *session) ReleaseAllTableLocks() { - s.lockedTables = make(map[int64]tableLockInfo) + s.lockedTables = make(map[int64]model.TableLockTpInfo) } // DDLOwnerChecker returns s.ddlOwnerChecker. @@ -1261,8 +1253,8 @@ func (s *session) ClearValue(key fmt.Stringer) { // Close should release the table locks which hold by the session. func (s *session) Close() { if len(s.lockedTables) > 0 { - tbIDs, dbIDs := s.GetAllTableLocks() - err := domain.GetDomain(s).DDL().UnlockTables(s, tbIDs, dbIDs) + lockedTables := s.GetAllTableLocks() + err := domain.GetDomain(s).DDL().UnlockTables(s, lockedTables) if err != nil { logutil.Logger(context.Background()).Error("release table lock failed", zap.Uint64("conn", s.sessionVars.ConnectionID)) } @@ -1516,7 +1508,7 @@ func createSession(store kv.Storage) (*session, error) { plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory) } s.mu.values = make(map[fmt.Stringer]interface{}) - s.lockedTables = make(map[int64]tableLockInfo) + s.lockedTables = make(map[int64]model.TableLockTpInfo) domain.BindDomain(s, dom) // session implements variable.GlobalVarAccessor. Bind it to ctx. s.sessionVars.GlobalVarsAccessor = s @@ -1540,7 +1532,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory) } s.mu.values = make(map[fmt.Stringer]interface{}) - s.lockedTables = make(map[int64]tableLockInfo) + s.lockedTables = make(map[int64]model.TableLockTpInfo) domain.BindDomain(s, dom) // session implements variable.GlobalVarAccessor. Bind it to ctx. s.sessionVars.GlobalVarsAccessor = s diff --git a/sessionctx/context.go b/sessionctx/context.go index 9fc640a4b0f3f..e651ea36fb678 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -84,11 +84,11 @@ type Context interface { // DDLOwnerChecker returns owner.DDLOwnerChecker. DDLOwnerChecker() owner.DDLOwnerChecker // AddTableLock adds table lock to the session lock map. - AddTableLock(tblID, dbID int64, tp model.TableLockType) + AddTableLock([]model.TableLockTpInfo) // CheckTableLocked checks the table lock. CheckTableLocked(tblID int64) (bool, model.TableLockType) - GetAllTableLocks() (tbIDs []int64, dbIDs []int64) + GetAllTableLocks() []model.TableLockTpInfo ReleaseAllTableLocks() } diff --git a/table/tables/tables.go b/table/tables/tables.go index 9a3861169fb34..fc8ff837f325c 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1206,7 +1206,7 @@ func (ctx *ctxForPartitionExpr) DDLOwnerChecker() owner.DDLOwnerChecker { } // AddTableLock implements the sessionctx.Context interface. -func (ctx *ctxForPartitionExpr) AddTableLock(_, _ int64, _ model.TableLockType) { +func (ctx *ctxForPartitionExpr) AddTableLock(_ []model.TableLockTpInfo) { } // CheckTableLocked implements the sessionctx.Context interface. @@ -1215,8 +1215,8 @@ func (ctx *ctxForPartitionExpr) CheckTableLocked(tblID int64) (bool, model.Table } // GetAllTableLocks implements the sessionctx.Context interface. -func (ctx *ctxForPartitionExpr) GetAllTableLocks() ([]int64, []int64) { - return nil, nil +func (ctx *ctxForPartitionExpr) GetAllTableLocks() []model.TableLockTpInfo { + return nil } // ReleaseAllTableLocks implements the sessionctx.Context interface. diff --git a/util/mock/context.go b/util/mock/context.go index 5255572190f71..bbde0cb38e752 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -217,17 +217,17 @@ func (c *Context) StmtAddDirtyTableOP(op int, tid int64, handle int64, row []typ } // AddTableLock implements the sessionctx.Context interface. -func (s *Context) AddTableLock(_, _ int64, _ model.TableLockType) { +func (s *Context) AddTableLock(_ []model.TableLockTpInfo) { } // CheckTableLocked implements the sessionctx.Context interface. -func (s *Context) CheckTableLocked(tblID int64) (bool, model.TableLockType) { +func (s *Context) CheckTableLocked(_ int64) (bool, model.TableLockType) { return false, model.TableLockNone } // GetAllTableLocks implements the sessionctx.Context interface. -func (s *Context) GetAllTableLocks() ([]int64, []int64) { - return nil, nil +func (s *Context) GetAllTableLocks() []model.TableLockTpInfo { + return nil } // ReleaseAllTableLocks implements the sessionctx.Context interface. From 7415cf7f6a390d240eb9baf6764fbcc30de2256e Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 3 May 2019 21:52:05 +0800 Subject: [PATCH 06/48] support lock multi-table lock but with diff schema version, todo: add check in sessino --- ddl/db_test.go | 20 ++++- ddl/ddl_api.go | 23 +++-- ddl/table_lock.go | 210 +++++++++++++++++++++++++++++++++------------- ddl/table_test.go | 10 ++- 4 files changed, 193 insertions(+), 70 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 16722f1885edb..a150fcee3aee2 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2755,15 +2755,31 @@ func (s *testDBSuite2) TestLockTables(c *C) { tk := s.tk tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - defer tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t1,t2") + defer tk.MustExec("drop table if exists t1,t2") tk.MustExec("create table t1 (a int)") + tk.MustExec("create table t2 (a int)") tk.MustExec("lock tables t1 write") checkTableLock(c, tk.Se, "test", "t1", model.TableLockWrite) tk.MustExec("lock tables t1 read") checkTableLock(c, tk.Se, "test", "t1", model.TableLockRead) + tk.MustExec("lock tables t1 write") + checkTableLock(c, tk.Se, "test", "t1", model.TableLockWrite) tk.MustExec("unlock tables") checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone) + + tk.MustExec("lock tables t1 write, t2 read") + checkTableLock(c, tk.Se, "test", "t1", model.TableLockWrite) + checkTableLock(c, tk.Se, "test", "t2", model.TableLockRead) + tk.MustExec("lock tables t1 read, t2 write") + checkTableLock(c, tk.Se, "test", "t1", model.TableLockRead) + checkTableLock(c, tk.Se, "test", "t2", model.TableLockWrite) + tk.MustExec("lock tables t2 write") + checkTableLock(c, tk.Se, "test", "t2", model.TableLockWrite) + checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone) + tk.MustExec("lock tables t1 write") + checkTableLock(c, tk.Se, "test", "t1", model.TableLockWrite) + checkTableLock(c, tk.Se, "test", "t2", model.TableLockNone) } func checkTableLock(c *C, se session.Session, dbName, tableName string, lockTp model.TableLockType) { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 75b6ac20cc8fb..9072fbe3c8741 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3129,8 +3129,10 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error arg := &lockTablesArg{ LockTables: lockTables, UnlockTables: unlockTables, - ServerID: d.GetID(), - SessionID: ctx.GetSessionVars().ConnectionID, + SessionInfo: model.SessionInfo{ + ServerID: d.GetID(), + SessionID: ctx.GetSessionVars().ConnectionID, + }, } job := &model.Job{ @@ -3140,6 +3142,8 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{arg}, } + // should check info schema lock. + ctx.ReleaseAllTableLocks() ctx.AddTableLock(lockTables) err := d.doDDLJob(ctx, job) err = d.callHookOnChanged(err) @@ -3152,8 +3156,10 @@ func (d *ddl) UnlockTables(ctx sessionctx.Context, unlockTables []model.TableLoc } arg := &lockTablesArg{ UnlockTables: unlockTables, - ServerID: d.GetID(), - SessionID: ctx.GetSessionVars().ConnectionID, + SessionInfo: model.SessionInfo{ + ServerID: d.GetID(), + SessionID: ctx.GetSessionVars().ConnectionID, + }, } job := &model.Job{ SchemaID: unlockTables[0].SchemaID, @@ -3172,8 +3178,9 @@ func (d *ddl) UnlockTables(ctx sessionctx.Context, unlockTables []model.TableLoc } type lockTablesArg struct { - LockTables []model.TableLockTpInfo - UnlockTables []model.TableLockTpInfo - ServerID string - SessionID uint64 + LockTables []model.TableLockTpInfo + IndexOfLock int + UnlockTables []model.TableLockTpInfo + IndexOfUnlock int + SessionInfo model.SessionInfo } diff --git a/ddl/table_lock.go b/ddl/table_lock.go index a629bd083ceb7..f83e4fd26b1a3 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -30,66 +30,126 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { } fmt.Printf("on lock table: arg: %#v\n---------\n\n", arg) - tbInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) - if err != nil { - return ver, err - } - if tbInfo.Lock == nil { - tbInfo.Lock = &model.TableLockInfo{} + // Unlock table first. + if arg.IndexOfUnlock < len(arg.UnlockTables) { + return unlockTableReq(t, job, arg) } - err = checkLockTable(tbInfo, 0, arg) - if err != nil { - job.State = model.JobStateCancelled - return ver, err + // Check table locked by other, this can be only checked at the first time. + if arg.IndexOfLock == 0 { + for i, tl := range arg.LockTables { + job.SchemaID = tl.SchemaID + job.TableID = tl.TableID + tbInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, err + } + err = checkTableLocked(tbInfo, i, arg) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + } + job.SchemaState = model.StateDeleteOnly } - switch tbInfo.Lock.State { - case model.TableLockStateNone: - // none -> pre_lock - tbInfo.Lock.State = model.TableLockStatePreLock - tbInfo.Lock.TS = t.StartTS - job.SchemaState = model.StateDeleteOnly - ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) - case model.TableLockStatePreLock, model.TableLockStatePublic: - tbInfo.Lock.State = model.TableLockStatePublic - tbInfo.Lock.TS = t.StartTS - ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) + // Lock table. + if arg.IndexOfLock < len(arg.LockTables) { + job.SchemaID = arg.LockTables[arg.IndexOfLock].SchemaID + job.TableID = arg.LockTables[arg.IndexOfLock].TableID + tbInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) if err != nil { - return ver, errors.Trace(err) + return ver, err } - // Finish this job. - job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) - default: - return ver, ErrInvalidTableLockState.GenWithStack("invalid table lock state %v", tbInfo.Lock.State) + err = checkAndLockTable(tbInfo, arg.IndexOfLock, arg) + if err != nil { + job.State = model.JobStateCancelled + return ver, err + } + + switch tbInfo.Lock.State { + case model.TableLockStateNone: + // none -> pre_lock + tbInfo.Lock.State = model.TableLockStatePreLock + tbInfo.Lock.TS = t.StartTS + ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) + case model.TableLockStatePreLock, model.TableLockStatePublic: + tbInfo.Lock.State = model.TableLockStatePublic + tbInfo.Lock.TS = t.StartTS + ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + arg.IndexOfLock++ + job.Args = []interface{}{arg} + if arg.IndexOfLock == len(arg.LockTables) { + // Finish this job. + // Do not use FinishTableJob function, no need to write binlog? + job.State = model.JobStateDone + job.SchemaState = model.StatePublic + } + default: + job.State = model.JobStateCancelled + return ver, ErrInvalidTableLockState.GenWithStack("invalid table lock state %v", tbInfo.Lock.State) + } } + return ver, err } -func checkLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error { +func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) error { + if tbInfo.Lock == nil { + return nil + } + sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg) + if sessionIndex < 0 { + return nil + // todo: when clean table lock , session maybe send unlock table even the table lock maybe not hold by the session. + //return errors.Errorf("%s isn't holding table %s lock", arg.SessionInfo, tbInfo.Name) + } + oldSessionInfo := tbInfo.Lock.Sessions + tbInfo.Lock.Sessions = oldSessionInfo[:sessionIndex] + tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, oldSessionInfo[sessionIndex+1:]...) + if len(tbInfo.Lock.Sessions) == 0 { + tbInfo.Lock = nil + } + return nil +} + +// +func indexOfLockHolder(sessions []model.SessionInfo, arg *lockTablesArg) int { + for i := range sessions { + if sessions[i].ServerID == arg.SessionInfo.ServerID && sessions[i].SessionID == arg.SessionInfo.SessionID { + return i + } + } + return -1 +} + +func checkAndLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error { if tbInfo.Lock == nil || len(tbInfo.Lock.Sessions) == 0 { tbInfo.Lock = &model.TableLockInfo{ Tp: arg.LockTables[idx].Tp, } - tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, model.SessionInfo{ServerID: arg.ServerID, SessionID: arg.SessionID}) + tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, arg.SessionInfo) return nil } if tbInfo.Lock.State == model.TableLockStatePreLock { return nil } if tbInfo.Lock.Tp == model.TableLockRead && arg.LockTables[idx].Tp == model.TableLockRead { - contain := hasServerAndSessionID(tbInfo.Lock.Sessions, arg.ServerID, arg.SessionID) + sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg) // repeat lock. - if contain { + if sessionIndex >= 0 { return nil } - tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, model.SessionInfo{ServerID: arg.ServerID, SessionID: arg.SessionID}) + tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, arg.SessionInfo) return nil } - contain := hasServerAndSessionID(tbInfo.Lock.Sessions, arg.ServerID, arg.SessionID) + sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg) // repeat lock. - if contain { + if sessionIndex >= 0 { if tbInfo.Lock.Tp == arg.LockTables[idx].Tp { return nil } @@ -98,42 +158,80 @@ func checkLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error tbInfo.Lock.Tp = arg.LockTables[idx].Tp return nil } - - // todo: release lock. - return infoschema.ErrTableLocked.GenWithStackByArgs(tbInfo.Name.L, tbInfo.Lock.Tp, tbInfo.Lock.Sessions[0]) } + return infoschema.ErrTableLocked.GenWithStackByArgs(tbInfo.Name.L, tbInfo.Lock.Tp, tbInfo.Lock.Sessions[0]) +} +func checkTableLocked(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error { + if tbInfo.Lock == nil || len(tbInfo.Lock.Sessions) == 0 { + tbInfo.Lock = &model.TableLockInfo{ + Tp: arg.LockTables[idx].Tp, + } + return nil + } + // remove this? + if tbInfo.Lock.State == model.TableLockStatePreLock { + return nil + } + if tbInfo.Lock.Tp == model.TableLockRead && arg.LockTables[idx].Tp == model.TableLockRead { + return nil + } + sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg) + // repeat lock. + if sessionIndex >= 0 { + if tbInfo.Lock.Tp == arg.LockTables[idx].Tp { + return nil + } + if len(tbInfo.Lock.Sessions) == 1 { + // just change lock tp directly. + return nil + } + } return infoschema.ErrTableLocked.GenWithStackByArgs(tbInfo.Name.L, tbInfo.Lock.Tp, tbInfo.Lock.Sessions[0]) } +func unlockTableReq(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, err error) { + // Unlock table first. + if arg.IndexOfUnlock < len(arg.UnlockTables) { + job.SchemaID = arg.UnlockTables[arg.IndexOfUnlock].SchemaID + job.TableID = arg.UnlockTables[arg.IndexOfUnlock].TableID + tbInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, err + } + err = unlockTable(tbInfo, arg) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + arg.IndexOfUnlock++ + job.Args = []interface{}{arg} + job.SchemaState = model.StateDeleteOnly + } + return ver, nil +} + func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { - lockTablesArg := &lockTablesArg{} - if err := job.DecodeArgs(lockTablesArg); err != nil { + arg := &lockTablesArg{} + if err := job.DecodeArgs(arg); err != nil { // Invalid arguments, cancel this job. job.State = model.JobStateCancelled return ver, errors.Trace(err) } - fmt.Printf("on unlock table: arg: %#v\n---------\n\n", lockTablesArg) + fmt.Printf("on unlock table: arg: %#v\n---------\n\n", arg) - tbInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) - if err != nil { - return ver, err - } - // Nothing need to do. - if tbInfo.Lock == nil { - // should never run to this. - job.State = model.JobStateCancelled - return ver, errors.Errorf("the lock of table %v was released, this should never hapen", tbInfo.Name.L) - } + ver, err = unlockTableReq(t, job, arg) - tbInfo.Lock = nil - ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) - if err != nil { - return ver, errors.Trace(err) + if arg.IndexOfUnlock == len(arg.UnlockTables) { + // Do not use FinishTableJob function, no need to write binlog? + job.State = model.JobStateDone + job.SchemaState = model.StateNone } - // Finish this job. - job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) - return ver, nil + return ver, err } func hasServerAndSessionID(sessions []model.SessionInfo, serverID string, sessionID uint64) bool { diff --git a/ddl/table_test.go b/ddl/table_test.go index c3ca00adfa43c..d49695b64e027 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -181,8 +181,10 @@ func testRenameTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID, oldSchem func testLockTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID int64, tblInfo *model.TableInfo, lockTp model.TableLockType) *model.Job { arg := &lockTablesArg{ LockTables: []model.TableLockTpInfo{{SchemaID: newSchemaID, TableID: tblInfo.ID, Tp: lockTp}}, - ServerID: d.GetID(), - SessionID: ctx.GetSessionVars().ConnectionID, + SessionInfo: model.SessionInfo{ + ServerID: d.GetID(), + SessionID: ctx.GetSessionVars().ConnectionID, + }, } job := &model.Job{ SchemaID: newSchemaID, @@ -199,7 +201,7 @@ func testLockTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID int64, tblI return job } -func checkTableLocked(c *C, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, serverID string, sessionID uint64, lockTp model.TableLockType) { +func checkTableLockedTest(c *C, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, serverID string, sessionID uint64, lockTp model.TableLockType) { err := kv.RunInNewTxn(d.store, false, func(txn kv.Transaction) error { t := meta.NewMeta(txn) info, err := t.GetTable(dbInfo.ID, tblInfo.ID) @@ -358,7 +360,7 @@ func (s *testTableSuite) TestTable(c *C) { testCheckTableState(c, d, dbInfo1, tblInfo, model.StatePublic) testCheckJobDone(c, d, job, true) - checkTableLocked(c, d, dbInfo1, tblInfo, d.GetID(), ctx.GetSessionVars().ConnectionID, model.TableLockWrite) + checkTableLockedTest(c, d, dbInfo1, tblInfo, d.GetID(), ctx.GetSessionVars().ConnectionID, model.TableLockWrite) } func (s *testTableSuite) TestTableResume(c *C) { From 07c721c9555d27709db6bae09ab3e665c50d79b4 Mon Sep 17 00:00:00 2001 From: crazycs Date: Sat, 4 May 2019 16:03:59 +0800 Subject: [PATCH 07/48] add check after privilege check --- ddl/ddl_api.go | 13 ++++-- ddl/table_lock.go | 31 ++++++-------- ddl/table_test.go | 2 +- go.mod | 2 +- go.sum | 20 +-------- infoschema/infoschema.go | 2 +- lock/lock.go | 82 +++++++++++++++++++++++++++++++++++++ planner/core/optimizer.go | 13 ++++++ planner/core/planbuilder.go | 8 +--- planner/optimize.go | 4 ++ session/session.go | 7 +++- sessionctx/context.go | 2 + table/tables/tables.go | 5 +++ util/mock/context.go | 5 +++ 14 files changed, 144 insertions(+), 52 deletions(-) create mode 100644 lock/lock.go diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 9072fbe3c8741..9635bd63aa317 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3111,6 +3111,10 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error lockTables := make([]model.TableLockTpInfo, 0, len(stmt.TableLocks)) is := d.infoHandle.Get() + sessionInfo := model.SessionInfo{ + ServerID: d.GetID(), + SessionID: ctx.GetSessionVars().ConnectionID, + } for _, tl := range stmt.TableLocks { tb := tl.Table schema, ok := is.SchemaByName(tb.Schema) @@ -3121,6 +3125,10 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error if err != nil { return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(tb.Schema, tb.Name)) } + err = checkTableLocked(t.Meta(), tl.Type, sessionInfo) + if err != nil { + return err + } lockTables = append(lockTables, model.TableLockTpInfo{SchemaID: schema.ID, TableID: t.Meta().ID, Tp: tl.Type}) } @@ -3129,10 +3137,7 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error arg := &lockTablesArg{ LockTables: lockTables, UnlockTables: unlockTables, - SessionInfo: model.SessionInfo{ - ServerID: d.GetID(), - SessionID: ctx.GetSessionVars().ConnectionID, - }, + SessionInfo: sessionInfo, } job := &model.Job{ diff --git a/ddl/table_lock.go b/ddl/table_lock.go index f83e4fd26b1a3..72715add25651 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -44,7 +44,7 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { if err != nil { return ver, err } - err = checkTableLocked(tbInfo, i, arg) + err = checkTableLocked(tbInfo, arg.LockTables[i].Tp, arg.SessionInfo) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -84,9 +84,7 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { job.Args = []interface{}{arg} if arg.IndexOfLock == len(arg.LockTables) { // Finish this job. - // Do not use FinishTableJob function, no need to write binlog? - job.State = model.JobStateDone - job.SchemaState = model.StatePublic + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, nil) } default: job.State = model.JobStateCancelled @@ -102,7 +100,7 @@ func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) error { if tbInfo.Lock == nil { return nil } - sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg) + sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg.SessionInfo) if sessionIndex < 0 { return nil // todo: when clean table lock , session maybe send unlock table even the table lock maybe not hold by the session. @@ -118,9 +116,9 @@ func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) error { } // -func indexOfLockHolder(sessions []model.SessionInfo, arg *lockTablesArg) int { +func indexOfLockHolder(sessions []model.SessionInfo, sessionInfo model.SessionInfo) int { for i := range sessions { - if sessions[i].ServerID == arg.SessionInfo.ServerID && sessions[i].SessionID == arg.SessionInfo.SessionID { + if sessions[i].ServerID == sessionInfo.ServerID && sessions[i].SessionID == sessionInfo.SessionID { return i } } @@ -139,7 +137,7 @@ func checkAndLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) err return nil } if tbInfo.Lock.Tp == model.TableLockRead && arg.LockTables[idx].Tp == model.TableLockRead { - sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg) + sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg.SessionInfo) // repeat lock. if sessionIndex >= 0 { return nil @@ -147,7 +145,7 @@ func checkAndLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) err tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, arg.SessionInfo) return nil } - sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg) + sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg.SessionInfo) // repeat lock. if sessionIndex >= 0 { if tbInfo.Lock.Tp == arg.LockTables[idx].Tp { @@ -162,24 +160,21 @@ func checkAndLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) err return infoschema.ErrTableLocked.GenWithStackByArgs(tbInfo.Name.L, tbInfo.Lock.Tp, tbInfo.Lock.Sessions[0]) } -func checkTableLocked(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error { +func checkTableLocked(tbInfo *model.TableInfo, lockTp model.TableLockType, sessionInfo model.SessionInfo) error { if tbInfo.Lock == nil || len(tbInfo.Lock.Sessions) == 0 { - tbInfo.Lock = &model.TableLockInfo{ - Tp: arg.LockTables[idx].Tp, - } return nil } // remove this? if tbInfo.Lock.State == model.TableLockStatePreLock { return nil } - if tbInfo.Lock.Tp == model.TableLockRead && arg.LockTables[idx].Tp == model.TableLockRead { + if tbInfo.Lock.Tp == model.TableLockRead && lockTp == model.TableLockRead { return nil } - sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg) + sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, sessionInfo) // repeat lock. if sessionIndex >= 0 { - if tbInfo.Lock.Tp == arg.LockTables[idx].Tp { + if tbInfo.Lock.Tp == lockTp { return nil } if len(tbInfo.Lock.Sessions) == 1 { @@ -227,9 +222,7 @@ func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { ver, err = unlockTableReq(t, job, arg) if arg.IndexOfUnlock == len(arg.UnlockTables) { - // Do not use FinishTableJob function, no need to write binlog? - job.State = model.JobStateDone - job.SchemaState = model.StateNone + job.FinishTableJob(model.JobStateDone, model.StateNone, ver, nil) } return ver, err } diff --git a/ddl/table_test.go b/ddl/table_test.go index d49695b64e027..4b37a8a0945d6 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -197,7 +197,7 @@ func testLockTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID int64, tblI c.Assert(err, IsNil) v := getSchemaVer(c, ctx) - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v}) return job } diff --git a/go.mod b/go.mod index bb51f348b43e8..f90aa60f5f1c9 100644 --- a/go.mod +++ b/go.mod @@ -73,4 +73,4 @@ require ( sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190430053142-0f4907461312 +replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190504080252-5fa8335e1490 diff --git a/go.sum b/go.sum index 23900f4336a0f..41abb315f7093 100644 --- a/go.sum +++ b/go.sum @@ -30,24 +30,8 @@ github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142/go.mod h1:F5haX7 github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbpBpLoyyu8B6e44T7hJy6potg= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= -github.com/crazycs520/parser v0.0.0-20190429053203-8106f70e326f h1:bVWi7q/REj0dmZMautvgv/SjRPDZHwlSpkerXNIjGmg= -github.com/crazycs520/parser v0.0.0-20190429053203-8106f70e326f/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= -github.com/crazycs520/parser v0.0.0-20190429063023-191de28158f7 h1:TE6vjepIgrvpjtUNRHODoBlNgSVUiKXSsJdrnTcBzGQ= -github.com/crazycs520/parser v0.0.0-20190429063023-191de28158f7/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= -github.com/crazycs520/parser v0.0.0-20190429080204-707399bd8341 h1:zvEf1pPdUA720KwxQ7zGScYFsOSNfcjAtRECxuWr9R4= -github.com/crazycs520/parser v0.0.0-20190429080204-707399bd8341/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= -github.com/crazycs520/parser v0.0.0-20190429091436-934a2b54c6e0 h1:nsSKc7e7bjda9aJZm+xwaEEZJbAF253H1dcd5QResoU= -github.com/crazycs520/parser v0.0.0-20190429091436-934a2b54c6e0/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= -github.com/crazycs520/parser v0.0.0-20190429094900-82615a39430f h1:qe5g2qkOnvcfYZ8y+AxgJs9S3ALkJzlp8Y6jaLyUkG0= -github.com/crazycs520/parser v0.0.0-20190429094900-82615a39430f/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= -github.com/crazycs520/parser v0.0.0-20190430020852-5cdecbe3e166 h1:4iHvnr5NAsbIpzoK6EGxUfURSQaNPwH6BoN9Q96mcvs= -github.com/crazycs520/parser v0.0.0-20190430020852-5cdecbe3e166/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= -github.com/crazycs520/parser v0.0.0-20190430032010-a4525fb1844c h1:7NscklE3FOuQyr9XeqVvroma+RZ/5RlRttEQyZ8O1JE= -github.com/crazycs520/parser v0.0.0-20190430032010-a4525fb1844c/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= -github.com/crazycs520/parser v0.0.0-20190430034757-798dc77c7d76 h1:1em8yDP3h8xnwCgiAxhY2jCpAbD50UEkMZYCM82L1iE= -github.com/crazycs520/parser v0.0.0-20190430034757-798dc77c7d76/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= -github.com/crazycs520/parser v0.0.0-20190430053142-0f4907461312 h1:W1OI+Z88CBsOVzIDl5bcxB9EOIPsRQS9AnJynldNBBw= -github.com/crazycs520/parser v0.0.0-20190430053142-0f4907461312/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= +github.com/crazycs520/parser v0.0.0-20190504080252-5fa8335e1490 h1:OEFMpdaGkCJ+PIXlNYJbKqYnGBzBC/F/UDTwFbQiDFI= +github.com/crazycs520/parser v0.0.0-20190504080252-5fa8335e1490/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= github.com/cznic/sortutil v0.0.0-20150617083342-4c7342852e65 h1:hxuZop6tSoOi0sxFzoGGYdRqNrPubyaIf9KoBG9tPiE= diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index dc2755ab27c59..4a112df014c9b 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -61,7 +61,7 @@ var ( ErrTableNotLockedForWrite = terror.ClassOptimizer.New(codeErrTableNotLockedForWrite, mysql.MySQLErrName[mysql.ErrTableNotLockedForWrite]) ErrTableNotLocked = terror.ClassOptimizer.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrTableNotLocked]) - ErrTableLocked = terror.ClassOptimizer.New(codeTableLocked, "Table '%s' was locked in %s by %s") + ErrTableLocked = terror.ClassOptimizer.New(codeTableLocked, "Table '%s' was locked in %s by %v") ) // InfoSchema is the interface used to retrieve the schema information. diff --git a/lock/lock.go b/lock/lock.go new file mode 100644 index 0000000000000..6777dd07f96df --- /dev/null +++ b/lock/lock.go @@ -0,0 +1,82 @@ +// Copyright 2016 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package lock + +import ( + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/sessionctx" +) + +type LockChecker struct { + ctx sessionctx.Context + is infoschema.InfoSchema +} + +func NewLockChecker(ctx sessionctx.Context, is infoschema.InfoSchema) *LockChecker { + return &LockChecker{ctx: ctx, is: is} +} + +func (c *LockChecker) CheckTableLock(db, table string, privilege mysql.PrivilegeType) error { + if db == "" || table == "" { + return nil + } + switch privilege { + case mysql.CreatePriv: + return nil + } + tb, err := c.is.TableByName(model.NewCIStr(db), model.NewCIStr(table)) + // TODO: remove this? + if infoschema.ErrTableNotExists.Equal(err) { + return nil + } + if err != nil { + return err + } + if c.ctx.HasLockedTables() { + if locked, tp := c.ctx.CheckTableLocked(tb.Meta().ID); locked { + if checkLockTpMeetPrivilege(tp, privilege) { + return nil + } + return infoschema.ErrTableNotLockedForWrite.GenWithStackByArgs(tb.Meta().Name) + } + return infoschema.ErrTableNotLocked.GenWithStackByArgs(tb.Meta().Name) + } + + if tb.Meta().Lock == nil { + return nil + } + + switch privilege { + case mysql.SelectPriv: + switch tb.Meta().Lock.Tp { + case model.TableLockRead, model.TableLockWriteLocal: + return nil + } + } + return infoschema.ErrTableLocked.GenWithStackByArgs(tb.Meta().Name.L, tb.Meta().Lock.Tp, tb.Meta().Lock.Sessions[0]) +} + +func checkLockTpMeetPrivilege(tp model.TableLockType, privilege mysql.PrivilegeType) bool { + switch tp { + case model.TableLockWrite, model.TableLockWriteLocal: + return true + case model.TableLockRead: + if privilege == mysql.SelectPriv { + return true + } + } + return false +} diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 1baf596854a66..66ebee155adb7 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -14,6 +14,7 @@ package core import ( + "github.com/pingcap/tidb/lock" "math" "github.com/pingcap/errors" @@ -96,6 +97,18 @@ func CheckPrivilege(activeRoles []*auth.RoleIdentity, pm privilege.Manager, vs [ return nil } +// CheckTableLock checks the table lock. +func CheckTableLock(ctx sessionctx.Context, is infoschema.InfoSchema, vs []visitInfo) error { + checker := lock.NewLockChecker(ctx, is) + for i := range vs { + err := checker.CheckTableLock(vs[i].db, vs[i].table, vs[i].privilege) + if err != nil { + return err + } + } + return nil +} + // DoOptimize optimizes a logical plan to a physical plan. func DoOptimize(flag uint64, logic LogicalPlan) (PhysicalPlan, error) { logic, err := logicalOptimize(flag, logic) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 01eecc1fe13fa..c4ffa734ecec5 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1751,13 +1751,7 @@ func (b *PlanBuilder) buildDDL(node ast.DDLNode) (Plan, error) { // Recover table command can only be executed by administrator. b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil) case *ast.LockTablesStmt: - for _, tl := range v.TableLocks { - if tl.Type == model.TableLockRead { - b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, tl.Table.Schema.L, tl.Table.Name.L, "", infoschema.ErrTableLocked) - } else { - b.visitInfo = appendVisitInfo(b.visitInfo, mysql.UpdatePriv, tl.Table.Schema.L, tl.Table.Name.L, "", infoschema.ErrTableLocked) - } - } + // do nothing. } p := &DDL{Statement: node} return p, nil diff --git a/planner/optimize.go b/planner/optimize.go index 820983d505b97..e658899b0efaa 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -50,6 +50,10 @@ func Optimize(ctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) ( } } + if err := plannercore.CheckTableLock(ctx, is, builder.GetVisitInfo()); err != nil { + return nil, err + } + // Handle the execute statement. if execPlan, ok := p.(*plannercore.Execute); ok { err := execPlan.OptimizePreparedPlan(ctx, is) diff --git a/session/session.go b/session/session.go index bad1c55ee29ee..1c3bd32afc553 100644 --- a/session/session.go +++ b/session/session.go @@ -211,10 +211,15 @@ func (s *session) GetAllTableLocks() []model.TableLockTpInfo { for _, tl := range s.lockedTables { lockTpInfo = append(lockTpInfo, tl) } - fmt.Printf("dbTB: %v\n--------\n\n", lockTpInfo) return lockTpInfo } +// HasLockedTables uses to check whether this session locked any tables. +// If so, the session can only visit the table which locked by self. +func (s *session) HasLockedTables() bool { + return len(s.lockedTables) > 0 +} + // ReleaseAllTableLocks releases all table locks hold by the session. func (s *session) ReleaseAllTableLocks() { s.lockedTables = make(map[int64]model.TableLockTpInfo) diff --git a/sessionctx/context.go b/sessionctx/context.go index e651ea36fb678..5fed00a9196d6 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -91,6 +91,8 @@ type Context interface { GetAllTableLocks() []model.TableLockTpInfo ReleaseAllTableLocks() + + HasLockedTables() bool } type basicCtxType int diff --git a/table/tables/tables.go b/table/tables/tables.go index fc8ff837f325c..b19a84b9a8a89 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1223,6 +1223,11 @@ func (ctx *ctxForPartitionExpr) GetAllTableLocks() []model.TableLockTpInfo { func (ctx *ctxForPartitionExpr) ReleaseAllTableLocks() { } +// HasLockedTables implements the sessionctx.Context interface. +func (s *ctxForPartitionExpr) HasLockedTables() bool { + return true +} + // Close implements the sessionctx.Context interface. func (ctx *ctxForPartitionExpr) Close() { } diff --git a/util/mock/context.go b/util/mock/context.go index bbde0cb38e752..7b353e2a52340 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -234,6 +234,11 @@ func (s *Context) GetAllTableLocks() []model.TableLockTpInfo { func (s *Context) ReleaseAllTableLocks() { } +// HasLockedTables implements the sessionctx.Context interface. +func (s *Context) HasLockedTables() bool { + return true +} + // Close implements the sessionctx.Context interface. func (s *Context) Close() { } From d7f73a6a95791309c5f6ff3ef0fa702fca6c8768 Mon Sep 17 00:00:00 2001 From: crazycs Date: Sun, 5 May 2019 11:46:21 +0800 Subject: [PATCH 08/48] fix unit test --- ddl/db_test.go | 3 +++ ddl/table_lock.go | 3 --- planner/core/logical_plan_test.go | 1 - table/tables/tables.go | 2 +- util/mock/context.go | 2 +- 5 files changed, 5 insertions(+), 6 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 2925fd380e45f..0ab030e21a01a 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2780,6 +2780,9 @@ func (s *testDBSuite2) TestLockTables(c *C) { tk.MustExec("lock tables t1 write") checkTableLock(c, tk.Se, "test", "t1", model.TableLockWrite) checkTableLock(c, tk.Se, "test", "t2", model.TableLockNone) + tk.MustExec("unlock tables") + checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone) + checkTableLock(c, tk.Se, "test", "t2", model.TableLockNone) } func checkTableLock(c *C, se session.Session, dbName, tableName string, lockTp model.TableLockType) { diff --git a/ddl/table_lock.go b/ddl/table_lock.go index 72715add25651..18ff38056db0f 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -14,7 +14,6 @@ package ddl import ( - "fmt" "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/infoschema" @@ -28,7 +27,6 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { job.State = model.JobStateCancelled return ver, errors.Trace(err) } - fmt.Printf("on lock table: arg: %#v\n---------\n\n", arg) // Unlock table first. if arg.IndexOfUnlock < len(arg.UnlockTables) { @@ -217,7 +215,6 @@ func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { job.State = model.JobStateCancelled return ver, errors.Trace(err) } - fmt.Printf("on unlock table: arg: %#v\n---------\n\n", arg) ver, err = unlockTableReq(t, job, arg) diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 2af974ba048c8..359884315980b 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -864,7 +864,6 @@ func (s *testPlanSuite) TestPlanBuilder(c *C) { comment := Commentf("for %s", ca.sql) stmt, err := s.ParseOneStmt(ca.sql, "", "") c.Assert(err, IsNil, comment) - s.ctx.GetSessionVars().HashJoinConcurrency = 1 Preprocess(s.ctx, stmt, s.is) p, err := BuildLogicalPlan(s.ctx, stmt, s.is) diff --git a/table/tables/tables.go b/table/tables/tables.go index b19a84b9a8a89..5933c87a74d23 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1225,7 +1225,7 @@ func (ctx *ctxForPartitionExpr) ReleaseAllTableLocks() { // HasLockedTables implements the sessionctx.Context interface. func (s *ctxForPartitionExpr) HasLockedTables() bool { - return true + return false } // Close implements the sessionctx.Context interface. diff --git a/util/mock/context.go b/util/mock/context.go index 7b353e2a52340..cbfdcd0f0914f 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -236,7 +236,7 @@ func (s *Context) ReleaseAllTableLocks() { // HasLockedTables implements the sessionctx.Context interface. func (s *Context) HasLockedTables() bool { - return true + return false } // Close implements the sessionctx.Context interface. From 68b611dad147282373e0c45b57ffaad97bf71454 Mon Sep 17 00:00:00 2001 From: crazycs Date: Sun, 5 May 2019 13:08:08 +0800 Subject: [PATCH 09/48] refine code --- ddl/ddl_api.go | 14 ++++---- ddl/ddl_worker.go | 1 - ddl/table_lock.go | 58 ++++++++++++------------------- infoschema/infoschema.go | 8 +++-- lock/lock.go | 4 +-- planner/core/logical_plan_test.go | 1 + planner/core/optimizer.go | 2 +- session/session.go | 7 ++++ sessionctx/context.go | 9 +++-- table/tables/tables.go | 4 +++ util/mock/context.go | 6 +++- 11 files changed, 61 insertions(+), 53 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 393c531518553..a5142f16d8f57 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3122,16 +3122,13 @@ func buildPartitionInfo(meta *model.TableInfo, d *ddl, spec *ast.AlterTableSpec) } func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error { - //if len(stmt.TableLocks) > 1 { - // return errors.New("Currently only support lock 1 table at the same time") - //} - lockTables := make([]model.TableLockTpInfo, 0, len(stmt.TableLocks)) is := d.infoHandle.Get() sessionInfo := model.SessionInfo{ ServerID: d.GetID(), SessionID: ctx.GetSessionVars().ConnectionID, } + // Check whether the table was already locked by other. for _, tl := range stmt.TableLocks { tb := tl.Table schema, ok := is.SchemaByName(tb.Schema) @@ -3150,7 +3147,6 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error } unlockTables := ctx.GetAllTableLocks() - arg := &lockTablesArg{ LockTables: lockTables, UnlockTables: unlockTables, @@ -3164,10 +3160,14 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{arg}, } - // should check info schema lock. - ctx.ReleaseAllTableLocks() + // AddTableLock here to avoid this ddl job was execute successful but the session was been kill before return. ctx.AddTableLock(lockTables) err := d.doDDLJob(ctx, job) + if err == nil { + // TODO: add mutex to avoid concurrency problem. + ctx.ReleaseTableLock(unlockTables) + ctx.AddTableLock(lockTables) + } err = d.callHookOnChanged(err) return errors.Trace(err) } diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index f5c23f365174e..eb0550be31173 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -538,7 +538,6 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, ver, err = onLockTables(t, job) case model.ActionUnlockTable: ver, err = onUnlockTables(t, job) - default: // Invalid job, cancel it. job.State = model.JobStateCancelled diff --git a/ddl/table_lock.go b/ddl/table_lock.go index 18ff38056db0f..63b5aafc3709f 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -1,4 +1,4 @@ -// Copyright 2015 PingCAP, Inc. +// Copyright 2019 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -51,7 +51,7 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { job.SchemaState = model.StateDeleteOnly } - // Lock table. + // Lock tables. if arg.IndexOfLock < len(arg.LockTables) { job.SchemaID = arg.LockTables[arg.IndexOfLock].SchemaID job.TableID = arg.LockTables[arg.IndexOfLock].TableID @@ -87,7 +87,6 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { default: job.State = model.JobStateCancelled return ver, ErrInvalidTableLockState.GenWithStack("invalid table lock state %v", tbInfo.Lock.State) - } } @@ -113,7 +112,7 @@ func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) error { return nil } -// +// indexOfLockHolder gets the index of sessionInfo in the sessions. return -1 if sessions doesn't contain sessionInfo. func indexOfLockHolder(sessions []model.SessionInfo, sessionInfo model.SessionInfo) int { for i := range sessions { if sessions[i].ServerID == sessionInfo.ServerID && sessions[i].SessionID == sessionInfo.SessionID { @@ -162,7 +161,6 @@ func checkTableLocked(tbInfo *model.TableInfo, lockTp model.TableLockType, sessi if tbInfo.Lock == nil || len(tbInfo.Lock.Sessions) == 0 { return nil } - // remove this? if tbInfo.Lock.State == model.TableLockStatePreLock { return nil } @@ -176,7 +174,6 @@ func checkTableLocked(tbInfo *model.TableInfo, lockTp model.TableLockType, sessi return nil } if len(tbInfo.Lock.Sessions) == 1 { - // just change lock tp directly. return nil } } @@ -184,27 +181,27 @@ func checkTableLocked(tbInfo *model.TableInfo, lockTp model.TableLockType, sessi } func unlockTableReq(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, err error) { - // Unlock table first. - if arg.IndexOfUnlock < len(arg.UnlockTables) { - job.SchemaID = arg.UnlockTables[arg.IndexOfUnlock].SchemaID - job.TableID = arg.UnlockTables[arg.IndexOfUnlock].TableID - tbInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) - if err != nil { - return ver, err - } - err = unlockTable(tbInfo, arg) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) - if err != nil { - return ver, errors.Trace(err) - } - arg.IndexOfUnlock++ - job.Args = []interface{}{arg} - job.SchemaState = model.StateDeleteOnly + if arg.IndexOfUnlock >= len(arg.UnlockTables) { + return ver, nil } + job.SchemaID = arg.UnlockTables[arg.IndexOfUnlock].SchemaID + job.TableID = arg.UnlockTables[arg.IndexOfUnlock].TableID + tbInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, err + } + err = unlockTable(tbInfo, arg) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + arg.IndexOfUnlock++ + job.Args = []interface{}{arg} + job.SchemaState = model.StateDeleteOnly return ver, nil } @@ -223,12 +220,3 @@ func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { } return ver, err } - -func hasServerAndSessionID(sessions []model.SessionInfo, serverID string, sessionID uint64) bool { - for i := range sessions { - if sessions[i].ServerID == serverID && sessions[i].SessionID == sessionID { - return true - } - } - return false -} diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 4a112df014c9b..25595d914d045 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -58,10 +58,12 @@ var ( ErrMultiplePriKey = terror.ClassSchema.New(codeMultiplePriKey, "Multiple primary key defined") // ErrTooManyKeyParts returns for too many key parts. ErrTooManyKeyParts = terror.ClassSchema.New(codeTooManyKeyParts, "Too many key parts specified; max %d parts allowed") - + // ErrTableNotLockedForWrite returns for write tables when only hold the table read lock. ErrTableNotLockedForWrite = terror.ClassOptimizer.New(codeErrTableNotLockedForWrite, mysql.MySQLErrName[mysql.ErrTableNotLockedForWrite]) - ErrTableNotLocked = terror.ClassOptimizer.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrTableNotLocked]) - ErrTableLocked = terror.ClassOptimizer.New(codeTableLocked, "Table '%s' was locked in %s by %v") + // ErrTableNotLocked returns when session has explicitly lock tables, then visit unlocked table will return this error. + ErrTableNotLocked = terror.ClassOptimizer.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrTableNotLocked]) + // ErrTableLocked returns when the table was locked by other session. + ErrTableLocked = terror.ClassOptimizer.New(codeTableLocked, "Table '%s' was locked in %s by %v") ) // InfoSchema is the interface used to retrieve the schema information. diff --git a/lock/lock.go b/lock/lock.go index 6777dd07f96df..d782b2f6e9e10 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -1,4 +1,4 @@ -// Copyright 2016 PingCAP, Inc. +// Copyright 2019 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -38,7 +38,7 @@ func (c *LockChecker) CheckTableLock(db, table string, privilege mysql.Privilege return nil } tb, err := c.is.TableByName(model.NewCIStr(db), model.NewCIStr(table)) - // TODO: remove this? + // TODO: remove this if infoschema.ErrTableNotExists.Equal(err) { return nil } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 359884315980b..2af974ba048c8 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -864,6 +864,7 @@ func (s *testPlanSuite) TestPlanBuilder(c *C) { comment := Commentf("for %s", ca.sql) stmt, err := s.ParseOneStmt(ca.sql, "", "") c.Assert(err, IsNil, comment) + s.ctx.GetSessionVars().HashJoinConcurrency = 1 Preprocess(s.ctx, stmt, s.is) p, err := BuildLogicalPlan(s.ctx, stmt, s.is) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 5fc89e7154736..9416bfda8229e 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -14,7 +14,6 @@ package core import ( - "github.com/pingcap/tidb/lock" "math" "github.com/pingcap/errors" @@ -22,6 +21,7 @@ import ( "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/lock" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" diff --git a/session/session.go b/session/session.go index db48bca69ce02..34a3715b5c22b 100644 --- a/session/session.go +++ b/session/session.go @@ -196,6 +196,13 @@ func (s *session) AddTableLock(locks []model.TableLockTpInfo) { } } +// ReleaseTableLock releases table lock in the session lock map. +func (s *session) ReleaseTableLock(locks []model.TableLockTpInfo) { + for _, l := range locks { + delete(s.lockedTables, l.TableID) + } +} + // CheckTableLocked checks the table lock. func (s *session) CheckTableLocked(tblID int64) (bool, model.TableLockType) { lt, ok := s.lockedTables[tblID] diff --git a/sessionctx/context.go b/sessionctx/context.go index 5fed00a9196d6..9ae13a544897b 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -16,6 +16,7 @@ package sessionctx import ( "context" "fmt" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/owner" @@ -85,13 +86,15 @@ type Context interface { DDLOwnerChecker() owner.DDLOwnerChecker // AddTableLock adds table lock to the session lock map. AddTableLock([]model.TableLockTpInfo) + // ReleaseTableLock releases table lock in the session lock map. + ReleaseTableLock(locks []model.TableLockTpInfo) // CheckTableLocked checks the table lock. CheckTableLocked(tblID int64) (bool, model.TableLockType) - + // GetAllTableLocks gets all table locks table id and db id hold by the session. GetAllTableLocks() []model.TableLockTpInfo - + // ReleaseAllTableLocks releases all table locks hold by the session. ReleaseAllTableLocks() - + // HasLockedTables uses to check whether this session locked any tables. HasLockedTables() bool } diff --git a/table/tables/tables.go b/table/tables/tables.go index 5933c87a74d23..ae6533bc8e0d9 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1209,6 +1209,10 @@ func (ctx *ctxForPartitionExpr) DDLOwnerChecker() owner.DDLOwnerChecker { func (ctx *ctxForPartitionExpr) AddTableLock(_ []model.TableLockTpInfo) { } +// ReleaseTableLock implements the sessionctx.Context interface. +func (s *ctxForPartitionExpr) ReleaseTableLock(locks []model.TableLockTpInfo) { +} + // CheckTableLocked implements the sessionctx.Context interface. func (ctx *ctxForPartitionExpr) CheckTableLocked(tblID int64) (bool, model.TableLockType) { return false, model.TableLockNone diff --git a/util/mock/context.go b/util/mock/context.go index cbfdcd0f0914f..e97dd9ce83bcc 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -17,11 +17,11 @@ package mock import ( "context" "fmt" - "github.com/pingcap/parser/model" "sync" "time" "github.com/pingcap/errors" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/sessionctx" @@ -220,6 +220,10 @@ func (c *Context) StmtAddDirtyTableOP(op int, tid int64, handle int64, row []typ func (s *Context) AddTableLock(_ []model.TableLockTpInfo) { } +// ReleaseTableLock implements the sessionctx.Context interface. +func (s *Context) ReleaseTableLock(locks []model.TableLockTpInfo) { +} + // CheckTableLocked implements the sessionctx.Context interface. func (s *Context) CheckTableLocked(_ int64) (bool, model.TableLockType) { return false, model.TableLockNone From 9e507a45fef0b4163ad6ffc6dc8094b618b4336d Mon Sep 17 00:00:00 2001 From: crazycs Date: Sun, 5 May 2019 14:43:37 +0800 Subject: [PATCH 10/48] fix show privilege check table lock and not support lock table in INFORMATION_SCHEMA, PERFORMANCE_SCHEMA, mysql --- lock/lock.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/lock/lock.go b/lock/lock.go index d782b2f6e9e10..0dc4feb5366a1 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -33,8 +33,12 @@ func (c *LockChecker) CheckTableLock(db, table string, privilege mysql.Privilege if db == "" || table == "" { return nil } + // Below database are not support table lock. + if db == "INFORMATION_SCHEMA" || db == "PERFORMANCE_SCHEMA" || db == "mysql" { + return nil + } switch privilege { - case mysql.CreatePriv: + case mysql.CreatePriv, mysql.ShowDBPriv, mysql.AllPrivMask: return nil } tb, err := c.is.TableByName(model.NewCIStr(db), model.NewCIStr(table)) From 86c9be1998567e4add88273053f1d0fb501b2676 Mon Sep 17 00:00:00 2001 From: crazycs Date: Sun, 5 May 2019 15:21:01 +0800 Subject: [PATCH 11/48] add more unit test --- ddl/db_test.go | 67 +++++++++++++++++++++++++++++++++++++++++++++----- 1 file changed, 61 insertions(+), 6 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 0ab030e21a01a..453730eb20ad7 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2756,18 +2756,19 @@ func (s *testDBSuite2) TestLockTables(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2") - defer tk.MustExec("drop table if exists t1,t2") + defer tk.MustExec("unlock tables; drop table if exists t1,t2") tk.MustExec("create table t1 (a int)") tk.MustExec("create table t2 (a int)") + + // Test lock 1 table. tk.MustExec("lock tables t1 write") checkTableLock(c, tk.Se, "test", "t1", model.TableLockWrite) tk.MustExec("lock tables t1 read") checkTableLock(c, tk.Se, "test", "t1", model.TableLockRead) tk.MustExec("lock tables t1 write") checkTableLock(c, tk.Se, "test", "t1", model.TableLockWrite) - tk.MustExec("unlock tables") - checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone) + // Test lock multi tables. tk.MustExec("lock tables t1 write, t2 read") checkTableLock(c, tk.Se, "test", "t1", model.TableLockWrite) checkTableLock(c, tk.Se, "test", "t2", model.TableLockRead) @@ -2780,9 +2781,63 @@ func (s *testDBSuite2) TestLockTables(c *C) { tk.MustExec("lock tables t1 write") checkTableLock(c, tk.Se, "test", "t1", model.TableLockWrite) checkTableLock(c, tk.Se, "test", "t2", model.TableLockNone) - tk.MustExec("unlock tables") - checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone) - checkTableLock(c, tk.Se, "test", "t2", model.TableLockNone) + + tk2 := testkit.NewTestKit(c, s.store) + tk2.MustExec("use test") + defer tk2.MustExec("unlock tables") + + // Test read lock. + tk.MustExec("lock tables t1 read") + tk.MustQuery("select * from t1") + tk2.MustExec("select * from t1") + _, err := tk.Exec("insert into t1 set a=1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableNotLockedForWrite), IsTrue) + _, err = tk.Exec("update t1 set a=1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableNotLockedForWrite), IsTrue) + _, err = tk.Exec("delete from t1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableNotLockedForWrite), IsTrue) + + _, err = tk2.Exec("insert into t1 set a=1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + _, err = tk2.Exec("update t1 set a=1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + _, err = tk2.Exec("delete from t1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + tk2.MustExec("lock tables t1 read") + _, err = tk2.Exec("insert into t1 set a=1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableNotLockedForWrite), IsTrue) + + // Test write lock. + _, err = tk.Exec("lock tables t1 write") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + tk2.MustExec("unlock tables") + tk.MustExec("lock tables t1 write") + tk.MustQuery("select * from t1") + tk.MustExec("delete from t1") + tk.MustExec("insert into t1 set a=1") + + _, err = tk2.Exec("select * from t1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + _, err = tk2.Exec("insert into t1 set a=1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + _, err = tk2.Exec("lock tables t1 write") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + + // Test write local lock. + tk.MustExec("lock tables t1 write local") + tk.MustQuery("select * from t1") + tk.MustExec("delete from t1") + tk.MustExec("insert into t1 set a=1") + + tk2.MustQuery("select * from t1") + _, err = tk2.Exec("delete from t1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + _, err = tk2.Exec("insert into t1 set a=1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + _, err = tk2.Exec("lock tables t1 write") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + _, err = tk2.Exec("lock tables t1 read") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) } func checkTableLock(c *C, se session.Session, dbName, tableName string, lockTp model.TableLockType) { From 12d6fe10b91f3e8061ec12cc9ef1a2b5e59ba89e Mon Sep 17 00:00:00 2001 From: crazycs Date: Sun, 5 May 2019 15:40:50 +0800 Subject: [PATCH 12/48] add mutex in session.lockedtables to avoid concurrent problem --- ddl/ddl_api.go | 1 - session/session.go | 38 +++++++++++++++++++++++++++----------- 2 files changed, 27 insertions(+), 12 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index a5142f16d8f57..b5609b301691f 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3164,7 +3164,6 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error ctx.AddTableLock(lockTables) err := d.doDDLJob(ctx, job) if err == nil { - // TODO: add mutex to avoid concurrency problem. ctx.ReleaseTableLock(unlockTables) ctx.AddTableLock(lockTables) } diff --git a/session/session.go b/session/session.go index 34a3715b5c22b..a91f811b9041f 100644 --- a/session/session.go +++ b/session/session.go @@ -186,26 +186,35 @@ type session struct { // ddlOwnerChecker is used in `select tidb_is_ddl_owner()` statement; ddlOwnerChecker owner.DDLOwnerChecker // lockedTables use to record the table locks hold by the session. - lockedTables map[int64]model.TableLockTpInfo + lockedTables struct { + sync.RWMutex + holdLocks map[int64]model.TableLockTpInfo + } } // AddTableLock adds table lock to the session lock map. func (s *session) AddTableLock(locks []model.TableLockTpInfo) { + s.lockedTables.Lock() for _, l := range locks { - s.lockedTables[l.TableID] = l + s.lockedTables.holdLocks[l.TableID] = l } + s.lockedTables.Unlock() } // ReleaseTableLock releases table lock in the session lock map. func (s *session) ReleaseTableLock(locks []model.TableLockTpInfo) { + s.lockedTables.Lock() for _, l := range locks { - delete(s.lockedTables, l.TableID) + delete(s.lockedTables.holdLocks, l.TableID) } + s.lockedTables.Unlock() } // CheckTableLocked checks the table lock. func (s *session) CheckTableLocked(tblID int64) (bool, model.TableLockType) { - lt, ok := s.lockedTables[tblID] + s.lockedTables.RLock() + defer s.lockedTables.RUnlock() + lt, ok := s.lockedTables.holdLocks[tblID] if !ok { return false, model.TableLockNone } @@ -214,22 +223,29 @@ func (s *session) CheckTableLocked(tblID int64) (bool, model.TableLockType) { // GetAllTableLocks gets all table locks table id and db id hold by the session. func (s *session) GetAllTableLocks() []model.TableLockTpInfo { - lockTpInfo := make([]model.TableLockTpInfo, 0, len(s.lockedTables)) - for _, tl := range s.lockedTables { + s.lockedTables.RLock() + lockTpInfo := make([]model.TableLockTpInfo, 0, len(s.lockedTables.holdLocks)) + for _, tl := range s.lockedTables.holdLocks { lockTpInfo = append(lockTpInfo, tl) } + s.lockedTables.RUnlock() return lockTpInfo } // HasLockedTables uses to check whether this session locked any tables. // If so, the session can only visit the table which locked by self. func (s *session) HasLockedTables() bool { - return len(s.lockedTables) > 0 + s.lockedTables.RLock() + b := len(s.lockedTables.holdLocks) > 0 + s.lockedTables.RUnlock() + return b } // ReleaseAllTableLocks releases all table locks hold by the session. func (s *session) ReleaseAllTableLocks() { - s.lockedTables = make(map[int64]model.TableLockTpInfo) + s.lockedTables.Lock() + s.lockedTables.holdLocks = make(map[int64]model.TableLockTpInfo) + s.lockedTables.Unlock() } // DDLOwnerChecker returns s.ddlOwnerChecker. @@ -1323,7 +1339,7 @@ func (s *session) ClearValue(key fmt.Stringer) { // Close function does some clean work when session end. // Close should release the table locks which hold by the session. func (s *session) Close() { - if len(s.lockedTables) > 0 { + if s.HasLockedTables() { lockedTables := s.GetAllTableLocks() err := domain.GetDomain(s).DDL().UnlockTables(s, lockedTables) if err != nil { @@ -1579,7 +1595,7 @@ func createSession(store kv.Storage) (*session, error) { plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory.Load()) } s.mu.values = make(map[fmt.Stringer]interface{}) - s.lockedTables = make(map[int64]model.TableLockTpInfo) + s.lockedTables.holdLocks = make(map[int64]model.TableLockTpInfo) domain.BindDomain(s, dom) // session implements variable.GlobalVarAccessor. Bind it to ctx. s.sessionVars.GlobalVarsAccessor = s @@ -1603,7 +1619,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory.Load()) } s.mu.values = make(map[fmt.Stringer]interface{}) - s.lockedTables = make(map[int64]model.TableLockTpInfo) + s.lockedTables.holdLocks = make(map[int64]model.TableLockTpInfo) domain.BindDomain(s, dom) // session implements variable.GlobalVarAccessor. Bind it to ctx. s.sessionVars.GlobalVarsAccessor = s From 45e91203a829e77a1f961a7093436a6491ce05a7 Mon Sep 17 00:00:00 2001 From: crazycs Date: Sun, 5 May 2019 16:07:51 +0800 Subject: [PATCH 13/48] add none unique table check in lock tables --- ddl/db_test.go | 4 ++++ ddl/ddl_api.go | 5 +++++ infoschema/infoschema.go | 4 ++++ 3 files changed, 13 insertions(+) diff --git a/ddl/db_test.go b/ddl/db_test.go index 453730eb20ad7..ada52f54c186e 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2838,6 +2838,10 @@ func (s *testDBSuite2) TestLockTables(c *C) { c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) _, err = tk2.Exec("lock tables t1 read") c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + + // Test none unique table. + _, err = tk.Exec("lock tables t1 read, t1 write") + c.Assert(terror.ErrorEqual(err, infoschema.ErrNonuniqTable), IsTrue) } func checkTableLock(c *C, se session.Session, dbName, tableName string, lockTp model.TableLockType) { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index b5609b301691f..080352f70d53d 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3128,6 +3128,7 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error ServerID: d.GetID(), SessionID: ctx.GetSessionVars().ConnectionID, } + uniqueTableID := make(map[int64]struct{}) // Check whether the table was already locked by other. for _, tl := range stmt.TableLocks { tb := tl.Table @@ -3143,6 +3144,10 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error if err != nil { return err } + if _, ok := uniqueTableID[t.Meta().ID]; ok { + return infoschema.ErrNonuniqTable.GenWithStackByArgs(t.Meta().Name) + } + uniqueTableID[t.Meta().ID] = struct{}{} lockTables = append(lockTables, model.TableLockTpInfo{SchemaID: schema.ID, TableID: t.Meta().ID, Tp: tl.Type}) } diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 25595d914d045..e192a793b41a0 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -62,6 +62,8 @@ var ( ErrTableNotLockedForWrite = terror.ClassOptimizer.New(codeErrTableNotLockedForWrite, mysql.MySQLErrName[mysql.ErrTableNotLockedForWrite]) // ErrTableNotLocked returns when session has explicitly lock tables, then visit unlocked table will return this error. ErrTableNotLocked = terror.ClassOptimizer.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrTableNotLocked]) + // ErrNonuniqTable returns when none unique tables errors. + ErrNonuniqTable = terror.ClassOptimizer.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrNonuniqTable]) // ErrTableLocked returns when the table was locked by other session. ErrTableLocked = terror.ClassOptimizer.New(codeTableLocked, "Table '%s' was locked in %s by %v") ) @@ -329,6 +331,7 @@ const ( codeErrTableNotLockedForWrite = mysql.ErrTableNotLockedForWrite codeErrTableNotLocked = mysql.ErrTableNotLocked + codeErrNonuniqTable = mysql.ErrNonuniqTable ) func init() { @@ -351,6 +354,7 @@ func init() { codeKeyNotExists: mysql.ErrKeyDoesNotExist, codeErrTableNotLockedForWrite: mysql.ErrTableNotLockedForWrite, codeErrTableNotLocked: mysql.ErrTableNotLocked, + codeErrNonuniqTable: mysql.ErrNonuniqTable, } terror.ErrClassToMySQLCodes[terror.ClassSchema] = schemaMySQLErrCodes initInfoSchemaDB() From 9b4841a914970e3c0c6fb843ed8b831c0bf9a415 Mon Sep 17 00:00:00 2001 From: crazycs Date: Sun, 5 May 2019 16:20:34 +0800 Subject: [PATCH 14/48] fix ci --- lock/lock.go | 11 +++++++---- planner/core/optimizer.go | 2 +- table/tables/tables.go | 4 ++-- util/mock/context.go | 14 +++++++------- 4 files changed, 17 insertions(+), 14 deletions(-) diff --git a/lock/lock.go b/lock/lock.go index 0dc4feb5366a1..c485ab2380376 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -20,16 +20,19 @@ import ( "github.com/pingcap/tidb/sessionctx" ) -type LockChecker struct { +// Checker uses to check tables lock. +type Checker struct { ctx sessionctx.Context is infoschema.InfoSchema } -func NewLockChecker(ctx sessionctx.Context, is infoschema.InfoSchema) *LockChecker { - return &LockChecker{ctx: ctx, is: is} +// NewChecker return new lock Checker. +func NewChecker(ctx sessionctx.Context, is infoschema.InfoSchema) *Checker { + return &Checker{ctx: ctx, is: is} } -func (c *LockChecker) CheckTableLock(db, table string, privilege mysql.PrivilegeType) error { +// CheckTableLock uses to check table lock. +func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType) error { if db == "" || table == "" { return nil } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 9416bfda8229e..45d87c65f6194 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -100,7 +100,7 @@ func CheckPrivilege(activeRoles []*auth.RoleIdentity, pm privilege.Manager, vs [ // CheckTableLock checks the table lock. func CheckTableLock(ctx sessionctx.Context, is infoschema.InfoSchema, vs []visitInfo) error { - checker := lock.NewLockChecker(ctx, is) + checker := lock.NewChecker(ctx, is) for i := range vs { err := checker.CheckTableLock(vs[i].db, vs[i].table, vs[i].privilege) if err != nil { diff --git a/table/tables/tables.go b/table/tables/tables.go index ae6533bc8e0d9..5d27fb46a4df5 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1210,7 +1210,7 @@ func (ctx *ctxForPartitionExpr) AddTableLock(_ []model.TableLockTpInfo) { } // ReleaseTableLock implements the sessionctx.Context interface. -func (s *ctxForPartitionExpr) ReleaseTableLock(locks []model.TableLockTpInfo) { +func (ctx *ctxForPartitionExpr) ReleaseTableLock(locks []model.TableLockTpInfo) { } // CheckTableLocked implements the sessionctx.Context interface. @@ -1228,7 +1228,7 @@ func (ctx *ctxForPartitionExpr) ReleaseAllTableLocks() { } // HasLockedTables implements the sessionctx.Context interface. -func (s *ctxForPartitionExpr) HasLockedTables() bool { +func (ctx *ctxForPartitionExpr) HasLockedTables() bool { return false } diff --git a/util/mock/context.go b/util/mock/context.go index e97dd9ce83bcc..af877f71685ec 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -217,34 +217,34 @@ func (c *Context) StmtAddDirtyTableOP(op int, tid int64, handle int64, row []typ } // AddTableLock implements the sessionctx.Context interface. -func (s *Context) AddTableLock(_ []model.TableLockTpInfo) { +func (c *Context) AddTableLock(_ []model.TableLockTpInfo) { } // ReleaseTableLock implements the sessionctx.Context interface. -func (s *Context) ReleaseTableLock(locks []model.TableLockTpInfo) { +func (c *Context) ReleaseTableLock(locks []model.TableLockTpInfo) { } // CheckTableLocked implements the sessionctx.Context interface. -func (s *Context) CheckTableLocked(_ int64) (bool, model.TableLockType) { +func (c *Context) CheckTableLocked(_ int64) (bool, model.TableLockType) { return false, model.TableLockNone } // GetAllTableLocks implements the sessionctx.Context interface. -func (s *Context) GetAllTableLocks() []model.TableLockTpInfo { +func (c *Context) GetAllTableLocks() []model.TableLockTpInfo { return nil } // ReleaseAllTableLocks implements the sessionctx.Context interface. -func (s *Context) ReleaseAllTableLocks() { +func (c *Context) ReleaseAllTableLocks() { } // HasLockedTables implements the sessionctx.Context interface. -func (s *Context) HasLockedTables() bool { +func (c *Context) HasLockedTables() bool { return false } // Close implements the sessionctx.Context interface. -func (s *Context) Close() { +func (c *Context) Close() { } // NewContext creates a new mocked sessionctx.Context. From afd98cff4cef099d6699436856216c6ed133e107 Mon Sep 17 00:00:00 2001 From: crazycs Date: Sun, 5 May 2019 22:53:51 +0800 Subject: [PATCH 15/48] add table lock test in transaction --- ddl/db_test.go | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/ddl/db_test.go b/ddl/db_test.go index ada52f54c186e..b45876c38fdef 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2842,6 +2842,27 @@ func (s *testDBSuite2) TestLockTables(c *C) { // Test none unique table. _, err = tk.Exec("lock tables t1 read, t1 write") c.Assert(terror.ErrorEqual(err, infoschema.ErrNonuniqTable), IsTrue) + + // Test lock table by other session in transaction and commit without retry. + tk.MustExec("unlock tables") + tk2.MustExec("unlock tables") + tk.MustExec("set @@session.tidb_retry_limit=0") + tk.MustExec("begin") + tk.MustExec("insert into t1 set a=1") + tk2.MustExec("lock tables t1 write") + _, err = tk.Exec("commit") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[domain:2]Information schema is changed.") + + // Test lock table by other session in transaction and commit with retry. + tk.MustExec("unlock tables") + tk2.MustExec("unlock tables") + tk.MustExec("set @@session.tidb_retry_limit=1") + tk.MustExec("begin") + tk.MustExec("insert into t1 set a=1") + tk2.MustExec("lock tables t1 write") + _, err = tk.Exec("commit") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) } func checkTableLock(c *C, se session.Session, dbName, tableName string, lockTp model.TableLockType) { From 7886732687ac1eff432c8f730f6e81878459936a Mon Sep 17 00:00:00 2001 From: crazycs Date: Sun, 5 May 2019 23:58:08 +0800 Subject: [PATCH 16/48] add concurrent lock tables test --- ddl/db_test.go | 118 +++++++++++++++++++++++++++++++++++++++++++++- ddl/table_lock.go | 14 +----- 2 files changed, 119 insertions(+), 13 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index b45876c38fdef..87a86027fb2c2 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2753,7 +2753,6 @@ func (s *testDBSuite2) TestAlterShardRowIDBits(c *C) { func (s *testDBSuite2) TestLockTables(c *C) { s.tk = testkit.NewTestKit(c, s.store) tk := s.tk - tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2") defer tk.MustExec("unlock tables; drop table if exists t1,t2") @@ -2863,6 +2862,123 @@ func (s *testDBSuite2) TestLockTables(c *C) { tk2.MustExec("lock tables t1 write") _, err = tk.Exec("commit") c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + + // Test for lock the same table multiple times. + tk2.MustExec("lock tables t1 write") + tk2.MustExec("lock tables t1 write, t2 read") +} + +// TestConcurrentLockTables test concurrent lock/unlock tables. +func (s *testDBSuite2) TestConcurrentLockTables(c *C) { + s.tk = testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(c, s.store) + tk := s.tk + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + defer tk.MustExec("unlock tables; drop table if exists t1") + tk.MustExec("create table t1 (a int)") + tk2.MustExec("use test") + defer tk2.MustExec("unlock tables") + + // Test concurrent lock tables read. + sql1 := "lock tables t1 read" + sql2 := "lock tables t1 read" + s.testParallelExecSQL(c, sql1, sql2, tk.Se, tk2.Se, func(c *C, err1, err2 error) { + c.Assert(err1, IsNil) + c.Assert(err2, IsNil) + }) + tk.MustExec("unlock tables") + tk2.MustExec("unlock tables") + + // Test concurrent lock tables write. + sql1 = "lock tables t1 write" + sql2 = "lock tables t1 write" + s.testParallelExecSQL(c, sql1, sql2, tk.Se, tk2.Se, func(c *C, err1, err2 error) { + c.Assert(err1, IsNil) + c.Assert(terror.ErrorEqual(err2, infoschema.ErrTableLocked), IsTrue) + }) + tk.MustExec("unlock tables") + tk2.MustExec("unlock tables") + + // Test concurrent lock tables write local. + sql1 = "lock tables t1 write local" + sql2 = "lock tables t1 write local" + s.testParallelExecSQL(c, sql1, sql2, tk.Se, tk2.Se, func(c *C, err1, err2 error) { + c.Assert(err1, IsNil) + c.Assert(terror.ErrorEqual(err2, infoschema.ErrTableLocked), IsTrue) + }) + +} + +func (s *testDBSuite2) testParallelExecSQL(c *C, sql1, sql2 string, se1, se2 session.Session, f checkRet) { + + callback := &ddl.TestDDLCallback{} + times := 0 + callback.OnJobRunBeforeExported = func(job *model.Job) { + if times != 0 { + return + } + var qLen int + for { + err := kv.RunInNewTxn(s.store, false, func(txn kv.Transaction) error { + jobs, err1 := admin.GetDDLJobs(txn) + if err1 != nil { + return err1 + } + qLen = len(jobs) + return nil + }) + c.Assert(err, IsNil) + if qLen == 2 { + break + } + time.Sleep(5 * time.Millisecond) + } + times++ + } + d := s.dom.DDL() + originalCallback := d.GetHook() + defer d.(ddl.DDLForTest).SetHook(originalCallback) + d.(ddl.DDLForTest).SetHook(callback) + + wg := sync.WaitGroup{} + var err1 error + var err2 error + wg.Add(2) + ch := make(chan struct{}) + // Make sure the sql1 is put into the DDLJobQueue. + go func() { + var qLen int + for { + err := kv.RunInNewTxn(s.store, false, func(txn kv.Transaction) error { + jobs, err3 := admin.GetDDLJobs(txn) + if err3 != nil { + return err3 + } + qLen = len(jobs) + return nil + }) + c.Assert(err, IsNil) + if qLen == 1 { + // Make sure sql2 is executed after the sql1. + close(ch) + break + } + time.Sleep(5 * time.Millisecond) + } + }() + go func() { + defer wg.Done() + _, err1 = se1.Execute(context.Background(), sql1) + }() + go func() { + defer wg.Done() + <-ch + _, err2 = se2.Execute(context.Background(), sql2) + }() + + wg.Wait() + f(c, err1, err2) } func checkTableLock(c *C, se session.Session, dbName, tableName string, lockTp model.TableLockType) { diff --git a/ddl/table_lock.go b/ddl/table_lock.go index 63b5aafc3709f..cc082056e3d0d 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -142,18 +142,8 @@ func checkAndLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) err tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, arg.SessionInfo) return nil } - sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg.SessionInfo) - // repeat lock. - if sessionIndex >= 0 { - if tbInfo.Lock.Tp == arg.LockTables[idx].Tp { - return nil - } - if len(tbInfo.Lock.Sessions) == 1 { - // just change lock tp directly. - tbInfo.Lock.Tp = arg.LockTables[idx].Tp - return nil - } - } + // Unlock tables should execute before lock tables, so arg.SessionInfo in tbInfo.Lock.Sessions here is impossible. + return infoschema.ErrTableLocked.GenWithStackByArgs(tbInfo.Name.L, tbInfo.Lock.Tp, tbInfo.Lock.Sessions[0]) } From 3419f718ab50a053f9852104f0039511d4f2b7a0 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 6 May 2019 00:07:18 +0800 Subject: [PATCH 17/48] remove blank line --- ddl/db_test.go | 2 -- ddl/table_test.go | 1 - planner/core/planbuilder.go | 2 +- 3 files changed, 1 insertion(+), 4 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 87a86027fb2c2..2025e0a3e609e 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2907,11 +2907,9 @@ func (s *testDBSuite2) TestConcurrentLockTables(c *C) { c.Assert(err1, IsNil) c.Assert(terror.ErrorEqual(err2, infoschema.ErrTableLocked), IsTrue) }) - } func (s *testDBSuite2) testParallelExecSQL(c *C, sql1, sql2 string, se1, se2 session.Session, f checkRet) { - callback := &ddl.TestDDLCallback{} times := 0 callback.OnJobRunBeforeExported = func(job *model.Job) { diff --git a/ddl/table_test.go b/ddl/table_test.go index 4b37a8a0945d6..70602624dd454 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -359,7 +359,6 @@ func (s *testTableSuite) TestTable(c *C) { job = testLockTable(c, ctx, d, dbInfo1.ID, tblInfo, model.TableLockWrite) testCheckTableState(c, d, dbInfo1, tblInfo, model.StatePublic) testCheckJobDone(c, d, job, true) - checkTableLockedTest(c, d, dbInfo1, tblInfo, d.GetID(), ctx.GetSessionVars().ConnectionID, model.TableLockWrite) } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 74f9c39377c64..db0c2b7085758 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1768,7 +1768,7 @@ func (b *PlanBuilder) buildDDL(node ast.DDLNode) (Plan, error) { // Recover table command can only be executed by administrator. b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil) case *ast.LockTablesStmt: - // do nothing. + // Nothing need to do. } p := &DDL{Statement: node} return p, nil From 9e4b024d151a882cc146640ed8b101d9f4a605f8 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 6 May 2019 00:12:24 +0800 Subject: [PATCH 18/48] add todo: clean table lock when tidb-server was kill -9 --- session/session.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/session/session.go b/session/session.go index a91f811b9041f..3a37edf14d5b2 100644 --- a/session/session.go +++ b/session/session.go @@ -1339,6 +1339,8 @@ func (s *session) ClearValue(key fmt.Stringer) { // Close function does some clean work when session end. // Close should release the table locks which hold by the session. func (s *session) Close() { + // TODO: do clean table locks when session exited without execute Close. + // TODO: do clean table locks when tidb-server was `kill -9`. if s.HasLockedTables() { lockedTables := s.GetAllTableLocks() err := domain.GetDomain(s).DDL().UnlockTables(s, lockedTables) From 2126cd85ae7085f9f8cf94ee709f526282c6e25a Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 6 May 2019 10:10:11 +0800 Subject: [PATCH 19/48] add lock unsupport schema check,test and add comments --- ddl/db_test.go | 8 ++++++++ ddl/ddl_api.go | 6 +++++- ddl/table_lock.go | 49 ++++++++++++++++++++++++++--------------------- ddl/table_test.go | 2 ++ lock/lock.go | 2 +- 5 files changed, 43 insertions(+), 24 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 2025e0a3e609e..5913c7e68a59e 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2866,6 +2866,14 @@ func (s *testDBSuite2) TestLockTables(c *C) { // Test for lock the same table multiple times. tk2.MustExec("lock tables t1 write") tk2.MustExec("lock tables t1 write, t2 read") + + // Test for lock unsupported schema tables. + _, err = tk2.Exec("lock tables performance_schema.global_status write") + c.Assert(terror.ErrorEqual(err, table.ErrUnsupportedOp), IsTrue) + _, err = tk2.Exec("lock tables information_schema.tables write") + c.Assert(terror.ErrorEqual(err, table.ErrUnsupportedOp), IsTrue) + _, err = tk2.Exec("lock tables mysql.db write") + c.Assert(terror.ErrorEqual(err, table.ErrUnsupportedOp), IsTrue) } // TestConcurrentLockTables test concurrent lock/unlock tables. diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 080352f70d53d..1f4fa2a0ecce1 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3121,6 +3121,7 @@ func buildPartitionInfo(meta *model.TableInfo, d *ddl, spec *ast.AlterTableSpec) return part, nil } +// LockTables uses to execute lock tables statement. func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error { lockTables := make([]model.TableLockTpInfo, 0, len(stmt.TableLocks)) is := d.infoHandle.Get() @@ -3132,6 +3133,9 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error // Check whether the table was already locked by other. for _, tl := range stmt.TableLocks { tb := tl.Table + if tb.Schema.L == "information_schema" || tb.Schema.L == "performance_schema" || tb.Schema.L == "mysql" { + return table.ErrUnsupportedOp + } schema, ok := is.SchemaByName(tb.Schema) if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(tb.Schema) @@ -3157,7 +3161,6 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error UnlockTables: unlockTables, SessionInfo: sessionInfo, } - job := &model.Job{ SchemaID: lockTables[0].SchemaID, TableID: lockTables[0].TableID, @@ -3176,6 +3179,7 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error return errors.Trace(err) } +// UnlockTables uses to execute unlock tables statement. func (d *ddl) UnlockTables(ctx sessionctx.Context, unlockTables []model.TableLockTpInfo) error { if len(unlockTables) == 0 { return nil diff --git a/ddl/table_lock.go b/ddl/table_lock.go index cc082056e3d0d..2d0d757ac11e6 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -30,7 +30,7 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { // Unlock table first. if arg.IndexOfUnlock < len(arg.UnlockTables) { - return unlockTableReq(t, job, arg) + return unlockTables(t, job, arg) } // Check table locked by other, this can be only checked at the first time. @@ -48,6 +48,7 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { return ver, errors.Trace(err) } } + // Set job.schemaState to StateDeleteOnly just want to indicate this job was in running. job.SchemaState = model.StateDeleteOnly } @@ -93,25 +94,6 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { return ver, err } -func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) error { - if tbInfo.Lock == nil { - return nil - } - sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg.SessionInfo) - if sessionIndex < 0 { - return nil - // todo: when clean table lock , session maybe send unlock table even the table lock maybe not hold by the session. - //return errors.Errorf("%s isn't holding table %s lock", arg.SessionInfo, tbInfo.Name) - } - oldSessionInfo := tbInfo.Lock.Sessions - tbInfo.Lock.Sessions = oldSessionInfo[:sessionIndex] - tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, oldSessionInfo[sessionIndex+1:]...) - if len(tbInfo.Lock.Sessions) == 0 { - tbInfo.Lock = nil - } - return nil -} - // indexOfLockHolder gets the index of sessionInfo in the sessions. return -1 if sessions doesn't contain sessionInfo. func indexOfLockHolder(sessions []model.SessionInfo, sessionInfo model.SessionInfo) int { for i := range sessions { @@ -122,6 +104,7 @@ func indexOfLockHolder(sessions []model.SessionInfo, sessionInfo model.SessionIn return -1 } +// checkAndLockTable uses to check table locked and acquire the table lock for the request session. func checkAndLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error { if tbInfo.Lock == nil || len(tbInfo.Lock.Sessions) == 0 { tbInfo.Lock = &model.TableLockInfo{ @@ -147,6 +130,7 @@ func checkAndLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) err return infoschema.ErrTableLocked.GenWithStackByArgs(tbInfo.Name.L, tbInfo.Lock.Tp, tbInfo.Lock.Sessions[0]) } +// checkTableLocked uses to check whether table was locked. func checkTableLocked(tbInfo *model.TableInfo, lockTp model.TableLockType, sessionInfo model.SessionInfo) error { if tbInfo.Lock == nil || len(tbInfo.Lock.Sessions) == 0 { return nil @@ -170,7 +154,8 @@ func checkTableLocked(tbInfo *model.TableInfo, lockTp model.TableLockType, sessi return infoschema.ErrTableLocked.GenWithStackByArgs(tbInfo.Name.L, tbInfo.Lock.Tp, tbInfo.Lock.Sessions[0]) } -func unlockTableReq(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, err error) { +// unlockTables uses unlock a batch of table lock one by one. +func unlockTables(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, err error) { if arg.IndexOfUnlock >= len(arg.UnlockTables) { return ver, nil } @@ -195,6 +180,26 @@ func unlockTableReq(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64 return ver, nil } +// unlockTable uses to unlock table lock that hold by the session. +func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) error { + if tbInfo.Lock == nil { + return nil + } + sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg.SessionInfo) + if sessionIndex < 0 { + // When session clean table lock, session maybe send unlock table even the table lock maybe not hold by the session. + // so just return nil here. + return nil + } + oldSessionInfo := tbInfo.Lock.Sessions + tbInfo.Lock.Sessions = oldSessionInfo[:sessionIndex] + tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, oldSessionInfo[sessionIndex+1:]...) + if len(tbInfo.Lock.Sessions) == 0 { + tbInfo.Lock = nil + } + return nil +} + func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { arg := &lockTablesArg{} if err := job.DecodeArgs(arg); err != nil { @@ -203,7 +208,7 @@ func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { return ver, errors.Trace(err) } - ver, err = unlockTableReq(t, job, arg) + ver, err = unlockTables(t, job, arg) if arg.IndexOfUnlock == len(arg.UnlockTables) { job.FinishTableJob(model.JobStateDone, model.StateNone, ver, nil) diff --git a/ddl/table_test.go b/ddl/table_test.go index 70602624dd454..4d8f0650fb1af 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -174,7 +174,9 @@ func testRenameTable(c *C, ctx sessionctx.Context, d *ddl, newSchemaID, oldSchem c.Assert(err, IsNil) v := getSchemaVer(c, ctx) + tblInfo.State = model.StatePublic checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + tblInfo.State = model.StateNone return job } diff --git a/lock/lock.go b/lock/lock.go index c485ab2380376..73e7131114625 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -37,7 +37,7 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType return nil } // Below database are not support table lock. - if db == "INFORMATION_SCHEMA" || db == "PERFORMANCE_SCHEMA" || db == "mysql" { + if db == "information_schema" || db == "performance_schema" || db == "mysql" { return nil } switch privilege { From 5b1a0ca206b55984770bc298348173f2f9e198cd Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Tue, 7 May 2019 15:06:39 +0800 Subject: [PATCH 20/48] fix table lock bug in drop table/schema, truncate table bug --- ddl/db_test.go | 25 +++++++++++++++++++++++++ ddl/ddl_api.go | 28 ++++++++++++++++++++++++++++ ddl/table.go | 13 ++++++++++--- ddl/table_lock.go | 14 ++++++++++++-- infoschema/infoschema.go | 8 ++++---- 5 files changed, 79 insertions(+), 9 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 5913c7e68a59e..b8e664b2b74c4 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2867,6 +2867,31 @@ func (s *testDBSuite2) TestLockTables(c *C) { tk2.MustExec("lock tables t1 write") tk2.MustExec("lock tables t1 write, t2 read") + // Test lock tables and drop tables + tk.MustExec("unlock tables") + tk2.MustExec("unlock tables") + tk.MustExec("lock tables t1 write, t2 write") + tk.MustExec("drop table t1") + tk.MustExec("create table t1 (a int)") + tk.MustExec("lock tables t1 write, t2 read") + + // Test lock tables and drop database. + tk.MustExec("unlock tables") + tk.MustExec("create database test_lock") + tk.MustExec("create table test_lock.t3 (a int)") + tk.MustExec("lock tables t1 write, test_lock.t3 write") + tk.MustExec("drop database test_lock") + tk.MustExec("create table t3 (a int)") + tk.MustExec("lock tables t1 write, t3 read") + + // Test lock tables and truncate tables. + tk.MustExec("unlock tables") + tk.MustExec("lock tables t1 write, t2 read") + tk.MustExec("truncate table t1") + tk.MustExec("insert into t1 set a=1") + _, err = tk2.Exec("insert into t1 set a=1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + // Test for lock unsupported schema tables. _, err = tk2.Exec("lock tables performance_schema.global_status write") c.Assert(terror.ErrorEqual(err, table.ErrUnsupportedOp), IsTrue) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 1f4fa2a0ecce1..d53fd424526e9 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -97,6 +97,7 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) if !ok { return errors.Trace(infoschema.ErrDatabaseNotExists) } + tbs := is.SchemaTables(schema) job := &model.Job{ SchemaID: old.ID, Type: model.ActionDropSchema, @@ -104,6 +105,16 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) } err = d.doDDLJob(ctx, job) + if err == nil { + // clear table locks. + tableLocks := make([]model.TableLockTpInfo, 0) + for _, tb := range tbs { + if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { + tableLocks = append(tableLocks, model.TableLockTpInfo{SchemaID: old.ID, TableID: tb.Meta().ID}) + } + } + ctx.ReleaseTableLock(tableLocks) + } err = d.callHookOnChanged(err) return errors.Trace(err) } @@ -2759,6 +2770,12 @@ func (d *ddl) DropTable(ctx sessionctx.Context, ti ast.Ident) (err error) { } err = d.doDDLJob(ctx, job) + if err == nil { + if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { + ctx.ReleaseTableLock([]model.TableLockTpInfo{{SchemaID: schema.ID, TableID: tb.Meta().ID}}) + } + } + err = d.callHookOnChanged(err) return errors.Trace(err) } @@ -2802,7 +2819,18 @@ func (d *ddl) TruncateTable(ctx sessionctx.Context, ti ast.Ident) error { BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{newTableID}, } + if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { + // AddTableLock here to avoid this ddl job was execute successful but the session was been kill before return. + ctx.AddTableLock(([]model.TableLockTpInfo{{SchemaID: schema.ID, TableID: newTableID, Tp: tb.Meta().Lock.Tp}})) + } err = d.doDDLJob(ctx, job) + if err == nil { + if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { + ctx.ReleaseTableLock([]model.TableLockTpInfo{{SchemaID: schema.ID, TableID: tb.Meta().ID}}) + } + } else { + ctx.ReleaseTableLock([]model.TableLockTpInfo{{SchemaID: schema.ID, TableID: newTableID}}) + } err = d.callHookOnChanged(err) return errors.Trace(err) } diff --git a/ddl/table.go b/ddl/table.go index dd604a3d8c7c3..70b50bb7bef68 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -443,12 +443,20 @@ func getTableInfoAndCancelFaultJob(t *meta.Meta, job *model.Job, schemaID int64) } func checkTableExistAndCancelNonExistJob(t *meta.Meta, job *model.Job, schemaID int64) (*model.TableInfo, error) { - tableID := job.TableID + tblInfo, err := getTableInfo(t, job.TableID, schemaID) + if err != nil { + if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { + job.State = model.JobStateCancelled + } + } + return tblInfo, err +} + +func getTableInfo(t *meta.Meta, tableID, schemaID int64) (*model.TableInfo, error) { // Check this table's database. tblInfo, err := t.GetTable(schemaID, tableID) if err != nil { if meta.ErrDBNotExists.Equal(err) { - job.State = model.JobStateCancelled return nil, errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs( fmt.Sprintf("(Schema ID %d)", schemaID), )) @@ -458,7 +466,6 @@ func checkTableExistAndCancelNonExistJob(t *meta.Meta, job *model.Job, schemaID // Check the table. if tblInfo == nil { - job.State = model.JobStateCancelled return nil, errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs( fmt.Sprintf("(Schema ID %d)", schemaID), fmt.Sprintf("(Table ID %d)", tableID), diff --git a/ddl/table_lock.go b/ddl/table_lock.go index 2d0d757ac11e6..8b0dba300d6f0 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -161,10 +161,20 @@ func unlockTables(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, } job.SchemaID = arg.UnlockTables[arg.IndexOfUnlock].SchemaID job.TableID = arg.UnlockTables[arg.IndexOfUnlock].TableID - tbInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) + tbInfo, err := getTableInfo(t, job.TableID, job.SchemaID) if err != nil { - return ver, err + if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { + err = nil + // The table maybe has been dropped. just ignore this err and go on. + arg.IndexOfUnlock++ + job.Args = []interface{}{arg} + return ver, nil + } else { + job.State = model.JobStateCancelled + return ver, err + } } + err = unlockTable(tbInfo, arg) if err != nil { job.State = model.JobStateCancelled diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index e192a793b41a0..adbe6867bff44 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -59,13 +59,13 @@ var ( // ErrTooManyKeyParts returns for too many key parts. ErrTooManyKeyParts = terror.ClassSchema.New(codeTooManyKeyParts, "Too many key parts specified; max %d parts allowed") // ErrTableNotLockedForWrite returns for write tables when only hold the table read lock. - ErrTableNotLockedForWrite = terror.ClassOptimizer.New(codeErrTableNotLockedForWrite, mysql.MySQLErrName[mysql.ErrTableNotLockedForWrite]) + ErrTableNotLockedForWrite = terror.ClassSchema.New(codeErrTableNotLockedForWrite, mysql.MySQLErrName[mysql.ErrTableNotLockedForWrite]) // ErrTableNotLocked returns when session has explicitly lock tables, then visit unlocked table will return this error. - ErrTableNotLocked = terror.ClassOptimizer.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrTableNotLocked]) + ErrTableNotLocked = terror.ClassSchema.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrTableNotLocked]) // ErrNonuniqTable returns when none unique tables errors. - ErrNonuniqTable = terror.ClassOptimizer.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrNonuniqTable]) + ErrNonuniqTable = terror.ClassSchema.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrNonuniqTable]) // ErrTableLocked returns when the table was locked by other session. - ErrTableLocked = terror.ClassOptimizer.New(codeTableLocked, "Table '%s' was locked in %s by %v") + ErrTableLocked = terror.ClassSchema.New(codeTableLocked, "Table '%s' was locked in %s by %v") ) // InfoSchema is the interface used to retrieve the schema information. From 826381495cc2f219e941f7cab0aaf671dc0ec8ff Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Tue, 7 May 2019 15:28:48 +0800 Subject: [PATCH 21/48] fix test --- ddl/db_test.go | 3 ++- ddl/table_lock.go | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index b8e664b2b74c4..5325409ee0c52 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2882,7 +2882,8 @@ func (s *testDBSuite2) TestLockTables(c *C) { tk.MustExec("lock tables t1 write, test_lock.t3 write") tk.MustExec("drop database test_lock") tk.MustExec("create table t3 (a int)") - tk.MustExec("lock tables t1 write, t3 read") + tk.MustExec("lock tables t1 write, t3 write") + tk.MustExec("drop table t3") // Test lock tables and truncate tables. tk.MustExec("unlock tables") diff --git a/ddl/table_lock.go b/ddl/table_lock.go index 8b0dba300d6f0..9d5991302d6ec 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -164,7 +164,6 @@ func unlockTables(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, tbInfo, err := getTableInfo(t, job.TableID, job.SchemaID) if err != nil { if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { - err = nil // The table maybe has been dropped. just ignore this err and go on. arg.IndexOfUnlock++ job.Args = []interface{}{arg} From 919ac2c2f21a4c3be75471e73495b83d05c495c2 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 20 May 2019 17:26:53 +0800 Subject: [PATCH 22/48] fix test --- ddl/db_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index a3597c07ff10d..32fe0de51e61c 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2846,7 +2846,7 @@ func (s *testDBSuite2) TestLockTables(c *C) { // Test lock table by other session in transaction and commit without retry. tk.MustExec("unlock tables") tk2.MustExec("unlock tables") - tk.MustExec("set @@session.tidb_retry_limit=0") + tk.MustExec("set @@session.tidb_disable_txn_auto_retry=1") tk.MustExec("begin") tk.MustExec("insert into t1 set a=1") tk2.MustExec("lock tables t1 write") @@ -2859,7 +2859,7 @@ func (s *testDBSuite2) TestLockTables(c *C) { tk2.MustExec("unlock tables") tk.MustExec("begin") tk.MustExec("insert into t1 set a=1") - tk.MustExec("set @@session.tidb_retry_limit=10") + tk.MustExec("set @@session.tidb_disable_txn_auto_retry=0") tk2.MustExec("lock tables t1 write") _, err = tk.Exec("commit") c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue, Commentf("err: %v\n", err)) From 196eeeb36d2e23c4c9d026f1e1e0d32f1e7a4bec Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 20 May 2019 19:04:38 +0800 Subject: [PATCH 23/48] add config field to control enable table feature --- config/config.go | 4 ++++ config/config.toml.example | 3 +++ config/config_test.go | 2 ++ ddl/db_test.go | 30 ++++++++++++++++++++++++++---- ddl/test.log] | 0 executor/ddl.go | 3 +++ planner/core/optimizer.go | 4 ++++ sessionctx/variable/session.go | 2 ++ sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 2 ++ sessionctx/variable/varsutil.go | 2 ++ 11 files changed, 49 insertions(+), 4 deletions(-) create mode 100644 ddl/test.log] diff --git a/config/config.go b/config/config.go index 7313af776a9a0..6721adc4a8104 100644 --- a/config/config.go +++ b/config/config.go @@ -87,6 +87,9 @@ type Config struct { // TreatOldVersionUTF8AsUTF8MB4 is use to treat old version table/column UTF8 charset as UTF8MB4. This is for compatibility. // Currently not support dynamic modify, because this need to reload all old version schema. TreatOldVersionUTF8AsUTF8MB4 bool `toml:"treat-old-version-utf8-as-utf8mb4" json:"treat-old-version-utf8-as-utf8mb4"` + // EnableTableLock indicate whether enable table lock. + // TODO: remove this after table lock features stable. + EnableTableLock bool `toml:"enable-table-lock" json:"enable-table-lock"` } // Log is the log section of config. @@ -320,6 +323,7 @@ var defaultConf = Config{ EnableStreaming: false, CheckMb4ValueInUTF8: true, TreatOldVersionUTF8AsUTF8MB4: true, + EnableTableLock: false, TxnLocalLatches: TxnLocalLatches{ Enabled: true, Capacity: 2048000, diff --git a/config/config.toml.example b/config/config.toml.example index 589046ee46a1d..30c29347eb1da 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -54,6 +54,9 @@ check-mb4-value-in-utf8 = true # treat-old-version-utf8-as-utf8mb4 use for upgrade compatibility. Set to true will treat old version table/column UTF8 charset as UTF8MB4. treat-old-version-utf8-as-utf8mb4 = true +# enable-table-lock is used to control table lock feature. Default false means not enable table lock. +enable-table-lock = false + [log] # Log level: debug, info, warn, error, fatal. level = "info" diff --git a/config/config_test.go b/config/config_test.go index 237bf1a1b85ba..eafa2b28278dc 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -60,6 +60,7 @@ unrecognized-option-test = true _, err = f.WriteString(` token-limit = 0 +enable-table-lock = true [performance] [tikv-client] commit-timeout="41s" @@ -78,6 +79,7 @@ max-batch-size=128 c.Assert(conf.TiKVClient.CommitTimeout, Equals, "41s") c.Assert(conf.TiKVClient.MaxBatchSize, Equals, uint(128)) c.Assert(conf.TokenLimit, Equals, uint(1000)) + c.Assert(conf.EnableTableLock, IsTrue) c.Assert(f.Close(), IsNil) c.Assert(os.Remove(configFile), IsNil) diff --git a/ddl/db_test.go b/ddl/db_test.go index 32fe0de51e61c..1ad1f3c026a72 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -16,6 +16,7 @@ package ddl_test import ( "context" "fmt" + "github.com/pingcap/tidb/config" "io" "math" "math/rand" @@ -2756,10 +2757,20 @@ func (s *testDBSuite2) TestLockTables(c *C) { tk := s.tk tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2") - defer tk.MustExec("unlock tables; drop table if exists t1,t2") + defer tk.MustExec("drop table if exists t1,t2") tk.MustExec("create table t1 (a int)") tk.MustExec("create table t2 (a int)") + // recover table lock config. + originValue := config.GetGlobalConfig().EnableTableLock + defer tk.MustExec(fmt.Sprintf("set @@tidb_enable_table_lock=%v", originValue)) + + // Test for enable table lock config. + tk.MustExec("set @@tidb_enable_table_lock=0") + tk.MustExec("lock tables t1 write") + checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone) + tk.MustExec("set @@tidb_enable_table_lock=1") + // Test lock 1 table. tk.MustExec("lock tables t1 write") checkTableLock(c, tk.Se, "test", "t1", model.TableLockWrite) @@ -2784,7 +2795,6 @@ func (s *testDBSuite2) TestLockTables(c *C) { tk2 := testkit.NewTestKit(c, s.store) tk2.MustExec("use test") - defer tk2.MustExec("unlock tables") // Test read lock. tk.MustExec("lock tables t1 read") @@ -2901,6 +2911,9 @@ func (s *testDBSuite2) TestLockTables(c *C) { c.Assert(terror.ErrorEqual(err, table.ErrUnsupportedOp), IsTrue) _, err = tk2.Exec("lock tables mysql.db write") c.Assert(terror.ErrorEqual(err, table.ErrUnsupportedOp), IsTrue) + + tk.MustExec("unlock tables") + tk2.MustExec("unlock tables") } // TestConcurrentLockTables test concurrent lock/unlock tables. @@ -2910,10 +2923,16 @@ func (s *testDBSuite2) TestConcurrentLockTables(c *C) { tk := s.tk tk.MustExec("use test") tk.MustExec("drop table if exists t1") - defer tk.MustExec("unlock tables; drop table if exists t1") + defer tk.MustExec("drop table if exists t1") tk.MustExec("create table t1 (a int)") tk2.MustExec("use test") - defer tk2.MustExec("unlock tables") + + // recover table lock config. + originValue := config.GetGlobalConfig().EnableTableLock + defer tk.MustExec(fmt.Sprintf("set @@tidb_enable_table_lock=%v", originValue)) + + // Test for enable table lock config. + tk.MustExec("set @@tidb_enable_table_lock=1") // Test concurrent lock tables read. sql1 := "lock tables t1 read" @@ -2942,6 +2961,9 @@ func (s *testDBSuite2) TestConcurrentLockTables(c *C) { c.Assert(err1, IsNil) c.Assert(terror.ErrorEqual(err2, infoschema.ErrTableLocked), IsTrue) }) + + tk.MustExec("unlock tables") + tk2.MustExec("unlock tables") } func (s *testDBSuite2) testParallelExecSQL(c *C, sql1, sql2 string, se1, se2 session.Session, f checkRet) { diff --git a/ddl/test.log] b/ddl/test.log] new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/executor/ddl.go b/executor/ddl.go index 252c0d9d3ec11..3b1c69afda420 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -439,6 +439,9 @@ func (e *DDLExec) getRecoverTableByTableName(s *ast.RecoverTableStmt, t *meta.Me } func (e *DDLExec) executeLockTables(s *ast.LockTablesStmt) error { + if !config.GetGlobalConfig().EnableTableLock { + return nil + } err := domain.GetDomain(e.ctx).DDL().LockTables(e.ctx, s) return err } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 45d87c65f6194..d8dec08b1b212 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/lock" @@ -100,6 +101,9 @@ func CheckPrivilege(activeRoles []*auth.RoleIdentity, pm privilege.Manager, vs [ // CheckTableLock checks the table lock. func CheckTableLock(ctx sessionctx.Context, is infoschema.InfoSchema, vs []visitInfo) error { + if !config.GetGlobalConfig().EnableTableLock { + return nil + } checker := lock.NewChecker(ctx, is) for i := range vs { err := checker.CheckTableLock(vs[i].db, vs[i].table, vs[i].privilege) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 1deb125aafd17..3134966c39b41 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -785,6 +785,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.TiDBOptJoinReorderThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinReorderThreshold) case TiDBCheckMb4ValueInUTF8: config.GetGlobalConfig().CheckMb4ValueInUTF8 = TiDBOptOn(val) + case TiDBEnableTableLock: + config.GetGlobalConfig().EnableTableLock = TiDBOptOn(val) case TiDBSlowQueryFile: s.SlowQueryFile = val case TiDBEnableFastAnalyze: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index bd8cf2d326f88..333e36b317d08 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -694,6 +694,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBEnableRadixJoin, BoolToIntStr(DefTiDBUseRadixJoin)}, {ScopeGlobal | ScopeSession, TiDBOptJoinReorderThreshold, strconv.Itoa(DefTiDBOptJoinReorderThreshold)}, {ScopeSession, TiDBCheckMb4ValueInUTF8, BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8)}, + {ScopeSession, TiDBEnableTableLock, BoolToIntStr(config.GetGlobalConfig().EnableTableLock)}, {ScopeSession, TiDBSlowQueryFile, ""}, {ScopeSession, TiDBWaitTableSplitFinish, BoolToIntStr(DefTiDBWaitTableSplitFinish)}, } diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 2f39b2bec34cc..80c99e920a183 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -138,6 +138,8 @@ const ( // tidb_skip_isolation_level_check is used to control whether to return error when set unsupported transaction // isolation level. TiDBSkipIsolationLevelCheck = "tidb_skip_isolation_level_check" + // tidb_enable_table_lock is used to control table lock feature. + TiDBEnableTableLock = "tidb_enable_table_lock" ) // TiDB system variable names that both in session and global scope. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 075751ebda922..5092ebafbe5dd 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -124,6 +124,8 @@ func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) { return config.GetGlobalConfig().Plugin.Load, true, nil case TiDBCheckMb4ValueInUTF8: return BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8), true, nil + case TiDBEnableTableLock: + return BoolToIntStr(config.GetGlobalConfig().EnableTableLock), true, nil } sVal, ok := s.systems[key] if ok { From 5345a683f9f12cefb1c689098cd43b2f7d7b5fec Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Tue, 21 May 2019 10:45:21 +0800 Subject: [PATCH 24/48] remove redundant file --- ddl/test.log] | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 ddl/test.log] diff --git a/ddl/test.log] b/ddl/test.log] deleted file mode 100644 index e69de29bb2d1d..0000000000000 From 52239f9f91fe3cdd74262b1a33831393889c997d Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 27 May 2019 11:07:50 +0800 Subject: [PATCH 25/48] address comment --- ddl/db_test.go | 8 ++++---- ddl/ddl_api.go | 7 +++++-- infoschema/infoschema.go | 4 ++++ lock/lock.go | 5 ++++- 4 files changed, 17 insertions(+), 7 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index bc9138540a226..bd87c3d6f9155 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2806,7 +2806,7 @@ func (s *testDBSuite2) TestLockTables(c *C) { // Test read lock. tk.MustExec("lock tables t1 read") tk.MustQuery("select * from t1") - tk2.MustExec("select * from t1") + tk2.MustQuery("select * from t1") _, err := tk.Exec("insert into t1 set a=1") c.Assert(terror.ErrorEqual(err, infoschema.ErrTableNotLockedForWrite), IsTrue) _, err = tk.Exec("update t1 set a=1") @@ -2913,11 +2913,11 @@ func (s *testDBSuite2) TestLockTables(c *C) { // Test for lock unsupported schema tables. _, err = tk2.Exec("lock tables performance_schema.global_status write") - c.Assert(terror.ErrorEqual(err, table.ErrUnsupportedOp), IsTrue) + c.Assert(terror.ErrorEqual(err, infoschema.ErrAccessDenied), IsTrue) _, err = tk2.Exec("lock tables information_schema.tables write") - c.Assert(terror.ErrorEqual(err, table.ErrUnsupportedOp), IsTrue) + c.Assert(terror.ErrorEqual(err, infoschema.ErrAccessDenied), IsTrue) _, err = tk2.Exec("lock tables mysql.db write") - c.Assert(terror.ErrorEqual(err, table.ErrUnsupportedOp), IsTrue) + c.Assert(terror.ErrorEqual(err, infoschema.ErrAccessDenied), IsTrue) tk.MustExec("unlock tables") tk2.MustExec("unlock tables") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index affa6ae0e5691..ddb0aea3ac595 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3234,8 +3234,11 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error // Check whether the table was already locked by other. for _, tl := range stmt.TableLocks { tb := tl.Table - if tb.Schema.L == "information_schema" || tb.Schema.L == "performance_schema" || tb.Schema.L == "mysql" { - return table.ErrUnsupportedOp + if tb.Schema.L == strings.ToLower(infoschema.Name) || tb.Schema.L == "performance_schema" || tb.Schema.L == mysql.SystemDB { + if ctx.GetSessionVars().User != nil { + return infoschema.ErrAccessDenied.GenWithStackByArgs(ctx.GetSessionVars().User.Username, ctx.GetSessionVars().User.Hostname) + } + return infoschema.ErrAccessDenied } schema, ok := is.SchemaByName(tb.Schema) if !ok { diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 14350bede37d6..7d4a8fd461782 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -68,6 +68,8 @@ var ( ErrNonuniqTable = terror.ClassSchema.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrNonuniqTable]) // ErrTableLocked returns when the table was locked by other session. ErrTableLocked = terror.ClassSchema.New(codeTableLocked, "Table '%s' was locked in %s by %v") + // ErrAccessDenied return when the user doesn't have the permission to access the table. + ErrAccessDenied = terror.ClassSchema.New(codeErrAccessDenied, mysql.MySQLErrName[mysql.ErrAccessDenied]) ) // InfoSchema is the interface used to retrieve the schema information. @@ -335,6 +337,7 @@ const ( codeErrTableNotLockedForWrite = mysql.ErrTableNotLockedForWrite codeErrTableNotLocked = mysql.ErrTableNotLocked codeErrNonuniqTable = mysql.ErrNonuniqTable + codeErrAccessDenied = mysql.ErrAccessDenied ) func init() { @@ -359,6 +362,7 @@ func init() { codeErrTableNotLockedForWrite: mysql.ErrTableNotLockedForWrite, codeErrTableNotLocked: mysql.ErrTableNotLocked, codeErrNonuniqTable: mysql.ErrNonuniqTable, + mysql.ErrAccessDenied: mysql.ErrAccessDenied, } terror.ErrClassToMySQLCodes[terror.ClassSchema] = schemaMySQLErrCodes initInfoSchemaDB() diff --git a/lock/lock.go b/lock/lock.go index 73e7131114625..58f80c9b81b76 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -14,9 +14,12 @@ package lock import ( + "strings" + "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/infoschema/perfschema" "github.com/pingcap/tidb/sessionctx" ) @@ -37,7 +40,7 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType return nil } // Below database are not support table lock. - if db == "information_schema" || db == "performance_schema" || db == "mysql" { + if db == strings.ToLower(infoschema.Name) || db == strings.ToLower(perfschema.Name) || db == mysql.SystemDB { return nil } switch privilege { From 116803a07e6fc64f9fadf94584cb608ffe11fd85 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 29 May 2019 12:02:59 +0800 Subject: [PATCH 26/48] address comment and add test --- config/config.toml.example | 2 +- ddl/db_test.go | 18 +++++++++-- ddl/ddl_api.go | 63 +++++++++++++++++++++----------------- go.mod | 2 +- go.sum | 2 ++ infoschema/infoschema.go | 6 ++-- lock/lock.go | 7 ++++- session/session.go | 13 ++++++-- sessionctx/context.go | 6 ++-- util/mock/context.go | 8 +++-- 10 files changed, 85 insertions(+), 42 deletions(-) diff --git a/config/config.toml.example b/config/config.toml.example index 87a03ce00bebf..f4dc58a7e00de 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -54,7 +54,7 @@ check-mb4-value-in-utf8 = true # treat-old-version-utf8-as-utf8mb4 use for upgrade compatibility. Set to true will treat old version table/column UTF8 charset as UTF8MB4. treat-old-version-utf8-as-utf8mb4 = true -# enable-table-lock is used to control table lock feature. Default false means not enable table lock. +# enable-table-lock is used to control table lock feature. Default is false, indicate the table lock feature is disabled. enable-table-lock = false [log] diff --git a/ddl/db_test.go b/ddl/db_test.go index bd87c3d6f9155..6193229cb9309 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2890,7 +2890,7 @@ func (s *testDBSuite2) TestLockTables(c *C) { tk2.MustExec("unlock tables") tk.MustExec("lock tables t1 write, t2 write") tk.MustExec("drop table t1") - tk.MustExec("create table t1 (a int)") + tk2.MustExec("create table t1 (a int)") tk.MustExec("lock tables t1 write, t2 read") // Test lock tables and drop database. @@ -2899,7 +2899,7 @@ func (s *testDBSuite2) TestLockTables(c *C) { tk.MustExec("create table test_lock.t3 (a int)") tk.MustExec("lock tables t1 write, test_lock.t3 write") tk.MustExec("drop database test_lock") - tk.MustExec("create table t3 (a int)") + tk2.MustExec("create table t3 (a int)") tk.MustExec("lock tables t1 write, t3 write") tk.MustExec("drop table t3") @@ -2919,6 +2919,20 @@ func (s *testDBSuite2) TestLockTables(c *C) { _, err = tk2.Exec("lock tables mysql.db write") c.Assert(terror.ErrorEqual(err, infoschema.ErrAccessDenied), IsTrue) + // Test create table/view when session is holding the table locks. + tk.MustExec("unlock tables") + tk.MustExec("lock tables t1 write, t2 read") + _, err = tk.Exec("create table t3 (a int)") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableNotLocked), IsTrue) + _, err = tk.Exec("create view v1 as select * from t1;") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableNotLocked), IsTrue) + + // Test for lock view was not supported. + tk.MustExec("unlock tables") + tk.MustExec("create view v1 as select * from t1;") + _, err = tk.Exec("lock tables v1 read") + c.Assert(terror.ErrorEqual(err, table.ErrUnsupportedOp), IsTrue) + tk.MustExec("unlock tables") tk2.MustExec("unlock tables") } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index ddb0aea3ac595..1cecb7b860e38 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -154,7 +154,6 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) if !ok { return errors.Trace(infoschema.ErrDatabaseNotExists) } - tbs := is.SchemaTables(schema) job := &model.Job{ SchemaID: old.ID, Type: model.ActionDropSchema, @@ -162,18 +161,20 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) } err = d.doDDLJob(ctx, job) - if err == nil { - // clear table locks. - tableLocks := make([]model.TableLockTpInfo, 0) - for _, tb := range tbs { - if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { - tableLocks = append(tableLocks, model.TableLockTpInfo{SchemaID: old.ID, TableID: tb.Meta().ID}) - } + err = d.callHookOnChanged(err) + if err != nil { + return errors.Trace(err) + } + // clear table locks. + tbs := is.SchemaTables(schema) + lockTableIDs := make([]int64, 0) + for _, tb := range tbs { + if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { + lockTableIDs = append(lockTableIDs, tb.Meta().ID) } - ctx.ReleaseTableLock(tableLocks) } - err = d.callHookOnChanged(err) - return errors.Trace(err) + ctx.ReleaseTableLockByTableIDs(lockTableIDs) + return nil } func checkTooLongSchema(schema model.CIStr) error { @@ -2843,14 +2844,14 @@ func (d *ddl) DropTable(ctx sessionctx.Context, ti ast.Ident) (err error) { } err = d.doDDLJob(ctx, job) - if err == nil { - if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { - ctx.ReleaseTableLock([]model.TableLockTpInfo{{SchemaID: schema.ID, TableID: tb.Meta().ID}}) - } - } - err = d.callHookOnChanged(err) - return errors.Trace(err) + if err != nil { + return errors.Trace(err) + } + if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { + ctx.ReleaseTableLockByTableIDs([]int64{tb.Meta().ID}) + } + return nil } // DropView will proceed even if some view in the list does not exists. @@ -2894,18 +2895,21 @@ func (d *ddl) TruncateTable(ctx sessionctx.Context, ti ast.Ident) error { } if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { // AddTableLock here to avoid this ddl job was execute successful but the session was been kill before return. + // The session will release its all table locks, if don't add table lock of the new table id here, + // the session maybe forgot release the new table id lock when this ddl job was execute successful but the session was been kill before return. ctx.AddTableLock(([]model.TableLockTpInfo{{SchemaID: schema.ID, TableID: newTableID, Tp: tb.Meta().Lock.Tp}})) } err = d.doDDLJob(ctx, job) - if err == nil { - if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { - ctx.ReleaseTableLock([]model.TableLockTpInfo{{SchemaID: schema.ID, TableID: tb.Meta().ID}}) - } - } else { - ctx.ReleaseTableLock([]model.TableLockTpInfo{{SchemaID: schema.ID, TableID: newTableID}}) - } err = d.callHookOnChanged(err) - return errors.Trace(err) + if err != nil { + ctx.ReleaseTableLockByTableIDs([]int64{newTableID}) + return errors.Trace(err) + } + if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { + ctx.ReleaseTableLockByTableIDs([]int64{tb.Meta().ID}) + } + return nil + } func (d *ddl) RenameTable(ctx sessionctx.Context, oldIdent, newIdent ast.Ident, isAlterTable bool) error { @@ -3246,12 +3250,15 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error } t, err := is.TableByName(tb.Schema, tb.Name) if err != nil { - return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(tb.Schema, tb.Name)) + return infoschema.ErrTableNotExists.GenWithStackByArgs(tb.Schema, tb.Name) } err = checkTableLocked(t.Meta(), tl.Type, sessionInfo) if err != nil { return err } + if t.Meta().IsView() { + return table.ErrUnsupportedOp.GenWithStackByArgs() + } if _, ok := uniqueTableID[t.Meta().ID]; ok { return infoschema.ErrNonuniqTable.GenWithStackByArgs(t.Meta().Name) } @@ -3276,7 +3283,7 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error ctx.AddTableLock(lockTables) err := d.doDDLJob(ctx, job) if err == nil { - ctx.ReleaseTableLock(unlockTables) + ctx.ReleaseTableLocks(unlockTables) ctx.AddTableLock(lockTables) } err = d.callHookOnChanged(err) diff --git a/go.mod b/go.mod index a9b267b6f1c12..9c41bed3f44fc 100644 --- a/go.mod +++ b/go.mod @@ -74,4 +74,4 @@ require ( sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190527021240-0a76ab79f7e5 +replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190529024536-fad921c77dcd diff --git a/go.sum b/go.sum index 5bdb758687388..29338ace96e5e 100644 --- a/go.sum +++ b/go.sum @@ -31,6 +31,8 @@ github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbp github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/crazycs520/parser v0.0.0-20190527021240-0a76ab79f7e5 h1:zv7F1x4nOzd5Rd+7eXava06Ka5T5Nm8OVTjGzBvviNw= github.com/crazycs520/parser v0.0.0-20190527021240-0a76ab79f7e5/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= +github.com/crazycs520/parser v0.0.0-20190529024536-fad921c77dcd h1:gLG7fMJc7dX8VwLRkHnH7tEGlwtfFYPfxPmEAvCwALk= +github.com/crazycs520/parser v0.0.0-20190529024536-fad921c77dcd/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= github.com/cznic/sortutil v0.0.0-20150617083342-4c7342852e65 h1:hxuZop6tSoOi0sxFzoGGYdRqNrPubyaIf9KoBG9tPiE= diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 7d4a8fd461782..dd9285e2e2b62 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -67,7 +67,7 @@ var ( // ErrNonuniqTable returns when none unique tables errors. ErrNonuniqTable = terror.ClassSchema.New(codeErrTableNotLocked, mysql.MySQLErrName[mysql.ErrNonuniqTable]) // ErrTableLocked returns when the table was locked by other session. - ErrTableLocked = terror.ClassSchema.New(codeTableLocked, "Table '%s' was locked in %s by %v") + ErrTableLocked = terror.ClassSchema.New(codeTableLocked, mysql.MySQLErrName[mysql.ErrTableLocked]) // ErrAccessDenied return when the user doesn't have the permission to access the table. ErrAccessDenied = terror.ClassSchema.New(codeErrAccessDenied, mysql.MySQLErrName[mysql.ErrAccessDenied]) ) @@ -312,8 +312,6 @@ func (h *Handle) EmptyClone() *Handle { // Schema error codes. const ( - codeTableLocked terror.ErrCode = 1 - codeDBDropExists terror.ErrCode = 1008 codeDatabaseNotExists = 1049 codeTableNotExists = 1146 @@ -338,6 +336,7 @@ const ( codeErrTableNotLocked = mysql.ErrTableNotLocked codeErrNonuniqTable = mysql.ErrNonuniqTable codeErrAccessDenied = mysql.ErrAccessDenied + codeTableLocked = mysql.ErrTableLocked ) func init() { @@ -363,6 +362,7 @@ func init() { codeErrTableNotLocked: mysql.ErrTableNotLocked, codeErrNonuniqTable: mysql.ErrNonuniqTable, mysql.ErrAccessDenied: mysql.ErrAccessDenied, + codeTableLocked: mysql.ErrTableLocked, } terror.ErrClassToMySQLCodes[terror.ClassSchema] = schemaMySQLErrCodes initInfoSchemaDB() diff --git a/lock/lock.go b/lock/lock.go index 58f80c9b81b76..d8443928bc37d 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -44,7 +44,12 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType return nil } switch privilege { - case mysql.CreatePriv, mysql.ShowDBPriv, mysql.AllPrivMask: + case mysql.ShowDBPriv, mysql.AllPrivMask: + return nil + case mysql.CreatePriv, mysql.CreateViewPriv: + if c.ctx.HasLockedTables() { + return infoschema.ErrTableNotLocked.GenWithStackByArgs(table) + } return nil } tb, err := c.is.TableByName(model.NewCIStr(db), model.NewCIStr(table)) diff --git a/session/session.go b/session/session.go index 363645107d494..06ffd7f712518 100644 --- a/session/session.go +++ b/session/session.go @@ -201,8 +201,8 @@ func (s *session) AddTableLock(locks []model.TableLockTpInfo) { s.lockedTables.Unlock() } -// ReleaseTableLock releases table lock in the session lock map. -func (s *session) ReleaseTableLock(locks []model.TableLockTpInfo) { +// ReleaseTableLocks releases table lock in the session lock map. +func (s *session) ReleaseTableLocks(locks []model.TableLockTpInfo) { s.lockedTables.Lock() for _, l := range locks { delete(s.lockedTables.holdLocks, l.TableID) @@ -210,6 +210,15 @@ func (s *session) ReleaseTableLock(locks []model.TableLockTpInfo) { s.lockedTables.Unlock() } +// ReleaseTableLockByTableIDs releases table lock in the session lock map by table ID. +func (s *session) ReleaseTableLockByTableIDs(tableIDs []int64) { + s.lockedTables.Lock() + for _, tblID := range tableIDs { + delete(s.lockedTables.holdLocks, tblID) + } + s.lockedTables.Unlock() +} + // CheckTableLocked checks the table lock. func (s *session) CheckTableLocked(tblID int64) (bool, model.TableLockType) { s.lockedTables.RLock() diff --git a/sessionctx/context.go b/sessionctx/context.go index 9ae13a544897b..5b430dc938352 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -86,8 +86,10 @@ type Context interface { DDLOwnerChecker() owner.DDLOwnerChecker // AddTableLock adds table lock to the session lock map. AddTableLock([]model.TableLockTpInfo) - // ReleaseTableLock releases table lock in the session lock map. - ReleaseTableLock(locks []model.TableLockTpInfo) + // ReleaseTableLocks releases table locks in the session lock map. + ReleaseTableLocks(locks []model.TableLockTpInfo) + // ReleaseTableLockByTableID releases table locks in the session lock map by table ID. + ReleaseTableLockByTableIDs(tableIDs []int64) // CheckTableLocked checks the table lock. CheckTableLocked(tblID int64) (bool, model.TableLockType) // GetAllTableLocks gets all table locks table id and db id hold by the session. diff --git a/util/mock/context.go b/util/mock/context.go index af877f71685ec..befd00241b08f 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -220,8 +220,12 @@ func (c *Context) StmtAddDirtyTableOP(op int, tid int64, handle int64, row []typ func (c *Context) AddTableLock(_ []model.TableLockTpInfo) { } -// ReleaseTableLock implements the sessionctx.Context interface. -func (c *Context) ReleaseTableLock(locks []model.TableLockTpInfo) { +// ReleaseTableLocks implements the sessionctx.Context interface. +func (c *Context) ReleaseTableLocks(locks []model.TableLockTpInfo) { +} + +// ReleaseTableLockByTableIDs implements the sessionctx.Context interface. +func (c *Context) ReleaseTableLockByTableIDs(tableIDs []int64) { } // CheckTableLocked implements the sessionctx.Context interface. From 4bb0cdc0200413a76f5dfae68cf154467f66f7b6 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Fri, 31 May 2019 14:54:06 +0800 Subject: [PATCH 27/48] address comment --- ddl/db_test.go | 14 ++++++++---- ddl/ddl_api.go | 8 +++---- ddl/table_lock.go | 39 ++++++++++++++++---------------- executor/ddl.go | 3 +++ go.mod | 2 +- go.sum | 6 ++--- infoschema/infoschema.go | 3 ++- infoschema/perfschema/const.go | 3 ++- lock/lock.go | 4 +--- session/session.go | 17 +++++++------- sessionctx/variable/session.go | 2 -- sessionctx/variable/sysvar.go | 1 - sessionctx/variable/tidb_vars.go | 3 --- sessionctx/variable/varsutil.go | 2 -- 14 files changed, 52 insertions(+), 55 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index c9dc65ba18122..e1e28fc0e932f 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2770,13 +2770,15 @@ func (s *testDBSuite2) TestLockTables(c *C) { // recover table lock config. originValue := config.GetGlobalConfig().EnableTableLock - defer tk.MustExec(fmt.Sprintf("set @@tidb_enable_table_lock=%v", originValue)) + defer func() { + config.GetGlobalConfig().EnableTableLock = originValue + }() // Test for enable table lock config. - tk.MustExec("set @@tidb_enable_table_lock=0") + config.GetGlobalConfig().EnableTableLock = false tk.MustExec("lock tables t1 write") checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone) - tk.MustExec("set @@tidb_enable_table_lock=1") + config.GetGlobalConfig().EnableTableLock = true // Test lock 1 table. tk.MustExec("lock tables t1 write") @@ -2950,10 +2952,12 @@ func (s *testDBSuite2) TestConcurrentLockTables(c *C) { // recover table lock config. originValue := config.GetGlobalConfig().EnableTableLock - defer tk.MustExec(fmt.Sprintf("set @@tidb_enable_table_lock=%v", originValue)) + defer func() { + config.GetGlobalConfig().EnableTableLock = originValue + }() // Test for enable table lock config. - tk.MustExec("set @@tidb_enable_table_lock=1") + config.GetGlobalConfig().EnableTableLock = true // Test concurrent lock tables read. sql1 := "lock tables t1 read" diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 1cecb7b860e38..62892071e18f0 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3238,7 +3238,7 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error // Check whether the table was already locked by other. for _, tl := range stmt.TableLocks { tb := tl.Table - if tb.Schema.L == strings.ToLower(infoschema.Name) || tb.Schema.L == "performance_schema" || tb.Schema.L == mysql.SystemDB { + if tb.Schema.L == infoschema.LowerName || tb.Schema.L == "performance_schema" || tb.Schema.L == mysql.SystemDB { if ctx.GetSessionVars().User != nil { return infoschema.ErrAccessDenied.GenWithStackByArgs(ctx.GetSessionVars().User.Username, ctx.GetSessionVars().User.Hostname) } @@ -3252,13 +3252,13 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error if err != nil { return infoschema.ErrTableNotExists.GenWithStackByArgs(tb.Schema, tb.Name) } + if t.Meta().IsView() { + return table.ErrUnsupportedOp.GenWithStackByArgs() + } err = checkTableLocked(t.Meta(), tl.Type, sessionInfo) if err != nil { return err } - if t.Meta().IsView() { - return table.ErrUnsupportedOp.GenWithStackByArgs() - } if _, ok := uniqueTableID[t.Meta().ID]; ok { return infoschema.ErrNonuniqTable.GenWithStackByArgs(t.Meta().Name) } diff --git a/ddl/table_lock.go b/ddl/table_lock.go index 9d5991302d6ec..a65753528a4a1 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -44,6 +44,9 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { } err = checkTableLocked(tbInfo, arg.LockTables[i].Tp, arg.SessionInfo) if err != nil { + // If any request table was locked by other session, just cancel this job. + // No need to rolling back the unlocked tables, MySQL will release the lock first + // and block if the request table was locked by other. job.State = model.JobStateCancelled return ver, errors.Trace(err) } @@ -94,8 +97,8 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { return ver, err } -// indexOfLockHolder gets the index of sessionInfo in the sessions. return -1 if sessions doesn't contain sessionInfo. -func indexOfLockHolder(sessions []model.SessionInfo, sessionInfo model.SessionInfo) int { +// findSessionInfoIndex gets the index of sessionInfo in the sessions. return -1 if sessions doesn't contain the sessionInfo. +func findSessionInfoIndex(sessions []model.SessionInfo, sessionInfo model.SessionInfo) int { for i := range sessions { if sessions[i].ServerID == sessionInfo.ServerID && sessions[i].SessionID == sessionInfo.SessionID { return i @@ -106,7 +109,7 @@ func indexOfLockHolder(sessions []model.SessionInfo, sessionInfo model.SessionIn // checkAndLockTable uses to check table locked and acquire the table lock for the request session. func checkAndLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error { - if tbInfo.Lock == nil || len(tbInfo.Lock.Sessions) == 0 { + if !tbInfo.IsLocked() { tbInfo.Lock = &model.TableLockInfo{ Tp: arg.LockTables[idx].Tp, } @@ -117,7 +120,7 @@ func checkAndLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) err return nil } if tbInfo.Lock.Tp == model.TableLockRead && arg.LockTables[idx].Tp == model.TableLockRead { - sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg.SessionInfo) + sessionIndex := findSessionInfoIndex(tbInfo.Lock.Sessions, arg.SessionInfo) // repeat lock. if sessionIndex >= 0 { return nil @@ -125,14 +128,15 @@ func checkAndLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) err tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, arg.SessionInfo) return nil } - // Unlock tables should execute before lock tables, so arg.SessionInfo in tbInfo.Lock.Sessions here is impossible. + // Unlock tables should execute before lock tables. + // Normally execute to here is impossible. return infoschema.ErrTableLocked.GenWithStackByArgs(tbInfo.Name.L, tbInfo.Lock.Tp, tbInfo.Lock.Sessions[0]) } // checkTableLocked uses to check whether table was locked. func checkTableLocked(tbInfo *model.TableInfo, lockTp model.TableLockType, sessionInfo model.SessionInfo) error { - if tbInfo.Lock == nil || len(tbInfo.Lock.Sessions) == 0 { + if !tbInfo.IsLocked() { return nil } if tbInfo.Lock.State == model.TableLockStatePreLock { @@ -141,12 +145,13 @@ func checkTableLocked(tbInfo *model.TableInfo, lockTp model.TableLockType, sessi if tbInfo.Lock.Tp == model.TableLockRead && lockTp == model.TableLockRead { return nil } - sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, sessionInfo) - // repeat lock. + sessionIndex := findSessionInfoIndex(tbInfo.Lock.Sessions, sessionInfo) + // If the request session already locked the table before, In other words, repeat lock. if sessionIndex >= 0 { if tbInfo.Lock.Tp == lockTp { return nil } + // If no other session locked this table. if len(tbInfo.Lock.Sessions) == 1 { return nil } @@ -169,16 +174,11 @@ func unlockTables(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, job.Args = []interface{}{arg} return ver, nil } else { - job.State = model.JobStateCancelled return ver, err } } - err = unlockTable(tbInfo, arg) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } + unlockTable(tbInfo, arg) ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) if err != nil { return ver, errors.Trace(err) @@ -190,15 +190,15 @@ func unlockTables(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, } // unlockTable uses to unlock table lock that hold by the session. -func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) error { - if tbInfo.Lock == nil { - return nil +func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) { + if !tbInfo.IsLocked() { + return } - sessionIndex := indexOfLockHolder(tbInfo.Lock.Sessions, arg.SessionInfo) + sessionIndex := findSessionInfoIndex(tbInfo.Lock.Sessions, arg.SessionInfo) if sessionIndex < 0 { // When session clean table lock, session maybe send unlock table even the table lock maybe not hold by the session. // so just return nil here. - return nil + return } oldSessionInfo := tbInfo.Lock.Sessions tbInfo.Lock.Sessions = oldSessionInfo[:sessionIndex] @@ -206,7 +206,6 @@ func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) error { if len(tbInfo.Lock.Sessions) == 0 { tbInfo.Lock = nil } - return nil } func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { diff --git a/executor/ddl.go b/executor/ddl.go index 3b1c69afda420..802d6a140e9b8 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -447,6 +447,9 @@ func (e *DDLExec) executeLockTables(s *ast.LockTablesStmt) error { } func (e *DDLExec) executeUnlockTables(s *ast.UnlockTablesStmt) error { + if !config.GetGlobalConfig().EnableTableLock { + return nil + } lockedTables := e.ctx.GetAllTableLocks() err := domain.GetDomain(e.ctx).DDL().UnlockTables(e.ctx, lockedTables) return err diff --git a/go.mod b/go.mod index 15f86bfc4c27c..e2215642c4fcd 100644 --- a/go.mod +++ b/go.mod @@ -74,4 +74,4 @@ require ( sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190529024536-fad921c77dcd +replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190531051457-66a41ea67e20 diff --git a/go.sum b/go.sum index 6f887b337bbc6..d75b350808e28 100644 --- a/go.sum +++ b/go.sum @@ -29,10 +29,8 @@ github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142/go.mod h1:F5haX7 github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbpBpLoyyu8B6e44T7hJy6potg= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= -github.com/crazycs520/parser v0.0.0-20190527021240-0a76ab79f7e5 h1:zv7F1x4nOzd5Rd+7eXava06Ka5T5Nm8OVTjGzBvviNw= -github.com/crazycs520/parser v0.0.0-20190527021240-0a76ab79f7e5/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= -github.com/crazycs520/parser v0.0.0-20190529024536-fad921c77dcd h1:gLG7fMJc7dX8VwLRkHnH7tEGlwtfFYPfxPmEAvCwALk= -github.com/crazycs520/parser v0.0.0-20190529024536-fad921c77dcd/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= +github.com/crazycs520/parser v0.0.0-20190531051457-66a41ea67e20 h1:pZ1OOP0Me+7LoV+V44X8DyYB1kCKgcTOLIQsLq43bFU= +github.com/crazycs520/parser v0.0.0-20190531051457-66a41ea67e20/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= github.com/cznic/sortutil v0.0.0-20150617083342-4c7342852e65 h1:hxuZop6tSoOi0sxFzoGGYdRqNrPubyaIf9KoBG9tPiE= diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index dd9285e2e2b62..843b20f72a1bf 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -96,7 +96,8 @@ type InfoSchema interface { // Information Schema Name. const ( - Name = "INFORMATION_SCHEMA" + Name = "INFORMATION_SCHEMA" + LowerName = "information_schema" ) type sortedTables []table.Table diff --git a/infoschema/perfschema/const.go b/infoschema/perfschema/const.go index db8f9af0f5e51..68dbbff24a080 100644 --- a/infoschema/perfschema/const.go +++ b/infoschema/perfschema/const.go @@ -15,7 +15,8 @@ package perfschema // Performance Schema Name. const ( - Name = "PERFORMANCE_SCHEMA" + Name = "PERFORMANCE_SCHEMA" + LowerName = "performance_schema" ) // perfSchemaTables is a shortcut to involve all table names. diff --git a/lock/lock.go b/lock/lock.go index d8443928bc37d..984ecd35b9c7e 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -14,8 +14,6 @@ package lock import ( - "strings" - "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/infoschema" @@ -40,7 +38,7 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType return nil } // Below database are not support table lock. - if db == strings.ToLower(infoschema.Name) || db == strings.ToLower(perfschema.Name) || db == mysql.SystemDB { + if db == infoschema.LowerName || db == perfschema.LowerName || db == mysql.SystemDB { return nil } switch privilege { diff --git a/session/session.go b/session/session.go index 06ffd7f712518..54fffb968bafb 100644 --- a/session/session.go +++ b/session/session.go @@ -187,7 +187,7 @@ type session struct { ddlOwnerChecker owner.DDLOwnerChecker // lockedTables use to record the table locks hold by the session. lockedTables struct { - sync.RWMutex + sync.Mutex holdLocks map[int64]model.TableLockTpInfo } } @@ -221,32 +221,33 @@ func (s *session) ReleaseTableLockByTableIDs(tableIDs []int64) { // CheckTableLocked checks the table lock. func (s *session) CheckTableLocked(tblID int64) (bool, model.TableLockType) { - s.lockedTables.RLock() - defer s.lockedTables.RUnlock() + s.lockedTables.Lock() lt, ok := s.lockedTables.holdLocks[tblID] if !ok { + s.lockedTables.Unlock() return false, model.TableLockNone } + s.lockedTables.Unlock() return true, lt.Tp } // GetAllTableLocks gets all table locks table id and db id hold by the session. func (s *session) GetAllTableLocks() []model.TableLockTpInfo { - s.lockedTables.RLock() + s.lockedTables.Lock() lockTpInfo := make([]model.TableLockTpInfo, 0, len(s.lockedTables.holdLocks)) for _, tl := range s.lockedTables.holdLocks { lockTpInfo = append(lockTpInfo, tl) } - s.lockedTables.RUnlock() + s.lockedTables.Unlock() return lockTpInfo } // HasLockedTables uses to check whether this session locked any tables. // If so, the session can only visit the table which locked by self. func (s *session) HasLockedTables() bool { - s.lockedTables.RLock() + s.lockedTables.Lock() b := len(s.lockedTables.holdLocks) > 0 - s.lockedTables.RUnlock() + s.lockedTables.Unlock() return b } @@ -1356,7 +1357,7 @@ func (s *session) ClearValue(key fmt.Stringer) { func (s *session) Close() { // TODO: do clean table locks when session exited without execute Close. // TODO: do clean table locks when tidb-server was `kill -9`. - if s.HasLockedTables() { + if config.GetGlobalConfig().EnableTableLock && s.HasLockedTables() { lockedTables := s.GetAllTableLocks() err := domain.GetDomain(s).DDL().UnlockTables(s, lockedTables) if err != nil { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 1eda117d77394..1d27574a76875 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -788,8 +788,6 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.TiDBOptJoinReorderThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinReorderThreshold) case TiDBCheckMb4ValueInUTF8: config.GetGlobalConfig().CheckMb4ValueInUTF8 = TiDBOptOn(val) - case TiDBEnableTableLock: - config.GetGlobalConfig().EnableTableLock = TiDBOptOn(val) case TiDBSlowQueryFile: s.SlowQueryFile = val case TiDBEnableFastAnalyze: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index ce1efa4840184..4d30c6e383df1 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -694,7 +694,6 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBEnableRadixJoin, BoolToIntStr(DefTiDBUseRadixJoin)}, {ScopeGlobal | ScopeSession, TiDBOptJoinReorderThreshold, strconv.Itoa(DefTiDBOptJoinReorderThreshold)}, {ScopeSession, TiDBCheckMb4ValueInUTF8, BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8)}, - {ScopeSession, TiDBEnableTableLock, BoolToIntStr(config.GetGlobalConfig().EnableTableLock)}, {ScopeSession, TiDBSlowQueryFile, ""}, {ScopeSession, TiDBWaitTableSplitFinish, BoolToIntStr(DefTiDBWaitTableSplitFinish)}, {ScopeSession, TiDBLowResolutionTSO, "0"}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index bcae8dd70ffa3..9f4941f6a7fc3 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -141,9 +141,6 @@ const ( // TiDBLowResolutionTSO is used for reading data with low resolution TSO which is updated once every two seconds TiDBLowResolutionTSO = "tidb_low_resolution_tso" - - // tidb_enable_table_lock is used to control table lock feature. - TiDBEnableTableLock = "tidb_enable_table_lock" ) // TiDB system variable names that both in session and global scope. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 1032a88fcb2ea..2180b61391032 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -124,8 +124,6 @@ func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) { return config.GetGlobalConfig().Plugin.Load, true, nil case TiDBCheckMb4ValueInUTF8: return BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8), true, nil - case TiDBEnableTableLock: - return BoolToIntStr(config.GetGlobalConfig().EnableTableLock), true, nil } sVal, ok := s.systems[key] if ok { From 65005453887efd091423e0fab19d60fcb761aff2 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Fri, 31 May 2019 17:33:33 +0800 Subject: [PATCH 28/48] add todo comment --- planner/core/planbuilder.go | 2 +- session/session.go | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 3c0aee81bf473..054d391177b25 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1834,7 +1834,7 @@ func (b *PlanBuilder) buildDDL(node ast.DDLNode) (Plan, error) { // Recover table command can only be executed by administrator. b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil) case *ast.LockTablesStmt: - // Nothing need to do. + // TODO: add Lock Table privilege check. } p := &DDL{Statement: node} return p, nil diff --git a/session/session.go b/session/session.go index 54fffb968bafb..e157fed697b46 100644 --- a/session/session.go +++ b/session/session.go @@ -187,6 +187,7 @@ type session struct { ddlOwnerChecker owner.DDLOwnerChecker // lockedTables use to record the table locks hold by the session. lockedTables struct { + // TODO: use double-checked locking to optimize performance. sync.Mutex holdLocks map[int64]model.TableLockTpInfo } From 2338305fbf2115589ddb9de4891c63d9203c0fe5 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Fri, 31 May 2019 17:59:58 +0800 Subject: [PATCH 29/48] remove redundant code --- ddl/table_lock.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/ddl/table_lock.go b/ddl/table_lock.go index a65753528a4a1..9949f391b27d5 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -51,8 +51,6 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { return ver, errors.Trace(err) } } - // Set job.schemaState to StateDeleteOnly just want to indicate this job was in running. - job.SchemaState = model.StateDeleteOnly } // Lock tables. @@ -185,7 +183,6 @@ func unlockTables(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, } arg.IndexOfUnlock++ job.Args = []interface{}{arg} - job.SchemaState = model.StateDeleteOnly return ver, nil } From 12463122c48153bad456847b276622f72f618534 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 3 Jun 2019 16:05:38 +0800 Subject: [PATCH 30/48] address comment --- ddl/ddl_api.go | 7 ++++--- ddl/table_lock.go | 8 ++++---- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 62892071e18f0..a262e0fbe54f0 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2895,8 +2895,9 @@ func (d *ddl) TruncateTable(ctx sessionctx.Context, ti ast.Ident) error { } if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { // AddTableLock here to avoid this ddl job was execute successful but the session was been kill before return. - // The session will release its all table locks, if don't add table lock of the new table id here, - // the session maybe forgot release the new table id lock when this ddl job was execute successful but the session was been kill before return. + // The session will release all table locks it holds, if we don't add the new locking table id here, + // the session may forget to release the new locked table id when this ddl job was executed successfully + // but the session was killed before return. ctx.AddTableLock(([]model.TableLockTpInfo{{SchemaID: schema.ID, TableID: newTableID, Tp: tb.Meta().Lock.Tp}})) } err = d.doDDLJob(ctx, job) @@ -3279,7 +3280,7 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{arg}, } - // AddTableLock here to avoid this ddl job was execute successful but the session was been kill before return. + // AddTableLock here is avoiding this job was executed successfully but the session was killed before return. ctx.AddTableLock(lockTables) err := d.doDDLJob(ctx, job) if err == nil { diff --git a/ddl/table_lock.go b/ddl/table_lock.go index 9949f391b27d5..25ba037f8a44f 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -176,8 +176,7 @@ func unlockTables(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, } } - unlockTable(tbInfo, arg) - ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) + ver, err = unlockTableAndUpdate(t, job, tbInfo, arg) if err != nil { return ver, errors.Trace(err) } @@ -186,8 +185,8 @@ func unlockTables(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, return ver, nil } -// unlockTable uses to unlock table lock that hold by the session. -func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) { +// unlockTableAndUpdate uses to unlock table lock that hold by the session. +func unlockTableAndUpdate(t *meta.Meta, job *model.Job, tbInfo *model.TableInfo, arg *lockTablesArg) (ver int64, err error) { if !tbInfo.IsLocked() { return } @@ -203,6 +202,7 @@ func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) { if len(tbInfo.Lock.Sessions) == 0 { tbInfo.Lock = nil } + return updateVersionAndTableInfo(t, job, tbInfo, true) } func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { From 871ccbcfa66d11e2d46880e4bc369539f6200fa3 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 5 Jun 2019 15:39:00 +0800 Subject: [PATCH 31/48] address comment --- ddl/ddl_api.go | 2 ++ ddl/table_lock.go | 6 +++--- lock/lock.go | 6 ++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index a262e0fbe54f0..90a359bec677d 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3239,6 +3239,8 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error // Check whether the table was already locked by other. for _, tl := range stmt.TableLocks { tb := tl.Table + // TODO: replace const string "performance_schema" with xxx.LowerName. + // Currently use perfschema.LowerName will have import cycle problem. if tb.Schema.L == infoschema.LowerName || tb.Schema.L == "performance_schema" || tb.Schema.L == mysql.SystemDB { if ctx.GetSessionVars().User != nil { return infoschema.ErrAccessDenied.GenWithStackByArgs(ctx.GetSessionVars().User.Username, ctx.GetSessionVars().User.Hostname) diff --git a/ddl/table_lock.go b/ddl/table_lock.go index 25ba037f8a44f..3a032b1b13e06 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -61,7 +61,7 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { if err != nil { return ver, err } - err = checkAndLockTable(tbInfo, arg.IndexOfLock, arg) + err = lockTable(tbInfo, arg.IndexOfLock, arg) if err != nil { job.State = model.JobStateCancelled return ver, err @@ -105,8 +105,8 @@ func findSessionInfoIndex(sessions []model.SessionInfo, sessionInfo model.Sessio return -1 } -// checkAndLockTable uses to check table locked and acquire the table lock for the request session. -func checkAndLockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error { +// lockTable uses to check table locked and acquire the table lock for the request session. +func lockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error { if !tbInfo.IsLocked() { tbInfo.Lock = &model.TableLockInfo{ Tp: arg.LockTables[idx].Tp, diff --git a/lock/lock.go b/lock/lock.go index 984ecd35b9c7e..ba1c815ff2106 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -43,6 +43,7 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType } switch privilege { case mysql.ShowDBPriv, mysql.AllPrivMask: + // AllPrivMask only used in show create table statement now. return nil case mysql.CreatePriv, mysql.CreateViewPriv: if c.ctx.HasLockedTables() { @@ -50,11 +51,8 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType } return nil } + // TODO: try to remove this get for speed up. tb, err := c.is.TableByName(model.NewCIStr(db), model.NewCIStr(table)) - // TODO: remove this - if infoschema.ErrTableNotExists.Equal(err) { - return nil - } if err != nil { return err } From d25ad4966290036150dfd0658a0a9b787b24bedd Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 5 Jun 2019 16:04:00 +0800 Subject: [PATCH 32/48] add comment --- lock/lock.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/lock/lock.go b/lock/lock.go index ba1c815ff2106..fe3ecc80d04e9 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -85,6 +85,8 @@ func checkLockTpMeetPrivilege(tp model.TableLockType, privilege mysql.PrivilegeT case model.TableLockWrite, model.TableLockWriteLocal: return true case model.TableLockRead: + // ShowDBPriv, AllPrivMask,CreatePriv, CreateViewPriv already checked before. + // The other privilege in read lock was not allowed. if privilege == mysql.SelectPriv { return true } From dcd6e5bea0fdc7b61396d436f45c0ee2b75f914b Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 5 Jun 2019 16:22:29 +0800 Subject: [PATCH 33/48] ignore table not exists error for drop table not exists --- config/config.go | 5 +++++ ddl/ddl_api.go | 17 ++++++++++------- executor/ddl.go | 4 ++-- lock/lock.go | 4 ++++ planner/core/optimizer.go | 2 +- session/session.go | 2 +- 6 files changed, 23 insertions(+), 11 deletions(-) diff --git a/config/config.go b/config/config.go index 10f1a5ec68fd5..ab92358ef7095 100644 --- a/config/config.go +++ b/config/config.go @@ -576,6 +576,11 @@ func hasRootPrivilege() bool { return os.Geteuid() == 0 } +// TableLockEnabled uses to check whether enabled the table lock feature. +func TableLockEnabled() bool { + return GetGlobalConfig().EnableTableLock +} + // ToLogConfig converts *Log to *logutil.LogConfig. func (l *Log) ToLogConfig() *logutil.LogConfig { return logutil.NewLogConfig(l.Level, l.Format, l.SlowQueryFile, l.File, l.DisableTimestamp) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 90a359bec677d..6cbe8a1aed1b9 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" field_types "github.com/pingcap/parser/types" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta/autoid" @@ -165,15 +166,17 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) if err != nil { return errors.Trace(err) } - // clear table locks. - tbs := is.SchemaTables(schema) - lockTableIDs := make([]int64, 0) - for _, tb := range tbs { - if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { - lockTableIDs = append(lockTableIDs, tb.Meta().ID) + if config.TableLockEnabled() { + // Clear table locks hold by the session. + tbs := is.SchemaTables(schema) + lockTableIDs := make([]int64, 0) + for _, tb := range tbs { + if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { + lockTableIDs = append(lockTableIDs, tb.Meta().ID) + } } + ctx.ReleaseTableLockByTableIDs(lockTableIDs) } - ctx.ReleaseTableLockByTableIDs(lockTableIDs) return nil } diff --git a/executor/ddl.go b/executor/ddl.go index 802d6a140e9b8..7ed5efb7ab09e 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -439,7 +439,7 @@ func (e *DDLExec) getRecoverTableByTableName(s *ast.RecoverTableStmt, t *meta.Me } func (e *DDLExec) executeLockTables(s *ast.LockTablesStmt) error { - if !config.GetGlobalConfig().EnableTableLock { + if !config.TableLockEnabled() { return nil } err := domain.GetDomain(e.ctx).DDL().LockTables(e.ctx, s) @@ -447,7 +447,7 @@ func (e *DDLExec) executeLockTables(s *ast.LockTablesStmt) error { } func (e *DDLExec) executeUnlockTables(s *ast.UnlockTablesStmt) error { - if !config.GetGlobalConfig().EnableTableLock { + if !config.TableLockEnabled() { return nil } lockedTables := e.ctx.GetAllTableLocks() diff --git a/lock/lock.go b/lock/lock.go index fe3ecc80d04e9..d21bb865c8b74 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -53,6 +53,10 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType } // TODO: try to remove this get for speed up. tb, err := c.is.TableByName(model.NewCIStr(db), model.NewCIStr(table)) + // Ignore this error for "drop table if not exists t1" when t1 doesn't exists. + if infoschema.ErrTableNotExists.Equal(err) { + return nil + } if err != nil { return err } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index d8dec08b1b212..d41dadde11710 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -101,7 +101,7 @@ func CheckPrivilege(activeRoles []*auth.RoleIdentity, pm privilege.Manager, vs [ // CheckTableLock checks the table lock. func CheckTableLock(ctx sessionctx.Context, is infoschema.InfoSchema, vs []visitInfo) error { - if !config.GetGlobalConfig().EnableTableLock { + if !config.TableLockEnabled() { return nil } checker := lock.NewChecker(ctx, is) diff --git a/session/session.go b/session/session.go index f56e6bbd76950..dff2f4b6d971b 100644 --- a/session/session.go +++ b/session/session.go @@ -1358,7 +1358,7 @@ func (s *session) ClearValue(key fmt.Stringer) { func (s *session) Close() { // TODO: do clean table locks when session exited without execute Close. // TODO: do clean table locks when tidb-server was `kill -9`. - if config.GetGlobalConfig().EnableTableLock && s.HasLockedTables() { + if config.TableLockEnabled() && s.HasLockedTables() { lockedTables := s.GetAllTableLocks() err := domain.GetDomain(s).DDL().UnlockTables(s, lockedTables) if err != nil { From e97b08d132cfe304c7c002cd235a0e31c4345a8b Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 5 Jun 2019 16:35:15 +0800 Subject: [PATCH 34/48] refine comment --- ddl/ddl_api.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 6cbe8a1aed1b9..ac061341c1763 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2897,7 +2897,7 @@ func (d *ddl) TruncateTable(ctx sessionctx.Context, ti ast.Ident) error { Args: []interface{}{newTableID}, } if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { - // AddTableLock here to avoid this ddl job was execute successful but the session was been kill before return. + // AddTableLock here to avoid this ddl job was executed successfully but the session was been kill before return. // The session will release all table locks it holds, if we don't add the new locking table id here, // the session may forget to release the new locked table id when this ddl job was executed successfully // but the session was killed before return. From 276b6c2f3dc876c32aab4756bd2015d5ca678cdb Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 5 Jun 2019 16:52:17 +0800 Subject: [PATCH 35/48] refine code --- ddl/ddl_api.go | 31 ++++++++++++++++++++----------- 1 file changed, 20 insertions(+), 11 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index ac061341c1763..e0777a5aab23f 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -166,17 +166,18 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) if err != nil { return errors.Trace(err) } - if config.TableLockEnabled() { - // Clear table locks hold by the session. - tbs := is.SchemaTables(schema) - lockTableIDs := make([]int64, 0) - for _, tb := range tbs { - if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { - lockTableIDs = append(lockTableIDs, tb.Meta().ID) - } + if !config.TableLockEnabled() { + return nil + } + // Clear table locks hold by the session. + tbs := is.SchemaTables(schema) + lockTableIDs := make([]int64, 0) + for _, tb := range tbs { + if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { + lockTableIDs = append(lockTableIDs, tb.Meta().ID) } - ctx.ReleaseTableLockByTableIDs(lockTableIDs) } + ctx.ReleaseTableLockByTableIDs(lockTableIDs) return nil } @@ -2851,6 +2852,9 @@ func (d *ddl) DropTable(ctx sessionctx.Context, ti ast.Ident) (err error) { if err != nil { return errors.Trace(err) } + if !config.TableLockEnabled() { + return nil + } if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { ctx.ReleaseTableLockByTableIDs([]int64{tb.Meta().ID}) } @@ -2896,7 +2900,7 @@ func (d *ddl) TruncateTable(ctx sessionctx.Context, ti ast.Ident) error { BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{newTableID}, } - if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { + if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok && config.TableLockEnabled() { // AddTableLock here to avoid this ddl job was executed successfully but the session was been kill before return. // The session will release all table locks it holds, if we don't add the new locking table id here, // the session may forget to release the new locked table id when this ddl job was executed successfully @@ -2906,9 +2910,14 @@ func (d *ddl) TruncateTable(ctx sessionctx.Context, ti ast.Ident) error { err = d.doDDLJob(ctx, job) err = d.callHookOnChanged(err) if err != nil { - ctx.ReleaseTableLockByTableIDs([]int64{newTableID}) + if config.TableLockEnabled() { + ctx.ReleaseTableLockByTableIDs([]int64{newTableID}) + } return errors.Trace(err) } + if !config.TableLockEnabled() { + return nil + } if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { ctx.ReleaseTableLockByTableIDs([]int64{tb.Meta().ID}) } From b05c612f065c573ab82b509b9f00318b99a38475 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 5 Jun 2019 17:07:39 +0800 Subject: [PATCH 36/48] address comment --- ddl/ddl_api.go | 1 - ddl/table_lock.go | 16 ++++++++++------ 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index e0777a5aab23f..a80b17c86642a 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2922,7 +2922,6 @@ func (d *ddl) TruncateTable(ctx sessionctx.Context, ti ast.Ident) error { ctx.ReleaseTableLockByTableIDs([]int64{tb.Meta().ID}) } return nil - } func (d *ddl) RenameTable(ctx sessionctx.Context, oldIdent, newIdent ast.Ident, isAlterTable bool) error { diff --git a/ddl/table_lock.go b/ddl/table_lock.go index 3a032b1b13e06..e2e047ae09fa6 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -176,17 +176,21 @@ func unlockTables(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, } } - ver, err = unlockTableAndUpdate(t, job, tbInfo, arg) - if err != nil { - return ver, errors.Trace(err) + needUpdateTableInfo := unlockTable(tbInfo, arg) + if needUpdateTableInfo { + ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) + if err != nil { + return ver, errors.Trace(err) + } } + arg.IndexOfUnlock++ job.Args = []interface{}{arg} return ver, nil } -// unlockTableAndUpdate uses to unlock table lock that hold by the session. -func unlockTableAndUpdate(t *meta.Meta, job *model.Job, tbInfo *model.TableInfo, arg *lockTablesArg) (ver int64, err error) { +// unlockTable uses to unlock table lock that hold by the session. +func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) (needUpdateTableInfo bool) { if !tbInfo.IsLocked() { return } @@ -202,7 +206,7 @@ func unlockTableAndUpdate(t *meta.Meta, job *model.Job, tbInfo *model.TableInfo, if len(tbInfo.Lock.Sessions) == 0 { tbInfo.Lock = nil } - return updateVersionAndTableInfo(t, job, tbInfo, true) + return true } func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { From 5bdc187df5b73fa6fc5cce7871910a36ec46da3f Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Thu, 6 Jun 2019 12:01:20 +0800 Subject: [PATCH 37/48] address comment --- ddl/table_lock.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/ddl/table_lock.go b/ddl/table_lock.go index e2e047ae09fa6..d88f1631c9143 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -192,13 +192,13 @@ func unlockTables(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, // unlockTable uses to unlock table lock that hold by the session. func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) (needUpdateTableInfo bool) { if !tbInfo.IsLocked() { - return + return false } sessionIndex := findSessionInfoIndex(tbInfo.Lock.Sessions, arg.SessionInfo) if sessionIndex < 0 { // When session clean table lock, session maybe send unlock table even the table lock maybe not hold by the session. - // so just return nil here. - return + // so just ignore and return here. + return false } oldSessionInfo := tbInfo.Lock.Sessions tbInfo.Lock.Sessions = oldSessionInfo[:sessionIndex] From 926ca8ddd09a47f4a81e540a10c97913e8dfb658 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Thu, 6 Jun 2019 17:17:30 +0800 Subject: [PATCH 38/48] add comment --- ddl/table_lock.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/ddl/table_lock.go b/ddl/table_lock.go index d88f1631c9143..1ef810ca12721 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -73,6 +73,8 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { tbInfo.Lock.State = model.TableLockStatePreLock tbInfo.Lock.TS = t.StartTS ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) + // If the state of the lock is public, It means the lock is a read lock and already locked by other session, + // so this request of lock table doesn't need pre-lock state, just update the TS and table info is ok. case model.TableLockStatePreLock, model.TableLockStatePublic: tbInfo.Lock.State = model.TableLockStatePublic tbInfo.Lock.TS = t.StartTS @@ -114,6 +116,8 @@ func lockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error { tbInfo.Lock.Sessions = append(tbInfo.Lock.Sessions, arg.SessionInfo) return nil } + // If the state of the lock is in pre-lock, then the lock must be locked by the current request. So we can just return here. + // Because the lock/unlock job must be serial execution in DDL owner now. if tbInfo.Lock.State == model.TableLockStatePreLock { return nil } From 2daf292cb4e1df4e5174977902be81121c416ce6 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Thu, 6 Jun 2019 21:00:50 +0800 Subject: [PATCH 39/48] fix ci --- ddl/table_lock.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/ddl/table_lock.go b/ddl/table_lock.go index 1ef810ca12721..cb4525626ee20 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -57,7 +57,8 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { if arg.IndexOfLock < len(arg.LockTables) { job.SchemaID = arg.LockTables[arg.IndexOfLock].SchemaID job.TableID = arg.LockTables[arg.IndexOfLock].TableID - tbInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) + var tbInfo *model.TableInfo + tbInfo, err = getTableInfoAndCancelFaultJob(t, job, job.SchemaID) if err != nil { return ver, err } @@ -175,9 +176,8 @@ func unlockTables(t *meta.Meta, job *model.Job, arg *lockTablesArg) (ver int64, arg.IndexOfUnlock++ job.Args = []interface{}{arg} return ver, nil - } else { - return ver, err } + return ver, err } needUpdateTableInfo := unlockTable(tbInfo, arg) From 055f73f3b1b1d0f4386a14193be24b2b4f08a279 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 10 Jun 2019 09:52:34 +0800 Subject: [PATCH 40/48] update parser in go.mod --- go.mod | 4 +--- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index c98c16fd28eca..2ce023eb3e123 100644 --- a/go.mod +++ b/go.mod @@ -40,7 +40,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190528074401-b942b3f4108f github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190529103304-95494e4022ce + github.com/pingcap/parser v0.0.0-20190610014346-0ea1bf007f79 github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 @@ -74,5 +74,3 @@ require ( sourcegraph.com/sourcegraph/appdash v0.0.0-20180531100431-4c381bd170b4 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) - -replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190603081135-d3658ad11a24 diff --git a/go.sum b/go.sum index f5bc2ceb6497b..4904ddcebe1dd 100644 --- a/go.sum +++ b/go.sum @@ -29,8 +29,6 @@ github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142/go.mod h1:F5haX7 github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbpBpLoyyu8B6e44T7hJy6potg= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= -github.com/crazycs520/parser v0.0.0-20190603081135-d3658ad11a24 h1:4Yd6f6o4K1wxplBNGD5LbHjhg340N70Nv73CrYjnJww= -github.com/crazycs520/parser v0.0.0-20190603081135-d3658ad11a24/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= github.com/cznic/sortutil v0.0.0-20150617083342-4c7342852e65 h1:hxuZop6tSoOi0sxFzoGGYdRqNrPubyaIf9KoBG9tPiE= @@ -164,6 +162,8 @@ github.com/pingcap/kvproto v0.0.0-20190528074401-b942b3f4108f/go.mod h1:QMdbTAXC github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= +github.com/pingcap/parser v0.0.0-20190610014346-0ea1bf007f79 h1:fpmVSKVnJ3AoaF4mAdu+6pm98Ds9FvOHDFjI/+j8zI8= +github.com/pingcap/parser v0.0.0-20190610014346-0ea1bf007f79/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQkc/YLUttUdQXoH77gtuWCvL4= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= From d04d8cff73f3f4788f04593e49f020fa08f866ec Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 10 Jun 2019 10:36:58 +0800 Subject: [PATCH 41/48] fix race by avoid race --- session/session.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/session/session.go b/session/session.go index 3e418ac4ebf91..4e7ff051091bb 100644 --- a/session/session.go +++ b/session/session.go @@ -1361,7 +1361,7 @@ func (s *session) ClearValue(key fmt.Stringer) { func (s *session) Close() { // TODO: do clean table locks when session exited without execute Close. // TODO: do clean table locks when tidb-server was `kill -9`. - if config.TableLockEnabled() && s.HasLockedTables() { + if s.HasLockedTables() && config.TableLockEnabled() { lockedTables := s.GetAllTableLocks() err := domain.GetDomain(s).DDL().UnlockTables(s, lockedTables) if err != nil { From 101102a8d0f1baba164afdf89e89148ef7c4b7b2 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 17 Jun 2019 10:08:19 +0800 Subject: [PATCH 42/48] Address comment --- ddl/ddl_api.go | 9 ++------- ddl/table.go | 11 ++++++----- ddl/table_lock.go | 3 +-- executor/ddl.go | 6 ++---- lock/lock.go | 3 +-- 5 files changed, 12 insertions(+), 20 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index b8766b15661b6..b571200bf6d5e 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3234,7 +3234,6 @@ func buildPartitionInfo(meta *model.TableInfo, d *ddl, spec *ast.AlterTableSpec) // LockTables uses to execute lock tables statement. func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error { lockTables := make([]model.TableLockTpInfo, 0, len(stmt.TableLocks)) - is := d.infoHandle.Get() sessionInfo := model.SessionInfo{ ServerID: d.GetID(), SessionID: ctx.GetSessionVars().ConnectionID, @@ -3251,13 +3250,9 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error } return infoschema.ErrAccessDenied } - schema, ok := is.SchemaByName(tb.Schema) - if !ok { - return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(tb.Schema) - } - t, err := is.TableByName(tb.Schema, tb.Name) + schema, t, err := d.getSchemaAndTableByIdent(ctx, ast.Ident{Schema: tb.Schema, Name: tb.Name}) if err != nil { - return infoschema.ErrTableNotExists.GenWithStackByArgs(tb.Schema, tb.Name) + return errors.Trace(err) } if t.Meta().IsView() { return table.ErrUnsupportedOp.GenWithStackByArgs() diff --git a/ddl/table.go b/ddl/table.go index b7c6d5d715473..1ee96144732ac 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -455,12 +455,13 @@ func getTableInfoAndCancelFaultJob(t *meta.Meta, job *model.Job, schemaID int64) func checkTableExistAndCancelNonExistJob(t *meta.Meta, job *model.Job, schemaID int64) (*model.TableInfo, error) { tblInfo, err := getTableInfo(t, job.TableID, schemaID) - if err != nil { - if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { - job.State = model.JobStateCancelled - } + if err == nil { + return tblInfo, nil + } + if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { + job.State = model.JobStateCancelled } - return tblInfo, err + return nil, err } func getTableInfo(t *meta.Meta, tableID, schemaID int64) (*model.TableInfo, error) { diff --git a/ddl/table_lock.go b/ddl/table_lock.go index cb4525626ee20..03d798a3911ab 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -74,7 +74,7 @@ func onLockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { tbInfo.Lock.State = model.TableLockStatePreLock tbInfo.Lock.TS = t.StartTS ver, err = updateVersionAndTableInfo(t, job, tbInfo, true) - // If the state of the lock is public, It means the lock is a read lock and already locked by other session, + // If the state of the lock is public, it means the lock is a read lock and already locked by other session, // so this request of lock table doesn't need pre-lock state, just update the TS and table info is ok. case model.TableLockStatePreLock, model.TableLockStatePublic: tbInfo.Lock.State = model.TableLockStatePublic @@ -222,7 +222,6 @@ func onUnlockTables(t *meta.Meta, job *model.Job) (ver int64, err error) { } ver, err = unlockTables(t, job, arg) - if arg.IndexOfUnlock == len(arg.UnlockTables) { job.FinishTableJob(model.JobStateDone, model.StateNone, ver, nil) } diff --git a/executor/ddl.go b/executor/ddl.go index 7ed5efb7ab09e..d0d04f9d6e857 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -442,8 +442,7 @@ func (e *DDLExec) executeLockTables(s *ast.LockTablesStmt) error { if !config.TableLockEnabled() { return nil } - err := domain.GetDomain(e.ctx).DDL().LockTables(e.ctx, s) - return err + return domain.GetDomain(e.ctx).DDL().LockTables(e.ctx, s) } func (e *DDLExec) executeUnlockTables(s *ast.UnlockTablesStmt) error { @@ -451,6 +450,5 @@ func (e *DDLExec) executeUnlockTables(s *ast.UnlockTablesStmt) error { return nil } lockedTables := e.ctx.GetAllTableLocks() - err := domain.GetDomain(e.ctx).DDL().UnlockTables(e.ctx, lockedTables) - return err + return domain.GetDomain(e.ctx).DDL().UnlockTables(e.ctx, lockedTables) } diff --git a/lock/lock.go b/lock/lock.go index d21bb865c8b74..21bb1f63657d8 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -74,8 +74,7 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType return nil } - switch privilege { - case mysql.SelectPriv: + if privilege == mysql.SelectPriv { switch tb.Meta().Lock.Tp { case model.TableLockRead, model.TableLockWriteLocal: return nil From af345bd49a05663e2cf07e76a9ddfe354e59fcc8 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 17 Jun 2019 12:06:59 +0800 Subject: [PATCH 43/48] try to remove mutex --- session/session.go | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/session/session.go b/session/session.go index 4e7ff051091bb..a4de9a6594356 100644 --- a/session/session.go +++ b/session/session.go @@ -188,75 +188,59 @@ type session struct { // lockedTables use to record the table locks hold by the session. lockedTables struct { // TODO: use double-checked locking to optimize performance. - sync.Mutex holdLocks map[int64]model.TableLockTpInfo } } // AddTableLock adds table lock to the session lock map. func (s *session) AddTableLock(locks []model.TableLockTpInfo) { - s.lockedTables.Lock() for _, l := range locks { s.lockedTables.holdLocks[l.TableID] = l } - s.lockedTables.Unlock() } // ReleaseTableLocks releases table lock in the session lock map. func (s *session) ReleaseTableLocks(locks []model.TableLockTpInfo) { - s.lockedTables.Lock() for _, l := range locks { delete(s.lockedTables.holdLocks, l.TableID) } - s.lockedTables.Unlock() } // ReleaseTableLockByTableIDs releases table lock in the session lock map by table ID. func (s *session) ReleaseTableLockByTableIDs(tableIDs []int64) { - s.lockedTables.Lock() for _, tblID := range tableIDs { delete(s.lockedTables.holdLocks, tblID) } - s.lockedTables.Unlock() } // CheckTableLocked checks the table lock. func (s *session) CheckTableLocked(tblID int64) (bool, model.TableLockType) { - s.lockedTables.Lock() lt, ok := s.lockedTables.holdLocks[tblID] if !ok { - s.lockedTables.Unlock() return false, model.TableLockNone } - s.lockedTables.Unlock() return true, lt.Tp } // GetAllTableLocks gets all table locks table id and db id hold by the session. func (s *session) GetAllTableLocks() []model.TableLockTpInfo { - s.lockedTables.Lock() lockTpInfo := make([]model.TableLockTpInfo, 0, len(s.lockedTables.holdLocks)) for _, tl := range s.lockedTables.holdLocks { lockTpInfo = append(lockTpInfo, tl) } - s.lockedTables.Unlock() return lockTpInfo } // HasLockedTables uses to check whether this session locked any tables. // If so, the session can only visit the table which locked by self. func (s *session) HasLockedTables() bool { - s.lockedTables.Lock() b := len(s.lockedTables.holdLocks) > 0 - s.lockedTables.Unlock() return b } // ReleaseAllTableLocks releases all table locks hold by the session. func (s *session) ReleaseAllTableLocks() { - s.lockedTables.Lock() s.lockedTables.holdLocks = make(map[int64]model.TableLockTpInfo) - s.lockedTables.Unlock() } // DDLOwnerChecker returns s.ddlOwnerChecker. From a316a7b1ae32419d462759bf10f458001c75a8a9 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 17 Jun 2019 15:17:46 +0800 Subject: [PATCH 44/48] address comment --- ddl/db_test.go | 4 ++-- planner/core/planbuilder.go | 2 +- session/session.go | 25 +++++++++++-------------- 3 files changed, 14 insertions(+), 17 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 57739f757a548..483b1c1c435d1 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2940,7 +2940,7 @@ func (s *testDBSuite2) TestLockTables(c *C) { } // TestConcurrentLockTables test concurrent lock/unlock tables. -func (s *testDBSuite2) TestConcurrentLockTables(c *C) { +func (s *testDBSuite4) TestConcurrentLockTables(c *C) { s.tk = testkit.NewTestKit(c, s.store) tk2 := testkit.NewTestKit(c, s.store) tk := s.tk @@ -2991,7 +2991,7 @@ func (s *testDBSuite2) TestConcurrentLockTables(c *C) { tk2.MustExec("unlock tables") } -func (s *testDBSuite2) testParallelExecSQL(c *C, sql1, sql2 string, se1, se2 session.Session, f checkRet) { +func (s *testDBSuite4) testParallelExecSQL(c *C, sql1, sql2 string, se1, se2 session.Session, f checkRet) { callback := &ddl.TestDDLCallback{} times := 0 callback.OnJobRunBeforeExported = func(job *model.Job) { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index fbca834d52e09..29c981b1e649b 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1973,7 +1973,7 @@ func (b *PlanBuilder) buildDDL(node ast.DDLNode) (Plan, error) { case *ast.RecoverTableStmt: // Recover table command can only be executed by administrator. b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil) - case *ast.LockTablesStmt: + case *ast.LockTablesStmt, *ast.UnlockTablesStmt: // TODO: add Lock Table privilege check. } p := &DDL{Statement: node} diff --git a/session/session.go b/session/session.go index 3c44425bcd860..691548ddac471 100644 --- a/session/session.go +++ b/session/session.go @@ -186,36 +186,33 @@ type session struct { // ddlOwnerChecker is used in `select tidb_is_ddl_owner()` statement; ddlOwnerChecker owner.DDLOwnerChecker // lockedTables use to record the table locks hold by the session. - lockedTables struct { - // TODO: use double-checked locking to optimize performance. - holdLocks map[int64]model.TableLockTpInfo - } + lockedTables map[int64]model.TableLockTpInfo } // AddTableLock adds table lock to the session lock map. func (s *session) AddTableLock(locks []model.TableLockTpInfo) { for _, l := range locks { - s.lockedTables.holdLocks[l.TableID] = l + s.lockedTables[l.TableID] = l } } // ReleaseTableLocks releases table lock in the session lock map. func (s *session) ReleaseTableLocks(locks []model.TableLockTpInfo) { for _, l := range locks { - delete(s.lockedTables.holdLocks, l.TableID) + delete(s.lockedTables, l.TableID) } } // ReleaseTableLockByTableIDs releases table lock in the session lock map by table ID. func (s *session) ReleaseTableLockByTableIDs(tableIDs []int64) { for _, tblID := range tableIDs { - delete(s.lockedTables.holdLocks, tblID) + delete(s.lockedTables, tblID) } } // CheckTableLocked checks the table lock. func (s *session) CheckTableLocked(tblID int64) (bool, model.TableLockType) { - lt, ok := s.lockedTables.holdLocks[tblID] + lt, ok := s.lockedTables[tblID] if !ok { return false, model.TableLockNone } @@ -224,8 +221,8 @@ func (s *session) CheckTableLocked(tblID int64) (bool, model.TableLockType) { // GetAllTableLocks gets all table locks table id and db id hold by the session. func (s *session) GetAllTableLocks() []model.TableLockTpInfo { - lockTpInfo := make([]model.TableLockTpInfo, 0, len(s.lockedTables.holdLocks)) - for _, tl := range s.lockedTables.holdLocks { + lockTpInfo := make([]model.TableLockTpInfo, 0, len(s.lockedTables)) + for _, tl := range s.lockedTables { lockTpInfo = append(lockTpInfo, tl) } return lockTpInfo @@ -234,13 +231,13 @@ func (s *session) GetAllTableLocks() []model.TableLockTpInfo { // HasLockedTables uses to check whether this session locked any tables. // If so, the session can only visit the table which locked by self. func (s *session) HasLockedTables() bool { - b := len(s.lockedTables.holdLocks) > 0 + b := len(s.lockedTables) > 0 return b } // ReleaseAllTableLocks releases all table locks hold by the session. func (s *session) ReleaseAllTableLocks() { - s.lockedTables.holdLocks = make(map[int64]model.TableLockTpInfo) + s.lockedTables = make(map[int64]model.TableLockTpInfo) } // DDLOwnerChecker returns s.ddlOwnerChecker. @@ -1606,7 +1603,7 @@ func createSession(store kv.Storage) (*session, error) { plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory.Load()) } s.mu.values = make(map[fmt.Stringer]interface{}) - s.lockedTables.holdLocks = make(map[int64]model.TableLockTpInfo) + s.lockedTables = make(map[int64]model.TableLockTpInfo) domain.BindDomain(s, dom) // session implements variable.GlobalVarAccessor. Bind it to ctx. s.sessionVars.GlobalVarsAccessor = s @@ -1630,7 +1627,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory.Load()) } s.mu.values = make(map[fmt.Stringer]interface{}) - s.lockedTables.holdLocks = make(map[int64]model.TableLockTpInfo) + s.lockedTables = make(map[int64]model.TableLockTpInfo) domain.BindDomain(s, dom) // session implements variable.GlobalVarAccessor. Bind it to ctx. s.sessionVars.GlobalVarsAccessor = s From 099ca8f627a5f3682218228d82462b7aa4b64d27 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 17 Jun 2019 15:53:02 +0800 Subject: [PATCH 45/48] fix operate database when session holding table locks. --- ddl/db_test.go | 17 ++++++++++++++++- lock/lock.go | 27 ++++++++++++++++++++++++++- table/table.go | 5 ++++- 3 files changed, 46 insertions(+), 3 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 483b1c1c435d1..18db2b2178a37 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2900,7 +2900,6 @@ func (s *testDBSuite2) TestLockTables(c *C) { tk.MustExec("create database test_lock") tk.MustExec("create table test_lock.t3 (a int)") tk.MustExec("lock tables t1 write, test_lock.t3 write") - tk.MustExec("drop database test_lock") tk2.MustExec("create table t3 (a int)") tk.MustExec("lock tables t1 write, t3 write") tk.MustExec("drop table t3") @@ -2935,6 +2934,22 @@ func (s *testDBSuite2) TestLockTables(c *C) { _, err = tk.Exec("lock tables v1 read") c.Assert(terror.ErrorEqual(err, table.ErrUnsupportedOp), IsTrue) + // Test for create/drop/alter database when session is holding the table locks. + tk.MustExec("unlock tables") + tk.MustExec("lock table t1 write") + _, err = tk.Exec("drop database test") + c.Assert(terror.ErrorEqual(err, table.ErrLockOrActiveTransaction), IsTrue) + _, err = tk.Exec("create database test_lock") + c.Assert(terror.ErrorEqual(err, table.ErrLockOrActiveTransaction), IsTrue) + _, err = tk.Exec("alter database test charset='utf8mb4'") + c.Assert(terror.ErrorEqual(err, table.ErrLockOrActiveTransaction), IsTrue) + // Test alter/drop database when other session is holding the table locks of the database. + tk2.MustExec("create database test_lock2") + _, err = tk2.Exec("drop database test") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + _, err = tk2.Exec("alter database test charset='utf8mb4'") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + tk.MustExec("unlock tables") tk2.MustExec("unlock tables") } diff --git a/lock/lock.go b/lock/lock.go index 21bb1f63657d8..cda684de15988 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/infoschema/perfschema" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/table" ) // Checker uses to check tables lock. @@ -34,13 +35,17 @@ func NewChecker(ctx sessionctx.Context, is infoschema.InfoSchema) *Checker { // CheckTableLock uses to check table lock. func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType) error { - if db == "" || table == "" { + if db == "" && table == "" { return nil } // Below database are not support table lock. if db == infoschema.LowerName || db == perfschema.LowerName || db == mysql.SystemDB { return nil } + // check operation on database. + if table == "" { + return c.CheckLockInDB(db, privilege) + } switch privilege { case mysql.ShowDBPriv, mysql.AllPrivMask: // AllPrivMask only used in show create table statement now. @@ -96,3 +101,23 @@ func checkLockTpMeetPrivilege(tp model.TableLockType, privilege mysql.PrivilegeT } return false } + +func (c *Checker) CheckLockInDB(db string, privilege mysql.PrivilegeType) error { + if c.ctx.HasLockedTables() { + switch privilege { + case mysql.CreatePriv, mysql.DropPriv, mysql.AlterPriv: + return table.ErrLockOrActiveTransaction.GenWithStackByArgs() + } + } + if privilege == mysql.CreatePriv { + return nil + } + tables := c.is.SchemaTables(model.NewCIStr(db)) + for _, tbl := range tables { + err := c.CheckTableLock(db, tbl.Meta().Name.L, privilege) + if err != nil { + return err + } + } + return nil +} diff --git a/table/table.go b/table/table.go index 2fba027e6df1c..a67ecdf67f5ff 100644 --- a/table/table.go +++ b/table/table.go @@ -79,11 +79,12 @@ var ( ErrTruncateWrongValue = terror.ClassTable.New(codeTruncateWrongValue, "incorrect value") // ErrTruncatedWrongValueForField returns for truncate wrong value for field. ErrTruncatedWrongValueForField = terror.ClassTable.New(codeTruncateWrongValue, mysql.MySQLErrName[mysql.ErrTruncatedWrongValueForField]) - // ErrUnknownPartition returns unknown partition error. ErrUnknownPartition = terror.ClassTable.New(codeUnknownPartition, mysql.MySQLErrName[mysql.ErrUnknownPartition]) // ErrNoPartitionForGivenValue returns table has no partition for value. ErrNoPartitionForGivenValue = terror.ClassTable.New(codeNoPartitionForGivenValue, mysql.MySQLErrName[mysql.ErrNoPartitionForGivenValue]) + // ErrLockOrActiveTransaction returns when execute unsupported statement in a lock session or an active transaction. + ErrLockOrActiveTransaction = terror.ClassTable.New(codeLockOrActiveTransaction, mysql.MySQLErrName[mysql.ErrLockOrActiveTransaction]) ) // RecordIterFunc is used for low-level record iteration. @@ -212,6 +213,7 @@ const ( codeUnknownPartition = mysql.ErrUnknownPartition codeNoPartitionForGivenValue = mysql.ErrNoPartitionForGivenValue + codeLockOrActiveTransaction = mysql.ErrLockOrActiveTransaction ) // Slice is used for table sorting. @@ -234,6 +236,7 @@ func init() { codeTruncateWrongValue: mysql.ErrTruncatedWrongValueForField, codeUnknownPartition: mysql.ErrUnknownPartition, codeNoPartitionForGivenValue: mysql.ErrNoPartitionForGivenValue, + codeLockOrActiveTransaction: mysql.ErrLockOrActiveTransaction, } terror.ErrClassToMySQLCodes[terror.ClassTable] = tableMySQLErrCodes } From dcfe7f6185209d26bbfeb22b0780ce18e7e0b868 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 17 Jun 2019 17:13:07 +0800 Subject: [PATCH 46/48] fix ci --- lock/lock.go | 1 + 1 file changed, 1 insertion(+) diff --git a/lock/lock.go b/lock/lock.go index cda684de15988..c1247a4ea8e67 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -102,6 +102,7 @@ func checkLockTpMeetPrivilege(tp model.TableLockType, privilege mysql.PrivilegeT return false } +// CheckLockInDB uses to check operation on database. func (c *Checker) CheckLockInDB(db string, privilege mysql.PrivilegeType) error { if c.ctx.HasLockedTables() { switch privilege { From c511160914c8025a4731f27555d0db58e566c510 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Tue, 18 Jun 2019 11:33:47 +0800 Subject: [PATCH 47/48] add todo --- lock/lock.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/lock/lock.go b/lock/lock.go index c1247a4ea8e67..f96879b2a0a5d 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -52,6 +52,8 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType return nil case mysql.CreatePriv, mysql.CreateViewPriv: if c.ctx.HasLockedTables() { + // TODO: For `create table t_exists ...` statement, mysql will check out `t_exists` first, but in TiDB now, + // will return below error first. return infoschema.ErrTableNotLocked.GenWithStackByArgs(table) } return nil From defcb0a7b72e868ba6eaeba66566f3f787028fc3 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 18 Jun 2019 14:29:06 +0800 Subject: [PATCH 48/48] fix data race in test --- ddl/db_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 8c56a71eeb127..d176d60130f0b 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2959,7 +2959,7 @@ func (s *testDBSuite2) TestLockTables(c *C) { } // TestConcurrentLockTables test concurrent lock/unlock tables. -func (s *testDBSuite4) TestConcurrentLockTables(c *C) { +func (s *testDBSuite2) TestConcurrentLockTables(c *C) { s.tk = testkit.NewTestKit(c, s.store) tk2 := testkit.NewTestKit(c, s.store) tk := s.tk @@ -3010,7 +3010,7 @@ func (s *testDBSuite4) TestConcurrentLockTables(c *C) { tk2.MustExec("unlock tables") } -func (s *testDBSuite4) testParallelExecSQL(c *C, sql1, sql2 string, se1, se2 session.Session, f checkRet) { +func (s *testDBSuite2) testParallelExecSQL(c *C, sql1, sql2 string, se1, se2 session.Session, f checkRet) { callback := &ddl.TestDDLCallback{} times := 0 callback.OnJobRunBeforeExported = func(job *model.Job) {