Skip to content

Commit

Permalink
planner, executor: fix show view privileges for explain (#10585) (#10635
Browse files Browse the repository at this point in the history
)
  • Loading branch information
bb7133 authored and winkyao committed May 31, 2019
1 parent 52dde3b commit c3a89b0
Show file tree
Hide file tree
Showing 6 changed files with 94 additions and 2 deletions.
2 changes: 2 additions & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1380,6 +1380,8 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
sc.NotFillCache = !opts.SQLCache
}
sc.PadCharToFullLength = ctx.GetSessionVars().SQLMode.HasPadCharToFullLengthMode()
case *ast.ExplainStmt:
sc.InExplainStmt = true
case *ast.ShowStmt:
sc.IgnoreTruncate = true
sc.IgnoreZeroInDate = true
Expand Down
62 changes: 62 additions & 0 deletions executor/explain_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
// Copyright 2019 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package executor_test

import (
. "github.com/pingcap/check"
"github.com/pingcap/parser/auth"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/util/testkit"
)

func (s *testSuite1) TestExplainPriviliges(c *C) {
se, err := session.CreateSession4Test(s.store)
c.Assert(err, IsNil)
c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil), IsTrue)
tk := testkit.NewTestKit(c, s.store)
tk.Se = se

tk.MustExec("create database explaindatabase")
tk.MustExec("use explaindatabase")
tk.MustExec("create table t (id int)")
tk.MustExec("create view v as select * from t")
tk.MustExec(`create user 'explain'@'%'`)
tk.MustExec(`flush privileges`)

tk1 := testkit.NewTestKit(c, s.store)
se, err = session.CreateSession4Test(s.store)
c.Assert(err, IsNil)
c.Assert(se.Auth(&auth.UserIdentity{Username: "explain", Hostname: "%"}, nil, nil), IsTrue)
tk1.Se = se

tk.MustExec(`grant select on explaindatabase.v to 'explain'@'%'`)
tk.MustExec(`flush privileges`)
tk1.MustQuery("show databases").Check(testkit.Rows("INFORMATION_SCHEMA", "explaindatabase"))

tk1.MustExec("use explaindatabase")
tk1.MustQuery("select * from v")
err = tk1.ExecToErr("explain select * from v")
c.Assert(err.Error(), Equals, plannercore.ErrViewNoExplain.Error())

tk.MustExec(`grant show view on explaindatabase.v to 'explain'@'%'`)
tk.MustExec(`flush privileges`)
tk1.MustQuery("explain select * from v")

tk.MustExec(`revoke select on explaindatabase.v from 'explain'@'%'`)
tk.MustExec(`flush privileges`)

