planner: change SET_VAR behavior for non-verified vars and add some var (#47258)

ref pingcap/tidb#45892
This commit is contained in:
Yiding Cui
2023-09-26 01:42:15 +08:00
committed by GitHub
parent 8129a21c1f
commit 7c1d60bfef
8 changed files with 54 additions and 47 deletions

View File

@ -890,7 +890,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrRoleNotGranted: mysql.Message("%s is not granted to %s", nil),
ErrMaxExecTimeExceeded: mysql.Message("Query execution was interrupted, maximum statement execution time exceeded", nil),
ErrLockAcquireFailAndNoWaitSet: mysql.Message("Statement aborted because lock(s) could not be acquired immediately and NOWAIT is set.", nil),
ErrNotHintUpdatable: mysql.Message("Variable '%s' cannot be set using SET_VAR hint.", nil),
ErrNotHintUpdatable: mysql.Message("Variable '%s' might not be affected by SET_VAR hint.", nil),
ErrExistsInHistoryPassword: mysql.Message("Cannot use these credentials for '%s@%s' because they contradict the password history policy.", nil),
ErrInvalidDefaultUTF8MB4Collation: mysql.Message("Invalid default collation %s: utf8mb4_0900_ai_ci or utf8mb4_general_ci or utf8mb4_bin expected", nil),
ErrForeignKeyCannotDropParent: mysql.Message("Cannot drop table '%s' referenced by a foreign key constraint '%s' on table '%s'.", nil),

View File

@ -2488,7 +2488,7 @@ Argument %s of GROUPING function is not in GROUP BY
["planner:3637"]
error = '''
Variable '%s' cannot be set using SET_VAR hint.
Variable '%s' might not be affected by SET_VAR hint.
'''
["planner:8006"]

View File

@ -760,9 +760,8 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin
warns = append(warns, core.ErrUnresolvedHintName.GenWithStackByArgs(setVarHint.VarName, hint.HintName.String()))
continue
}
if !sysVar.IsHintUpdatable {
if !sysVar.IsHintUpdatableVerfied {
warns = append(warns, core.ErrNotHintUpdatable.GenWithStackByArgs(setVarHint.VarName))
continue
}
// If several hints with the same variable name appear in the same statement, the first one is applied and the others are ignored with a warning
if _, ok := setVars[setVarHint.VarName]; ok {

View File

@ -500,8 +500,9 @@ func TestSetVarHint(t *testing.T) {
require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0)
tk.MustExec("SELECT /*+ SET_VAR(collation_server = 'utf8') */ 1;")
require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1)
require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, "[planner:3637]Variable 'collation_server' cannot be set using SET_VAR hint.")
require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 2)
require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, "[planner:3637]Variable 'collation_server' might not be affected by SET_VAR hint.")
require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[1].Err, "[ddl:1273]Unknown collation: 'utf8'")
tk.MustExec("SELECT /*+ SET_VAR(max_size = 1G) */ 1;")
require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1)

View File

