Skip to content

Commit

Permalink
planner: add privilege check for lock/unlock stats (#46742)
Browse files Browse the repository at this point in the history
ref #46351
  • Loading branch information
Rustin170506 authored Sep 8, 2023
1 parent 5f1f0de commit 78d1897
Show file tree
Hide file tree
Showing 4 changed files with 176 additions and 48 deletions.
36 changes: 23 additions & 13 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -3141,17 +3141,8 @@ func (b *PlanBuilder) buildAnalyze(as *ast.AnalyzeTableStmt) (Plan, error) {
return nil, errors.Errorf("Fast analyze hasn't reached General Availability and only support analyze version 1 currently")
}

// Check privilege.
for _, tbl := range as.TableNames {
user := b.ctx.GetSessionVars().User
var insertErr, selectErr error
if user != nil {
insertErr = ErrTableaccessDenied.GenWithStackByArgs("INSERT", user.AuthUsername, user.AuthHostname, tbl.Name.O)
selectErr = ErrTableaccessDenied.GenWithStackByArgs("SELECT", user.AuthUsername, user.AuthHostname, tbl.Name.O)
}
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.InsertPriv, tbl.Schema.O, tbl.Name.O, "", insertErr)
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, tbl.Schema.O, tbl.Name.O, "", selectErr)
}
// Require INSERT and SELECT privilege for tables.
b.requireInsertAndSelectPriv(as.TableNames)

