Skip to content

Commit c879480

Browse files
authored
*: add admin cleanup table lock syntax support (pingcap#10423)
1 parent 04cfda8 commit c879480

File tree

10 files changed

+134
-15
lines changed

10 files changed

+134
-15
lines changed

ddl/db_test.go

+16
Original file line numberDiff line numberDiff line change
@@ -3172,6 +3172,22 @@ func (s *testDBSuite2) TestLockTables(c *C) {
31723172
_, err = tk2.Exec("alter database test charset='utf8mb4'")
31733173
c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue)
31743174

3175+
// Test for admin cleanup table locks.
3176+
tk.MustExec("unlock tables")
3177+
tk.MustExec("lock table t1 write, t2 write")
3178+
_, err = tk2.Exec("lock tables t1 write, t2 read")
3179+
c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue)
3180+
tk2.MustExec("admin cleanup table lock t1,t2")
3181+
checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone)
3182+
checkTableLock(c, tk.Se, "test", "t2", model.TableLockNone)
3183+
// cleanup unlocked table.
3184+
tk2.MustExec("admin cleanup table lock t1,t2")
3185+
checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone)
3186+
checkTableLock(c, tk.Se, "test", "t2", model.TableLockNone)
3187+
tk2.MustExec("lock tables t1 write, t2 read")
3188+
checkTableLock(c, tk2.Se, "test", "t1", model.TableLockWrite)
3189+
checkTableLock(c, tk2.Se, "test", "t2", model.TableLockRead)
3190+
31753191
tk.MustExec("unlock tables")
31763192
tk2.MustExec("unlock tables")
31773193
}

ddl/ddl.go

