diff --git a/ddl/db_test.go b/ddl/db_test.go index eae4e09abe0ba..ab23e40a458e7 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -5630,6 +5630,78 @@ func (s *testDBSuite4) TestConcurrentLockTables(c *C) { tk2.MustExec("unlock tables") } +func (s *testDBSuite4) TestLockTableReadOnly(c *C) { + if israce.RaceEnabled { + c.Skip("skip race test") + } + tk := testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(c, s.store) + tk2.MustExec("use test") + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1,t2") + defer func() { + tk.MustExec("alter table t1 read write") + tk.MustExec("alter table t2 read write") + tk.MustExec("drop table if exists t1,t2") + }() + tk.MustExec("create table t1 (a int key, b int)") + tk.MustExec("create table t2 (a int key)") + + tk.MustExec("alter table t1 read only") + tk.MustQuery("select * from t1") + tk2.MustQuery("select * from t1") + _, err := tk.Exec("insert into t1 set a=1, b=2") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + _, err = tk.Exec("update t1 set a=1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + _, err = tk.Exec("delete from t1") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + + _, err = tk2.Exec("insert into t1 set a=1, b=2") + 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("alter table t1 read only") + _, err = tk2.Exec("insert into t1 set a=1, b=2") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableLocked), IsTrue) + tk.MustExec("alter table t1 read write") + + tk.MustExec("lock tables t1 read") + c.Assert(terror.ErrorEqual(tk.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked), IsTrue) + c.Assert(terror.ErrorEqual(tk2.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked), IsTrue) + tk.MustExec("lock tables t1 write") + c.Assert(terror.ErrorEqual(tk.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked), IsTrue) + c.Assert(terror.ErrorEqual(tk2.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked), IsTrue) + tk.MustExec("lock tables t1 write local") + c.Assert(terror.ErrorEqual(tk.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked), IsTrue) + c.Assert(terror.ErrorEqual(tk2.ExecToErr("alter table t1 read only"), infoschema.ErrTableLocked), IsTrue) + tk.MustExec("unlock tables") + + tk.MustExec("alter table t1 read only") + c.Assert(terror.ErrorEqual(tk.ExecToErr("lock tables t1 read"), infoschema.ErrTableLocked), IsTrue) + c.Assert(terror.ErrorEqual(tk2.ExecToErr("lock tables t1 read"), infoschema.ErrTableLocked), IsTrue) + c.Assert(terror.ErrorEqual(tk.ExecToErr("lock tables t1 write"), infoschema.ErrTableLocked), IsTrue) + c.Assert(terror.ErrorEqual(tk2.ExecToErr("lock tables t1 write"), infoschema.ErrTableLocked), IsTrue) + c.Assert(terror.ErrorEqual(tk.ExecToErr("lock tables t1 write local"), infoschema.ErrTableLocked), IsTrue) + c.Assert(terror.ErrorEqual(tk2.ExecToErr("lock tables t1 write local"), infoschema.ErrTableLocked), IsTrue) + tk.MustExec("admin cleanup table lock t1") + tk2.MustExec("insert into t1 set a=1, b=2") + + tk.MustExec("set tidb_enable_amend_pessimistic_txn = 1") + tk.MustExec("begin pessimistic") + tk.MustQuery("select * from t1 where a = 1").Check(testkit.Rows("1 2")) + tk2.MustExec("update t1 set b = 3") + tk2.MustExec("alter table t1 read only") + tk2.MustQuery("select * from t1 where a = 1").Check(testkit.Rows("1 3")) + tk.MustQuery("select * from t1 where a = 1").Check(testkit.Rows("1 2")) + tk.MustExec("update t1 set b = 4") + c.Assert(terror.ErrorEqual(tk.ExecToErr("commit"), domain.ErrInfoSchemaChanged), IsTrue) + tk2.MustExec("alter table t1 read write") +} + func (s *testDBSuite4) testParallelExecSQL(c *C, sql1, sql2 string, se1, se2 session.Session, f checkRet) { callback := &ddl.TestDDLCallback{} times := 0 diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 0e819144f297d..e9dd8151c2bfa 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2425,6 +2425,24 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A err = d.DropTablePartition(ctx, ident, spec) case ast.AlterTableTruncatePartition: err = d.TruncateTablePartition(ctx, ident, spec) + case ast.AlterTableWriteable: + if !config.TableLockEnabled() { + return nil + } + tName := &ast.TableName{Schema: ident.Schema, Name: ident.Name} + if spec.Writeable { + err = d.CleanupTableLock(ctx, []*ast.TableName{tName}) + } else { + lockStmt := &ast.LockTablesStmt{ + TableLocks: []ast.TableLock{ + { + Table: tName, + Type: model.TableLockReadOnly, + }, + }, + } + err = d.LockTables(ctx, lockStmt) + } case ast.AlterTableExchangePartition: err = d.ExchangeTablePartition(ctx, ident, spec) case ast.AlterTableAddConstraint: diff --git a/ddl/table_lock.go b/ddl/table_lock.go index 1b24c95114204..6309c8ca1aff7 100644 --- a/ddl/table_lock.go +++ b/ddl/table_lock.go @@ -122,7 +122,8 @@ func lockTable(tbInfo *model.TableInfo, idx int, arg *lockTablesArg) error { 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 && arg.LockTables[idx].Tp == model.TableLockRead) || + (tbInfo.Lock.Tp == model.TableLockReadOnly && arg.LockTables[idx].Tp == model.TableLockReadOnly) { sessionIndex := findSessionInfoIndex(tbInfo.Lock.Sessions, arg.SessionInfo) // repeat lock. if sessionIndex >= 0 { @@ -145,7 +146,8 @@ func checkTableLocked(tbInfo *model.TableInfo, lockTp model.TableLockType, sessi if tbInfo.Lock.State == model.TableLockStatePreLock { return nil } - if tbInfo.Lock.Tp == model.TableLockRead && lockTp == model.TableLockRead { + if (tbInfo.Lock.Tp == model.TableLockRead && lockTp == model.TableLockRead) || + (tbInfo.Lock.Tp == model.TableLockReadOnly && lockTp == model.TableLockReadOnly) { return nil } sessionIndex := findSessionInfoIndex(tbInfo.Lock.Sessions, sessionInfo) @@ -154,8 +156,8 @@ func checkTableLocked(tbInfo *model.TableInfo, lockTp model.TableLockType, sessi if tbInfo.Lock.Tp == lockTp { return nil } - // If no other session locked this table. - if len(tbInfo.Lock.Sessions) == 1 { + // If no other session locked this table, and it is not a read only lock (session unrelated). + if len(tbInfo.Lock.Sessions) == 1 && tbInfo.Lock.Tp != model.TableLockReadOnly { return nil } } diff --git a/executor/ddl.go b/executor/ddl.go index 81e02f3d366c4..2ebd12037dc82 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -599,7 +599,7 @@ func (e *DDLExec) executeLockTables(s *ast.LockTablesStmt) error { return domain.GetDomain(e.ctx).DDL().LockTables(e.ctx, s) } -func (e *DDLExec) executeUnlockTables(s *ast.UnlockTablesStmt) error { +func (e *DDLExec) executeUnlockTables(_ *ast.UnlockTablesStmt) error { if !config.TableLockEnabled() { return nil } diff --git a/executor/point_get.go b/executor/point_get.go index 3d397e0c1a450..e72a4a2b66414 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -340,26 +340,26 @@ func (e *PointGetExecutor) get(ctx context.Context, key kv.Key) ([]byte, error) // fallthrough to snapshot get. } - isLocked := e.tblInfo.IsLocked() - if !isLocked || e.tblInfo.Lock.Tp != model.TableLockRead { // if not read lock or table was unlock then snapshot get - return e.snapshot.Get(ctx, key) - } - - cacheDB := e.ctx.GetStore().GetMemCache() - val = cacheDB.Get(ctx, e.tblInfo.ID, key) - // key does not exist then get from snapshot and set to cache - if val == nil { - val, err = e.snapshot.Get(ctx, key) - if err != nil { - return nil, err - } + lock := e.tblInfo.Lock + if lock != nil && (lock.Tp == model.TableLockRead || lock.Tp == model.TableLockReadOnly) { + cacheDB := e.ctx.GetStore().GetMemCache() + val = cacheDB.Get(ctx, e.tblInfo.ID, key) + // key does not exist then get from snapshot and set to cache + if val == nil { + val, err = e.snapshot.Get(ctx, key) + if err != nil { + return nil, err + } - err := cacheDB.Set(e.tblInfo.ID, key, val) - if err != nil { - return nil, err + err := cacheDB.Set(e.tblInfo.ID, key, val) + if err != nil { + return nil, err + } } + return val, nil } - return val, nil + // if not read lock or table was unlock then snapshot get + return e.snapshot.Get(ctx, key) } // EncodeUniqueIndexKey encodes a unique index key. diff --git a/lock/lock.go b/lock/lock.go index 8e92e2a62ccc5..ecfc8093a5243 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -34,7 +34,7 @@ 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 { +func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType, alterWriteable bool) error { if db == "" && table == "" { return nil } @@ -43,7 +43,7 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType return nil } // check operation on database. - if table == "" { + if !alterWriteable && table == "" { return c.CheckLockInDB(db, privilege) } switch privilege { @@ -67,7 +67,7 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType if err != nil { return err } - if c.ctx.HasLockedTables() { + if !alterWriteable && c.ctx.HasLockedTables() { if locked, tp := c.ctx.CheckTableLocked(tb.Meta().ID); locked { if checkLockTpMeetPrivilege(tp, privilege) { return nil @@ -83,19 +83,24 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType if privilege == mysql.SelectPriv { switch tb.Meta().Lock.Tp { - case model.TableLockRead, model.TableLockWriteLocal: + case model.TableLockRead, model.TableLockWriteLocal, model.TableLockReadOnly: return nil } } + if alterWriteable && tb.Meta().Lock.Tp == model.TableLockReadOnly { + 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 { + // TableLockReadOnly doesn't need to check in this, because it is session unrelated. switch tp { case model.TableLockWrite, model.TableLockWriteLocal: return true case model.TableLockRead: - // ShowDBPriv, AllPrivMask,CreatePriv, CreateViewPriv already checked before. + // ShowDBPriv, AllPrivMask, CreatePriv, CreateViewPriv already checked before. // The other privilege in read lock was not allowed. if privilege == mysql.SelectPriv { return true @@ -117,7 +122,7 @@ func (c *Checker) CheckLockInDB(db string, privilege mysql.PrivilegeType) error } tables := c.is.SchemaTables(model.NewCIStr(db)) for _, tbl := range tables { - err := c.CheckTableLock(db, tbl.Meta().Name.L, privilege) + err := c.CheckTableLock(db, tbl.Meta().Name.L, privilege, false) if err != nil { return err } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index ee8d39b1017a6..9cc51f6fe2ea5 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -923,145 +923,145 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { { sql: "insert into t (a) values (1)", ans: []visitInfo{ - {mysql.InsertPriv, "test", "t", "", nil}, + {mysql.InsertPriv, "test", "t", "", nil, false}, }, }, { sql: "delete from t where a = 1", ans: []visitInfo{ - {mysql.DeletePriv, "test", "t", "", nil}, - {mysql.SelectPriv, "test", "t", "", nil}, + {mysql.DeletePriv, "test", "t", "", nil, false}, + {mysql.SelectPriv, "test", "t", "", nil, false}, }, }, { sql: "delete from a1 using t as a1 inner join t as a2 where a1.a = a2.a", ans: []visitInfo{ - {mysql.DeletePriv, "test", "t", "", nil}, - {mysql.SelectPriv, "test", "t", "", nil}, + {mysql.DeletePriv, "test", "t", "", nil, false}, + {mysql.SelectPriv, "test", "t", "", nil, false}, }, }, { sql: "update t set a = 7 where a = 1", ans: []visitInfo{ - {mysql.UpdatePriv, "test", "t", "", nil}, - {mysql.SelectPriv, "test", "t", "", nil}, + {mysql.UpdatePriv, "test", "t", "", nil, false}, + {mysql.SelectPriv, "test", "t", "", nil, false}, }, }, { sql: "update t, (select * from t) a1 set t.a = a1.a;", ans: []visitInfo{ - {mysql.UpdatePriv, "test", "t", "", nil}, - {mysql.SelectPriv, "test", "t", "", nil}, + {mysql.UpdatePriv, "test", "t", "", nil, false}, + {mysql.SelectPriv, "test", "t", "", nil, false}, }, }, { sql: "update t a1 set a1.a = a1.a + 1", ans: []visitInfo{ - {mysql.UpdatePriv, "test", "t", "", nil}, - {mysql.SelectPriv, "test", "t", "", nil}, + {mysql.UpdatePriv, "test", "t", "", nil, false}, + {mysql.SelectPriv, "test", "t", "", nil, false}, }, }, { sql: "select a, sum(e) from t group by a", ans: []visitInfo{ - {mysql.SelectPriv, "test", "t", "", nil}, + {mysql.SelectPriv, "test", "t", "", nil, false}, }, }, { sql: "truncate table t", ans: []visitInfo{ - {mysql.DropPriv, "test", "t", "", nil}, + {mysql.DropPriv, "test", "t", "", nil, false}, }, }, { sql: "drop table t", ans: []visitInfo{ - {mysql.DropPriv, "test", "t", "", nil}, + {mysql.DropPriv, "test", "t", "", nil, false}, }, }, { sql: "create table t (a int)", ans: []visitInfo{ - {mysql.CreatePriv, "test", "t", "", nil}, + {mysql.CreatePriv, "test", "t", "", nil, false}, }, }, { sql: "create table t1 like t", ans: []visitInfo{ - {mysql.CreatePriv, "test", "t1", "", nil}, - {mysql.SelectPriv, "test", "t", "", nil}, + {mysql.CreatePriv, "test", "t1", "", nil, false}, + {mysql.SelectPriv, "test", "t", "", nil, false}, }, }, { sql: "create database test", ans: []visitInfo{ - {mysql.CreatePriv, "test", "", "", nil}, + {mysql.CreatePriv, "test", "", "", nil, false}, }, }, { sql: "drop database test", ans: []visitInfo{ - {mysql.DropPriv, "test", "", "", nil}, + {mysql.DropPriv, "test", "", "", nil, false}, }, }, { sql: "create index t_1 on t (a)", ans: []visitInfo{ - {mysql.IndexPriv, "test", "t", "", nil}, + {mysql.IndexPriv, "test", "t", "", nil, false}, }, }, { sql: "drop index e on t", ans: []visitInfo{ - {mysql.IndexPriv, "test", "t", "", nil}, + {mysql.IndexPriv, "test", "t", "", nil, false}, }, }, { sql: `grant all privileges on test.* to 'test'@'%'`, ans: []visitInfo{ - {mysql.SelectPriv, "test", "", "", nil}, - {mysql.InsertPriv, "test", "", "", nil}, - {mysql.UpdatePriv, "test", "", "", nil}, - {mysql.DeletePriv, "test", "", "", nil}, - {mysql.CreatePriv, "test", "", "", nil}, - {mysql.DropPriv, "test", "", "", nil}, - {mysql.GrantPriv, "test", "", "", nil}, - {mysql.AlterPriv, "test", "", "", nil}, - {mysql.ExecutePriv, "test", "", "", nil}, - {mysql.IndexPriv, "test", "", "", nil}, - {mysql.CreateViewPriv, "test", "", "", nil}, - {mysql.ShowViewPriv, "test", "", "", nil}, + {mysql.SelectPriv, "test", "", "", nil, false}, + {mysql.InsertPriv, "test", "", "", nil, false}, + {mysql.UpdatePriv, "test", "", "", nil, false}, + {mysql.DeletePriv, "test", "", "", nil, false}, + {mysql.CreatePriv, "test", "", "", nil, false}, + {mysql.DropPriv, "test", "", "", nil, false}, + {mysql.GrantPriv, "test", "", "", nil, false}, + {mysql.AlterPriv, "test", "", "", nil, false}, + {mysql.ExecutePriv, "test", "", "", nil, false}, + {mysql.IndexPriv, "test", "", "", nil, false}, + {mysql.CreateViewPriv, "test", "", "", nil, false}, + {mysql.ShowViewPriv, "test", "", "", nil, false}, }, }, { sql: `grant select on test.ttt to 'test'@'%'`, ans: []visitInfo{ - {mysql.SelectPriv, "test", "ttt", "", nil}, - {mysql.GrantPriv, "test", "ttt", "", nil}, + {mysql.SelectPriv, "test", "ttt", "", nil, false}, + {mysql.GrantPriv, "test", "ttt", "", nil, false}, }, }, { sql: `grant select on ttt to 'test'@'%'`, ans: []visitInfo{ - {mysql.SelectPriv, "test", "ttt", "", nil}, - {mysql.GrantPriv, "test", "ttt", "", nil}, + {mysql.SelectPriv, "test", "ttt", "", nil, false}, + {mysql.GrantPriv, "test", "ttt", "", nil, false}, }, }, { sql: `revoke all privileges on test.* from 'test'@'%'`, ans: []visitInfo{ - {mysql.SelectPriv, "test", "", "", nil}, - {mysql.InsertPriv, "test", "", "", nil}, - {mysql.UpdatePriv, "test", "", "", nil}, - {mysql.DeletePriv, "test", "", "", nil}, - {mysql.CreatePriv, "test", "", "", nil}, - {mysql.DropPriv, "test", "", "", nil}, - {mysql.GrantPriv, "test", "", "", nil}, - {mysql.AlterPriv, "test", "", "", nil}, - {mysql.ExecutePriv, "test", "", "", nil}, - {mysql.IndexPriv, "test", "", "", nil}, - {mysql.CreateViewPriv, "test", "", "", nil}, - {mysql.ShowViewPriv, "test", "", "", nil}, + {mysql.SelectPriv, "test", "", "", nil, false}, + {mysql.InsertPriv, "test", "", "", nil, false}, + {mysql.UpdatePriv, "test", "", "", nil, false}, + {mysql.DeletePriv, "test", "", "", nil, false}, + {mysql.CreatePriv, "test", "", "", nil, false}, + {mysql.DropPriv, "test", "", "", nil, false}, + {mysql.GrantPriv, "test", "", "", nil, false}, + {mysql.AlterPriv, "test", "", "", nil, false}, + {mysql.ExecutePriv, "test", "", "", nil, false}, + {mysql.IndexPriv, "test", "", "", nil, false}, + {mysql.CreateViewPriv, "test", "", "", nil, false}, + {mysql.ShowViewPriv, "test", "", "", nil, false}, }, }, { @@ -1071,44 +1071,44 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { { sql: `show create table test.ttt`, ans: []visitInfo{ - {mysql.AllPrivMask, "test", "ttt", "", nil}, + {mysql.AllPrivMask, "test", "ttt", "", nil, false}, }, }, { sql: "alter table t add column a int(4)", ans: []visitInfo{ - {mysql.AlterPriv, "test", "t", "", nil}, + {mysql.AlterPriv, "test", "t", "", nil, false}, }, }, { sql: "rename table t_old to t_new", ans: []visitInfo{ - {mysql.AlterPriv, "test", "t_old", "", nil}, - {mysql.DropPriv, "test", "t_old", "", nil}, - {mysql.CreatePriv, "test", "t_new", "", nil}, - {mysql.InsertPriv, "test", "t_new", "", nil}, + {mysql.AlterPriv, "test", "t_old", "", nil, false}, + {mysql.DropPriv, "test", "t_old", "", nil, false}, + {mysql.CreatePriv, "test", "t_new", "", nil, false}, + {mysql.InsertPriv, "test", "t_new", "", nil, false}, }, }, { sql: "alter table t_old rename to t_new", ans: []visitInfo{ - {mysql.AlterPriv, "test", "t_old", "", nil}, - {mysql.DropPriv, "test", "t_old", "", nil}, - {mysql.CreatePriv, "test", "t_new", "", nil}, - {mysql.InsertPriv, "test", "t_new", "", nil}, + {mysql.AlterPriv, "test", "t_old", "", nil, false}, + {mysql.DropPriv, "test", "t_old", "", nil, false}, + {mysql.CreatePriv, "test", "t_new", "", nil, false}, + {mysql.InsertPriv, "test", "t_new", "", nil, false}, }, }, { sql: "alter table t drop partition p0;", ans: []visitInfo{ - {mysql.AlterPriv, "test", "t", "", nil}, - {mysql.DropPriv, "test", "t", "", nil}, + {mysql.AlterPriv, "test", "t", "", nil, false}, + {mysql.DropPriv, "test", "t", "", nil, false}, }, }, { sql: "flush privileges", ans: []visitInfo{ - {mysql.ReloadPriv, "", "", "", ErrSpecificAccessDenied}, + {mysql.ReloadPriv, "", "", "", ErrSpecificAccessDenied, false}, }, }, } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 983713c36acaa..2db34e9ba19f5 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -111,7 +111,7 @@ func CheckTableLock(ctx sessionctx.Context, is infoschema.InfoSchema, vs []visit } checker := lock.NewChecker(ctx, is) for i := range vs { - err := checker.CheckTableLock(vs[i].db, vs[i].table, vs[i].privilege) + err := checker.CheckTableLock(vs[i].db, vs[i].table, vs[i].privilege, vs[i].alterWritable) if err != nil { return err } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 5c1705f33eb34..8ea9382cecec7 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -57,11 +57,12 @@ import ( ) type visitInfo struct { - privilege mysql.PrivilegeType - db string - table string - column string - err error + privilege mysql.PrivilegeType + db string + table string + column string + err error + alterWritable bool } type indexNestedLoopJoinTables struct { @@ -2987,6 +2988,8 @@ func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (Plan, err } b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DropPriv, v.Table.Schema.L, v.Table.Name.L, "", authErr) + } else if spec.Tp == ast.AlterTableWriteable { + b.visitInfo[0].alterWritable = true } } case *ast.AlterSequenceStmt: diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 92de0d683705f..c62d1181e1ab2 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -875,7 +875,7 @@ func newPointGetPlan(ctx sessionctx.Context, dbName string, schema *expression.S func checkFastPlanPrivilege(ctx sessionctx.Context, dbName, tableName string, checkTypes ...mysql.PrivilegeType) error { pm := privilege.GetPrivilegeManager(ctx) - visitInfos := []visitInfo{} + var visitInfos []visitInfo for _, checkType := range checkTypes { if pm != nil && !pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, dbName, tableName, "", checkType) { return errors.New("privilege check fail") diff --git a/session/session.go b/session/session.go index 19bfea96d2a79..2c3c175fcdc48 100644 --- a/session/session.go +++ b/session/session.go @@ -197,7 +197,10 @@ type session struct { // AddTableLock adds table lock to the session lock map. func (s *session) AddTableLock(locks []model.TableLockTpInfo) { for _, l := range locks { - s.lockedTables[l.TableID] = l + // read only lock is session unrelated, skip it when adding lock to session. + if l.Tp != model.TableLockReadOnly { + s.lockedTables[l.TableID] = l + } } }