Skip to content

Commit

Permalink
*: add admin cleanup table lock syntax
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 committed Jun 19, 2019
1 parent b0d6c5b commit 47d218a
Show file tree
Hide file tree
Showing 8 changed files with 91 additions and 2 deletions.
16 changes: 16 additions & 0 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2954,6 +2954,22 @@ func (s *testDBSuite2) TestLockTables(c *C) {
_, err = tk2.Exec("alter database test charset='utf8mb4'")
c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue)

// Test for admin cleanup table locks.
tk.MustExec("unlock tables")
tk.MustExec("lock table t1 write, t2 write")
_, err = tk2.Exec("lock tables t1 write, t2 read")
c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue)
tk2.MustExec("admin cleanup table lock t1,t2")
checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone)
checkTableLock(c, tk.Se, "test", "t2", model.TableLockNone)
// cleanup unlocked table.
tk2.MustExec("admin cleanup table lock t1,t2")
checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone)
checkTableLock(c, tk.Se, "test", "t2", model.TableLockNone)
tk2.MustExec("lock tables t1 write, t2 read")
checkTableLock(c, tk2.Se, "test", "t1", model.TableLockWrite)
checkTableLock(c, tk2.Se, "test", "t2", model.TableLockRead)

tk.MustExec("unlock tables")
tk2.MustExec("unlock tables")
}
Expand Down
1 change: 1 addition & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -251,6 +251,7 @@ type DDL interface {
RenameTable(ctx sessionctx.Context, oldTableIdent, newTableIdent ast.Ident, isAlterTable bool) error
LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error
UnlockTables(ctx sessionctx.Context, lockedTables []model.TableLockTpInfo) error
CleanupTableLock(ctx sessionctx.Context, tables []*ast.TableName) error

// GetLease returns current schema lease time.
GetLease() time.Duration
Expand Down
53 changes: 53 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -3366,10 +3366,63 @@ func (d *ddl) UnlockTables(ctx sessionctx.Context, unlockTables []model.TableLoc
return errors.Trace(err)
}

func (d *ddl) CleanupTableLock(ctx sessionctx.Context, tables []*ast.TableName) error {
is := d.infoHandle.Get()
uniqueTableID := make(map[int64]struct{})
cleanupTables := make([]model.TableLockTpInfo, 0, len(tables))
// Check whether the table was already locked by other.
for _, tb := range tables {
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)
}
t, err := is.TableByName(tb.Schema, tb.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(tb.Schema, tb.Name))
}
tbInfo := t.Meta()

if tbInfo.Lock == nil || len(tbInfo.Lock.Sessions) == 0 {
continue
}

if _, ok := uniqueTableID[t.Meta().ID]; ok {
return infoschema.ErrNonuniqTable.GenWithStackByArgs(t.Meta().Name)
}
uniqueTableID[t.Meta().ID] = struct{}{}
cleanupTables = append(cleanupTables, model.TableLockTpInfo{SchemaID: schema.ID, TableID: t.Meta().ID})
}
if len(cleanupTables) == 0 {
return nil
}

arg := &lockTablesArg{
UnlockTables: cleanupTables,
IsCleanup: true,
}
job := &model.Job{
SchemaID: cleanupTables[0].SchemaID,
TableID: cleanupTables[0].TableID,
Type: model.ActionUnlockTable,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{arg},
}
err := d.doDDLJob(ctx, job)
if err == nil {
ctx.ReleaseTableLocks(cleanupTables)
}
err = d.callHookOnChanged(err)
return errors.Trace(err)
}

type lockTablesArg struct {
LockTables []model.TableLockTpInfo
IndexOfLock int
UnlockTables []model.TableLockTpInfo
IndexOfUnlock int
SessionInfo model.SessionInfo
IsCleanup bool
}
5 changes: 5 additions & 0 deletions ddl/table_lock.go
Original file line number Diff line number Diff line change
Expand Up @@ -198,6 +198,11 @@ func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) (needUpdateTableIn
if !tbInfo.IsLocked() {
return false
}
if arg.IsCleanup {
tbInfo.Lock = nil
return true
}

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.
Expand Down
8 changes: 8 additions & 0 deletions executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -113,6 +113,9 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.RecordBatch) (err error)
err = e.executeLockTables(x)
case *ast.UnlockTablesStmt:
err = e.executeUnlockTables(x)
case *ast.CleanupTableLockStmt:
err = e.executeCleanupTableLock(x)

}
if err != nil {
return e.toErr(err)
Expand Down Expand Up @@ -452,3 +455,8 @@ func (e *DDLExec) executeUnlockTables(s *ast.UnlockTablesStmt) error {
lockedTables := e.ctx.GetAllTableLocks()
return domain.GetDomain(e.ctx).DDL().UnlockTables(e.ctx, lockedTables)
}

func (e *DDLExec) executeCleanupTableLock(s *ast.CleanupTableLockStmt) error {
err := domain.GetDomain(e.ctx).DDL().CleanupTableLock(e.ctx, s.Tables)
return err
}
2 changes: 2 additions & 0 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -93,3 +93,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-20190619022326-690470b074e1
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@ github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142 h1:3jFq2xL4ZajGK
github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
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-20190619022326-690470b074e1 h1:stj/09+uCI3xogQMbpA8uN003Dpgc12wwoTdBHiARVE=
github.com/crazycs520/parser v0.0.0-20190619022326-690470b074e1/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=
Expand Down Expand Up @@ -133,8 +135,6 @@ github.com/pingcap/kvproto v0.0.0-20190528074401-b942b3f4108f h1:EXZvZmZl+n4PGSR
github.com/pingcap/kvproto v0.0.0-20190528074401-b942b3f4108f/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY=
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-20190612052718-3b36f86d9b7b h1:4/+CIoFd4AMLZbjDpqhoa9IByT/lVcg+13/W/UgNVXM=
github.com/pingcap/parser v0.0.0-20190612052718-3b36f86d9b7b/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
github.com/pingcap/pd v2.1.11+incompatible h1:LHn92NDzkkpivZj+hyyuXIoSdyMMQbLRqQZg8ZQcz6o=
github.com/pingcap/pd v2.1.11+incompatible/go.mod h1:nD3+EoYes4+aNNODO99ES59V83MZSI+dFbhyr667a0E=
github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU=
Expand Down
4 changes: 4 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1975,6 +1975,10 @@ func (b *PlanBuilder) buildDDL(node ast.DDLNode) (Plan, error) {
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil)
case *ast.LockTablesStmt, *ast.UnlockTablesStmt:
// TODO: add Lock Table privilege check.
case *ast.CleanupTableLockStmt:
// This command can only be executed by administrator.
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil)

}
p := &DDL{Statement: node}
return p, nil
Expand Down

0 comments on commit 47d218a

Please sign in to comment.