opts, err := handleAnalyzeOptions(as.AnalyzeOpts, statsVersion)
if err != nil {
Expand Down Expand Up @@ -4530,16 +4521,35 @@ func (*PlanBuilder) buildLoadStats(ld *ast.LoadStatsStmt) Plan {
return p
}

func (*PlanBuilder) buildLockStats(ld *ast.LockStatsStmt) Plan {
func (b *PlanBuilder) buildLockStats(ld *ast.LockStatsStmt) Plan {
p := &LockStats{Tables: ld.Tables}
b.requireInsertAndSelectPriv(ld.Tables)

return p
}

func (*PlanBuilder) buildUnlockStats(ld *ast.UnlockStatsStmt) Plan {
// buildUnlockStats requires INSERT and SELECT privilege for the tables same as buildAnalyze.
func (b *PlanBuilder) buildUnlockStats(ld *ast.UnlockStatsStmt) Plan {
p := &UnlockStats{Tables: ld.Tables}
b.requireInsertAndSelectPriv(ld.Tables)

return p
}

// requireInsertAndSelectPriv requires INSERT and SELECT privilege for the tables.
func (b *PlanBuilder) requireInsertAndSelectPriv(tables []*ast.TableName) {
for _, tbl := range tables {
user := b.ctx.GetSessionVars().User
var insertErr, selectErr error
if user != nil {
insertErr = ErrTableaccessDenied.GenWithStackByArgs("INSERT", user.AuthUsername, user.AuthHostname, tbl.Name.O)
selectErr = ErrTableaccessDenied.GenWithStackByArgs("SELECT", user.AuthUsername, user.AuthHostname, tbl.Name.O)
}
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.InsertPriv, tbl.Schema.O, tbl.Name.O, "", insertErr)
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, tbl.Schema.O, tbl.Name.O, "", selectErr)
}
}

func (*PlanBuilder) buildIndexAdvise(node *ast.IndexAdviseStmt) Plan {
p := &IndexAdvise{
IsLocal: node.IsLocal,
Expand Down
26 changes: 26 additions & 0 deletions planner/core/planbuilder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -707,3 +707,29 @@ func TestGetFullAnalyzeColumnsInfo(t *testing.T) {
require.NoError(t, err)
require.Equal(t, specifiedCols, cols)
}

func TestRequireInsertAndSelectPriv(t *testing.T) {
ctx := MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
pb, _ := NewPlanBuilder().Init(ctx, nil, &hint.BlockHintProcessor{})

tables := []*ast.TableName{
{
Schema: model.NewCIStr("test"),
Name: model.NewCIStr("t1"),
},
{
Schema: model.NewCIStr("test"),
Name: model.NewCIStr("t2"),
},
}

pb.requireInsertAndSelectPriv(tables)
require.Len(t, pb.visitInfo, 4)
require.Equal(t, "test", pb.visitInfo[0].db)
require.Equal(t, "t1", pb.visitInfo[0].table)
require.Equal(t, mysql.InsertPriv, pb.visitInfo[0].privilege)
require.Equal(t, mysql.SelectPriv, pb.visitInfo[1].privilege)
}
5 changes: 4 additions & 1 deletion statistics/handle/handletest/statslock/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,11 +8,14 @@ go_test(
"stats_lock_test.go",
],
flaky = True,
shard_count = 3,
shard_count = 5,
deps = [
"//config",
"//domain",
"//kv",
"//parser/auth",
"//parser/model",
"//session",
"//testkit",
"//testkit/testsetup",
"@com_github_stretchr_testify//require",
Expand Down
157 changes: 123 additions & 34 deletions statistics/handle/handletest/statslock/stats_lock_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,29 +21,19 @@ import (

"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/parser/auth"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/testkit"
"github.com/stretchr/testify/require"
)

func TestStatsLockAndUnlockTable(t *testing.T) {
restore := config.RestoreFunc()
defer restore()
config.UpdateGlobal(func(conf *config.Config) {
conf.Performance.EnableStatsCacheMemQuota = true
})
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set @@tidb_analyze_version = 1")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b varchar(10), index idx_b (b))")
tk.MustExec("analyze table test.t")
tbl, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.Nil(t, err)
_, tk, tbl := setupTestEnvironmentWithTableT(t)

handle := domain.GetDomain(tk.Session()).StatsHandle()
tblStats := handle.GetTableStats(tbl.Meta())
tblStats := handle.GetTableStats(tbl)
for _, col := range tblStats.Columns {
require.True(t, col.IsStatsInitialized())
}
Expand All @@ -57,7 +47,7 @@ func TestStatsLockAndUnlockTable(t *testing.T) {
tk.MustExec("insert into t(a, b) values(2,'b')")

tk.MustExec("analyze table test.t")
tblStats1 := handle.GetTableStats(tbl.Meta())
tblStats1 := handle.GetTableStats(tbl)
require.Equal(t, tblStats, tblStats1)

lockedTables, err := handle.GetTableLockedAndClearForTest()
Expand All @@ -70,28 +60,15 @@ func TestStatsLockAndUnlockTable(t *testing.T) {
require.Equal(t, num, 0)

tk.MustExec("analyze table test.t")
tblStats2 := handle.GetTableStats(tbl.Meta())
tblStats2 := handle.GetTableStats(tbl)
require.Equal(t, int64(2), tblStats2.RealtimeCount)
}

func TestStatsLockTableAndUnlockTableRepeatedly(t *testing.T) {
restore := config.RestoreFunc()
defer restore()
config.UpdateGlobal(func(conf *config.Config) {
conf.Performance.EnableStatsCacheMemQuota = true
})
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set @@tidb_analyze_version = 1")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b varchar(10), index idx_b (b))")
tk.MustExec("analyze table test.t")
tbl, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.Nil(t, err)
_, tk, tbl := setupTestEnvironmentWithTableT(t)

handle := domain.GetDomain(tk.Session()).StatsHandle()
tblStats := handle.GetTableStats(tbl.Meta())
tblStats := handle.GetTableStats(tbl)
for _, col := range tblStats.Columns {
require.True(t, col.IsStatsInitialized())
}
Expand All @@ -105,7 +82,7 @@ func TestStatsLockTableAndUnlockTableRepeatedly(t *testing.T) {
tk.MustExec("insert into t(a, b) values(2,'b')")

tk.MustExec("analyze table test.t")
tblStats1 := handle.GetTableStats(tbl.Meta())
tblStats1 := handle.GetTableStats(tbl)
require.Equal(t, tblStats, tblStats1)

// Lock the table again and check the warning.
Expand All @@ -127,7 +104,7 @@ func TestStatsLockTableAndUnlockTableRepeatedly(t *testing.T) {
require.Equal(t, num, 0)

tk.MustExec("analyze table test.t")
tblStats2 := handle.GetTableStats(tbl.Meta())
tblStats2 := handle.GetTableStats(tbl)
require.Equal(t, int64(2), tblStats2.RealtimeCount)

// Unlock the table again and check the warning.
Expand Down Expand Up @@ -203,3 +180,115 @@ func TestStatsLockAndUnlockTables(t *testing.T) {
tbl2Stats2 := handle.GetTableStats(tbl2.Meta())
require.Equal(t, int64(2), tbl2Stats2.RealtimeCount)
}

func TestLockAndUnlockTablePrivilege(t *testing.T) {
store, tk, tbl := setupTestEnvironmentWithTableT(t)

handle := domain.GetDomain(tk.Session()).StatsHandle()
tblStats := handle.GetTableStats(tbl)
for _, col := range tblStats.Columns {
require.True(t, col.IsStatsInitialized())
}
// With privilege.
tk.MustExec("lock stats t")
rows := tk.MustQuery("select count(*) from mysql.stats_table_locked").Rows()
num, _ := strconv.Atoi(rows[0][0].(string))
require.Equal(t, num, 1)
tk.MustExec("unlock stats t")
rows = tk.MustQuery("select count(*) from mysql.stats_table_locked").Rows()
num, _ = strconv.Atoi(rows[0][0].(string))
require.Equal(t, num, 0)

// Add a user.
tk.MustExec("drop user if exists myuser@localhost")
tk.MustExec("create user myuser@localhost")
// Only grant delete privilege.
tk.MustExec("grant delete on test.* to myuser@localhost")

// Without privilege.
tk1 := testkit.NewTestKit(t, store)
se, err := session.CreateSession4Test(store)
require.NoError(t, err)
require.NoError(t, se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil, nil))
tk1.SetSession(se)

tk1.MustExec("use test")
_, err = tk1.Exec("lock stats t")
require.Error(t, err)
require.Equal(t, "[planner:1142]INSERT command denied to user 'myuser'@'localhost' for table 't'", err.Error())
_, err = tk1.Exec("unlock stats t")
require.Error(t, err)
require.Equal(t, "[planner:1142]INSERT command denied to user 'myuser'@'localhost' for table 't'", err.Error())

// Grant INSERT privilege.
tk.MustExec("grant insert on test.* to myuser@localhost")
tk.MustExec("flush privileges")

// Try again.
_, err = tk1.Exec("lock stats t")
require.Error(t, err)
require.Equal(t, "[planner:1142]SELECT command denied to user 'myuser'@'localhost' for table 't'", err.Error())
_, err = tk1.Exec("unlock stats t")
require.Error(t, err)
require.Equal(t, "[planner:1142]SELECT command denied to user 'myuser'@'localhost' for table 't'", err.Error())

// Grant SELECT privilege.
tk.MustExec("grant select on test.* to myuser@localhost")
tk.MustExec("flush privileges")

// Try again
tk1.MustExec("lock stats t")
tk1.MustExec("unlock stats t")
}

func TestShowStatsLockedTablePrivilege(t *testing.T) {
store, tk, tbl := setupTestEnvironmentWithTableT(t)

handle := domain.GetDomain(tk.Session()).StatsHandle()
tblStats := handle.GetTableStats(tbl)
for _, col := range tblStats.Columns {
require.True(t, col.IsStatsInitialized())
}
// With privilege.
tk.MustExec("lock stats t")
rows := tk.MustQuery("show stats_locked").Rows()
require.Len(t, rows, 1)

// Add a user.
tk.MustExec("drop user if exists myuser@localhost")
tk.MustExec("create user myuser@localhost")
// Only grant INSERT privilege.
tk.MustExec("grant insert on mysql.* to myuser@localhost")

// Without privilege.
tk1 := testkit.NewTestKit(t, store)
se, err := session.CreateSession4Test(store)
require.NoError(t, err)
require.NoError(t, se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil, nil))
tk1.SetSession(se)
_, err = tk1.Exec("show stats_locked")
require.Error(t, err)
require.Equal(t, "[planner:1142]SHOW command denied to user 'myuser'@'localhost' for table 'stats_table_locked'", err.Error())

// Grant SELECT privilege.
tk.MustExec("grant select on mysql.* to myuser@localhost")
tk.MustExec("flush privileges")

// Try again
rows = tk.MustQuery("show stats_locked").Rows()
require.Len(t, rows, 1)
}

func setupTestEnvironmentWithTableT(t *testing.T) (kv.Storage, *testkit.TestKit, *model.TableInfo) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set @@tidb_analyze_version = 1")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b varchar(10), index idx_b (b))")
tk.MustExec("analyze table test.t")
tbl, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.Nil(t, err)

return store, tk, tbl.Meta()
}

0 comments on commit 78d1897

Please sign in to comment.