@ -44,7 +44,7 @@ var noopSysVars = []*SysVar{
}},
{Scope: ScopeGlobal, Name: ConnectTimeout, Value: "10", Type: TypeUnsigned, MinValue: 2, MaxValue: secondsPerYear},
{Scope: ScopeGlobal | ScopeSession, Name: QueryCacheWlockInvalidate, Value: Off, Type: TypeBool},
{Scope: ScopeGlobal | ScopeSession, Name: "sql_buffer_result", Value: Off, IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "sql_buffer_result", Value: Off, IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal, Name: MyISAMUseMmap, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true},
{Scope: ScopeGlobal, Name: "gtid_mode", Value: Off, Type: TypeBool},
{Scope: ScopeGlobal, Name: FlushTime, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: secondsPerYear},
@ -86,7 +86,7 @@ var noopSysVars = []*SysVar{
{Scope: ScopeNone, Name: "have_query_cache", Value: "YES"},
{Scope: ScopeGlobal, Name: "innodb_flush_log_at_timeout", Value: "1"},
{Scope: ScopeGlobal, Name: "innodb_max_undo_log_size", Value: ""},
{Scope: ScopeGlobal | ScopeSession, Name: "range_alloc_block_size", Value: "4096", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "range_alloc_block_size", Value: "4096", IsHintUpdatableVerfied: true},
{Scope: ScopeNone, Name: "have_rtree_keys", Value: "YES"},
{Scope: ScopeGlobal, Name: "innodb_old_blocks_pct", Value: "37"},
{Scope: ScopeGlobal, Name: "innodb_file_format", Value: "Barracuda", Type: TypeEnum, PossibleValues: []string{"Antelope", "Barracuda"}},
@ -138,7 +138,7 @@ var noopSysVars = []*SysVar{
{Scope: ScopeGlobal, Name: "log_warnings", Value: "1"},
{Scope: ScopeGlobal | ScopeSession, Name: InnodbStrictMode, Value: On, Type: TypeBool, AutoConvertNegativeBool: true},
{Scope: ScopeGlobal, Name: "innodb_rollback_segments", Value: "128"},
{Scope: ScopeGlobal | ScopeSession, Name: "join_buffer_size", Value: "262144", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "join_buffer_size", Value: "262144", IsHintUpdatableVerfied: true},
{Scope: ScopeNone, Name: "innodb_mirrored_log_groups", Value: "1"},
{Scope: ScopeGlobal, Name: "max_binlog_size", Value: "1073741824"},
{Scope: ScopeGlobal, Name: "concurrent_insert", Value: "AUTO"},
@ -150,7 +150,7 @@ var noopSysVars = []*SysVar{
{Scope: ScopeNone, Name: "innodb_file_format_check", Value: "1"},
{Scope: ScopeNone, Name: "myisam_mmap_size", Value: "18446744073709551615"},
{Scope: ScopeNone, Name: "innodb_buffer_pool_instances", Value: "8"},
{Scope: ScopeGlobal | ScopeSession, Name: "max_length_for_sort_data", Value: "1024", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "max_length_for_sort_data", Value: "1024", IsHintUpdatableVerfied: true},
{Scope: ScopeNone, Name: CharacterSetSystem, Value: "utf8"},
{Scope: ScopeGlobal | ScopeSession, Name: CharacterSetFilesystem, Value: "binary", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
return checkCharacterSet(normalizedValue, CharacterSetFilesystem)
@ -175,7 +175,7 @@ var noopSysVars = []*SysVar{
{Scope: ScopeNone, Name: "innodb_undo_tablespaces", Value: "0"},
{Scope: ScopeGlobal, Name: InnodbStatusOutputLocks, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true},
{Scope: ScopeNone, Name: "performance_schema_accounts_size", Value: "100"},
{Scope: ScopeGlobal | ScopeSession, Name: "max_error_count", Value: "64", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "max_error_count", Value: "64", IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal, Name: "max_write_lock_count", Value: "18446744073709551615"},
{Scope: ScopeNone, Name: "performance_schema_max_socket_instances", Value: "322"},
{Scope: ScopeNone, Name: "performance_schema_max_table_instances", Value: "12500"},
@ -189,12 +189,12 @@ var noopSysVars = []*SysVar{
{Scope: ScopeNone, Name: "ft_stopword_file", Value: "(built-in)"},
{Scope: ScopeGlobal, Name: "innodb_max_dirty_pages_pct_lwm", Value: "0"},
{Scope: ScopeGlobal, Name: LogQueriesNotUsingIndexes, Value: Off, Type: TypeBool},
{Scope: ScopeGlobal | ScopeSession, Name: "max_heap_table_size", Value: "16777216", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "tmp_table_size", Value: "16777216", Type: TypeUnsigned, MinValue: 1024, MaxValue: math.MaxUint64, IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "div_precision_increment", Value: "4", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "max_heap_table_size", Value: "16777216", IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal | ScopeSession, Name: "tmp_table_size", Value: "16777216", Type: TypeUnsigned, MinValue: 1024, MaxValue: math.MaxUint64, IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal | ScopeSession, Name: "div_precision_increment", Value: "4", IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal, Name: "innodb_lru_scan_depth", Value: "1024"},
{Scope: ScopeGlobal, Name: "innodb_purge_rseg_truncate_frequency", Value: ""},
{Scope: ScopeGlobal | ScopeSession, Name: SQLAutoIsNull, Value: Off, Type: TypeBool, IsHintUpdatable: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
{Scope: ScopeGlobal | ScopeSession, Name: SQLAutoIsNull, Value: Off, Type: TypeBool, IsHintUpdatableVerfied: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
// checkSQLAutoIsNull requires TiDBEnableNoopFuncs != OFF for the same scope otherwise an error will be returned.
// See also https://github.com/pingcap/tidb/issues/28230
errMsg := ErrFunctionsNoopImpl.GenWithStackByArgs("sql_auto_is_null")
@ -225,7 +225,7 @@ var noopSysVars = []*SysVar{
{Scope: ScopeNone, Name: "server_id_bits", Value: "32"},
{Scope: ScopeGlobal, Name: "innodb_log_checksum_algorithm", Value: ""},
{Scope: ScopeNone, Name: "innodb_buffer_pool_load_at_startup", Value: "1"},
{Scope: ScopeGlobal | ScopeSession, Name: "sort_buffer_size", Value: "262144", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "sort_buffer_size", Value: "262144", IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal, Name: "innodb_flush_neighbors", Value: "1"},
{Scope: ScopeNone, Name: "innodb_use_sys_malloc", Value: "1"},
{Scope: ScopeNone, Name: "performance_schema_max_socket_classes", Value: "10"},
@ -238,13 +238,13 @@ var noopSysVars = []*SysVar{
{Scope: ScopeGlobal, Name: "myisam_data_pointer_size", Value: "6"},
{Scope: ScopeGlobal, Name: "ndb_optimization_delay", Value: ""},
{Scope: ScopeGlobal, Name: "innodb_ft_num_word_optimize", Value: "2000"},
{Scope: ScopeGlobal | ScopeSession, Name: "max_join_size", Value: "18446744073709551615", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "max_join_size", Value: "18446744073709551615", IsHintUpdatableVerfied: true},
{Scope: ScopeNone, Name: CoreFile, Value: Off, Type: TypeBool},
{Scope: ScopeGlobal | ScopeSession, Name: "max_seeks_for_key", Value: "18446744073709551615", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "max_seeks_for_key", Value: "18446744073709551615", IsHintUpdatableVerfied: true},
{Scope: ScopeNone, Name: "innodb_log_buffer_size", Value: "8388608"},
{Scope: ScopeGlobal, Name: "delayed_insert_timeout", Value: "300"},
{Scope: ScopeGlobal, Name: "max_relay_log_size", Value: "0"},
{Scope: ScopeGlobal | ScopeSession, Name: MaxSortLength, Value: "1024", Type: TypeUnsigned, MinValue: 4, MaxValue: 8388608, IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: MaxSortLength, Value: "1024", Type: TypeUnsigned, MinValue: 4, MaxValue: 8388608, IsHintUpdatableVerfied: true},
{Scope: ScopeNone, Name: "metadata_locks_hash_instances", Value: "8"},
{Scope: ScopeGlobal, Name: "ndb_eventbuffer_free_percent", Value: ""},
{Scope: ScopeNone, Name: "large_files_support", Value: "1"},
@ -293,17 +293,17 @@ var noopSysVars = []*SysVar{
{Scope: ScopeGlobal, Name: InnodbStatsAutoRecalc, Value: "1"},
// lc_messages cannot be read_only, see https://github.com/pingcap/tidb/issues/38231.
{Scope: ScopeGlobal | ScopeSession, Name: "lc_messages", Value: "en_US"},
{Scope: ScopeGlobal | ScopeSession, Name: "bulk_insert_buffer_size", Value: "8388608", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "bulk_insert_buffer_size", Value: "8388608", IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal | ScopeSession, Name: BinlogDirectNonTransactionalUpdates, Value: Off, Type: TypeBool},
{Scope: ScopeGlobal, Name: "innodb_change_buffering", Value: "all"},
{Scope: ScopeGlobal | ScopeSession, Name: SQLBigSelects, Value: On, Type: TypeBool, IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: SQLBigSelects, Value: On, Type: TypeBool, IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal, Name: "innodb_max_purge_lag_delay", Value: "0"},
{Scope: ScopeGlobal | ScopeSession, Name: "session_track_schema", Value: ""},
{Scope: ScopeGlobal, Name: "innodb_io_capacity_max", Value: "2000"},
{Scope: ScopeGlobal, Name: "innodb_autoextend_increment", Value: "64"},
{Scope: ScopeGlobal | ScopeSession, Name: "binlog_format", Value: "STATEMENT"},
{Scope: ScopeGlobal | ScopeSession, Name: "optimizer_trace", Value: "enabled=off,one_line=off"},
{Scope: ScopeGlobal | ScopeSession, Name: "read_rnd_buffer_size", Value: "262144", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "read_rnd_buffer_size", Value: "262144", IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal | ScopeSession, Name: NetWriteTimeout, Value: "60"},
{Scope: ScopeGlobal, Name: InnodbBufferPoolLoadAbort, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true},
{Scope: ScopeGlobal | ScopeSession, Name: "transaction_prealloc_size", Value: "4096"},
@ -326,7 +326,7 @@ var noopSysVars = []*SysVar{
{Scope: ScopeNone, Name: "table_open_cache_instances", Value: "1"},
{Scope: ScopeGlobal, Name: InnodbStatsPersistent, Value: On, Type: TypeBool, AutoConvertNegativeBool: true},
{Scope: ScopeGlobal | ScopeSession, Name: "session_track_state_change", Value: ""},
{Scope: ScopeNone, Name: OptimizerSwitch, Value: "index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,engine_condition_pushdown=on,index_condition_pushdown=on,mrr=on,mrr_cost_based=on,block_nested_loop=on,batched_key_access=off,materialization=on,semijoin=on,loosescan=on,firstmatch=on,subquery_materialization_cost_based=on,use_index_extensions=on", IsHintUpdatable: true},
{Scope: ScopeNone, Name: OptimizerSwitch, Value: "index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,engine_condition_pushdown=on,index_condition_pushdown=on,mrr=on,mrr_cost_based=on,block_nested_loop=on,batched_key_access=off,materialization=on,semijoin=on,loosescan=on,firstmatch=on,subquery_materialization_cost_based=on,use_index_extensions=on", IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal, Name: "delayed_queue_size", Value: "1000"},
{Scope: ScopeNone, Name: "innodb_read_only", Value: "0"},
{Scope: ScopeNone, Name: "datetime_format", Value: "%Y-%m-%d %H:%i:%s"},
@ -358,7 +358,7 @@ var noopSysVars = []*SysVar{
}},
{Scope: ScopeNone, Name: "max_tmp_tables", Value: "32"},
{Scope: ScopeGlobal, Name: InnodbRandomReadAhead, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true},
{Scope: ScopeGlobal | ScopeSession, Name: UniqueChecks, Value: On, Type: TypeBool, IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: UniqueChecks, Value: On, Type: TypeBool, IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal, Name: "internal_tmp_disk_storage_engine", Value: ""},
{Scope: ScopeGlobal | ScopeSession, Name: "myisam_repair_threads", Value: "1"},
{Scope: ScopeGlobal, Name: "ndb_eventbuffer_max_alloc", Value: ""},
@ -368,7 +368,7 @@ var noopSysVars = []*SysVar{
{Scope: ScopeGlobal, Name: "gtid_purged", Value: ""},
{Scope: ScopeGlobal, Name: "max_binlog_stmt_cache_size", Value: "18446744073709547520"},
{Scope: ScopeGlobal | ScopeSession, Name: "lock_wait_timeout", Value: "31536000"},
{Scope: ScopeGlobal | ScopeSession, Name: "read_buffer_size", Value: "131072", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "read_buffer_size", Value: "131072", IsHintUpdatableVerfied: true},
{Scope: ScopeNone, Name: "innodb_read_io_threads", Value: "4"},
{Scope: ScopeGlobal | ScopeSession, Name: MaxSpRecursionDepth, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 255},
{Scope: ScopeNone, Name: "ignore_builtin_innodb", Value: "0"},
@ -388,9 +388,9 @@ var noopSysVars = []*SysVar{
{Scope: ScopeGlobal, Name: "table_open_cache", Value: "2000"},
{Scope: ScopeNone, Name: "performance_schema_events_stages_history_long_size", Value: "10000"},
{Scope: ScopeSession, Name: "insert_id", Value: ""},
{Scope: ScopeGlobal | ScopeSession, Name: "default_tmp_storage_engine", Value: "InnoDB", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "optimizer_search_depth", Value: "62", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "max_points_in_geometry", Value: "65536", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "default_tmp_storage_engine", Value: "InnoDB", IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal | ScopeSession, Name: "optimizer_search_depth", Value: "62", IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal | ScopeSession, Name: "max_points_in_geometry", Value: "65536", IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal, Name: "innodb_stats_sample_pages", Value: "8"},
{Scope: ScopeGlobal | ScopeSession, Name: "profiling_history_size", Value: "15"},
{Scope: ScopeNone, Name: "have_symlink", Value: "YES"},
@ -414,8 +414,8 @@ var noopSysVars = []*SysVar{
{Scope: ScopeGlobal, Name: "innodb_flush_log_at_trx_commit", Value: "1"},
{Scope: ScopeGlobal, Name: "rewriter_enabled", Value: ""},
{Scope: ScopeGlobal, Name: "query_cache_min_res_unit", Value: "4096"},
{Scope: ScopeGlobal | ScopeSession, Name: "updatable_views_with_limit", Value: "YES", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "optimizer_prune_level", Value: "1", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "updatable_views_with_limit", Value: "YES", IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal | ScopeSession, Name: "optimizer_prune_level", Value: "1", IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal | ScopeSession, Name: "completion_type", Value: "NO_CHAIN"},
{Scope: ScopeGlobal, Name: "binlog_checksum", Value: "CRC32"},
{Scope: ScopeNone, Name: "report_port", Value: "3306"},
@ -456,7 +456,7 @@ var noopSysVars = []*SysVar{
{Scope: ScopeNone, Name: "performance_schema_max_cond_instances", Value: "3504"},
{Scope: ScopeGlobal, Name: "delayed_insert_limit", Value: "100"},
{Scope: ScopeGlobal, Name: Flush, Value: Off, Type: TypeBool},
{Scope: ScopeGlobal | ScopeSession, Name: "eq_range_index_dive_limit", Value: "200", IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: "eq_range_index_dive_limit", Value: "200", IsHintUpdatableVerfied: true},
{Scope: ScopeNone, Name: "performance_schema_events_stages_history_size", Value: "10"},
{Scope: ScopeGlobal | ScopeSession, Name: "ndb_join_pushdown", Value: ""},
{Scope: ScopeNone, Name: "performance_schema_max_thread_instances", Value: "402"},
@ -472,13 +472,13 @@ var noopSysVars = []*SysVar{
{Scope: ScopeNone, Name: "innodb_undo_directory", Value: "."},
{Scope: ScopeNone, Name: "bind_address", Value: "*"},
{Scope: ScopeGlobal, Name: "innodb_sync_spin_loops", Value: "30"},
{Scope: ScopeGlobal | ScopeSession, Name: SQLSafeUpdates, Value: Off, Type: TypeBool, IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: SQLSafeUpdates, Value: Off, Type: TypeBool, IsHintUpdatableVerfied: true},
{Scope: ScopeNone, Name: "tmpdir", Value: "/var/tmp/"},
{Scope: ScopeGlobal, Name: "innodb_thread_concurrency", Value: "0"},
{Scope: ScopeGlobal, Name: "innodb_buffer_pool_dump_pct", Value: ""},
{Scope: ScopeGlobal | ScopeSession, Name: "lc_time_names", Value: "en_US", ReadOnly: true},
{Scope: ScopeGlobal | ScopeSession, Name: "max_statement_time", Value: ""},
{Scope: ScopeGlobal | ScopeSession, Name: EndMarkersInJSON, Value: Off, Type: TypeBool, IsHintUpdatable: true},
{Scope: ScopeGlobal | ScopeSession, Name: EndMarkersInJSON, Value: Off, Type: TypeBool, IsHintUpdatableVerfied: true},
{Scope: ScopeGlobal, Name: AvoidTemporalUpgrade, Value: Off, Type: TypeBool},
{Scope: ScopeGlobal, Name: "key_cache_age_threshold", Value: "300"},
{Scope: ScopeGlobal, Name: InnodbStatusOutput, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true},

View File

@ -14,7 +14,7 @@
package variable
var isHintUpdatable = map[string]struct{}{
var isHintUpdatableVerified = map[string]struct{}{
"tidb_opt_agg_push_down": {},
"tidb_opt_derive_topn": {},
"tidb_opt_broadcast_cartesian_join": {},
@ -96,6 +96,13 @@ var isHintUpdatable = map[string]struct{}{
"tidb_opt_fix_control": {},
"tidb_runtime_filter_type": {},
"tidb_runtime_filter_mode": {},
"tidb_session_alias": {},
"tidb_opt_objective": {},
"mpp_exchange_compression_mode": {},
"tidb_allow_fallback_to_tikv": {},
"tiflash_fastscan": {},
"tiflash_fine_grained_shuffle_batch_size": {},
"tiflash_find_grained_shuffle_stream_count": {},
// Variables that is compatible with MySQL.
"cte_max_recursion_depth": {},
"sql_mode": {},
@ -104,8 +111,8 @@ var isHintUpdatable = map[string]struct{}{
func setHintUpdatable(vars []*SysVar) {
for _, v := range vars {
if _, ok := isHintUpdatable[v.Name]; ok {
v.IsHintUpdatable = true
if _, ok := isHintUpdatableVerified[v.Name]; ok {
v.IsHintUpdatableVerfied = true
}
}
}

View File

@ -1408,7 +1408,7 @@ var defaultSysVars = []*SysVar{
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: DefaultWeekFormat, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 7},
{Scope: ScopeGlobal | ScopeSession, Name: SQLModeVar, Value: mysql.DefaultSQLMode, IsHintUpdatable: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
{Scope: ScopeGlobal | ScopeSession, Name: SQLModeVar, Value: mysql.DefaultSQLMode, IsHintUpdatableVerfied: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
// Ensure the SQL mode parses
normalizedValue = mysql.FormatSQLModeStr(normalizedValue)
if _, err := mysql.GetSQLMode(normalizedValue); err != nil {
@ -1427,12 +1427,12 @@ var defaultSysVars = []*SysVar{
s.SetStatusFlag(mysql.ServerStatusNoBackslashEscaped, sqlMode.HasNoBackslashEscapesMode())
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: MaxExecutionTime, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, IsHintUpdatable: true, SetSession: func(s *SessionVars, val string) error {
{Scope: ScopeGlobal | ScopeSession, Name: MaxExecutionTime, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, IsHintUpdatableVerfied: true, SetSession: func(s *SessionVars, val string) error {
timeoutMS := tidbOptPositiveInt32(val, 0)
s.MaxExecutionTime = uint64(timeoutMS)
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiKVClientReadTimeout, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, IsHintUpdatable: true, SetSession: func(s *SessionVars, val string) error {
{Scope: ScopeGlobal | ScopeSession, Name: TiKVClientReadTimeout, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, IsHintUpdatableVerfied: true, SetSession: func(s *SessionVars, val string) error {
timeoutMS := tidbOptPositiveInt32(val, 0)
s.TiKVClientReadTimeout = uint64(timeoutMS)
return nil
@ -1456,7 +1456,7 @@ var defaultSysVars = []*SysVar{
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: SQLLogBin, Value: On, Type: TypeBool},
{Scope: ScopeGlobal | ScopeSession, Name: TimeZone, Value: "SYSTEM", IsHintUpdatable: true, Validation: func(varErrFunctionsNoopImpls *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
{Scope: ScopeGlobal | ScopeSession, Name: TimeZone, Value: "SYSTEM", IsHintUpdatableVerfied: true, Validation: func(varErrFunctionsNoopImpls *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
if strings.EqualFold(normalizedValue, "SYSTEM") {
return "SYSTEM", nil
}
@ -1554,7 +1554,7 @@ var defaultSysVars = []*SysVar{
s.LockWaitTimeout = lockWaitSec * 1000
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: GroupConcatMaxLen, Value: "1024", IsHintUpdatable: true, Type: TypeUnsigned, MinValue: 4, MaxValue: math.MaxUint64, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
{Scope: ScopeGlobal | ScopeSession, Name: GroupConcatMaxLen, Value: "1024", IsHintUpdatableVerfied: true, Type: TypeUnsigned, MinValue: 4, MaxValue: math.MaxUint64, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) {
// https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_group_concat_max_len
// Minimum Value 4
// Maximum Value (64-bit platforms) 18446744073709551615
@ -1615,7 +1615,7 @@ var defaultSysVars = []*SysVar{
return nil
},
},
{Scope: ScopeGlobal | ScopeSession, Name: WindowingUseHighPrecision, Value: On, Type: TypeBool, IsHintUpdatable: true, SetSession: func(s *SessionVars, val string) error {
{Scope: ScopeGlobal | ScopeSession, Name: WindowingUseHighPrecision, Value: On, Type: TypeBool, IsHintUpdatableVerfied: true, SetSession: func(s *SessionVars, val string) error {
s.WindowingUseHighPrecision = TiDBOptOn(val)
return nil
}},
@ -1674,7 +1674,7 @@ var defaultSysVars = []*SysVar{
s.SetAllowInSubqToJoinAndAgg(TiDBOptOn(val))
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptPreferRangeScan, Value: BoolToOnOff(DefOptPreferRangeScan), Type: TypeBool, IsHintUpdatable: true, SetSession: func(s *SessionVars, val string) error {
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptPreferRangeScan, Value: BoolToOnOff(DefOptPreferRangeScan), Type: TypeBool, IsHintUpdatableVerfied: true, SetSession: func(s *SessionVars, val string) error {
s.SetAllowPreferRangeScan(TiDBOptOn(val))
return nil
}},
@ -2596,7 +2596,7 @@ var defaultSysVars = []*SysVar{
s.EnableMPPSharedCTEExecution = TiDBOptOn(val)
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptFixControl, Value: "", Type: TypeStr, IsHintUpdatable: true,
{Scope: ScopeGlobal | ScopeSession, Name: TiDBOptFixControl, Value: "", Type: TypeStr, IsHintUpdatableVerfied: true,
SetGlobal: func(ctx context.Context, vars *SessionVars, val string) error {
// validation logic for setting global
// we don't put this in Validation to avoid repeating the checking logic for setting session.

View File

@ -140,8 +140,8 @@ type SysVar struct {
SetSession func(*SessionVars, string) error
// SetGlobal is called after validation
SetGlobal func(context.Context, *SessionVars, string) error
// IsHintUpdatable indicate whether it's updatable via SET_VAR() hint (optional)
IsHintUpdatable bool
// IsHintUpdatableVerfied indicate whether we've confirmed that SET_VAR() hint is worked for this hint.
IsHintUpdatableVerfied bool
// Deprecated: Hidden previously meant that the variable still responds to SET but doesn't show up in SHOW VARIABLES
// However, this feature is no longer used. All variables are visble.
Hidden bool
@ -639,7 +639,7 @@ func init() {
sysVars = make(map[string]*SysVar)
setHintUpdatable(defaultSysVars)
// Destroy the map after init.
maps.Clear(isHintUpdatable)
maps.Clear(isHintUpdatableVerified)
for _, v := range defaultSysVars {
RegisterSysVar(v)
}