+1
Original file line numberDiff line numberDiff line change
@@ -249,6 +249,7 @@ type DDL interface {
249249
RenameTable(ctx sessionctx.Context, oldTableIdent, newTableIdent ast.Ident, isAlterTable bool) error
250250
LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error
251251
UnlockTables(ctx sessionctx.Context, lockedTables []model.TableLockTpInfo) error
252+
CleanupTableLock(ctx sessionctx.Context, tables []*ast.TableName) error
252253

253254
// GetLease returns current schema lease time.
254255
GetLease() time.Duration

ddl/ddl_api.go

+70-8
Original file line numberDiff line numberDiff line change
@@ -45,6 +45,7 @@ import (
4545
"github.com/pingcap/tidb/table/tables"
4646
"github.com/pingcap/tidb/types"
4747
driver "github.com/pingcap/tidb/types/parser_driver"
48+
"github.com/pingcap/tidb/util"
4849
"github.com/pingcap/tidb/util/chunk"
4950
"github.com/pingcap/tidb/util/logutil"
5051
"github.com/pingcap/tidb/util/mock"
@@ -3386,16 +3387,12 @@ func (d *ddl) LockTables(ctx sessionctx.Context, stmt *ast.LockTablesStmt) error
33863387
SessionID: ctx.GetSessionVars().ConnectionID,
33873388
}
33883389
uniqueTableID := make(map[int64]struct{})
3389-
// Check whether the table was already locked by other.
3390+
// Check whether the table was already locked by another.
33903391
for _, tl := range stmt.TableLocks {
33913392
tb := tl.Table
3392-
// TODO: replace const string "performance_schema" with xxx.LowerName.
3393-
// Currently use perfschema.LowerName will have import cycle problem.
3394-
if tb.Schema.L == infoschema.LowerName || tb.Schema.L == "performance_schema" || tb.Schema.L == mysql.SystemDB {
3395-
if ctx.GetSessionVars().User != nil {
3396-
return infoschema.ErrAccessDenied.GenWithStackByArgs(ctx.GetSessionVars().User.Username, ctx.GetSessionVars().User.Hostname)
3397-
}
3398-
return infoschema.ErrAccessDenied
3393+
err := throwErrIfInMemOrSysDB(ctx, tb.Schema.L)
3394+
if err != nil {
3395+
return err
33993396
}
34003397
schema, t, err := d.getSchemaAndTableByIdent(ctx, ast.Ident{Schema: tb.Schema, Name: tb.Name})
34013398
if err != nil {
@@ -3467,10 +3464,75 @@ func (d *ddl) UnlockTables(ctx sessionctx.Context, unlockTables []model.TableLoc
34673464
return errors.Trace(err)
34683465
}
34693466

3467+
func throwErrIfInMemOrSysDB(ctx sessionctx.Context, dbLowerName string) error {
3468+
if util.IsMemOrSysDB(dbLowerName) {
3469+
if ctx.GetSessionVars().User != nil {
3470+
return infoschema.ErrAccessDenied.GenWithStackByArgs(ctx.GetSessionVars().User.Username, ctx.GetSessionVars().User.Hostname)
3471+
}
3472+
return infoschema.ErrAccessDenied.GenWithStackByArgs("", "")
3473+
}
3474+
return nil
3475+
}
3476+
3477+
func (d *ddl) CleanupTableLock(ctx sessionctx.Context, tables []*ast.TableName) error {
3478+
uniqueTableID := make(map[int64]struct{})
3479+
cleanupTables := make([]model.TableLockTpInfo, 0, len(tables))
3480+
unlockedTablesNum := 0
3481+
// Check whether the table was already locked by another.
3482+
for _, tb := range tables {
3483+
err := throwErrIfInMemOrSysDB(ctx, tb.Schema.L)
3484+
if err != nil {
3485+
return err
3486+
}
3487+
schema, t, err := d.getSchemaAndTableByIdent(ctx, ast.Ident{Schema: tb.Schema, Name: tb.Name})
3488+
if err != nil {
3489+
return errors.Trace(err)
3490+
}
3491+
if t.Meta().IsView() {
3492+
return table.ErrUnsupportedOp
3493+
}
3494+
// Maybe the table t was not locked, but still try to unlock this table.
3495+
// If we skip unlock the table here, the job maybe not consistent with the job.Query.
3496+
// eg: unlock tables t1,t2; If t2 is not locked and skip here, then the job will only unlock table t1,
3497+
// and this behaviour is not consistent with the sql query.
3498+
if !t.Meta().IsLocked() {
3499+
unlockedTablesNum++
3500+
}
3501+
if _, ok := uniqueTableID[t.Meta().ID]; ok {
3502+
return infoschema.ErrNonuniqTable.GenWithStackByArgs(t.Meta().Name)
3503+
}
3504+
uniqueTableID[t.Meta().ID] = struct{}{}
3505+
cleanupTables = append(cleanupTables, model.TableLockTpInfo{SchemaID: schema.ID, TableID: t.Meta().ID})
3506+
}
3507+
// If the num of cleanupTables is 0, or all cleanupTables is unlocked, just return here.
3508+
if len(cleanupTables) == 0 || len(cleanupTables) == unlockedTablesNum {
3509+
return nil
3510+
}
3511+
3512+
arg := &lockTablesArg{
3513+
UnlockTables: cleanupTables,
3514+
IsCleanup: true,
3515+
}
3516+
job := &model.Job{
3517+
SchemaID: cleanupTables[0].SchemaID,
3518+
TableID: cleanupTables[0].TableID,
3519+
Type: model.ActionUnlockTable,
3520+
BinlogInfo: &model.HistoryInfo{},
3521+
Args: []interface{}{arg},
3522+
}
3523+
err := d.doDDLJob(ctx, job)
3524+
if err == nil {
3525+
ctx.ReleaseTableLocks(cleanupTables)
3526+
}
3527+
err = d.callHookOnChanged(err)
3528+
return errors.Trace(err)
3529+
}
3530+
34703531
type lockTablesArg struct {
34713532
LockTables []model.TableLockTpInfo
34723533
IndexOfLock int
34733534
UnlockTables []model.TableLockTpInfo
34743535
IndexOfUnlock int
34753536
SessionInfo model.SessionInfo
3537+
IsCleanup bool
34763538
}

ddl/table_lock.go

+5
Original file line numberDiff line numberDiff line change
@@ -198,6 +198,11 @@ func unlockTable(tbInfo *model.TableInfo, arg *lockTablesArg) (needUpdateTableIn
198198
if !tbInfo.IsLocked() {
199199
return false
200200
}
201+
if arg.IsCleanup {
202+
tbInfo.Lock = nil
203+
return true
204+
}
205+
201206
sessionIndex := findSessionInfoIndex(tbInfo.Lock.Sessions, arg.SessionInfo)
202207
if sessionIndex < 0 {
203208
// When session clean table lock, session maybe send unlock table even the table lock maybe not hold by the session.

executor/ddl.go

+8
Original file line numberDiff line numberDiff line change
@@ -113,6 +113,9 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) {
113113
err = e.executeLockTables(x)
114114
case *ast.UnlockTablesStmt:
115115
err = e.executeUnlockTables(x)
116+
case *ast.CleanupTableLockStmt:
117+
err = e.executeCleanupTableLock(x)
118+
116119
}
117120
if err != nil {
118121
return e.toErr(err)
@@ -453,3 +456,8 @@ func (e *DDLExec) executeUnlockTables(s *ast.UnlockTablesStmt) error {
453456
lockedTables := e.ctx.GetAllTableLocks()
454457
return domain.GetDomain(e.ctx).DDL().UnlockTables(e.ctx, lockedTables)
455458
}
459+
460+
func (e *DDLExec) executeCleanupTableLock(s *ast.CleanupTableLockStmt) error {
461+
err := domain.GetDomain(e.ctx).DDL().CleanupTableLock(e.ctx, s.Tables)
462+
return err
463+
}

infoschema/infoschema.go

+3-2
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ import (
2323
"github.com/pingcap/tidb/kv"
2424
"github.com/pingcap/tidb/meta/autoid"
2525
"github.com/pingcap/tidb/table"
26+
"github.com/pingcap/tidb/util"
2627
)
2728

2829
var (
@@ -96,8 +97,8 @@ type InfoSchema interface {
9697

9798
// Information Schema Name.
9899
const (
99-
Name = "INFORMATION_SCHEMA"
100-
LowerName = "information_schema"
100+
Name = util.InformationSchemaName
101+
LowerName = util.InformationSchemaLowerName
101102
)
102103

103104
type sortedTables []table.Table

infoschema/perfschema/const.go

+4-2
Original file line numberDiff line numberDiff line change
@@ -13,10 +13,12 @@
1313

1414
package perfschema
1515

16+
import "github.com/pingcap/tidb/util"
17+
1618
// Performance Schema Name.
1719
const (
18-
Name = "PERFORMANCE_SCHEMA"
19-
LowerName = "performance_schema"
20+
Name = util.PerformanceSchemaName
21+
LowerName = util.PerformanceSchemaLowerName
2022
)
2123

2224
// perfSchemaTables is a shortcut to involve all table names.

lock/lock.go

+3-3
Original file line numberDiff line numberDiff line change
@@ -17,9 +17,9 @@ import (
1717
"github.com/pingcap/parser/model"
1818
"github.com/pingcap/parser/mysql"
1919
"github.com/pingcap/tidb/infoschema"
20-
"github.com/pingcap/tidb/infoschema/perfschema"
2120
"github.com/pingcap/tidb/sessionctx"
2221
"github.com/pingcap/tidb/table"
22+
"github.com/pingcap/tidb/util"
2323
)
2424

2525
// Checker uses to check tables lock.
@@ -38,8 +38,8 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType
3838
if db == "" && table == "" {
3939
return nil
4040
}
41-
// Below database are not support table lock.
42-
if db == infoschema.LowerName || db == perfschema.LowerName || db == mysql.SystemDB {
41+
// System DB and memory DB are not support table lock.
42+
if util.IsMemOrSysDB(db) {
4343
return nil
4444
}
4545
// check operation on database.

planner/core/planbuilder.go

+3
Original file line numberDiff line numberDiff line change
@@ -1984,6 +1984,9 @@ func (b *PlanBuilder) buildDDL(node ast.DDLNode) (Plan, error) {
19841984
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil)
19851985
case *ast.LockTablesStmt, *ast.UnlockTablesStmt:
19861986
// TODO: add Lock Table privilege check.
1987+
case *ast.CleanupTableLockStmt:
1988+
// This command can only be executed by administrator.
1989+
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil)
19871990
}
19881991
p := &DDL{Statement: node}
19891992
return p, nil

util/misc.go

+21
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ import (
2020

2121
"github.com/pingcap/errors"
2222
"github.com/pingcap/parser"
23+
"github.com/pingcap/parser/mysql"
2324
"github.com/pingcap/parser/terror"
2425
"github.com/pingcap/tidb/util/logutil"
2526
"go.uber.org/zap"
@@ -130,3 +131,23 @@ func SyntaxWarn(err error) error {
130131
}
131132
return parser.ErrParse.GenWithStackByArgs(syntaxErrorPrefix, err.Error())
132133
}
134+
135+
const (
136+
// InformationSchemaName is the `INFORMATION_SCHEMA` database name.
137+
InformationSchemaName = "INFORMATION_SCHEMA"
138+
// InformationSchemaLowerName is the `INFORMATION_SCHEMA` database lower name.
139+
InformationSchemaLowerName = "information_schema"
140+
// PerformanceSchemaName is the `PERFORMANCE_SCHEMA` database name.
141+
PerformanceSchemaName = "PERFORMANCE_SCHEMA"
142+
// PerformanceSchemaLowerName is the `PERFORMANCE_SCHEMA` database lower name.
143+
PerformanceSchemaLowerName = "performance_schema"
144+
)
145+
146+
// IsMemOrSysDB uses to check whether dbLowerName is memory database or system database.
147+
func IsMemOrSysDB(dbLowerName string) bool {
148+
switch dbLowerName {
149+
case InformationSchemaLowerName, PerformanceSchemaLowerName, mysql.SystemDB:
150+
return true
151+
}
152+
return false
153+
}

0 commit comments

Comments
 (0)