From c354b8bd07de1e9ff1822eaa9dd4c4de484307ea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Tue, 15 Nov 2022 12:11:55 +0800 Subject: [PATCH] extension: add support for extension to custom table and sysvar privileges (#39137) close pingcap/tidb#39136 --- executor/set.go | 19 +++ expression/extension.go | 10 +- extension/BUILD.bazel | 1 + extension/extensions.go | 15 +++ extension/function.go | 8 +- extension/function_test.go | 31 +++-- extension/manifest.go | 12 ++ extension/registry_test.go | 187 +++++++++++++++++++++++++++-- privilege/privileges/privileges.go | 20 ++- server/rpc_server.go | 9 +- session/session.go | 6 +- sessionctx/variable/variable.go | 2 + 12 files changed, 282 insertions(+), 38 deletions(-) diff --git a/executor/set.go b/executor/set.go index b3b8c00c1d..2396356c12 100644 --- a/executor/set.go +++ b/executor/set.go @@ -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 diff --git a/expression/extension.go b/expression/extension.go index 2fca7fdfab..9ab506213d 100644 --- a/expression/extension.go +++ b/expression/extension.go @@ -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) diff --git a/extension/BUILD.bazel b/extension/BUILD.bazel index 67b3351e01..12ca672b3c 100644 --- a/extension/BUILD.bazel +++ b/extension/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//parser", "//parser/ast", "//parser/auth", + "//parser/mysql", "//sessionctx/stmtctx", "//sessionctx/variable", "//types", diff --git a/extension/extensions.go b/extension/extensions.go index 68bcffd585..55f9c0bb83 100644 --- a/extension/extensions.go +++ b/extension/extensions.go @@ -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 { diff --git a/extension/function.go b/extension/function.go index d01519e9b7..cb7c19de0a 100644 --- a/extension/function.go +++ b/extension/function.go @@ -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 diff --git a/extension/function_test.go b/extension/function_test.go index c9521e91ed..86fcd0be84 100644 --- a/extension/function_test.go +++ b/extension/function_test.go @@ -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 }, diff --git a/extension/manifest.go b/extension/manifest.go index 4a11b93beb..436067c11d 100644 --- a/extension/manifest.go +++ b/extension/manifest.go @@ -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() } diff --git a/extension/registry_test.go b/extension/registry_test.go index 6137d3b0ff..ba1130d815 100644 --- a/extension/registry_test.go +++ b/extension/registry_test.go @@ -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")) +} diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index 58be6097c2..e983b81f0a 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -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) } diff --git a/server/rpc_server.go b/server/rpc_server.go index 4dcad639ad..f92deaf802 100644 --- a/server/rpc_server.go +++ b/server/rpc_server.go @@ -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 diff --git a/session/session.go b/session/session.go index 680ddda98d..4b220af345 100644 --- a/session/session.go +++ b/session/session.go @@ -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 diff --git a/sessionctx/variable/variable.go b/sessionctx/variable/variable.go index 3acecb6b83..4b7faa0948 100644 --- a/sessionctx/variable/variable.go +++ b/sessionctx/variable/variable.go @@ -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.