From 68c03cfb656436ab4082134872b8cf9afb86cd78 Mon Sep 17 00:00:00 2001 From: ekexium Date: Fri, 15 Mar 2024 16:59:11 +0800 Subject: [PATCH] txn: enable pipelined dml by hint (#51770) ref pingcap/tidb#50215 --- pkg/planner/optimize.go | 2 +- pkg/session/session.go | 3 + pkg/sessionctx/variable/noop.go | 239 ++++++++++++++---- pkg/sessionctx/variable/setvar_affect.go | 2 +- pkg/sessionctx/variable/sysvar.go | 222 ++++++++++------ pkg/sessionctx/variable/variable.go | 4 +- .../pipelineddmltest/pipelineddml_test.go | 15 +- 7 files changed, 360 insertions(+), 127 deletions(-) diff --git a/pkg/planner/optimize.go b/pkg/planner/optimize.go index 62a581a50bb5b..915fc537fcbc9 100644 --- a/pkg/planner/optimize.go +++ b/pkg/planner/optimize.go @@ -578,7 +578,7 @@ func setVarHintChecker(varName, hint string) (ok bool, warning error) { if sysVar == nil { // no such a variable return false, plannererrors.ErrUnresolvedHintName.FastGenByArgs(varName, hint) } - if !sysVar.IsHintUpdatableVerfied { + if !sysVar.IsHintUpdatableVerified { warning = plannererrors.ErrNotHintUpdatable.FastGenByArgs(varName) } return true, warning diff --git a/pkg/session/session.go b/pkg/session/session.go index 91c52b74bace2..a2d86b0353c23 100644 --- a/pkg/session/session.go +++ b/pkg/session/session.go @@ -4305,6 +4305,9 @@ func (s *session) usePipelinedDmlOrWarn() bool { if stmtCtx == nil { return false } + if stmtCtx.IsReadOnly { + return false + } vars := s.GetSessionVars() if !vars.TxnCtx.EnableMDL { stmtCtx.AppendWarning( diff --git a/pkg/sessionctx/variable/noop.go b/pkg/sessionctx/variable/noop.go index 2e89da57e5532..bcfa6d63a0f0a 100644 --- a/pkg/sessionctx/variable/noop.go +++ b/pkg/sessionctx/variable/noop.go @@ -46,7 +46,12 @@ 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, IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "sql_buffer_result", + Value: Off, + IsHintUpdatableVerified: 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}, @@ -88,7 +93,12 @@ 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", IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "range_alloc_block_size", + Value: "4096", + IsHintUpdatableVerified: 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"}}, @@ -140,7 +150,12 @@ 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", IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "join_buffer_size", + Value: "262144", + IsHintUpdatableVerified: 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"}, @@ -152,7 +167,12 @@ 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", IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "max_length_for_sort_data", + Value: "1024", + IsHintUpdatableVerified: 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) @@ -176,7 +196,12 @@ 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", IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "max_error_count", + Value: "64", + IsHintUpdatableVerified: 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"}, @@ -190,42 +215,68 @@ 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", IsHintUpdatableVerfied: true}, - {Scope: ScopeGlobal | ScopeSession, Name: "tmp_table_size", Value: "16777216", Type: TypeUnsigned, MinValue: 1024, MaxValue: math.MaxUint64, IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "max_heap_table_size", + Value: "16777216", + IsHintUpdatableVerified: true, + }, + { + Scope: ScopeGlobal | ScopeSession, + Name: "tmp_table_size", + Value: "16777216", + Type: TypeUnsigned, + MinValue: 1024, + MaxValue: math.MaxUint64, + IsHintUpdatableVerified: 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, 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.FastGenByArgs("sql_auto_is_null") - if TiDBOptOn(normalizedValue) { - if scope == ScopeSession && vars.NoopFuncsMode != OnInt { - if vars.NoopFuncsMode == OffInt { - return Off, errors.Trace(errMsg) - } - vars.StmtCtx.AppendWarning(errMsg) - } - if scope == ScopeGlobal { - val, err := vars.GlobalVarsAccessor.GetGlobalSysVar(TiDBEnableNoopFuncs) - if err != nil { - return originalValue, errUnknownSystemVariable.GenWithStackByArgs(TiDBEnableNoopFuncs) - } - if val == Off { - return Off, errors.Trace(errMsg) - } - if val == Warn { + { + Scope: ScopeGlobal | ScopeSession, + Name: SQLAutoIsNull, + Value: Off, + Type: TypeBool, + IsHintUpdatableVerified: 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.FastGenByArgs("sql_auto_is_null") + if TiDBOptOn(normalizedValue) { + if scope == ScopeSession && vars.NoopFuncsMode != OnInt { + if vars.NoopFuncsMode == OffInt { + return Off, errors.Trace(errMsg) + } vars.StmtCtx.AppendWarning(errMsg) } + if scope == ScopeGlobal { + val, err := vars.GlobalVarsAccessor.GetGlobalSysVar(TiDBEnableNoopFuncs) + if err != nil { + return originalValue, errUnknownSystemVariable.GenWithStackByArgs(TiDBEnableNoopFuncs) + } + if val == Off { + return Off, errors.Trace(errMsg) + } + if val == Warn { + vars.StmtCtx.AppendWarning(errMsg) + } + } } - } - return normalizedValue, nil - }}, + return normalizedValue, nil + }}, {Scope: ScopeNone, Name: "innodb_api_enable_binlog", Value: "0"}, {Scope: ScopeGlobal | ScopeSession, Name: "innodb_ft_user_stopword_table", Value: ""}, {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", IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "sort_buffer_size", + Value: "262144", + IsHintUpdatableVerified: 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 +289,31 @@ 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", IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "max_join_size", + Value: "18446744073709551615", + IsHintUpdatableVerified: true, + }, {Scope: ScopeNone, Name: CoreFile, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: "max_seeks_for_key", Value: "18446744073709551615", IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "max_seeks_for_key", + Value: "18446744073709551615", + IsHintUpdatableVerified: 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, IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: MaxSortLength, + Value: "1024", + Type: TypeUnsigned, + MinValue: 4, + MaxValue: 8388608, + IsHintUpdatableVerified: 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 +362,33 @@ 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", IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "bulk_insert_buffer_size", + Value: "8388608", + IsHintUpdatableVerified: 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, IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: SQLBigSelects, + Value: On, + Type: TypeBool, + IsHintUpdatableVerified: 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", IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "read_rnd_buffer_size", + Value: "262144", + IsHintUpdatableVerified: 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 +411,12 @@ 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", IsHintUpdatableVerfied: 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", + IsHintUpdatableVerified: 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 +448,13 @@ 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, IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: UniqueChecks, + Value: On, + Type: TypeBool, + IsHintUpdatableVerified: 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 +464,12 @@ 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", IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "read_buffer_size", + Value: "131072", + IsHintUpdatableVerified: 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 +489,24 @@ 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", 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 | ScopeSession, + Name: "default_tmp_storage_engine", + Value: "InnoDB", + IsHintUpdatableVerified: true, + }, + { + Scope: ScopeGlobal | ScopeSession, + Name: "optimizer_search_depth", + Value: "62", + IsHintUpdatableVerified: true, + }, + { + Scope: ScopeGlobal | ScopeSession, + Name: "max_points_in_geometry", + Value: "65536", + IsHintUpdatableVerified: 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 +530,18 @@ 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", IsHintUpdatableVerfied: true}, - {Scope: ScopeGlobal | ScopeSession, Name: "optimizer_prune_level", Value: "1", IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "updatable_views_with_limit", + Value: "YES", + IsHintUpdatableVerified: true, + }, + { + Scope: ScopeGlobal | ScopeSession, + Name: "optimizer_prune_level", + Value: "1", + IsHintUpdatableVerified: 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 +582,12 @@ 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", IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: "eq_range_index_dive_limit", + Value: "200", + IsHintUpdatableVerified: 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 +603,25 @@ 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, IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: SQLSafeUpdates, + Value: Off, + Type: TypeBool, + IsHintUpdatableVerified: 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, IsHintUpdatableVerfied: true}, + { + Scope: ScopeGlobal | ScopeSession, + Name: EndMarkersInJSON, + Value: Off, + Type: TypeBool, + IsHintUpdatableVerified: 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}, diff --git a/pkg/sessionctx/variable/setvar_affect.go b/pkg/sessionctx/variable/setvar_affect.go index 866dd611fcf33..6437e75249dd0 100644 --- a/pkg/sessionctx/variable/setvar_affect.go +++ b/pkg/sessionctx/variable/setvar_affect.go @@ -117,7 +117,7 @@ var isHintUpdatableVerified = map[string]struct{}{ func setHintUpdatable(vars []*SysVar) { for _, v := range vars { if _, ok := isHintUpdatableVerified[v.Name]; ok { - v.IsHintUpdatableVerfied = true + v.IsHintUpdatableVerified = true } } } diff --git a/pkg/sessionctx/variable/sysvar.go b/pkg/sessionctx/variable/sysvar.go index ecf55bf83516a..acadfe39167d3 100644 --- a/pkg/sessionctx/variable/sysvar.go +++ b/pkg/sessionctx/variable/sysvar.go @@ -1243,10 +1243,16 @@ var defaultSysVars = []*SysVar{ s.EnableNonPreparedPlanCacheForDML = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptEnableFuzzyBinding, Value: BoolToOnOff(false), Type: TypeBool, IsHintUpdatableVerfied: true, SetSession: func(s *SessionVars, val string) error { - s.EnableFuzzyBinding = TiDBOptOn(val) - return nil - }}, + { + Scope: ScopeGlobal | ScopeSession, + Name: TiDBOptEnableFuzzyBinding, + Value: BoolToOnOff(false), + Type: TypeBool, + IsHintUpdatableVerified: true, + SetSession: func(s *SessionVars, val string) error { + s.EnableFuzzyBinding = TiDBOptOn(val) + return nil + }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBNonPreparedPlanCacheSize, Value: strconv.FormatUint(uint64(DefTiDBNonPreparedPlanCacheSize), 10), Type: TypeUnsigned, MinValue: 1, MaxValue: 100000, SetSession: func(s *SessionVars, val string) error { uVal, err := strconv.ParseUint(val, 10, 64) if err == nil { @@ -1462,39 +1468,70 @@ 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, 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 { - return originalValue, err - } - return normalizedValue, nil - }, SetSession: func(s *SessionVars, val string) error { - val = mysql.FormatSQLModeStr(val) - // Modes is a list of different modes separated by commas. - sqlMode, err := mysql.GetSQLMode(val) - if err != nil { - return errors.Trace(err) - } - s.SQLMode = sqlMode - s.SetStatusFlag(mysql.ServerStatusNoBackslashEscaped, sqlMode.HasNoBackslashEscapesMode()) - return nil - }}, - {Scope: ScopeGlobal, Name: TiDBLoadBindingTimeout, Value: "200", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, IsHintUpdatableVerfied: false, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { - timeoutMS := tidbOptPositiveInt32(s, 0) - vars.LoadBindingTimeout = uint64(timeoutMS) - return nil - }}, - {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, IsHintUpdatableVerfied: true, SetSession: func(s *SessionVars, val string) error { - timeoutMS := tidbOptPositiveInt32(val, 0) - s.TiKVClientReadTimeout = uint64(timeoutMS) - return nil - }}, + { + Scope: ScopeGlobal | ScopeSession, + Name: SQLModeVar, + Value: mysql.DefaultSQLMode, + IsHintUpdatableVerified: 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 { + return originalValue, err + } + return normalizedValue, nil + }, SetSession: func(s *SessionVars, val string) error { + val = mysql.FormatSQLModeStr(val) + // Modes is a list of different modes separated by commas. + sqlMode, err := mysql.GetSQLMode(val) + if err != nil { + return errors.Trace(err) + } + s.SQLMode = sqlMode + s.SetStatusFlag(mysql.ServerStatusNoBackslashEscaped, sqlMode.HasNoBackslashEscapesMode()) + return nil + }}, + { + Scope: ScopeGlobal, + Name: TiDBLoadBindingTimeout, + Value: "200", + Type: TypeUnsigned, + MinValue: 0, + MaxValue: math.MaxInt32, + IsHintUpdatableVerified: false, + SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + timeoutMS := tidbOptPositiveInt32(s, 0) + vars.LoadBindingTimeout = uint64(timeoutMS) + return nil + }}, + { + Scope: ScopeGlobal | ScopeSession, + Name: MaxExecutionTime, + Value: "0", + Type: TypeUnsigned, + MinValue: 0, + MaxValue: math.MaxInt32, + IsHintUpdatableVerified: 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, + IsHintUpdatableVerified: true, + SetSession: func(s *SessionVars, val string) error { + timeoutMS := tidbOptPositiveInt32(val, 0) + s.TiKVClientReadTimeout = uint64(timeoutMS) + return nil + }}, {Scope: ScopeGlobal | ScopeSession, Name: CollationServer, Value: mysql.DefaultCollationName, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCollation(vars, normalizedValue, originalValue, scope) }, SetSession: func(s *SessionVars, val string) error { @@ -1514,20 +1551,28 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: SQLLogBin, Value: On, Type: TypeBool}, - {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 - } - _, err := timeutil.ParseTimeZone(normalizedValue) - return normalizedValue, err - }, SetSession: func(s *SessionVars, val string) error { - tz, err := timeutil.ParseTimeZone(val) - if err != nil { - return err - } - s.TimeZone = tz - return nil - }}, + { + Scope: ScopeGlobal | ScopeSession, + Name: TimeZone, + Value: "SYSTEM", + IsHintUpdatableVerified: true, + Validation: func( + varErrFunctionsNoopImpls *SessionVars, normalizedValue string, originalValue string, + scope ScopeFlag, + ) (string, error) { + if strings.EqualFold(normalizedValue, "SYSTEM") { + return "SYSTEM", nil + } + _, err := timeutil.ParseTimeZone(normalizedValue) + return normalizedValue, err + }, SetSession: func(s *SessionVars, val string) error { + tz, err := timeutil.ParseTimeZone(val) + if err != nil { + return err + } + s.TimeZone = tz + return nil + }}, {Scope: ScopeGlobal | ScopeSession, Name: ForeignKeyChecks, Value: BoolToOnOff(DefTiDBForeignKeyChecks), Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if TiDBOptOn(normalizedValue) { vars.ForeignKeyChecks = true @@ -1612,21 +1657,31 @@ var defaultSysVars = []*SysVar{ s.LockWaitTimeout = lockWaitSec * 1000 return nil }}, - {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 - // Maximum Value (32-bit platforms) 4294967295 - if mathutil.IntBits == 32 { - if val, err := strconv.ParseUint(normalizedValue, 10, 64); err == nil { - if val > uint64(math.MaxUint32) { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(GroupConcatMaxLen, originalValue)) - return strconv.FormatInt(int64(math.MaxUint32), 10), nil + { + Scope: ScopeGlobal | ScopeSession, + Name: GroupConcatMaxLen, + Value: "1024", + IsHintUpdatableVerified: 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 + // Maximum Value (32-bit platforms) 4294967295 + if mathutil.IntBits == 32 { + if val, err := strconv.ParseUint(normalizedValue, 10, 64); err == nil { + if val > uint64(math.MaxUint32) { + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(GroupConcatMaxLen, originalValue)) + return strconv.FormatInt(int64(math.MaxUint32), 10), nil + } } } - } - return normalizedValue, nil - }}, + return normalizedValue, nil + }}, {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetConnection, Value: mysql.DefaultCharset, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCharacterSet(normalizedValue, CharacterSetConnection) }, SetSession: func(s *SessionVars, val string) error { @@ -1673,10 +1728,16 @@ var defaultSysVars = []*SysVar{ return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: WindowingUseHighPrecision, Value: On, Type: TypeBool, IsHintUpdatableVerfied: true, SetSession: func(s *SessionVars, val string) error { - s.WindowingUseHighPrecision = TiDBOptOn(val) - return nil - }}, + { + Scope: ScopeGlobal | ScopeSession, + Name: WindowingUseHighPrecision, + Value: On, + Type: TypeBool, + IsHintUpdatableVerified: true, + SetSession: func(s *SessionVars, val string) error { + s.WindowingUseHighPrecision = TiDBOptOn(val) + return nil + }}, {Scope: ScopeGlobal | ScopeSession, Name: BlockEncryptionMode, Value: "aes-128-ecb", Type: TypeEnum, PossibleValues: []string{"aes-128-ecb", "aes-192-ecb", "aes-256-ecb", "aes-128-cbc", "aes-192-cbc", "aes-256-cbc", "aes-128-ofb", "aes-192-ofb", "aes-256-ofb", "aes-128-cfb", "aes-192-cfb", "aes-256-cfb"}}, /* TiDB specific variables */ {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Type: TypeBool, Value: BoolToOnOff(DefTiDBAllowMPPExecution), Depended: true, SetSession: func(s *SessionVars, val string) error { @@ -1737,10 +1798,16 @@ var defaultSysVars = []*SysVar{ s.SetAllowInSubqToJoinAndAgg(TiDBOptOn(val)) return nil }}, - {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 - }}, + { + Scope: ScopeGlobal | ScopeSession, + Name: TiDBOptPreferRangeScan, + Value: BoolToOnOff(DefOptPreferRangeScan), + Type: TypeBool, + IsHintUpdatableVerified: true, + SetSession: func(s *SessionVars, val string) error { + s.SetAllowPreferRangeScan(TiDBOptOn(val)) + return nil + }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptLimitPushDownThreshold, Value: strconv.Itoa(DefOptLimitPushDownThreshold), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.LimitPushDownThreshold = TidbOptInt64(val, DefOptLimitPushDownThreshold) return nil @@ -2715,7 +2782,12 @@ var defaultSysVars = []*SysVar{ s.EnableMPPSharedCTEExecution = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptFixControl, Value: "", Type: TypeStr, IsHintUpdatableVerfied: true, + { + Scope: ScopeGlobal | ScopeSession, + Name: TiDBOptFixControl, + Value: "", + Type: TypeStr, + IsHintUpdatableVerified: 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. @@ -3082,7 +3154,9 @@ var defaultSysVars = []*SysVar{ return nil } return errors.Errorf("unsupport DML type: %s", val) - }}, + }, + IsHintUpdatableVerified: true, + }, } // GlobalSystemVariableInitialValue gets the default value for a system variable including ones that are dynamically set (e.g. based on the store) diff --git a/pkg/sessionctx/variable/variable.go b/pkg/sessionctx/variable/variable.go index 5c66ce2e4ec25..a05dcd792fe4c 100644 --- a/pkg/sessionctx/variable/variable.go +++ b/pkg/sessionctx/variable/variable.go @@ -142,8 +142,8 @@ type SysVar struct { SetSession func(*SessionVars, string) error // SetGlobal is called after validation SetGlobal func(context.Context, *SessionVars, string) error - // IsHintUpdatableVerfied indicate whether we've confirmed that SET_VAR() hint is worked for this hint. - IsHintUpdatableVerfied bool + // IsHintUpdatableVerified indicate whether we've confirmed that SET_VAR() hint is worked for this hint. + IsHintUpdatableVerified 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 diff --git a/tests/realtikvtest/pipelineddmltest/pipelineddml_test.go b/tests/realtikvtest/pipelineddmltest/pipelineddml_test.go index ec414e2fe5788..fa61e23f70fa4 100644 --- a/tests/realtikvtest/pipelineddmltest/pipelineddml_test.go +++ b/tests/realtikvtest/pipelineddmltest/pipelineddml_test.go @@ -84,7 +84,7 @@ func TestPipelinedDMLPositive(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a int primary key, b int)") + tk.MustExec("create table t (a int, b int)") tk.MustExec("insert into t values(1, 1)") tk.MustExec("set session tidb_dml_type = bulk") for _, stmt := range stmts { @@ -123,6 +123,19 @@ func TestPipelinedDMLPositive(t *testing.T) { require.True(t, strings.Contains(err.Error(), "pipelined memdb is enabled"), err.Error()) tk.MustQuery("show warnings").CheckContain("pessimistic-auto-commit config is ignored in favor of Pipelined DML") config.GetGlobalConfig().PessimisticTxn.PessimisticAutoCommit.Store(false) + + // enable by hint + // Hint works for DELETE and UPDATE, but not for INSERT if the hint is in its select clause. + tk.MustExec("set @@tidb_dml_type = standard") + err = panicToErr( + func() error { + _, err := tk.Exec("delete /*+ SET_VAR(tidb_dml_type=bulk) */ from t") + // "insert into t select /*+ SET_VAR(tidb_dml_type=bulk) */ * from t" won't work + return err + }, + ) + require.Error(t, err) + require.True(t, strings.Contains(err.Error(), "pipelined memdb is enabled"), err.Error()) } func TestPipelinedDMLNegative(t *testing.T) {