err = tk1.ExecToErr("explain select * from v")
c.Assert(err.Error(), Equals, plannercore.ErrTableaccessDenied.GenWithStackByArgs("SELECT", "explain", "%", "v").Error())
}
3 changes: 3 additions & 0 deletions planner/core/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ const (
codeDBaccessDenied = mysql.ErrDBaccessDenied
codeTableaccessDenied = mysql.ErrTableaccessDenied
codeSpecificAccessDenied = mysql.ErrSpecificAccessDenied
codeViewNoExplain = mysql.ErrViewNoExplain
codeWindowFrameStartIllegal = mysql.ErrWindowFrameStartIllegal
codeWindowFrameEndIllegal = mysql.ErrWindowFrameEndIllegal
codeWindowFrameIllegal = mysql.ErrWindowFrameIllegal
Expand Down Expand Up @@ -128,6 +129,7 @@ var (
ErrDBaccessDenied = terror.ClassOptimizer.New(mysql.ErrDBaccessDenied, mysql.MySQLErrName[mysql.ErrDBaccessDenied])
ErrTableaccessDenied = terror.ClassOptimizer.New(mysql.ErrTableaccessDenied, mysql.MySQLErrName[mysql.ErrTableaccessDenied])
ErrSpecificAccessDenied = terror.ClassOptimizer.New(mysql.ErrSpecificAccessDenied, mysql.MySQLErrName[mysql.ErrSpecificAccessDenied])
ErrViewNoExplain = terror.ClassOptimizer.New(mysql.ErrViewNoExplain, mysql.MySQLErrName[mysql.ErrViewNoExplain])
ErrWindowFrameStartIllegal = terror.ClassOptimizer.New(codeWindowFrameStartIllegal, mysql.MySQLErrName[mysql.ErrWindowFrameStartIllegal])
ErrWindowFrameEndIllegal = terror.ClassOptimizer.New(codeWindowFrameEndIllegal, mysql.MySQLErrName[mysql.ErrWindowFrameEndIllegal])
ErrWindowFrameIllegal = terror.ClassOptimizer.New(codeWindowFrameIllegal, mysql.MySQLErrName[mysql.ErrWindowFrameIllegal])
Expand Down Expand Up @@ -183,6 +185,7 @@ func init() {
codeDBaccessDenied: mysql.ErrDBaccessDenied,
codeTableaccessDenied: mysql.ErrTableaccessDenied,
codeSpecificAccessDenied: mysql.ErrSpecificAccessDenied,
codeViewNoExplain: mysql.ErrViewNoExplain,
codeWindowFrameStartIllegal: mysql.ErrWindowFrameStartIllegal,
codeWindowFrameEndIllegal: mysql.ErrWindowFrameEndIllegal,
codeWindowFrameIllegal: mysql.ErrWindowFrameIllegal,
Expand Down
6 changes: 5 additions & 1 deletion planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2178,7 +2178,7 @@ func (b *PlanBuilder) buildDataSource(tn *ast.TableName) (LogicalPlan, error) {
tableInfo := tbl.Meta()
var authErr error
if b.ctx.GetSessionVars().User != nil {
authErr = ErrTableaccessDenied.GenWithStackByArgs("SELECT", b.ctx.GetSessionVars().User.Hostname, b.ctx.GetSessionVars().User.Username, tableInfo.Name.L)
authErr = ErrTableaccessDenied.GenWithStackByArgs("SELECT", b.ctx.GetSessionVars().User.Username, b.ctx.GetSessionVars().User.Hostname, tableInfo.Name.L)
}
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, dbName.L, tableInfo.Name.L, "", authErr)

Expand Down Expand Up @@ -2324,6 +2324,10 @@ func (b *PlanBuilder) BuildDataSourceFromView(dbName model.CIStr, tableInfo *mod
}
b.visitInfo = append(originalVisitInfo, b.visitInfo...)

if b.ctx.GetSessionVars().StmtCtx.InExplainStmt {
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ShowViewPriv, dbName.L, tableInfo.Name.L, "", ErrViewNoExplain)
}

projSchema := expression.NewSchema(make([]*expression.Column, 0, len(tableInfo.View.Cols))...)
projExprs := make([]expression.Expression, 0, len(tableInfo.View.Cols))
for i := range tableInfo.View.Cols {
Expand Down
22 changes: 21 additions & 1 deletion privilege/privileges/privileges_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -162,6 +162,26 @@ func (s *testPrivilegeSuite) TestCheckTablePrivilege(c *C) {
c.Assert(pc2.RequestVerification(activeRoles, "test", "test", "", mysql.IndexPriv), IsTrue)
}

func (s *testPrivilegeSuite) TestCheckViewPrivilege(c *C) {
rootSe := newSession(c, s.store, s.dbName)
mustExec(c, rootSe, `CREATE USER 'vuser'@'localhost';`)
mustExec(c, rootSe, `CREATE VIEW v AS SELECT * FROM test;`)

se := newSession(c, s.store, s.dbName)
activeRoles := make([]*auth.RoleIdentity, 0)
c.Assert(se.Auth(&auth.UserIdentity{Username: "vuser", Hostname: "localhost"}, nil, nil), IsTrue)
pc := privilege.GetPrivilegeManager(se)
c.Assert(pc.RequestVerification(activeRoles, "test", "v", "", mysql.SelectPriv), IsFalse)

mustExec(c, rootSe, `GRANT SELECT ON test.v TO 'vuser'@'localhost';`)
c.Assert(pc.RequestVerification(activeRoles, "test", "v", "", mysql.SelectPriv), IsTrue)
c.Assert(pc.RequestVerification(activeRoles, "test", "v", "", mysql.ShowViewPriv), IsFalse)

mustExec(c, rootSe, `GRANT SHOW VIEW ON test.v TO 'vuser'@'localhost';`)
c.Assert(pc.RequestVerification(activeRoles, "test", "v", "", mysql.SelectPriv), IsTrue)
c.Assert(pc.RequestVerification(activeRoles, "test", "v", "", mysql.ShowViewPriv), IsTrue)
}

func (s *testPrivilegeSuite) TestCheckPrivilegeWithRoles(c *C) {
rootSe := newSession(c, s.store, s.dbName)
mustExec(c, rootSe, `CREATE USER 'test_role'@'localhost';`)
Expand Down Expand Up @@ -556,7 +576,7 @@ func (s *testPrivilegeSuite) TestAnalyzeTable(c *C) {
c.Assert(err.Error(), Equals, "[planner:1142]INSERT command denied to user 'anobody'@'%' for table 't1'")

_, err = se.Execute(context.Background(), "select * from t1")
c.Assert(err.Error(), Equals, "[planner:1142]SELECT command denied to user 'localhost'@'anobody' for table 't1'")
c.Assert(err.Error(), Equals, "[planner:1142]SELECT command denied to user 'anobody'@'localhost' for table 't1'")

// try again after SELECT privilege granted
c.Assert(se.Auth(&auth.UserIdentity{Username: "asuper", Hostname: "localhost", AuthUsername: "asuper", AuthHostname: "%"}, nil, nil), IsTrue)
Expand Down
1 change: 1 addition & 0 deletions sessionctx/stmtctx/stmtctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ type StatementContext struct {
InDeleteStmt bool
InSelectStmt bool
InLoadDataStmt bool
InExplainStmt bool
IgnoreTruncate bool
IgnoreZeroInDate bool
DupKeyAsWarning bool
Expand Down

0 comments on commit c3a89b0

Please sign in to comment.