Skip to content

Commit

Permalink
Merge branch 'master' into issue#29993
Browse files Browse the repository at this point in the history
  • Loading branch information
Reminiscent authored Nov 23, 2021
2 parents 934f846 + e453873 commit 5314597
Show file tree
Hide file tree
Showing 16 changed files with 90 additions and 44 deletions.
3 changes: 2 additions & 1 deletion ddl/column_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,8 @@ func (s *testColumnChangeSuite) SetUpSuite(c *C) {
err := d.Stop()
c.Assert(err, IsNil)
}()
s.dbInfo = testSchemaInfo(c, d, "test_index_change")
s.dbInfo, err = testSchemaInfo(d, "test_index_change")
c.Assert(err, IsNil)
testCreateSchema(c, testNewContext(d), d, s.dbInfo)
}

Expand Down
3 changes: 2 additions & 1 deletion ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,8 @@ func (s *testColumnSuite) SetUpSuite(c *C) {
)
c.Assert(err, IsNil)

s.dbInfo = testSchemaInfo(c, d, "test_column")
s.dbInfo, err = testSchemaInfo(d, "test_column")
c.Assert(err, IsNil)
testCreateSchema(c, testNewContext(d), d, s.dbInfo)
c.Assert(d.Stop(), IsNil)
}
Expand Down
27 changes: 18 additions & 9 deletions ddl/ddl_worker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -254,7 +254,8 @@ func (s *testDDLSuite) TestTableError(c *C) {
// Schema ID is wrong, so dropping table is failed.
doDDLJobErr(c, -1, 1, model.ActionDropTable, nil, ctx, d)
// Table ID is wrong, so dropping table is failed.
dbInfo := testSchemaInfo(c, d, "test_ddl")
dbInfo, err := testSchemaInfo(d, "test_ddl")
c.Assert(err, IsNil)
testCreateSchema(c, testNewContext(d), d, dbInfo)
job := doDDLJobErr(c, dbInfo.ID, -1, model.ActionDropTable, nil, ctx, d)

Expand Down Expand Up @@ -302,7 +303,8 @@ func (s *testDDLSuite) TestViewError(c *C) {
c.Assert(err, IsNil)
}()
ctx := testNewContext(d)
dbInfo := testSchemaInfo(c, d, "test_ddl")
dbInfo, err := testSchemaInfo(d, "test_ddl")
c.Assert(err, IsNil)
testCreateSchema(c, testNewContext(d), d, dbInfo)

// Table ID or schema ID is wrong, so getting table is failed.
Expand Down Expand Up @@ -370,7 +372,8 @@ func (s *testDDLSuite) TestForeignKeyError(c *C) {
doDDLJobErr(c, -1, 1, model.ActionAddForeignKey, nil, ctx, d)
doDDLJobErr(c, -1, 1, model.ActionDropForeignKey, nil, ctx, d)

dbInfo := testSchemaInfo(c, d, "test_ddl")
dbInfo, err := testSchemaInfo(d, "test_ddl")
c.Assert(err, IsNil)
tblInfo := testTableInfo(c, d, "t", 3)
testCreateSchema(c, ctx, d, dbInfo)
testCreateTable(c, ctx, d, dbInfo, tblInfo)
Expand Down Expand Up @@ -400,7 +403,8 @@ func (s *testDDLSuite) TestIndexError(c *C) {
doDDLJobErr(c, -1, 1, model.ActionAddIndex, nil, ctx, d)
doDDLJobErr(c, -1, 1, model.ActionDropIndex, nil, ctx, d)

dbInfo := testSchemaInfo(c, d, "test_ddl")
dbInfo, err := testSchemaInfo(d, "test_ddl")
c.Assert(err, IsNil)
tblInfo := testTableInfo(c, d, "t", 3)
testCreateSchema(c, ctx, d, dbInfo)
testCreateTable(c, ctx, d, dbInfo, tblInfo)
Expand Down Expand Up @@ -442,7 +446,8 @@ func (s *testDDLSuite) TestColumnError(c *C) {
}()
ctx := testNewContext(d)

dbInfo := testSchemaInfo(c, d, "test_ddl")
dbInfo, err := testSchemaInfo(d, "test_ddl")
c.Assert(err, IsNil)
tblInfo := testTableInfo(c, d, "t", 3)
testCreateSchema(c, ctx, d, dbInfo)
testCreateTable(c, ctx, d, dbInfo, tblInfo)
Expand Down Expand Up @@ -739,7 +744,8 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) {
err := d.Stop()
c.Assert(err, IsNil)
}()
dbInfo := testSchemaInfo(c, d, "test_cancel_job")
dbInfo, err := testSchemaInfo(d, "test_cancel_job")
c.Assert(err, IsNil)
testCreateSchema(c, testNewContext(d), d, dbInfo)
// create a partition table.
partitionTblInfo := testTableInfoWithPartition(c, d, "t_partition", 5)
Expand Down Expand Up @@ -898,7 +904,8 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) {
testCheckTableState(c, d, dbInfo, tblInfo1, model.StateNone)

// for create database
dbInfo1 := testSchemaInfo(c, d, "test_cancel_job1")
dbInfo1, err := testSchemaInfo(d, "test_cancel_job1")
c.Assert(err, IsNil)
updateTest(&tests[9])
doDDLJobErrWithSchemaState(ctx, d, c, dbInfo1.ID, 0, model.ActionCreateSchema, []interface{}{dbInfo1}, &cancelState)
c.Check(checkErr, IsNil)
Expand Down Expand Up @@ -1469,7 +1476,8 @@ func (s *testDDLSuite) TestParallelDDL(c *C) {
}
*/
// create database test_parallel_ddl_1;
dbInfo1 := testSchemaInfo(c, d, "test_parallel_ddl_1")
dbInfo1, err := testSchemaInfo(d, "test_parallel_ddl_1")
c.Assert(err, IsNil)
testCreateSchema(c, ctx, d, dbInfo1)
// create table t1 (c1 int, c2 int);
tblInfo1 := testTableInfo(c, d, "t1", 2)
Expand All @@ -1494,7 +1502,8 @@ func (s *testDDLSuite) TestParallelDDL(c *C) {
_, err = tbl2.AddRecord(ctx, types.MakeDatums(3, 3, 3))
c.Assert(err, IsNil)
// create database test_parallel_ddl_2;
dbInfo2 := testSchemaInfo(c, d, "test_parallel_ddl_2")
dbInfo2, err := testSchemaInfo(d, "test_parallel_ddl_2")
c.Assert(err, IsNil)
testCreateSchema(c, ctx, d, dbInfo2)
// create table t3 (c1 int, c2 int, c3 int, c4 int);
tblInfo3 := testTableInfo(c, d, "t3", 4)
Expand Down
3 changes: 2 additions & 1 deletion ddl/foreign_key_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,8 @@ func (s *testForeignKeySuite) TestForeignKey(c *C) {
c.Assert(err, IsNil)
}()
s.d = d
s.dbInfo = testSchemaInfo(c, d, "test_foreign")
s.dbInfo, err = testSchemaInfo(d, "test_foreign")
c.Assert(err, IsNil)
ctx := testNewContext(d)
s.ctx = ctx
testCreateSchema(c, ctx, d, s.dbInfo)
Expand Down
3 changes: 2 additions & 1 deletion ddl/index_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,8 @@ func (s *testIndexChangeSuite) SetUpSuite(c *C) {
err := d.Stop()
c.Assert(err, IsNil)
}()
s.dbInfo = testSchemaInfo(c, d, "test_index_change")
s.dbInfo, err = testSchemaInfo(d, "test_index_change")
c.Assert(err, IsNil)
testCreateSchema(c, testNewContext(d), d, s.dbInfo)
}

Expand Down
3 changes: 2 additions & 1 deletion ddl/partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,8 @@ func (s *testPartitionSuite) TestDropAndTruncatePartition(c *C) {
err := d.Stop()
c.Assert(err, IsNil)
}()
dbInfo := testSchemaInfo(c, d, "test_partition")
dbInfo, err := testSchemaInfo(d, "test_partition")
c.Assert(err, IsNil)
testCreateSchema(c, testNewContext(d), d, dbInfo)
// generate 5 partition in tableInfo.
tblInfo, partIDs := buildTableInfoWithPartition(c, d)
Expand Down
9 changes: 6 additions & 3 deletions ddl/placement_policy_ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,11 +64,13 @@ func (s *testDDLSuite) TestPlacementPolicyInUse(c *C) {
c.Assert(err, IsNil)
sctx := testNewContext(d)

db1 := testSchemaInfo(c, d, "db1")
db1, err := testSchemaInfo(d, "db1")
c.Assert(err, IsNil)
testCreateSchema(c, sctx, d, db1)
db1.State = model.StatePublic

db2 := testSchemaInfo(c, d, "db2")
db2, err := testSchemaInfo(d, "db2")
c.Assert(err, IsNil)
testCreateSchema(c, sctx, d, db2)
db2.State = model.StatePublic

Expand Down Expand Up @@ -102,7 +104,8 @@ func (s *testDDLSuite) TestPlacementPolicyInUse(c *C) {
t3.State = model.StatePublic
db1.Tables = append(db1.Tables, t3)

dbP := testSchemaInfo(c, d, "db_p")
dbP, err := testSchemaInfo(d, "db_p")
c.Assert(err, IsNil)
dbP.PlacementPolicyRef = &model.PolicyRefInfo{ID: p4.ID, Name: p4.Name}
dbP.State = model.StatePublic
testCreateSchema(c, sctx, d, dbP)
Expand Down
3 changes: 2 additions & 1 deletion ddl/reorg_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -218,7 +218,8 @@ func (s *testDDLSuite) TestReorgOwner(c *C) {
c.Assert(err, IsNil)
}()

dbInfo := testSchemaInfo(c, d1, "test_reorg")
dbInfo, err := testSchemaInfo(d1, "test_reorg")
c.Assert(err, IsNil)
testCreateSchema(c, ctx, d1, dbInfo)

tblInfo := testTableInfo(c, d1, "t", 3)
Expand Down
6 changes: 4 additions & 2 deletions ddl/restart_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,8 @@ func (s *testSchemaSuite) TestSchemaResume(c *C) {

testCheckOwner(c, d1, true)

dbInfo := testSchemaInfo(c, d1, "test_restart")
dbInfo, err := testSchemaInfo(d1, "test_restart")
c.Assert(err, IsNil)
job := &model.Job{
SchemaID: dbInfo.ID,
Type: model.ActionCreateSchema,
Expand Down Expand Up @@ -160,7 +161,8 @@ func (s *testStatSuite) TestStat(c *C) {
c.Assert(err, IsNil)
}()

dbInfo := testSchemaInfo(c, d, "test_restart")
dbInfo, err := testSchemaInfo(d, "test_restart")
c.Assert(err, IsNil)
testCreateSchema(c, testNewContext(d), d, dbInfo)

// TODO: Get this information from etcd.
Expand Down
17 changes: 11 additions & 6 deletions ddl/schema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,14 +39,16 @@ func (s *testSchemaSuite) SetUpSuite(c *C) {
func (s *testSchemaSuite) TearDownSuite(c *C) {
}

func testSchemaInfo(c *C, d *ddl, name string) *model.DBInfo {
func testSchemaInfo(d *ddl, name string) (*model.DBInfo, error) {
dbInfo := &model.DBInfo{
Name: model.NewCIStr(name),
}
genIDs, err := d.genGlobalIDs(1)
c.Assert(err, IsNil)
if err != nil {
return nil, err
}
dbInfo.ID = genIDs[0]
return dbInfo
return dbInfo, nil
}

func testCreateSchema(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo) *model.Job {
Expand Down Expand Up @@ -140,7 +142,8 @@ func (s *testSchemaSuite) TestSchema(c *C) {
c.Assert(err, IsNil)
}()
ctx := testNewContext(d)
dbInfo := testSchemaInfo(c, d, "test_schema")
dbInfo, err := testSchemaInfo(d, "test_schema")
c.Assert(err, IsNil)

// create a database.
job := testCreateSchema(c, ctx, d, dbInfo)
Expand Down Expand Up @@ -185,7 +188,8 @@ func (s *testSchemaSuite) TestSchema(c *C) {
c.Assert(terror.ErrorEqual(err, infoschema.ErrDatabaseDropExists), IsTrue, Commentf("err %v", err))

// Drop a database without a table.
dbInfo1 := testSchemaInfo(c, d, "test1")
dbInfo1, err := testSchemaInfo(d, "test1")
c.Assert(err, IsNil)
job = testCreateSchema(c, ctx, d, dbInfo1)
testCheckSchemaState(c, d, dbInfo1, model.StatePublic)
testCheckJobDone(c, d, job, true)
Expand Down Expand Up @@ -229,7 +233,8 @@ func (s *testSchemaSuite) TestSchemaWaitJob(c *C) {
// d2 must not be owner.
d2.ownerManager.RetireOwner()

dbInfo := testSchemaInfo(c, d2, "test_schema")
dbInfo, err := testSchemaInfo(d2, "test_schema")
c.Assert(err, IsNil)
testCreateSchema(c, ctx, d2, dbInfo)
testCheckSchemaState(c, d2, dbInfo, model.StatePublic)

Expand Down
3 changes: 2 additions & 1 deletion ddl/stat_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,8 @@ func (s *testSerialStatSuite) TestDDLStatsInfo(c *C) {
c.Assert(err, IsNil)
}()

dbInfo := testSchemaInfo(c, d, "test_stat")
dbInfo, err := testSchemaInfo(d, "test_stat")
c.Assert(err, IsNil)
testCreateSchema(c, testNewContext(d), d, dbInfo)
tblInfo := testTableInfo(c, d, "t", 2)
ctx := testNewContext(d)
Expand Down
6 changes: 4 additions & 2 deletions ddl/table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -357,7 +357,8 @@ func (s *testTableSuite) SetUpSuite(c *C) {
c.Assert(err, IsNil)
s.d = ddl

s.dbInfo = testSchemaInfo(c, s.d, "test_table")
s.dbInfo, err = testSchemaInfo(s.d, "test_table")
c.Assert(err, IsNil)
testCreateSchema(c, testNewContext(s.d), s.d, s.dbInfo)
}

Expand Down Expand Up @@ -403,7 +404,8 @@ func (s *testTableSuite) TestTable(c *C) {
testCheckJobDone(c, d, job, true)

// for rename table
dbInfo1 := testSchemaInfo(c, s.d, "test_rename_table")
dbInfo1, err := testSchemaInfo(s.d, "test_rename_table")
c.Assert(err, IsNil)
testCreateSchema(c, testNewContext(s.d), s.d, dbInfo1)
job = testRenameTable(c, ctx, d, dbInfo1.ID, s.dbInfo.ID, s.dbInfo.Name, tblInfo)
testCheckTableState(c, d, dbInfo1, tblInfo, model.StatePublic)
Expand Down
6 changes: 3 additions & 3 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -1083,7 +1083,7 @@ func getPlanTree(sctx sessionctx.Context, p plannercore.Plan) string {
if atomic.LoadUint32(&cfg.Log.RecordPlanInSlowLog) == 0 {
return ""
}
planTree, _ := getEncodedPlan(sctx, p, false, nil)
planTree, _ := getEncodedPlan(sctx, p, false)
if len(planTree) == 0 {
return planTree
}
Expand All @@ -1103,7 +1103,7 @@ func getPlanDigest(sctx sessionctx.Context, p plannercore.Plan) (string, *parser
}

// getEncodedPlan gets the encoded plan, and generates the hint string if indicated.
func getEncodedPlan(sctx sessionctx.Context, p plannercore.Plan, genHint bool, n ast.StmtNode) (encodedPlan, hintStr string) {
func getEncodedPlan(sctx sessionctx.Context, p plannercore.Plan, genHint bool) (encodedPlan, hintStr string) {
var hintSet bool
encodedPlan = sctx.GetSessionVars().StmtCtx.GetEncodedPlan()
hintStr, hintSet = sctx.GetSessionVars().StmtCtx.GetPlanHint()
Expand Down Expand Up @@ -1161,7 +1161,7 @@ func (a *ExecStmt) SummaryStmt(succ bool) {

// No need to encode every time, so encode lazily.
planGenerator := func() (string, string) {
return getEncodedPlan(a.Ctx, a.Plan, !sessVars.InRestrictedSQL, a.StmtNode)
return getEncodedPlan(a.Ctx, a.Plan, !sessVars.InRestrictedSQL)
}
// Generating plan digest is slow, only generate it once if it's 'Point_Get'.
// If it's a point get, different SQLs leads to different plans, so SQL digest
Expand Down
25 changes: 13 additions & 12 deletions executor/grant.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,8 +72,20 @@ func (e *GrantExec) Next(ctx context.Context, req *chunk.Chunk) error {
dbName = e.ctx.GetSessionVars().CurrentDB
}

// Make sure the table exist.
// For table & column level, check whether table exists and privilege is valid
if e.Level.Level == ast.GrantLevelTable {
// Return if privilege is invalid, to fail before not existing table, see issue #29302
for _, p := range e.Privs {
if len(p.Cols) == 0 {
if !mysql.AllTablePrivs.Has(p.Priv) && p.Priv != mysql.AllPriv && p.Priv != mysql.UsagePriv && p.Priv != mysql.GrantPriv && p.Priv != mysql.ExtendedPriv {
return ErrIllegalGrantForTable
}
} else {
if !mysql.AllColumnPrivs.Has(p.Priv) && p.Priv != mysql.AllPriv && p.Priv != mysql.UsagePriv {
return ErrWrongUsage.GenWithStackByArgs("COLUMN GRANT", "NON-COLUMN PRIVILEGES")
}
}
}
dbNameStr := model.NewCIStr(dbName)
schema := e.ctx.GetInfoSchema().(infoschema.InfoSchema)
tbl, err := schema.TableByName(dbNameStr, model.NewCIStr(e.Level.TableName))
Expand Down Expand Up @@ -633,13 +645,6 @@ func composeDBPrivUpdate(sql *strings.Builder, priv mysql.PrivilegeType, value s
func composeTablePrivUpdateForGrant(ctx sessionctx.Context, sql *strings.Builder, priv mysql.PrivilegeType, name string, host string, db string, tbl string) error {
var newTablePriv, newColumnPriv []string
if priv != mysql.AllPriv {
// TODO: https://github.com/pingcap/parser/pull/581 removed privs from all priv lists
// it is to avoid add GRANT in GRANT ALL SQLs
// WithGRANT seems broken, fix it later
if priv != mysql.GrantPriv && !mysql.AllTablePrivs.Has(priv) {
return ErrIllegalGrantForTable
}

currTablePriv, currColumnPriv, err := getTablePriv(ctx, name, host, db, tbl)
if err != nil {
return err
Expand Down Expand Up @@ -669,10 +674,6 @@ func composeTablePrivUpdateForGrant(ctx sessionctx.Context, sql *strings.Builder
func composeColumnPrivUpdateForGrant(ctx sessionctx.Context, sql *strings.Builder, priv mysql.PrivilegeType, name string, host string, db string, tbl string, col string) error {
var newColumnPriv []string
if priv != mysql.AllPriv {
if !mysql.AllColumnPrivs.Has(priv) {
return ErrWrongUsage.GenWithStackByArgs("COLUMN GRANT", "NON-COLUMN PRIVILEGES")
}

currColumnPriv, err := getColumnPriv(ctx, name, host, db, tbl, col)
if err != nil {
return err
Expand Down
16 changes: 16 additions & 0 deletions executor/grant_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -615,3 +615,19 @@ func TestGrantDynamicPrivs(t *testing.T) {
tk.MustQuery("SELECT Grant_Priv FROM mysql.user WHERE `Host` = '%' AND `User` = 'dyn'").Check(testkit.Rows("Y"))
tk.MustQuery("SELECT WITH_GRANT_OPTION FROM mysql.global_grants WHERE `Host` = '%' AND `User` = 'dyn' AND Priv='CONNECTION_ADMIN'").Check(testkit.Rows("Y"))
}

func TestNonExistTableIllegalGrant(t *testing.T) {
t.Parallel()

store, clean := testkit.CreateMockStore(t)
defer clean()

tk := testkit.NewTestKit(t, store)
tk.MustExec("create user u29302")
defer tk.MustExec("drop user u29302")
// Table level, not existing table, illegal privilege
tk.MustGetErrCode("grant create temporary tables on NotExistsD29302.NotExistsT29302 to u29302", mysql.ErrIllegalGrantForTable)
tk.MustGetErrCode("grant lock tables on test.NotExistsT29302 to u29302", mysql.ErrIllegalGrantForTable)
// Column level, not existing table, illegal privilege
tk.MustGetErrCode("grant create temporary tables (NotExistsCol) on NotExistsD29302.NotExistsT29302 to u29302;", mysql.ErrWrongUsage)
}
1 change: 1 addition & 0 deletions privilege/privileges/privileges_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2668,6 +2668,7 @@ func TestGrantCreateTmpTables(t *testing.T) {
tk.MustExec("CREATE TABLE create_tmp_table_table (a int)")
tk.MustExec("GRANT CREATE TEMPORARY TABLES on create_tmp_table_db.* to u1")
tk.MustExec("GRANT CREATE TEMPORARY TABLES on *.* to u1")
tk.MustGetErrCode("GRANT CREATE TEMPORARY TABLES on create_tmp_table_db.tmp to u1", mysql.ErrIllegalGrantForTable)
// Must set a session user to avoid null pointer dereference
tk.Session().Auth(&auth.UserIdentity{
Username: "root",
Expand Down

0 comments on commit 5314597

Please sign in to comment.