Skip to content

Commit

Permalink
variable: add system variable 'tidb_resource_control_strict_mode' (#5…
Browse files Browse the repository at this point in the history
  • Loading branch information
glorv authored Jun 20, 2024
1 parent 1a24c03 commit 6fe4f9e
Show file tree
Hide file tree
Showing 8 changed files with 148 additions and 109 deletions.
2 changes: 1 addition & 1 deletion br/pkg/restore/snap_client/systable_restore_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,5 +115,5 @@ func TestCheckSysTableCompatibility(t *testing.T) {
//
// The above variables are in the file br/pkg/restore/systable_restore.go
func TestMonitorTheSystemTableIncremental(t *testing.T) {
require.Equal(t, int64(198), session.CurrentBootstrapVersion)
require.Equal(t, int64(199), session.CurrentBootstrapVersion)
}
6 changes: 4 additions & 2 deletions pkg/planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -3398,8 +3398,10 @@ func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (base.
p.StaleTxnStartTS = startTS
}
case *ast.SetResourceGroupStmt:
err := plannererrors.ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or RESOURCE_GROUP_ADMIN or RESOURCE_GROUP_USER")
b.visitInfo = appendDynamicVisitInfo(b.visitInfo, []string{"RESOURCE_GROUP_ADMIN", "RESOURCE_GROUP_USER"}, false, err)
if variable.EnableResourceControlStrictMode.Load() {
err := plannererrors.ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or RESOURCE_GROUP_ADMIN or RESOURCE_GROUP_USER")
b.visitInfo = appendDynamicVisitInfo(b.visitInfo, []string{"RESOURCE_GROUP_ADMIN", "RESOURCE_GROUP_USER"}, false, err)
}
}
return p, nil
}
Expand Down
13 changes: 8 additions & 5 deletions pkg/planner/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -175,12 +175,15 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
// Override the resource group if the hint is set.
if retErr == nil && sessVars.StmtCtx.StmtHints.HasResourceGroup {
if variable.EnableResourceControl.Load() {
checker := privilege.GetPrivilegeManager(sctx)
hasPriv := true
if checker != nil {
hasRgAdminPriv := checker.RequestDynamicVerification(sctx.GetSessionVars().ActiveRoles, "RESOURCE_GROUP_ADMIN", false)
hasRgUserPriv := checker.RequestDynamicVerification(sctx.GetSessionVars().ActiveRoles, "RESOURCE_GROUP_USER", false)
hasPriv = hasRgAdminPriv || hasRgUserPriv
// only check dynamic privilege when strict-mode is enabled.
if variable.EnableResourceControlStrictMode.Load() {
checker := privilege.GetPrivilegeManager(sctx)
if checker != nil {
hasRgAdminPriv := checker.RequestDynamicVerification(sctx.GetSessionVars().ActiveRoles, "RESOURCE_GROUP_ADMIN", false)
hasRgUserPriv := checker.RequestDynamicVerification(sctx.GetSessionVars().ActiveRoles, "RESOURCE_GROUP_USER", false)
hasPriv = hasRgAdminPriv || hasRgUserPriv
}
}
if hasPriv {
sessVars.StmtCtx.ResourceGroupName = sessVars.StmtCtx.StmtHints.ResourceGroup
Expand Down
130 changes: 38 additions & 92 deletions pkg/session/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -1097,11 +1097,15 @@ const (
// version 198
// add column `owner_id` for `mysql.tidb_mdl_info` table
version198 = 198

// version 199
// sets `tidb_resource_control_strict_mode` to off when a cluster upgrades from some version lower than v8.2.
version199 = 199
)

// currentBootstrapVersion is defined as a variable, so we can modify its value for testing.
// please make sure this is the largest version
var currentBootstrapVersion int64 = version198
var currentBootstrapVersion int64 = version199

// DDL owner key's expired time is ManagerSessionTTL seconds, we should wait the time and give more time to have a chance to finish it.
var internalSQLTimeout = owner.ManagerSessionTTL + 15
Expand Down Expand Up @@ -1265,6 +1269,7 @@ var (
upgradeToVer196,
upgradeToVer197,
upgradeToVer198,
upgradeToVer199,
}
)

Expand Down Expand Up @@ -2369,19 +2374,7 @@ func upgradeToVer80(s sessiontypes.Session, ver int64) {
}
// Check if tidb_analyze_version exists in mysql.GLOBAL_VARIABLES.
// If not, insert "tidb_analyze_version | 1" since this is the old behavior before we introduce this variable.
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
rs, err := s.ExecuteInternal(ctx, "SELECT VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME=%?;",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBAnalyzeVersion)
terror.MustNil(err)
req := rs.NewChunk(nil)
err = rs.Next(ctx, req)
terror.MustNil(err)
if req.NumRows() != 0 {
return
}

mustExecute(s, "INSERT HIGH_PRIORITY IGNORE INTO %n.%n VALUES (%?, %?);",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBAnalyzeVersion, 1)
initGlobalVariableIfNotExists(s, variable.TiDBAnalyzeVersion, 1)
}

// For users that upgrade TiDB from a pre-4.0 version, we want to disable index merge by default.
Expand All @@ -2392,19 +2385,7 @@ func upgradeToVer81(s sessiontypes.Session, ver int64) {
}
// Check if tidb_enable_index_merge exists in mysql.GLOBAL_VARIABLES.
// If not, insert "tidb_enable_index_merge | off".
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
rs, err := s.ExecuteInternal(ctx, "SELECT VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME=%?;",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableIndexMerge)
terror.MustNil(err)
req := rs.NewChunk(nil)
err = rs.Next(ctx, req)
terror.MustNil(err)
if req.NumRows() != 0 {
return
}

mustExecute(s, "INSERT HIGH_PRIORITY IGNORE INTO %n.%n VALUES (%?, %?);",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableIndexMerge, variable.Off)
initGlobalVariableIfNotExists(s, variable.TiDBEnableIndexMerge, variable.Off)
}

func upgradeToVer82(s sessiontypes.Session, ver int64) {
Expand Down Expand Up @@ -2539,19 +2520,7 @@ func upgradeToVer97(s sessiontypes.Session, ver int64) {
}
// Check if tidb_opt_range_max_size exists in mysql.GLOBAL_VARIABLES.
// If not, insert "tidb_opt_range_max_size | 0" since this is the old behavior before we introduce this variable.
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
rs, err := s.ExecuteInternal(ctx, "SELECT VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME=%?;",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBOptRangeMaxSize)
terror.MustNil(err)
req := rs.NewChunk(nil)
err = rs.Next(ctx, req)
terror.MustNil(err)
if req.NumRows() != 0 {
return
}

mustExecute(s, "INSERT HIGH_PRIORITY IGNORE INTO %n.%n VALUES (%?, %?);",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBOptRangeMaxSize, 0)
initGlobalVariableIfNotExists(s, variable.TiDBOptRangeMaxSize, 0)
}

func upgradeToVer98(s sessiontypes.Session, ver int64) {
Expand Down Expand Up @@ -2620,19 +2589,7 @@ func upgradeToVer105(s sessiontypes.Session, ver int64) {
if ver >= version105 {
return
}
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
rs, err := s.ExecuteInternal(ctx, "SELECT VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME=%?;",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBCostModelVersion)
terror.MustNil(err)
req := rs.NewChunk(nil)
err = rs.Next(ctx, req)
terror.MustNil(err)
if req.NumRows() != 0 {
return
}

mustExecute(s, "INSERT HIGH_PRIORITY IGNORE INTO %n.%n VALUES (%?, %?);",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBCostModelVersion, "1")
initGlobalVariableIfNotExists(s, variable.TiDBCostModelVersion, "1")
}

func upgradeToVer106(s sessiontypes.Session, ver int64) {
Expand Down Expand Up @@ -2724,20 +2681,7 @@ func upgradeToVer135(s sessiontypes.Session, ver int64) {
if ver >= version135 {
return
}

ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
rs, err := s.ExecuteInternal(ctx, "SELECT VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME=%?;",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBOptAdvancedJoinHint)
terror.MustNil(err)
req := rs.NewChunk(nil)
err = rs.Next(ctx, req)
terror.MustNil(err)
if req.NumRows() != 0 {
return
}

mustExecute(s, "INSERT HIGH_PRIORITY IGNORE INTO %n.%n VALUES (%?, %?);",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBOptAdvancedJoinHint, false)
initGlobalVariableIfNotExists(s, variable.TiDBOptAdvancedJoinHint, false)
}

func upgradeToVer136(s sessiontypes.Session, ver int64) {
Expand Down Expand Up @@ -2801,19 +2745,7 @@ func upgradeToVer142(s sessiontypes.Session, ver int64) {
if ver >= version142 {
return
}
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
rs, err := s.ExecuteInternal(ctx, "SELECT VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME=%?;",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableNonPreparedPlanCache)
terror.MustNil(err)
req := rs.NewChunk(nil)
err = rs.Next(ctx, req)
terror.MustNil(err)
if req.NumRows() != 0 {
return
}

mustExecute(s, "INSERT HIGH_PRIORITY IGNORE INTO %n.%n VALUES (%?, %?);",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableNonPreparedPlanCache, variable.Off)
initGlobalVariableIfNotExists(s, variable.TiDBEnableNonPreparedPlanCache, variable.Off)
}

func upgradeToVer143(s sessiontypes.Session, ver int64) {
Expand All @@ -2828,19 +2760,8 @@ func upgradeToVer144(s sessiontypes.Session, ver int64) {
if ver >= version144 {
return
}
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
rs, err := s.ExecuteInternal(ctx, "SELECT VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME=%?;",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBPlanCacheInvalidationOnFreshStats)
terror.MustNil(err)
req := rs.NewChunk(nil)
err = rs.Next(ctx, req)
terror.MustNil(err)
if req.NumRows() != 0 {
return
}

mustExecute(s, "INSERT HIGH_PRIORITY IGNORE INTO %n.%n VALUES (%?, %?);",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBPlanCacheInvalidationOnFreshStats, variable.Off)
initGlobalVariableIfNotExists(s, variable.TiDBPlanCacheInvalidationOnFreshStats, variable.Off)
}

func upgradeToVer146(s sessiontypes.Session, ver int64) {
Expand Down Expand Up @@ -3117,6 +3038,31 @@ func upgradeToVer198(s sessiontypes.Session, ver int64) {
doReentrantDDL(s, "ALTER TABLE mysql.tidb_mdl_info ADD COLUMN owner_id VARCHAR(64) NOT NULL DEFAULT '';", infoschema.ErrColumnExists)
}

func upgradeToVer199(s sessiontypes.Session, ver int64) {
if ver >= version199 {
return
}

initGlobalVariableIfNotExists(s, variable.TiDBResourceControlStrictMode, variable.Off)
}

// initGlobalVariableIfNotExists initialize a global variable with specific val if it does not exist.
func initGlobalVariableIfNotExists(s sessiontypes.Session, name string, val any) {
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
rs, err := s.ExecuteInternal(ctx, "SELECT VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME=%?;",
mysql.SystemDB, mysql.GlobalVariablesTable, name)
terror.MustNil(err)
req := rs.NewChunk(nil)
err = rs.Next(ctx, req)
terror.MustNil(err)
if req.NumRows() != 0 {
return
}

mustExecute(s, "INSERT HIGH_PRIORITY IGNORE INTO %n.%n VALUES (%?, %?);",
mysql.SystemDB, mysql.GlobalVariablesTable, name, val)
}

func writeOOMAction(s sessiontypes.Session) {
comment := "oom-action is `log` by default in v3.0.x, `cancel` by default in v4.0.11+"
mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES (%?, %?, %?) ON DUPLICATE KEY UPDATE VARIABLE_VALUE= %?`,
Expand Down
47 changes: 47 additions & 0 deletions pkg/session/bootstrap_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2249,6 +2249,53 @@ func testTiDBUpgradeWithDistTask(t *testing.T, injectQuery string, fatal bool) {
require.Equal(t, fatal, fatal2panic)
}

func TestTiDBUpgradeToVer199(t *testing.T) {
ctx := context.Background()
store, _ := CreateStoreAndBootstrap(t)
defer func() { require.NoError(t, store.Close()) }()

// bootstrap as version198
ver198 := version198
seV198 := CreateSessionAndSetID(t, store)
txn, err := store.Begin()
require.NoError(t, err)
m := meta.NewMeta(txn)
err = m.FinishBootstrap(int64(ver198))
require.NoError(t, err)
revertVersionAndVariables(t, seV198, ver198)
// simulate a real ver198 where `tidb_resource_control_strict_mode` doesn't exist yet
MustExec(t, seV198, "delete from mysql.GLOBAL_VARIABLES where variable_name='tidb_resource_control_strict_mode'")
err = txn.Commit(context.Background())
require.NoError(t, err)
unsetStoreBootstrapped(store.UUID())

// upgrade to ver199
domCurVer, err := BootstrapSession(store)
require.NoError(t, err)
defer domCurVer.Close()
seCurVer := CreateSessionAndSetID(t, store)
ver, err := getBootstrapVersion(seCurVer)
require.NoError(t, err)
require.Equal(t, currentBootstrapVersion, ver)

// the value in the table is set to OFF automatically
res := MustExecToRecodeSet(t, seCurVer, "select * from mysql.GLOBAL_VARIABLES where variable_name='tidb_resource_control_strict_mode'")
chk := res.NewChunk(nil)
require.NoError(t, res.Next(ctx, chk))
require.Equal(t, 1, chk.NumRows())
row := chk.GetRow(0)
require.Equal(t, "OFF", row.GetString(1))

// the global variable is also OFF
res = MustExecToRecodeSet(t, seCurVer, "select @@global.tidb_resource_control_strict_mode")
chk = res.NewChunk(nil)
require.NoError(t, res.Next(ctx, chk))
require.Equal(t, 1, chk.NumRows())
row = chk.GetRow(0)
require.Equal(t, int64(0), row.GetInt64(0))
require.Equal(t, false, variable.EnableResourceControlStrictMode.Load())
}

func TestTiDBUpgradeWithDistTaskEnable(t *testing.T) {
t.Run("test enable dist task", func(t *testing.T) { testTiDBUpgradeWithDistTask(t, "set global tidb_enable_dist_task = 1", true) })
t.Run("test disable dist task", func(t *testing.T) { testTiDBUpgradeWithDistTask(t, "set global tidb_enable_dist_task = 0", false) })
Expand Down
10 changes: 10 additions & 0 deletions pkg/sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -2783,6 +2783,16 @@ var defaultSysVars = []*SysVar{
}, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) {
return BoolToOnOff(EnableResourceControl.Load()), nil
}},
{Scope: ScopeGlobal, Name: TiDBResourceControlStrictMode, Value: BoolToOnOff(DefTiDBResourceControlStrictMode), Type: TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error {
opOn := TiDBOptOn(s)
if opOn != EnableResourceControlStrictMode.Load() {
EnableResourceControlStrictMode.Store(opOn)
logutil.BgLogger().Info("change resource control strict mode", zap.Bool("enable", TiDBOptOn(s)))
}
return nil
}, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) {
return BoolToOnOff(EnableResourceControlStrictMode.Load()), nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBPessimisticTransactionFairLocking, Value: BoolToOnOff(DefTiDBPessimisticTransactionFairLocking), Type: TypeBool, SetSession: func(s *SessionVars, val string) error {
s.PessimisticTransactionFairLocking = TiDBOptOn(val)
return nil
Expand Down
26 changes: 26 additions & 0 deletions pkg/sessionctx/variable/sysvar_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1326,6 +1326,32 @@ func TestTiDBEnableResourceControl(t *testing.T) {
require.Equal(t, enable, true)
}

func TestTiDBResourceControlStrictMode(t *testing.T) {
vars := NewSessionVars(nil)
mock := NewMockGlobalAccessor4Tests()
mock.SessionVars = vars
vars.GlobalVarsAccessor = mock
resourceControlStrictMode := GetSysVar(TiDBResourceControlStrictMode)

// Default true
require.Equal(t, resourceControlStrictMode.Value, On)
require.Equal(t, EnableResourceControlStrictMode.Load(), true)

// Set to Off
err := mock.SetGlobalSysVar(context.Background(), TiDBResourceControlStrictMode, Off)
require.NoError(t, err)
val, err1 := mock.GetGlobalSysVar(TiDBResourceControlStrictMode)
require.NoError(t, err1)
require.Equal(t, Off, val)

// Set to On again
err = mock.SetGlobalSysVar(context.Background(), TiDBResourceControlStrictMode, On)
require.NoError(t, err)
val, err1 = mock.GetGlobalSysVar(TiDBResourceControlStrictMode)
require.NoError(t, err1)
require.Equal(t, On, val)
}

func TestTiDBEnableRowLevelChecksum(t *testing.T) {
ctx := context.Background()
vars := NewSessionVars(nil)
Expand Down
23 changes: 14 additions & 9 deletions pkg/sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -1071,6 +1071,9 @@ const (
TiDBEnableHistoricalStatsForCapture = "tidb_enable_historical_stats_for_capture"
// TiDBEnableResourceControl indicates whether resource control feature is enabled
TiDBEnableResourceControl = "tidb_enable_resource_control"
// TiDBResourceControlStrictMode indicates whether resource control strict mode is enabled.
// When strict mode is enabled, user need certain privilege to change session or statement resource group.
TiDBResourceControlStrictMode = "tidb_resource_control_strict_mode"
// TiDBStmtSummaryEnablePersistent indicates whether to enable file persistence for stmtsummary.
TiDBStmtSummaryEnablePersistent = "tidb_stmt_summary_enable_persistent"
// TiDBStmtSummaryFilename indicates the file name written by stmtsummary.
Expand Down Expand Up @@ -1448,6 +1451,7 @@ const (
DefTiDBTTLDeleteWorkerCount = 4
DefaultExchangeCompressionMode = kv.ExchangeCompressionModeUnspecified
DefTiDBEnableResourceControl = true
DefTiDBResourceControlStrictMode = true
DefTiDBPessimisticTransactionFairLocking = false
DefTiDBEnablePlanCacheForParamLimit = true
DefTiFlashComputeDispatchPolicy = tiflashcompute.DispatchPolicyConsistentHashStr
Expand Down Expand Up @@ -1585,15 +1589,16 @@ var (
TTLRunningTasks = atomic.NewInt32(DefTiDBTTLRunningTasks)
// always set the default value to false because the resource control in kv-client is not inited
// It will be initialized to the right value after the first call of `rebuildSysVarCache`
EnableResourceControl = atomic.NewBool(false)
EnableCheckConstraint = atomic.NewBool(DefTiDBEnableCheckConstraint)
SkipMissingPartitionStats = atomic.NewBool(DefTiDBSkipMissingPartitionStats)
TiFlashEnablePipelineMode = atomic.NewBool(DefTiDBEnableTiFlashPipelineMode)
ServiceScope = atomic.NewString("")
SchemaVersionCacheLimit = atomic.NewInt64(DefTiDBSchemaVersionCacheLimit)
CloudStorageURI = atomic.NewString("")
IgnoreInlistPlanDigest = atomic.NewBool(DefTiDBIgnoreInlistPlanDigest)
TxnEntrySizeLimit = atomic.NewUint64(DefTiDBTxnEntrySizeLimit)
EnableResourceControl = atomic.NewBool(false)
EnableResourceControlStrictMode = atomic.NewBool(true)
EnableCheckConstraint = atomic.NewBool(DefTiDBEnableCheckConstraint)
SkipMissingPartitionStats = atomic.NewBool(DefTiDBSkipMissingPartitionStats)
TiFlashEnablePipelineMode = atomic.NewBool(DefTiDBEnableTiFlashPipelineMode)
ServiceScope = atomic.NewString("")
SchemaVersionCacheLimit = atomic.NewInt64(DefTiDBSchemaVersionCacheLimit)
CloudStorageURI = atomic.NewString("")
IgnoreInlistPlanDigest = atomic.NewBool(DefTiDBIgnoreInlistPlanDigest)
TxnEntrySizeLimit = atomic.NewUint64(DefTiDBTxnEntrySizeLimit)

SchemaCacheSize = atomic.NewInt64(DefTiDBSchemaCacheSize)
)
Expand Down

0 comments on commit 6fe4f9e

Please sign in to comment.