extension: add support for extension to custom table and sysvar privileges (#39137)

close pingcap/tidb#39136
This commit is contained in:
王超
2022-11-15 12:11:55 +08:00
committed by GitHub
parent b03690a3ae
commit c354b8bd07
12 changed files with 282 additions and 38 deletions

View File

@ -24,7 +24,9 @@ import (
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/charset"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/plugin"
"github.com/pingcap/tidb/privilege"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/table/temptable"
@ -32,6 +34,7 @@ import (
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/gcutil"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/sem"
"go.uber.org/zap"
)
@ -109,6 +112,22 @@ func (e *SetExecutor) setSysVariable(ctx context.Context, name string, v *expres
}
return variable.ErrUnknownSystemVar.GenWithStackByArgs(name)
}
if sysVar.RequireDynamicPrivileges != nil {
semEnabled := sem.IsEnabled()
pm := privilege.GetPrivilegeManager(e.ctx)
privs := sysVar.RequireDynamicPrivileges(v.IsGlobal, semEnabled)
for _, priv := range privs {
if !pm.RequestDynamicVerification(sessionVars.ActiveRoles, priv, false) {
msg := priv
if !semEnabled {
msg = "SUPER or " + msg
}
return core.ErrSpecificAccessDenied.GenWithStackByArgs(msg)
}
}
}
if sysVar.IsNoop && !variable.EnableNoopVariables.Load() {
// The variable is a noop. For compatibility we allow it to still
// be changed, but we append a warning since users might be expecting

View File

@ -114,11 +114,13 @@ func (c *extensionFuncClass) getFunction(ctx sessionctx.Context, args []Expressi
}
func (c *extensionFuncClass) checkPrivileges(ctx sessionctx.Context) error {
privs := c.funcDef.RequireDynamicPrivileges
if semPrivs := c.funcDef.SemRequireDynamicPrivileges; len(semPrivs) > 0 && sem.IsEnabled() {
privs = semPrivs
fn := c.funcDef.RequireDynamicPrivileges
if fn == nil {
return nil
}
semEnabled := sem.IsEnabled()
privs := fn(semEnabled)
if len(privs) == 0 {
return nil
}
@ -129,7 +131,7 @@ func (c *extensionFuncClass) checkPrivileges(ctx sessionctx.Context) error {
for _, priv := range privs {
if !manager.RequestDynamicVerification(activeRoles, priv, false) {
msg := priv
if !sem.IsEnabled() {
if !semEnabled {
msg = "SUPER or " + msg
}
return errSpecificAccessDenied.GenWithStackByArgs(msg)

View File

@ -16,6 +16,7 @@ go_library(
"//parser",
"//parser/ast",
"//parser/auth",
"//parser/mysql",
"//sessionctx/stmtctx",
"//sessionctx/variable",
"//types",

View File

@ -45,6 +45,21 @@ func (es *Extensions) Bootstrap(ctx BootstrapContext) error {
return nil
}
// GetAccessCheckFuncs returns spec functions of the custom access check
func (es *Extensions) GetAccessCheckFuncs() (funcs []AccessCheckFunc) {
if es == nil {
return nil
}
for _, m := range es.manifests {
if m.accessCheckFunc != nil {
funcs = append(funcs, m.accessCheckFunc)
}
}
return funcs
}
// NewSessionExtensions creates a new ConnExtensions object
func (es *Extensions) NewSessionExtensions() *SessionExtensions {
if es == nil {

View File

@ -48,12 +48,8 @@ type FunctionDef struct {
EvalStringFunc func(ctx FunctionContext, row chunk.Row) (string, bool, error)
// EvalIntFunc is the eval function when `EvalTp` is `types.ETInt`
EvalIntFunc func(ctx FunctionContext, row chunk.Row) (int64, bool, error)
// RequireDynamicPrivileges is the dynamic privileges needed to invoke the function
// If `RequireDynamicPrivileges` is empty, it means every one can invoke this function
RequireDynamicPrivileges []string
// SemRequireDynamicPrivileges is the dynamic privileges needed to invoke the function in sem mode
// If `SemRequireDynamicPrivileges` is empty, `DynamicPrivileges` will be used in sem mode
SemRequireDynamicPrivileges []string
// RequireDynamicPrivileges is a function to return a list of dynamic privileges to check.
RequireDynamicPrivileges func(sem bool) []string
}
// Validate validates the function definition

View File

@ -283,26 +283,37 @@ func TestExtensionFuncPrivilege(t *testing.T) {
},
},
{
Name: "custom_only_dyn_priv_func",
EvalTp: types.ETString,
RequireDynamicPrivileges: []string{"CUSTOM_DYN_PRIV_1"},
Name: "custom_only_dyn_priv_func",
EvalTp: types.ETString,
RequireDynamicPrivileges: func(sem bool) []string {
return []string{"CUSTOM_DYN_PRIV_1"}
},
EvalStringFunc: func(ctx extension.FunctionContext, row chunk.Row) (string, bool, error) {
return "abc", false, nil
},
},
{
Name: "custom_only_sem_dyn_priv_func",
EvalTp: types.ETString,
SemRequireDynamicPrivileges: []string{"RESTRICTED_CUSTOM_DYN_PRIV_2"},
Name: "custom_only_sem_dyn_priv_func",
EvalTp: types.ETString,
RequireDynamicPrivileges: func(sem bool) []string {
if sem {
return []string{"RESTRICTED_CUSTOM_DYN_PRIV_2"}
}
return nil
},
EvalStringFunc: func(ctx extension.FunctionContext, row chunk.Row) (string, bool, error) {
return "def", false, nil
},
},
{
Name: "custom_both_dyn_priv_func",
EvalTp: types.ETString,
RequireDynamicPrivileges: []string{"CUSTOM_DYN_PRIV_1"},
SemRequireDynamicPrivileges: []string{"RESTRICTED_CUSTOM_DYN_PRIV_2"},
Name: "custom_both_dyn_priv_func",
EvalTp: types.ETString,
RequireDynamicPrivileges: func(sem bool) []string {
if sem {
return []string{"RESTRICTED_CUSTOM_DYN_PRIV_2"}
}
return []string{"CUSTOM_DYN_PRIV_1"}
},
EvalStringFunc: func(ctx extension.FunctionContext, row chunk.Row) (string, bool, error) {
return "ghi", false, nil
},

View File

@ -19,6 +19,7 @@ import (
"github.com/ngaut/pools"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/chunk"
clientv3 "go.etcd.io/etcd/client/v3"
@ -54,6 +55,16 @@ func WithCustomFunctions(funcs []*FunctionDef) Option {
}
}
// AccessCheckFunc is a function that returns a dynamic privilege list for db/tbl/column access
type AccessCheckFunc func(db, tbl, column string, priv mysql.PrivilegeType, sem bool) []string
// WithCustomAccessCheck specifies the custom db/tbl/column dynamic privilege check
func WithCustomAccessCheck(fn AccessCheckFunc) Option {
return func(m *Manifest) {
m.accessCheckFunc = fn
}
}
// WithSessionHandlerFactory specifies a factory function to handle session
func WithSessionHandlerFactory(factory func() *SessionHandler) Option {
return func(m *Manifest) {
@ -106,6 +117,7 @@ type Manifest struct {
dynPrivs []string
bootstrap func(BootstrapContext) error
funcs []*FunctionDef
accessCheckFunc AccessCheckFunc
sessionHandlerFactory func() *SessionHandler
close func()
}

View File

@ -19,8 +19,12 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/tidb/extension"
"github.com/pingcap/tidb/parser/auth"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/privilege/privileges"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/util/sem"
"github.com/stretchr/testify/require"
)
@ -47,9 +51,7 @@ func TestSetupExtensions(t *testing.T) {
}
func TestExtensionRegisterName(t *testing.T) {
defer func() {
extension.Reset()
}()
defer extension.Reset()
// test empty name
extension.Reset()
@ -62,9 +64,7 @@ func TestExtensionRegisterName(t *testing.T) {
}
func TestRegisterExtensionWithClose(t *testing.T) {
defer func() {
extension.Reset()
}()
defer extension.Reset()
// normal register
extension.Reset()
@ -97,9 +97,7 @@ func TestRegisterExtensionWithClose(t *testing.T) {
}
func TestRegisterExtensionWithDyncPrivs(t *testing.T) {
defer func() {
extension.Reset()
}()
defer extension.Reset()
origDynPrivs := privileges.GetDynamicPrivileges()
origDynPrivs = append([]string{}, origDynPrivs...)
@ -132,9 +130,7 @@ func TestRegisterExtensionWithDyncPrivs(t *testing.T) {
}
func TestRegisterExtensionWithSysVars(t *testing.T) {
defer func() {
extension.Reset()
}()
defer extension.Reset()
sysVar1 := &variable.SysVar{
Scope: variable.ScopeGlobal | variable.ScopeSession,
@ -184,3 +180,170 @@ func TestRegisterExtensionWithSysVars(t *testing.T) {
require.Nil(t, variable.GetSysVar("var1"))
require.Nil(t, variable.GetSysVar("var2"))
}
func TestSetVariablePrivilege(t *testing.T) {
defer extension.Reset()
sysVar1 := &variable.SysVar{
Scope: variable.ScopeGlobal | variable.ScopeSession,
Name: "var1",
Value: "1",
MinValue: 0,
MaxValue: 100,
Type: variable.TypeInt,
RequireDynamicPrivileges: func(isGlobal bool, sem bool) []string {
privs := []string{"priv1"}
if isGlobal {
privs = append(privs, "priv2")
}
if sem {
privs = append(privs, "restricted_priv3")
}
return privs
},
}
extension.Reset()
require.NoError(t, extension.Register(
"test",
extension.WithCustomSysVariables([]*variable.SysVar{sysVar1}),
extension.WithCustomDynPrivs([]string{"priv1", "priv2", "restricted_priv3"}),
))
require.NoError(t, extension.Setup())
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("create user u2@localhost")
tk1 := testkit.NewTestKit(t, store)
require.NoError(t, tk1.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost"}, nil, nil))
tk2 := testkit.NewTestKit(t, store)
require.NoError(t, tk2.Session().Auth(&auth.UserIdentity{Username: "u2", Hostname: "localhost"}, nil, nil))
sem.Disable()
tk1.MustExec("set @@var1=7")
tk1.MustQuery("select @@var1").Check(testkit.Rows("7"))
require.EqualError(t, tk2.ExecToErr("set @@var1=10"), "[planner:1227]Access denied; you need (at least one of) the SUPER or priv1 privilege(s) for this operation")
tk2.MustQuery("select @@var1").Check(testkit.Rows("1"))
tk.MustExec("GRANT priv1 on *.* TO u2@localhost")
tk2.MustExec("set @@var1=8")
tk2.MustQuery("select @@var1").Check(testkit.Rows("8"))
tk1.MustExec("set @@global.var1=17")
tk1.MustQuery("select @@global.var1").Check(testkit.Rows("17"))
tk.MustExec("GRANT SYSTEM_VARIABLES_ADMIN on *.* TO u2@localhost")
require.EqualError(t, tk2.ExecToErr("set @@global.var1=18"), "[planner:1227]Access denied; you need (at least one of) the SUPER or priv2 privilege(s) for this operation")
tk2.MustQuery("select @@global.var1").Check(testkit.Rows("17"))
tk.MustExec("GRANT priv2 on *.* TO u2@localhost")
tk2.MustExec("set @@global.var1=18")
tk2.MustQuery("select @@global.var1").Check(testkit.Rows("18"))
sem.Enable()
defer sem.Disable()
require.EqualError(t, tk1.ExecToErr("set @@global.var1=27"), "[planner:1227]Access denied; you need (at least one of) the restricted_priv3 privilege(s) for this operation")
tk1.MustQuery("select @@global.var1").Check(testkit.Rows("18"))
require.EqualError(t, tk2.ExecToErr("set @@global.var1=27"), "[planner:1227]Access denied; you need (at least one of) the restricted_priv3 privilege(s) for this operation")
tk2.MustQuery("select @@global.var1").Check(testkit.Rows("18"))
tk.MustExec("GRANT restricted_priv3 on *.* TO u2@localhost")
tk2.MustExec("set @@global.var1=28")
tk2.MustQuery("select @@global.var1").Check(testkit.Rows("28"))
}
func TestCustomAccessCheck(t *testing.T) {
defer extension.Reset()
extension.Reset()
require.NoError(t, extension.Register(
"test",
extension.WithCustomDynPrivs([]string{"priv1", "priv2", "restricted_priv3"}),
extension.WithCustomAccessCheck(func(db, tbl, column string, priv mysql.PrivilegeType, sem bool) []string {
if db != "test" || tbl != "t1" {
return nil
}
var privs []string
if priv == mysql.SelectPriv {
privs = append(privs, "priv1")
} else if priv == mysql.UpdatePriv {
privs = append(privs, "priv2")
if sem {
privs = append(privs, "restricted_priv3")
}
} else {
return nil
}
return privs
}),
))
require.NoError(t, extension.Setup())
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("create user u2@localhost")
tk1 := testkit.NewTestKit(t, store)
require.NoError(t, tk1.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost"}, nil, nil))
tk1.MustExec("use test")
tk2 := testkit.NewTestKit(t, store)
require.NoError(t, tk2.Session().Auth(&auth.UserIdentity{Username: "u2", Hostname: "localhost"}, nil, nil))
tk.MustExec("GRANT all on test.t1 TO u2@localhost")
tk2.MustExec("use test")
tk1.MustExec("create table t1(id int primary key, v int)")
tk1.MustExec("insert into t1 values (1, 10), (2, 20)")
tk1.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10"))
tk1.MustQuery("select * from t1").Check(testkit.Rows("1 10", "2 20"))
require.EqualError(t, tk2.ExecToErr("select * from t1 where id=1"), "[planner:1142]SELECT command denied to user 'u2'@'localhost' for table 't1'")
require.EqualError(t, tk2.ExecToErr("select * from t1"), "[planner:1142]SELECT command denied to user 'u2'@'localhost' for table 't1'")
tk.MustExec("GRANT priv1 on *.* TO u2@localhost")
tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10"))
tk2.MustQuery("select * from t1").Check(testkit.Rows("1 10", "2 20"))
require.EqualError(t, tk2.ExecToErr("update t1 set v=11 where id=1"), "[planner:8121]privilege check for 'Update' fail")
require.EqualError(t, tk2.ExecToErr("update t1 set v=11 where id<2"), "[planner:8121]privilege check for 'Update' fail")
tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10"))
tk.MustExec("GRANT priv2 on *.* TO u2@localhost")
tk2.MustExec("update t1 set v=11 where id=1")
tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 11"))
tk2.MustExec("update t1 set v=12 where id<2")
tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 12"))
sem.Enable()
defer sem.Disable()
require.EqualError(t, tk1.ExecToErr("update t1 set v=21 where id=1"), "[planner:8121]privilege check for 'Update' fail")
require.EqualError(t, tk1.ExecToErr("update t1 set v=21 where id<2"), "[planner:8121]privilege check for 'Update' fail")
tk1.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 12"))
require.EqualError(t, tk2.ExecToErr("update t1 set v=21 where id=1"), "[planner:8121]privilege check for 'Update' fail")
require.EqualError(t, tk2.ExecToErr("update t1 set v=21 where id<2"), "[planner:8121]privilege check for 'Update' fail")
tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 12"))
tk.MustExec("GRANT restricted_priv3 on *.* TO u2@localhost")
tk2.MustExec("update t1 set v=31 where id=1")
tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 31"))
tk2.MustExec("update t1 set v=32 where id<2")
tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 32"))
}

View File

@ -68,6 +68,15 @@ type UserPrivileges struct {
user string
host string
*Handle
extensionAccessCheckFuncs []extension.AccessCheckFunc
}
// NewUserPrivileges creates a new UserPrivileges
func NewUserPrivileges(handle *Handle, extension *extension.Extensions) *UserPrivileges {
return &UserPrivileges{
Handle: handle,
extensionAccessCheckFuncs: extension.GetAccessCheckFuncs(),
}
}
// RequestDynamicVerificationWithUser implements the Manager interface.
@ -128,7 +137,8 @@ func (p *UserPrivileges) RequestVerification(activeRoles []*auth.RoleIdentity, d
tblLowerName := strings.ToLower(table)
// If SEM is enabled and the user does not have the RESTRICTED_TABLES_ADMIN privilege
// There are some hard rules which overwrite system tables and schemas as read-only at most.
if sem.IsEnabled() && !p.RequestDynamicVerification(activeRoles, "RESTRICTED_TABLES_ADMIN", false) {
semEnabled := sem.IsEnabled()
if semEnabled && !p.RequestDynamicVerification(activeRoles, "RESTRICTED_TABLES_ADMIN", false) {
if sem.IsInvisibleTable(dbLowerName, tblLowerName) {
return false
}
@ -158,6 +168,14 @@ func (p *UserPrivileges) RequestVerification(activeRoles []*auth.RoleIdentity, d
}
}
for _, fn := range p.extensionAccessCheckFuncs {
for _, dynPriv := range fn(db, table, column, priv, semEnabled) {
if !p.RequestDynamicVerification(activeRoles, dynPriv, false) {
return false
}
}
}
mysqlPriv := p.Handle.Get()
return mysqlPriv.RequestVerification(activeRoles, p.user, p.host, db, table, column, priv)
}

View File

@ -27,6 +27,7 @@ import (
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/extension"
"github.com/pingcap/tidb/privilege"
"github.com/pingcap/tidb/privilege/privileges"
"github.com/pingcap/tidb/session"
@ -216,11 +217,13 @@ func (s *rpcServer) createSession() (session.Session, error) {
if err != nil {
return nil, err
}
extensions, err := extension.GetExtensions()
if err != nil {
return nil, err
}
do := domain.GetDomain(se)
is := do.InfoSchema()
pm := &privileges.UserPrivileges{
Handle: do.PrivilegeHandle(),
}
pm := privileges.NewUserPrivileges(do.PrivilegeHandle(), extensions)
privilege.BindPrivilegeManager(se, pm)
vars := se.GetSessionVars()
vars.TxnCtx.InfoSchema = is

View File

@ -2711,9 +2711,11 @@ func CreateSessionWithOpt(store kv.Storage, opt *Opt) (Session, error) {
if err != nil {
return nil, err
}
pm := &privileges.UserPrivileges{
Handle: do.PrivilegeHandle(),
extensions, err := extension.GetExtensions()
if err != nil {
return nil, err
}
pm := privileges.NewUserPrivileges(do.PrivilegeHandle(), extensions)
privilege.BindPrivilegeManager(s, pm)
// Add stats collector, and it will be freed by background stats worker

View File

@ -164,6 +164,8 @@ type SysVar struct {
// If the global variable has the global config name,
// it should store the global config into PD(etcd) too when set global variable.
GlobalConfigName string
// RequireDynamicPrivileges is a function to return a dynamic privilege list to check the set sysvar privilege
RequireDynamicPrivileges func(isGlobal bool, sem bool) []string
}
// GetGlobalFromHook calls the GetSession func if it exists.