Skip to content

Commit

Permalink
planner, privilege: Add security enhanced mode part 4 (#24416)
Browse files Browse the repository at this point in the history
  • Loading branch information
morgo authored May 17, 2021
1 parent 557c94b commit f2cb145
Show file tree
Hide file tree
Showing 5 changed files with 106 additions and 1 deletion.
1 change: 1 addition & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -357,6 +357,7 @@ func (s *testSuiteP1) TestShow(c *C) {
"CONNECTION_ADMIN Server Admin ",
"RESTRICTED_TABLES_ADMIN Server Admin ",
"RESTRICTED_STATUS_ADMIN Server Admin ",
"RESTRICTED_USER_ADMIN Server Admin ",
))
c.Assert(len(tk.MustQuery("show table status").Rows()), Equals, 1)
}
Expand Down
40 changes: 39 additions & 1 deletion planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/parser"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/auth"
"github.com/pingcap/parser/charset"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
Expand All @@ -36,6 +37,7 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/planner/property"
"github.com/pingcap/tidb/planner/util"
"github.com/pingcap/tidb/privilege"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
Expand Down Expand Up @@ -2276,9 +2278,16 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) {
p.setSchemaAndNames(buildBRIESchema())
err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or BACKUP_ADMIN")
b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "BACKUP_ADMIN", false, err)
case *ast.GrantRoleStmt, *ast.RevokeRoleStmt:
case *ast.GrantRoleStmt:
err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or ROLE_ADMIN")
b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "ROLE_ADMIN", false, err)
case *ast.RevokeRoleStmt:
err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or ROLE_ADMIN")
b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "ROLE_ADMIN", false, err)
// Check if any of the users are RESTRICTED
for _, user := range raw.Users {
b.visitInfo = appendVisitInfoIsRestrictedUser(b.visitInfo, b.ctx, user, "RESTRICTED_USER_ADMIN")
}
case *ast.RevokeStmt:
b.visitInfo = collectVisitInfoFromRevokeStmt(b.ctx, b.visitInfo, raw)
case *ast.KillStmt:
Expand All @@ -2292,12 +2301,23 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) {
err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or CONNECTION_ADMIN")
b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "CONNECTION_ADMIN", false, err)
}
b.visitInfo = appendVisitInfoIsRestrictedUser(b.visitInfo, b.ctx, &auth.UserIdentity{Username: pi.User, Hostname: pi.Host}, "RESTRICTED_CONNECTION_ADMIN")
}
}
case *ast.UseStmt:
if raw.DBName == "" {
return nil, ErrNoDB
}
case *ast.DropUserStmt:
// The main privilege checks for DROP USER are currently performed in executor/simple.go
// because they use complex OR conditions (not supported by visitInfo).
for _, user := range raw.UserList {
b.visitInfo = appendVisitInfoIsRestrictedUser(b.visitInfo, b.ctx, user, "RESTRICTED_USER_ADMIN")
}
case *ast.SetPwdStmt:
if raw.User != nil {
b.visitInfo = appendVisitInfoIsRestrictedUser(b.visitInfo, b.ctx, raw.User, "RESTRICTED_USER_ADMIN")
}
case *ast.ShutdownStmt:
b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ShutdownPriv, "", "", "", nil)
}
Expand Down Expand Up @@ -2339,6 +2359,10 @@ func collectVisitInfoFromRevokeStmt(sctx sessionctx.Context, vi []visitInfo, stm
for _, priv := range allPrivs {
vi = appendVisitInfo(vi, priv, dbName, tableName, "", nil)
}
for _, u := range stmt.Users {
// For SEM, make sure the users are not restricted
vi = appendVisitInfoIsRestrictedUser(vi, sctx, u.User, "RESTRICTED_USER_ADMIN")
}
if nonDynamicPrivilege {
// Dynamic privileges use their own GRANT OPTION. If there were any non-dynamic privilege requests,
// we need to attach the "GLOBAL" version of the GRANT OPTION.
Expand All @@ -2347,6 +2371,20 @@ func collectVisitInfoFromRevokeStmt(sctx sessionctx.Context, vi []visitInfo, stm
return vi
}

// appendVisitInfoIsRestrictedUser appends additional visitInfo if the user has a
// special privilege called "RESTRICTED_USER_ADMIN". It only applies when SEM is enabled.
func appendVisitInfoIsRestrictedUser(visitInfo []visitInfo, sctx sessionctx.Context, user *auth.UserIdentity, priv string) []visitInfo {
if !sem.IsEnabled() {
return visitInfo
}
checker := privilege.GetPrivilegeManager(sctx)
if checker != nil && checker.RequestDynamicVerificationWithUser("RESTRICTED_USER_ADMIN", false, user) {
err := ErrSpecificAccessDenied.GenWithStackByArgs(priv)
visitInfo = appendDynamicVisitInfo(visitInfo, priv, false, err)
}
return visitInfo
}

func collectVisitInfoFromGrantStmt(sctx sessionctx.Context, vi []visitInfo, stmt *ast.GrantStmt) []visitInfo {
// To use GRANT, you must have the GRANT OPTION privilege,
// and you must have the privileges that you are granting.
Expand Down
3 changes: 3 additions & 0 deletions privilege/privilege.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,9 @@ type Manager interface {
// Dynamic privileges are only assignable globally, and have their own grantable attribute.
RequestDynamicVerification(activeRoles []*auth.RoleIdentity, privName string, grantable bool) bool

// RequestDynamicVerification verifies a DYNAMIC privilege for a specific user.
RequestDynamicVerificationWithUser(privName string, grantable bool, user *auth.UserIdentity) bool

// ConnectionVerification verifies user privilege for connection.
ConnectionVerification(user, host string, auth, salt []byte, tlsState *tls.ConnectionState) (string, string, bool)

Expand Down
16 changes: 16 additions & 0 deletions privilege/privileges/privileges.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ var dynamicPrivs = []string{
"CONNECTION_ADMIN",
"RESTRICTED_TABLES_ADMIN", // Can see system tables when SEM is enabled
"RESTRICTED_STATUS_ADMIN", // Can see all status vars when SEM is enabled.
"RESTRICTED_USER_ADMIN", // User can not have their access revoked by SUPER users.
}
var dynamicPrivLock sync.Mutex

Expand All @@ -56,6 +57,21 @@ type UserPrivileges struct {
*Handle
}

// RequestDynamicVerificationWithUser implements the Manager interface.
func (p *UserPrivileges) RequestDynamicVerificationWithUser(privName string, grantable bool, user *auth.UserIdentity) bool {
if SkipWithGrant {
return true
}

if user == nil {
return false
}

mysqlPriv := p.Handle.Get()
roles := mysqlPriv.getDefaultRoles(user.Username, user.Hostname)
return mysqlPriv.RequestDynamicVerification(roles, user.Username, user.Hostname, privName, grantable)
}

// RequestDynamicVerification implements the Manager interface.
func (p *UserPrivileges) RequestDynamicVerification(activeRoles []*auth.RoleIdentity, privName string, grantable bool) bool {
if SkipWithGrant {
Expand Down
47 changes: 47 additions & 0 deletions privilege/privileges/privileges_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1428,6 +1428,53 @@ func (s *testPrivilegeSuite) TestViewDefiner(c *C) {
tk.MustExec("select * from test_view2")
}

func (s *testPrivilegeSuite) TestSecurityEnhancedModeRestrictedUsers(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("CREATE USER ruroot1, ruroot2, ruroot3")
tk.MustExec("CREATE ROLE notimportant")
tk.MustExec("GRANT SUPER, CREATE USER ON *.* to ruroot1 WITH GRANT OPTION")
tk.MustExec("SET tidb_enable_dynamic_privileges=1")
tk.MustExec("GRANT SUPER, RESTRICTED_USER_ADMIN, CREATE USER ON *.* to ruroot2 WITH GRANT OPTION")
tk.MustExec("GRANT RESTRICTED_USER_ADMIN ON *.* to ruroot3")
tk.MustExec("GRANT notimportant TO ruroot2, ruroot3")

sem.Enable()
defer sem.Disable()

stmts := []string{
"SET PASSWORD for ruroot3 = 'newpassword'",
"REVOKE notimportant FROM ruroot3",
"REVOKE SUPER ON *.* FROM ruroot3",
"DROP USER ruroot3",
}

// ruroot1 has SUPER but in SEM will be restricted
tk.Se.Auth(&auth.UserIdentity{
Username: "ruroot1",
Hostname: "localhost",
AuthUsername: "uroot",
AuthHostname: "%",
}, nil, nil)

for _, stmt := range stmts {
err := tk.ExecToErr(stmt)
c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RESTRICTED_USER_ADMIN privilege(s) for this operation")
}

// Switch to ruroot2, it should be permitted
tk.Se.Auth(&auth.UserIdentity{
Username: "ruroot2",
Hostname: "localhost",
AuthUsername: "uroot",
AuthHostname: "%",
}, nil, nil)

for _, stmt := range stmts {
err := tk.ExecToErr(stmt)
c.Assert(err, IsNil)
}
}

func (s *testPrivilegeSuite) TestDynamicPrivsRegistration(c *C) {
se := newSession(c, s.store, s.dbName)
pm := privilege.GetPrivilegeManager(se)
Expand Down

0 comments on commit f2cb145

Please sign in to comment.