|
|
|
|
@ -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},
|
|
|
|
|
|