From cf4d252af8aa22bdae97e7d7565281f694eab464 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 23 Jan 2025 17:24:44 +0800 Subject: [PATCH] *: split definition out of variable pkg and refactor some reorg methods (#59146) ref pingcap/tidb#57229, ref pingcap/tidb#57497 --- br/pkg/backup/BUILD.bazel | 2 +- br/pkg/backup/schema_test.go | 12 +- br/pkg/checksum/BUILD.bazel | 4 +- br/pkg/checksum/executor.go | 4 +- br/pkg/checksum/executor_test.go | 8 +- br/pkg/task/BUILD.bazel | 2 +- br/pkg/task/common.go | 6 +- cmd/ddltest/BUILD.bazel | 2 +- cmd/ddltest/ddl_test.go | 6 +- cmd/tidb-server/BUILD.bazel | 2 + cmd/tidb-server/main.go | 55 +- cmd/tidb-server/main_test.go | 21 +- go.mod | 2 +- lightning/pkg/importer/BUILD.bazel | 2 +- lightning/pkg/importer/import.go | 4 +- lightning/pkg/importer/tidb.go | 22 +- pkg/bindinfo/BUILD.bazel | 3 +- pkg/bindinfo/binding_cache.go | 4 +- pkg/bindinfo/binding_cache_test.go | 8 +- pkg/bindinfo/global_handle.go | 5 +- pkg/config/BUILD.bazel | 2 +- pkg/config/config.go | 9 +- pkg/config/tiflash.go | 70 + pkg/ddl/BUILD.bazel | 3 + pkg/ddl/add_column.go | 4 +- pkg/ddl/backfilling.go | 20 +- pkg/ddl/backfilling_operators.go | 9 +- pkg/ddl/backfilling_test.go | 13 +- pkg/ddl/backfilling_txn_executor.go | 26 +- pkg/ddl/cluster.go | 25 +- pkg/ddl/cluster_test.go | 26 +- pkg/ddl/column.go | 10 +- pkg/ddl/create_table.go | 14 +- pkg/ddl/db_integration_test.go | 4 +- pkg/ddl/db_test.go | 12 +- pkg/ddl/ddl.go | 9 +- pkg/ddl/executor.go | 48 +- pkg/ddl/foreign_key.go | 20 +- pkg/ddl/index.go | 7 +- pkg/ddl/index_change_test.go | 10 +- pkg/ddl/index_modify_test.go | 16 +- pkg/ddl/ingest/BUILD.bazel | 2 +- pkg/ddl/ingest/backend_mgr.go | 5 +- pkg/ddl/ingest/config.go | 6 +- pkg/ddl/ingest/disk_root.go | 8 +- pkg/ddl/job_scheduler.go | 9 +- pkg/ddl/job_submitter.go | 4 +- pkg/ddl/job_worker.go | 18 +- pkg/ddl/metabuild_test.go | 14 +- pkg/ddl/modify_column.go | 4 +- pkg/ddl/modify_column_test.go | 8 +- pkg/ddl/multi_schema_change_test.go | 10 +- pkg/ddl/partition.go | 24 +- pkg/ddl/placement_policy.go | 18 +- pkg/ddl/placement_policy_ddl_test.go | 4 +- pkg/ddl/primary_key_handle_test.go | 10 +- pkg/ddl/reorg.go | 5 +- pkg/ddl/rollingback.go | 4 +- pkg/ddl/schematracker/BUILD.bazel | 1 + pkg/ddl/schematracker/checker.go | 3 +- pkg/ddl/schematracker/dm_tracker.go | 4 +- pkg/ddl/schemaver/BUILD.bazel | 4 +- pkg/ddl/schemaver/mem_syncer.go | 6 +- pkg/ddl/schemaver/syncer.go | 10 +- pkg/ddl/schemaver/syncer_nokit_test.go | 4 +- pkg/ddl/schemaver/syncer_test.go | 4 +- pkg/ddl/serverstate/BUILD.bazel | 2 +- pkg/ddl/serverstate/syncer_test.go | 4 +- pkg/ddl/split_region.go | 10 +- pkg/ddl/stat.go | 3 +- pkg/ddl/table.go | 4 +- pkg/ddl/tests/fail/BUILD.bazel | 2 +- pkg/ddl/tests/fail/fail_db_test.go | 16 +- pkg/ddl/tests/partition/BUILD.bazel | 2 +- pkg/ddl/tests/partition/db_partition_test.go | 4 +- pkg/ddl/tests/serial/BUILD.bazel | 2 +- pkg/ddl/tests/serial/serial_test.go | 8 +- pkg/ddl/util/BUILD.bazel | 2 +- pkg/ddl/util/util.go | 6 +- pkg/distsql/BUILD.bazel | 4 +- pkg/distsql/context_test.go | 20 +- pkg/distsql/distsql.go | 16 +- pkg/distsql/request_builder_test.go | 16 +- pkg/disttask/framework/storage/BUILD.bazel | 4 +- pkg/disttask/framework/storage/table_test.go | 6 +- pkg/disttask/framework/storage/task_table.go | 10 +- .../framework/taskexecutor/BUILD.bazel | 2 +- .../framework/taskexecutor/manager.go | 4 +- pkg/domain/BUILD.bazel | 2 + pkg/domain/db_test.go | 4 +- pkg/domain/domain.go | 31 +- pkg/domain/domain_sysvars.go | 21 +- pkg/domain/domain_test.go | 3 +- pkg/domain/infosync/BUILD.bazel | 2 +- pkg/domain/infosync/info.go | 4 +- pkg/domain/plan_replayer.go | 3 +- pkg/domain/plan_replayer_dump.go | 7 +- pkg/domain/schema_validator.go | 8 +- pkg/domain/schema_validator_test.go | 18 +- pkg/domain/sysvar_cache.go | 5 +- pkg/executor/BUILD.bazel | 2 + pkg/executor/adapter.go | 25 +- pkg/executor/adapter_test.go | 6 +- pkg/executor/aggfuncs/BUILD.bazel | 2 +- .../aggfuncs/func_group_concat_test.go | 6 +- pkg/executor/aggregate/BUILD.bazel | 4 +- pkg/executor/aggregate/agg_hash_executor.go | 6 +- pkg/executor/aggregate/agg_spill_test.go | 6 +- pkg/executor/analyze.go | 3 +- pkg/executor/analyze_utils.go | 14 +- pkg/executor/batch_point_get_test.go | 4 +- pkg/executor/benchmark_test.go | 32 +- pkg/executor/builder.go | 5 +- pkg/executor/checksum.go | 4 +- pkg/executor/coprocessor.go | 3 +- pkg/executor/cte.go | 6 +- pkg/executor/ddl.go | 11 +- pkg/executor/delete.go | 4 +- pkg/executor/delete_test.go | 4 +- pkg/executor/executor_required_rows_test.go | 6 +- pkg/executor/explain_unit_test.go | 6 +- pkg/executor/explainfor_test.go | 4 +- pkg/executor/grant.go | 4 +- pkg/executor/historical_stats_test.go | 4 +- pkg/executor/import_into.go | 6 +- pkg/executor/importer/BUILD.bazel | 4 +- pkg/executor/importer/import.go | 8 +- pkg/executor/importer/import_test.go | 10 +- .../importer/importer_testkit_test.go | 10 +- pkg/executor/importer/table_import.go | 6 +- pkg/executor/infoschema_reader.go | 5 +- pkg/executor/insert_common.go | 6 +- pkg/executor/insert_test.go | 4 +- pkg/executor/internal/builder/BUILD.bazel | 2 +- .../internal/builder/builder_utils.go | 4 +- .../internal/calibrateresource/BUILD.bazel | 2 +- .../calibrateresource/calibrate_resource.go | 4 +- pkg/executor/internal/mpp/BUILD.bazel | 2 +- .../internal/mpp/local_mpp_coordinator.go | 4 +- pkg/executor/internal/querywatch/BUILD.bazel | 2 +- .../internal/querywatch/query_watch_test.go | 4 +- pkg/executor/internal/testutil/BUILD.bazel | 2 +- pkg/executor/internal/testutil/agg.go | 6 +- pkg/executor/internal/testutil/limit.go | 6 +- pkg/executor/internal/testutil/sort.go | 10 +- pkg/executor/internal/testutil/window.go | 6 +- pkg/executor/join/BUILD.bazel | 4 +- pkg/executor/join/hash_join_v1.go | 4 +- pkg/executor/join/hash_join_v2.go | 4 +- pkg/executor/join/joiner_test.go | 6 +- pkg/executor/join/merge_join.go | 4 +- pkg/executor/join/merge_join_test.go | 6 +- pkg/executor/main_test.go | 4 +- pkg/executor/plan_replayer.go | 5 +- pkg/executor/point_get_test.go | 6 +- pkg/executor/prepared_test.go | 5 +- pkg/executor/sample_test.go | 4 +- pkg/executor/select.go | 13 +- pkg/executor/set.go | 41 +- pkg/executor/set_test.go | 117 +- pkg/executor/show.go | 15 +- pkg/executor/show_ddl_jobs.go | 14 +- pkg/executor/show_ddl_jobs_test.go | 14 +- pkg/executor/simple.go | 29 +- pkg/executor/slow_query.go | 3 +- pkg/executor/slow_query_test.go | 12 +- pkg/executor/sortexec/BUILD.bazel | 5 +- pkg/executor/sortexec/benchmark_test.go | 8 +- pkg/executor/sortexec/sort.go | 4 +- pkg/executor/sortexec/sort_test.go | 8 +- pkg/executor/sortexec/topn.go | 4 +- pkg/executor/stmtsummary_test.go | 8 +- pkg/executor/test/admintest/BUILD.bazel | 2 +- pkg/executor/test/admintest/admin_test.go | 10 +- pkg/executor/test/analyzetest/BUILD.bazel | 1 + pkg/executor/test/analyzetest/analyze_test.go | 3 +- .../analyzetest/memorycontrol/BUILD.bazel | 2 +- .../analyzetest/memorycontrol/main_test.go | 4 +- pkg/executor/test/ddl/BUILD.bazel | 1 + pkg/executor/test/ddl/ddl_test.go | 45 +- pkg/executor/test/distsqltest/BUILD.bazel | 2 +- pkg/executor/test/distsqltest/distsql_test.go | 4 +- pkg/executor/test/executor/BUILD.bazel | 1 + pkg/executor/test/executor/executor_test.go | 7 +- .../test/jointest/hashjoin/BUILD.bazel | 2 +- .../test/jointest/hashjoin/hash_join_test.go | 6 +- pkg/executor/test/passwordtest/BUILD.bazel | 2 +- .../passwordtest/password_management_test.go | 4 +- pkg/executor/test/seqtest/BUILD.bazel | 1 + .../test/seqtest/seq_executor_test.go | 3 +- pkg/executor/test/showtest/BUILD.bazel | 1 + pkg/executor/test/showtest/show_test.go | 5 +- pkg/executor/test/splittest/BUILD.bazel | 2 +- .../test/splittest/split_table_test.go | 4 +- pkg/executor/test/tiflashtest/tiflash_test.go | 8 +- pkg/executor/update_test.go | 10 +- pkg/executor/write.go | 4 +- pkg/expression/BUILD.bazel | 2 + pkg/expression/aggregation/BUILD.bazel | 1 + .../aggregation/aggregation_test.go | 3 +- pkg/expression/bench_test.go | 6 +- pkg/expression/builtin_encryption.go | 3 +- pkg/expression/builtin_encryption_test.go | 45 +- pkg/expression/builtin_encryption_vec.go | 3 +- pkg/expression/builtin_math_test.go | 4 +- pkg/expression/builtin_miscellaneous.go | 3 +- pkg/expression/builtin_string_test.go | 18 +- pkg/expression/exprstatic/BUILD.bazel | 2 + pkg/expression/exprstatic/evalctx.go | 29 +- pkg/expression/exprstatic/evalctx_test.go | 11 +- pkg/expression/exprstatic/exprctx.go | 25 +- pkg/expression/exprstatic/exprctx_test.go | 11 +- pkg/expression/integration_test/BUILD.bazel | 1 + .../integration_test/integration_test.go | 5 +- pkg/expression/sessionexpr/BUILD.bazel | 1 + pkg/expression/sessionexpr/sessionctx.go | 5 +- pkg/expression/typeinfer_test.go | 6 +- pkg/extension/BUILD.bazel | 1 + pkg/extension/auth_test.go | 21 +- pkg/extension/enterprise | 2 +- pkg/extension/registry_test.go | 23 +- pkg/infoschema/BUILD.bazel | 3 +- pkg/infoschema/bench_test.go | 4 +- pkg/infoschema/builder.go | 4 +- pkg/infoschema/infoschema_test.go | 26 +- pkg/infoschema/infoschema_v2_test.go | 12 +- .../test/infoschemav2test/BUILD.bazel | 2 +- .../test/infoschemav2test/v2_test.go | 6 +- pkg/kv/BUILD.bazel | 1 + pkg/kv/mpp.go | 54 - pkg/kv/version_test.go | 29 +- pkg/lightning/backend/kv/BUILD.bazel | 1 + pkg/lightning/backend/kv/context_test.go | 23 +- pkg/lightning/backend/kv/kv2sql_test.go | 6 +- pkg/lightning/backend/local/BUILD.bazel | 2 +- pkg/lightning/backend/local/checksum.go | 6 +- pkg/lightning/common/BUILD.bazel | 2 +- pkg/lightning/common/util.go | 6 +- pkg/lightning/errormanager/BUILD.bazel | 2 +- .../errormanager/resolveconflict_test.go | 10 +- pkg/meta/metabuild/BUILD.bazel | 3 +- pkg/meta/metabuild/context.go | 16 +- pkg/meta/metabuild/context_test.go | 5 +- pkg/meta/model/BUILD.bazel | 2 + pkg/meta/model/job.go | 11 +- pkg/meta/model/reorg.go | 21 +- pkg/planner/BUILD.bazel | 1 + pkg/planner/cardinality/BUILD.bazel | 2 +- pkg/planner/cardinality/selectivity_test.go | 4 +- pkg/planner/core/BUILD.bazel | 2 + pkg/planner/core/casetest/cbotest/BUILD.bazel | 2 +- pkg/planner/core/casetest/cbotest/cbo_test.go | 4 +- .../core/casetest/partition/BUILD.bazel | 2 +- .../partition/partition_pruner_test.go | 4 +- pkg/planner/core/explain.go | 8 +- pkg/planner/core/expression_rewriter.go | 7 +- pkg/planner/core/integration_test.go | 6 +- pkg/planner/core/logical_plan_builder.go | 5 +- pkg/planner/core/logical_plans_test.go | 14 +- pkg/planner/core/mock.go | 4 +- pkg/planner/core/optimizer.go | 3 +- pkg/planner/core/physical_plans.go | 3 +- pkg/planner/core/plan_cache.go | 4 +- pkg/planner/core/plan_cache_utils.go | 5 +- pkg/planner/core/plan_cost_ver1.go | 6 +- pkg/planner/core/plan_cost_ver2.go | 4 +- pkg/planner/core/planbuilder.go | 27 +- pkg/planner/core/rule_collect_plan_stats.go | 6 +- pkg/planner/memo/BUILD.bazel | 2 +- pkg/planner/memo/group_test.go | 6 +- pkg/planner/optimize.go | 7 +- pkg/plugin/conn_ip_example/BUILD.bazel | 1 + pkg/plugin/conn_ip_example/conn_ip_example.go | 7 +- pkg/privilege/privileges/BUILD.bazel | 2 + pkg/privilege/privileges/cache.go | 5 +- pkg/privilege/privileges/cache_test.go | 4 +- pkg/privilege/privileges/privileges.go | 5 +- pkg/privilege/privileges/privileges_test.go | 11 +- pkg/server/BUILD.bazel | 4 +- pkg/server/conn.go | 28 +- pkg/server/conn_stmt.go | 4 +- pkg/server/conn_test.go | 8 +- pkg/server/handler/optimizor/BUILD.bazel | 2 +- .../handler/optimizor/statistics_handler.go | 4 +- pkg/server/handler/tests/BUILD.bazel | 2 +- .../handler/tests/http_handler_serial_test.go | 24 +- pkg/server/handler/tikvhandler/BUILD.bazel | 1 + .../handler/tikvhandler/tikv_handler.go | 19 +- pkg/server/internal/BUILD.bazel | 2 +- pkg/server/internal/packetio.go | 4 +- pkg/server/rpc_server.go | 6 +- pkg/server/server.go | 5 +- pkg/server/stat.go | 9 +- pkg/session/BUILD.bazel | 2 + pkg/session/bootstrap.go | 123 +- pkg/session/bootstrap_test.go | 97 +- pkg/session/clusteredindextest/BUILD.bazel | 2 +- .../clustered_index_test.go | 4 +- pkg/session/nontransactional.go | 4 +- pkg/session/schematest/BUILD.bazel | 1 + pkg/session/schematest/schema_test.go | 7 +- pkg/session/session.go | 77 +- pkg/session/test/BUILD.bazel | 1 + pkg/session/test/session_test.go | 5 +- pkg/session/test/variable/BUILD.bazel | 1 + pkg/session/test/variable/variable_test.go | 13 +- pkg/session/test/vars/BUILD.bazel | 1 + pkg/session/test/vars/vars_test.go | 5 +- pkg/session/tidb_test.go | 4 +- pkg/sessionctx/sessionstates/BUILD.bazel | 2 +- .../sessionstates/session_states_test.go | 36 +- pkg/sessionctx/vardef/BUILD.bazel | 21 + pkg/sessionctx/vardef/OWNERS | 8 + pkg/sessionctx/vardef/sysvar.go | 342 +++ pkg/sessionctx/vardef/tidb_vars.go | 2014 +++++++++++++++ pkg/sessionctx/variable/BUILD.bazel | 4 +- .../variable/mock_globalaccessor.go | 10 +- .../variable/mock_globalaccessor_test.go | 7 +- pkg/sessionctx/variable/noop.go | 915 +++---- pkg/sessionctx/variable/removed_test.go | 5 +- pkg/sessionctx/variable/session.go | 325 ++- pkg/sessionctx/variable/session_test.go | 21 +- pkg/sessionctx/variable/statusvar.go | 17 +- pkg/sessionctx/variable/statusvar_test.go | 9 +- pkg/sessionctx/variable/sysvar.go | 2220 +++++++---------- pkg/sessionctx/variable/sysvar_test.go | 893 +++---- pkg/sessionctx/variable/tidb_vars.go | 1702 ------------- pkg/sessionctx/variable/variable.go | 176 +- pkg/sessionctx/variable/variable_test.go | 307 +-- pkg/sessionctx/variable/varsutil.go | 153 +- pkg/sessionctx/variable/varsutil_test.go | 433 ++-- pkg/sessiontxn/isolation/BUILD.bazel | 1 + pkg/sessiontxn/isolation/base.go | 3 +- pkg/sessiontxn/isolation/readcommitted.go | 3 +- pkg/sessiontxn/staleread/BUILD.bazel | 1 + pkg/sessiontxn/staleread/provider.go | 3 +- pkg/statistics/BUILD.bazel | 1 + pkg/statistics/handle/BUILD.bazel | 2 +- pkg/statistics/handle/autoanalyze/BUILD.bazel | 2 + .../handle/autoanalyze/autoanalyze.go | 11 +- .../handle/autoanalyze/autoanalyze_test.go | 3 +- .../handle/autoanalyze/exec/BUILD.bazel | 2 +- .../handle/autoanalyze/exec/exec.go | 14 +- .../autoanalyze/priorityqueue/BUILD.bazel | 1 + .../dynamic_partitioned_table_analysis_job.go | 6 +- .../handle/autoanalyze/priorityqueue/queue.go | 5 +- .../priorityqueue/queue_ddl_handler.go | 5 +- .../handle/autoanalyze/refresher/BUILD.bazel | 1 + .../handle/autoanalyze/refresher/refresher.go | 11 +- pkg/statistics/handle/bootstrap.go | 4 +- pkg/statistics/handle/cache/BUILD.bazel | 2 +- .../handle/cache/statscacheinner.go | 4 +- pkg/statistics/handle/ddl/BUILD.bazel | 1 + pkg/statistics/handle/ddl/subscriber.go | 5 +- pkg/statistics/handle/storage/BUILD.bazel | 1 + pkg/statistics/handle/storage/gc.go | 10 +- .../handle/storage/stats_read_writer.go | 5 +- pkg/statistics/handle/syncload/BUILD.bazel | 1 + .../handle/syncload/stats_syncload.go | 3 +- pkg/statistics/handle/util/BUILD.bazel | 1 + pkg/statistics/handle/util/util.go | 19 +- pkg/statistics/histogram.go | 3 +- pkg/store/copr/BUILD.bazel | 2 +- pkg/store/copr/batch_coprocessor.go | 4 +- pkg/store/copr/coprocessor.go | 4 +- pkg/store/driver/BUILD.bazel | 1 - pkg/store/driver/client_test.go | 9 +- pkg/store/gcworker/BUILD.bazel | 1 + pkg/store/gcworker/gc_worker.go | 3 +- pkg/store/mockstore/mockcopr/BUILD.bazel | 2 +- .../mockstore/mockcopr/cop_handler_dag.go | 4 +- .../mockstore/unistore/cophandler/BUILD.bazel | 2 +- .../unistore/cophandler/cop_handler.go | 4 +- pkg/table/tables/BUILD.bazel | 2 + pkg/table/tables/bench_test.go | 6 +- pkg/table/tables/partition.go | 6 +- pkg/timer/tablestore/BUILD.bazel | 2 +- pkg/timer/tablestore/store.go | 4 +- pkg/ttl/session/BUILD.bazel | 4 +- pkg/ttl/session/session.go | 6 +- pkg/ttl/session/sysvar_test.go | 46 +- pkg/ttl/ttlworker/BUILD.bazel | 3 +- pkg/ttl/ttlworker/del.go | 8 +- pkg/ttl/ttlworker/del_test.go | 30 +- pkg/ttl/ttlworker/job_manager.go | 10 +- .../ttlworker/job_manager_integration_test.go | 20 +- pkg/ttl/ttlworker/scan.go | 4 +- pkg/ttl/ttlworker/scan_test.go | 20 +- pkg/ttl/ttlworker/session.go | 4 +- pkg/ttl/ttlworker/task_manager.go | 16 +- pkg/ttl/ttlworker/task_manager_test.go | 14 +- pkg/ttl/ttlworker/timer.go | 6 +- pkg/ttl/ttlworker/timer_test.go | 18 +- pkg/util/expensivequery/BUILD.bazel | 2 +- pkg/util/expensivequery/expensivequery.go | 12 +- pkg/util/gcutil/BUILD.bazel | 1 + pkg/util/gcutil/gcutil.go | 7 +- pkg/util/logutil/BUILD.bazel | 4 +- pkg/util/logutil/log.go | 12 +- pkg/util/logutil/log_test.go | 14 +- pkg/util/memoryusagealarm/BUILD.bazel | 4 +- pkg/util/memoryusagealarm/memoryusagealarm.go | 8 +- .../memoryusagealarm/memoryusagealarm_test.go | 10 +- pkg/util/mock/BUILD.bazel | 1 + pkg/util/mock/context.go | 13 +- pkg/util/password-validation/BUILD.bazel | 2 + .../password_validation.go | 15 +- .../password_validation_test.go | 25 +- pkg/util/sem/BUILD.bazel | 3 +- pkg/util/sem/sem.go | 55 +- pkg/util/sem/sem_test.go | 50 +- pkg/util/tiflash/BUILD.bazel | 1 + pkg/util/tiflash/tiflash_replica_read.go | 25 +- pkg/util/tiflashcompute/BUILD.bazel | 2 + pkg/util/tiflashcompute/dispatch_policy.go | 22 +- pkg/util/tiflashcompute/topo_fetcher.go | 64 +- pkg/util/tracing/BUILD.bazel | 2 - pkg/util/tracing/util.go | 15 +- pkg/util/tracing/util_test.go | 3 +- pkg/util/workloadrepo/BUILD.bazel | 1 + pkg/util/workloadrepo/worker.go | 19 +- tests/realtikvtest/BUILD.bazel | 2 +- tests/realtikvtest/addindextest1/BUILD.bazel | 2 +- .../addindextest1/disttask_test.go | 12 +- tests/realtikvtest/addindextest2/BUILD.bazel | 2 +- .../addindextest2/global_sort_test.go | 6 +- tests/realtikvtest/addindextest3/BUILD.bazel | 2 +- .../realtikvtest/addindextest3/ingest_test.go | 4 +- .../realtikvtest/pessimistictest/BUILD.bazel | 2 +- .../pessimistictest/pessimistic_test.go | 6 +- .../realtikvtest/pipelineddmltest/BUILD.bazel | 2 +- .../pipelineddmltest/pipelineddml_test.go | 6 +- tests/realtikvtest/sessiontest/BUILD.bazel | 2 +- .../sessiontest/session_fail_test.go | 14 +- tests/realtikvtest/testkit.go | 4 +- 435 files changed, 6803 insertions(+), 6544 deletions(-) create mode 100644 pkg/config/tiflash.go create mode 100644 pkg/sessionctx/vardef/BUILD.bazel create mode 100644 pkg/sessionctx/vardef/OWNERS create mode 100644 pkg/sessionctx/vardef/sysvar.go create mode 100644 pkg/sessionctx/vardef/tidb_vars.go diff --git a/br/pkg/backup/BUILD.bazel b/br/pkg/backup/BUILD.bazel index 2febe37641b01..371c3f0f74efc 100644 --- a/br/pkg/backup/BUILD.bazel +++ b/br/pkg/backup/BUILD.bazel @@ -79,7 +79,7 @@ go_test( "//br/pkg/storage", "//br/pkg/utils", "//pkg/meta/model", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/testkit/testsetup", "//pkg/util/table-filter", diff --git a/br/pkg/backup/schema_test.go b/br/pkg/backup/schema_test.go index b45a96fb54970..38a9b8473fc13 100644 --- a/br/pkg/backup/schema_test.go +++ b/br/pkg/backup/schema_test.go @@ -18,7 +18,7 @@ import ( "github.com/pingcap/tidb/br/pkg/mock" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/utils" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" filter "github.com/pingcap/tidb/pkg/util/table-filter" "github.com/stretchr/testify/require" @@ -151,7 +151,7 @@ func TestBuildBackupRangeAndSchema(t *testing.T) { metaWriter := metautil.NewMetaWriter(es, metautil.MetaFileSize, false, "", &cipher) ctx := context.Background() err = backupSchemas.BackupSchemas( - ctx, metaWriter, nil, m.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) + ctx, metaWriter, nil, m.Storage, nil, math.MaxUint64, 1, vardef.DefChecksumTableConcurrency, skipChecksum, updateCh) require.Equal(t, int64(1), updateCh.get()) require.NoError(t, err) err = metaWriter.FlushBackupMeta(ctx) @@ -180,7 +180,7 @@ func TestBuildBackupRangeAndSchema(t *testing.T) { es2 := GetRandomStorage(t) metaWriter2 := metautil.NewMetaWriter(es2, metautil.MetaFileSize, false, "", &cipher) err = backupSchemas.BackupSchemas( - ctx, metaWriter2, nil, m.Storage, nil, math.MaxUint64, 2, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) + ctx, metaWriter2, nil, m.Storage, nil, math.MaxUint64, 2, vardef.DefChecksumTableConcurrency, skipChecksum, updateCh) require.Equal(t, int64(2), updateCh.get()) require.NoError(t, err) err = metaWriter2.FlushBackupMeta(ctx) @@ -234,7 +234,7 @@ func TestBuildBackupRangeAndSchemaWithBrokenStats(t *testing.T) { metaWriter := metautil.NewMetaWriter(es, metautil.MetaFileSize, false, "", &cipher) ctx := context.Background() err = backupSchemas.BackupSchemas( - ctx, metaWriter, nil, m.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) + ctx, metaWriter, nil, m.Storage, nil, math.MaxUint64, 1, vardef.DefChecksumTableConcurrency, skipChecksum, updateCh) require.NoError(t, err) err = metaWriter.FlushBackupMeta(ctx) require.NoError(t, err) @@ -262,7 +262,7 @@ func TestBuildBackupRangeAndSchemaWithBrokenStats(t *testing.T) { es2 := GetRandomStorage(t) metaWriter2 := metautil.NewMetaWriter(es2, metautil.MetaFileSize, false, "", &cipher) err = backupSchemas.BackupSchemas( - ctx, metaWriter2, nil, m.Storage, statsHandle, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) + ctx, metaWriter2, nil, m.Storage, statsHandle, math.MaxUint64, 1, vardef.DefChecksumTableConcurrency, skipChecksum, updateCh) require.NoError(t, err) err = metaWriter2.FlushBackupMeta(ctx) require.NoError(t, err) @@ -306,7 +306,7 @@ func TestBackupSchemasForSystemTable(t *testing.T) { metaWriter2 := metautil.NewMetaWriter(es2, metautil.MetaFileSize, false, "", &cipher) err = backupSchemas.BackupSchemas(ctx, metaWriter2, nil, m.Storage, nil, - math.MaxUint64, 1, variable.DefChecksumTableConcurrency, true, updateCh) + math.MaxUint64, 1, vardef.DefChecksumTableConcurrency, true, updateCh) require.NoError(t, err) err = metaWriter2.FlushBackupMeta(ctx) require.NoError(t, err) diff --git a/br/pkg/checksum/BUILD.bazel b/br/pkg/checksum/BUILD.bazel index 035819c5ab724..0334dda084e75 100644 --- a/br/pkg/checksum/BUILD.bazel +++ b/br/pkg/checksum/BUILD.bazel @@ -17,7 +17,7 @@ go_library( "//pkg/distsql", "//pkg/kv", "//pkg/meta/model", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/tablecodec", "//pkg/util/ranger", "@com_github_gogo_protobuf//proto", @@ -46,7 +46,7 @@ go_test( "//pkg/kv", "//pkg/meta/model", "//pkg/parser/ast", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/testkit/testsetup", "@com_github_pingcap_failpoint//:failpoint", diff --git a/br/pkg/checksum/executor.go b/br/pkg/checksum/executor.go index 3cd0405470562..40832d0434e40 100644 --- a/br/pkg/checksum/executor.go +++ b/br/pkg/checksum/executor.go @@ -14,7 +14,7 @@ import ( "github.com/pingcap/tidb/pkg/distsql" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/util/ranger" "github.com/pingcap/tipb/go-tipb" @@ -44,7 +44,7 @@ func NewExecutorBuilder(table *model.TableInfo, ts uint64) *ExecutorBuilder { table: table, ts: ts, - concurrency: variable.DefDistSQLScanConcurrency, + concurrency: vardef.DefDistSQLScanConcurrency, } } diff --git a/br/pkg/checksum/executor_test.go b/br/pkg/checksum/executor_test.go index 93483664ea8fa..afe87c433ab63 100644 --- a/br/pkg/checksum/executor_test.go +++ b/br/pkg/checksum/executor_test.go @@ -15,7 +15,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -44,7 +44,7 @@ func TestChecksumContextDone(t *testing.T) { tk.MustExec("insert into t1 values (10, 10);") tableInfo1 := getTableInfo(t, mock, "test", "t1") exe, err := checksum.NewExecutorBuilder(tableInfo1, math.MaxUint64). - SetConcurrency(variable.DefChecksumTableConcurrency). + SetConcurrency(vardef.DefChecksumTableConcurrency). Build() require.NoError(t, err) @@ -73,12 +73,12 @@ func TestChecksum(t *testing.T) { tk.MustExec("insert into t1 values (10);") tableInfo1 := getTableInfo(t, mock, "test", "t1") exe1, err := checksum.NewExecutorBuilder(tableInfo1, math.MaxUint64). - SetConcurrency(variable.DefChecksumTableConcurrency). + SetConcurrency(vardef.DefChecksumTableConcurrency). Build() require.NoError(t, err) require.NoError(t, exe1.Each(func(r *kv.Request) error { require.True(t, r.NotFillCache) - require.Equal(t, variable.DefChecksumTableConcurrency, r.Concurrency) + require.Equal(t, vardef.DefChecksumTableConcurrency, r.Concurrency) return nil })) require.Equal(t, 1, exe1.Len()) diff --git a/br/pkg/task/BUILD.bazel b/br/pkg/task/BUILD.bazel index 2fde9e8981389..625baff340ed4 100644 --- a/br/pkg/task/BUILD.bazel +++ b/br/pkg/task/BUILD.bazel @@ -63,7 +63,7 @@ go_library( "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/sessionctx/stmtctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/statistics/handle", "//pkg/types", "//pkg/util", diff --git a/br/pkg/task/common.go b/br/pkg/task/common.go index c95bee08c59d8..1f613ba9f934c 100644 --- a/br/pkg/task/common.go +++ b/br/pkg/task/common.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/br/pkg/metautil" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/utils" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" filter "github.com/pingcap/tidb/pkg/util/table-filter" "github.com/spf13/cobra" "github.com/spf13/pflag" @@ -294,7 +294,7 @@ func DefineCommonFlags(flags *pflag.FlagSet) { flags.String(flagCA, "", "CA certificate path for TLS connection") flags.String(flagCert, "", "Certificate path for TLS connection") flags.String(flagKey, "", "Private key path for TLS connection") - flags.Uint(flagChecksumConcurrency, variable.DefChecksumTableConcurrency, "The concurrency of checksumming in one table") + flags.Uint(flagChecksumConcurrency, vardef.DefChecksumTableConcurrency, "The concurrency of checksumming in one table") flags.Uint64(flagRateLimit, unlimited, "The rate limit of the task, MB/s per node") flags.Bool(FlagChecksum, true, "Run checksum at end of task") @@ -947,7 +947,7 @@ func (cfg *Config) adjust() { cfg.GRPCKeepaliveTimeout = defaultGRPCKeepaliveTimeout } if cfg.ChecksumConcurrency == 0 { - cfg.ChecksumConcurrency = variable.DefChecksumTableConcurrency + cfg.ChecksumConcurrency = vardef.DefChecksumTableConcurrency } if cfg.MetadataDownloadBatchSize == 0 { cfg.MetadataDownloadBatchSize = defaultMetadataDownloadBatchSize diff --git a/cmd/ddltest/BUILD.bazel b/cmd/ddltest/BUILD.bazel index 3b0340ecbc66f..3bd6f1add42ee 100644 --- a/cmd/ddltest/BUILD.bazel +++ b/cmd/ddltest/BUILD.bazel @@ -24,7 +24,7 @@ go_test( "//pkg/session", "//pkg/session/types", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/sessiontxn", "//pkg/store", "//pkg/store/driver", diff --git a/cmd/ddltest/ddl_test.go b/cmd/ddltest/ddl_test.go index e810b1d02f06d..21606da8477e9 100644 --- a/cmd/ddltest/ddl_test.go +++ b/cmd/ddltest/ddl_test.go @@ -43,7 +43,7 @@ import ( "github.com/pingcap/tidb/pkg/session" sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/store" tidbdriver "github.com/pingcap/tidb/pkg/store/driver" @@ -565,7 +565,7 @@ func (s *ddlSuite) Bootstrap(t *testing.T) { tk.MustExec("create table test_mixed (c1 int, c2 int, primary key(c1))") tk.MustExec("create table test_inc (c1 int, c2 int, primary key(c1))") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("drop table if exists test_insert_common, test_conflict_insert_common, " + "test_update_common, test_conflict_update_common, test_delete_common, test_conflict_delete_common, " + "test_mixed_common, test_inc_common") @@ -577,7 +577,7 @@ func (s *ddlSuite) Bootstrap(t *testing.T) { tk.MustExec("create table test_conflict_delete_common (c1 int, c2 int, primary key(c1, c2))") tk.MustExec("create table test_mixed_common (c1 int, c2 int, primary key(c1, c2))") tk.MustExec("create table test_inc_common (c1 int, c2 int, primary key(c1, c2))") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly } func TestSimple(t *testing.T) { diff --git a/cmd/tidb-server/BUILD.bazel b/cmd/tidb-server/BUILD.bazel index 138df3e421e7a..639257f1d2ce7 100644 --- a/cmd/tidb-server/BUILD.bazel +++ b/cmd/tidb-server/BUILD.bazel @@ -27,6 +27,7 @@ go_library( "//pkg/server", "//pkg/session", "//pkg/session/txninfo", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/store", @@ -104,6 +105,7 @@ go_test( deps = [ "//pkg/config", "//pkg/parser/mysql", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/testkit/testsetup", "@com_github_stretchr_testify//require", diff --git a/cmd/tidb-server/main.go b/cmd/tidb-server/main.go index 4855dccfdca5c..2089c5b8efe1b 100644 --- a/cmd/tidb-server/main.go +++ b/cmd/tidb-server/main.go @@ -52,6 +52,7 @@ import ( "github.com/pingcap/tidb/pkg/server" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/session/txninfo" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" kvstore "github.com/pingcap/tidb/pkg/store" @@ -279,7 +280,7 @@ func main() { err := metricsutil.RegisterMetrics() terror.MustNil(err) - if variable.EnableTmpStorageOnOOM.Load() { + if vardef.EnableTmpStorageOnOOM.Load() { config.GetGlobalConfig().UpdateTempStoragePath() err := disk.InitializeTempDir() terror.MustNil(err) @@ -741,48 +742,48 @@ func setGlobalVars() { kv.TxnEntrySizeLimit.Store(cfg.Performance.TxnEntrySizeLimit) priority := mysql.Str2Priority(cfg.Instance.ForcePriority) - variable.ForcePriority = int32(priority) + vardef.ForcePriority = int32(priority) - variable.ProcessGeneralLog.Store(cfg.Instance.TiDBGeneralLog) - variable.EnablePProfSQLCPU.Store(cfg.Instance.EnablePProfSQLCPU) - variable.EnableRCReadCheckTS.Store(cfg.Instance.TiDBRCReadCheckTS) - variable.IsSandBoxModeEnabled.Store(!cfg.Security.DisconnectOnExpiredPassword) - atomic.StoreUint32(&variable.DDLSlowOprThreshold, cfg.Instance.DDLSlowOprThreshold) - atomic.StoreUint64(&variable.ExpensiveQueryTimeThreshold, cfg.Instance.ExpensiveQueryTimeThreshold) - atomic.StoreUint64(&variable.ExpensiveTxnTimeThreshold, cfg.Instance.ExpensiveTxnTimeThreshold) + vardef.ProcessGeneralLog.Store(cfg.Instance.TiDBGeneralLog) + vardef.EnablePProfSQLCPU.Store(cfg.Instance.EnablePProfSQLCPU) + vardef.EnableRCReadCheckTS.Store(cfg.Instance.TiDBRCReadCheckTS) + vardef.IsSandBoxModeEnabled.Store(!cfg.Security.DisconnectOnExpiredPassword) + atomic.StoreUint32(&vardef.DDLSlowOprThreshold, cfg.Instance.DDLSlowOprThreshold) + atomic.StoreUint64(&vardef.ExpensiveQueryTimeThreshold, cfg.Instance.ExpensiveQueryTimeThreshold) + atomic.StoreUint64(&vardef.ExpensiveTxnTimeThreshold, cfg.Instance.ExpensiveTxnTimeThreshold) if len(cfg.ServerVersion) > 0 { mysql.ServerVersion = cfg.ServerVersion - variable.SetSysVar(variable.Version, cfg.ServerVersion) + variable.SetSysVar(vardef.Version, cfg.ServerVersion) } if len(cfg.TiDBEdition) > 0 { versioninfo.TiDBEdition = cfg.TiDBEdition - variable.SetSysVar(variable.VersionComment, "TiDB Server (Apache License 2.0) "+versioninfo.TiDBEdition+" Edition, MySQL 8.0 compatible") + variable.SetSysVar(vardef.VersionComment, "TiDB Server (Apache License 2.0) "+versioninfo.TiDBEdition+" Edition, MySQL 8.0 compatible") } if len(cfg.VersionComment) > 0 { - variable.SetSysVar(variable.VersionComment, cfg.VersionComment) + variable.SetSysVar(vardef.VersionComment, cfg.VersionComment) } if len(cfg.TiDBReleaseVersion) > 0 { mysql.TiDBReleaseVersion = cfg.TiDBReleaseVersion } - variable.SetSysVar(variable.TiDBForcePriority, mysql.Priority2Str[priority]) - variable.SetSysVar(variable.TiDBOptDistinctAggPushDown, variable.BoolToOnOff(cfg.Performance.DistinctAggPushDown)) - variable.SetSysVar(variable.TiDBOptProjectionPushDown, variable.BoolToOnOff(cfg.Performance.ProjectionPushDown)) - variable.SetSysVar(variable.Port, fmt.Sprintf("%d", cfg.Port)) + variable.SetSysVar(vardef.TiDBForcePriority, mysql.Priority2Str[priority]) + variable.SetSysVar(vardef.TiDBOptDistinctAggPushDown, variable.BoolToOnOff(cfg.Performance.DistinctAggPushDown)) + variable.SetSysVar(vardef.TiDBOptProjectionPushDown, variable.BoolToOnOff(cfg.Performance.ProjectionPushDown)) + variable.SetSysVar(vardef.Port, fmt.Sprintf("%d", cfg.Port)) cfg.Socket = strings.Replace(cfg.Socket, "{Port}", fmt.Sprintf("%d", cfg.Port), 1) - variable.SetSysVar(variable.Socket, cfg.Socket) - variable.SetSysVar(variable.DataDir, cfg.Path) - variable.SetSysVar(variable.TiDBSlowQueryFile, cfg.Log.SlowQueryFile) - variable.SetSysVar(variable.TiDBIsolationReadEngines, strings.Join(cfg.IsolationRead.Engines, ",")) - variable.SetSysVar(variable.TiDBEnforceMPPExecution, variable.BoolToOnOff(config.GetGlobalConfig().Performance.EnforceMPP)) - variable.MemoryUsageAlarmRatio.Store(cfg.Instance.MemoryUsageAlarmRatio) - variable.SetSysVar(variable.TiDBConstraintCheckInPlacePessimistic, variable.BoolToOnOff(cfg.PessimisticTxn.ConstraintCheckInPlacePessimistic)) + variable.SetSysVar(vardef.Socket, cfg.Socket) + variable.SetSysVar(vardef.DataDir, cfg.Path) + variable.SetSysVar(vardef.TiDBSlowQueryFile, cfg.Log.SlowQueryFile) + variable.SetSysVar(vardef.TiDBIsolationReadEngines, strings.Join(cfg.IsolationRead.Engines, ",")) + variable.SetSysVar(vardef.TiDBEnforceMPPExecution, variable.BoolToOnOff(config.GetGlobalConfig().Performance.EnforceMPP)) + vardef.MemoryUsageAlarmRatio.Store(cfg.Instance.MemoryUsageAlarmRatio) + variable.SetSysVar(vardef.TiDBConstraintCheckInPlacePessimistic, variable.BoolToOnOff(cfg.PessimisticTxn.ConstraintCheckInPlacePessimistic)) if hostname, err := os.Hostname(); err == nil { - variable.SetSysVar(variable.Hostname, hostname) + variable.SetSysVar(vardef.Hostname, hostname) } - variable.GlobalLogMaxDays.Store(int32(config.GetGlobalConfig().Log.File.MaxDays)) + vardef.GlobalLogMaxDays.Store(int32(config.GetGlobalConfig().Log.File.MaxDays)) if cfg.Security.EnableSEM { sem.Enable() @@ -790,7 +791,7 @@ func setGlobalVars() { // For CI environment we default enable prepare-plan-cache. if config.CheckTableBeforeDrop { // only for test - variable.SetSysVar(variable.TiDBEnablePrepPlanCache, variable.BoolToOnOff(true)) + variable.SetSysVar(vardef.TiDBEnablePrepPlanCache, variable.BoolToOnOff(true)) } // use server-memory-quota as max-plan-cache-memory plannercore.PreparedPlanCacheMaxMemory.Store(cfg.Performance.ServerMemoryQuota) @@ -827,7 +828,7 @@ func setGlobalVars() { chunk.InitChunkAllocSize(cfg.TiDBMaxReuseChunk, cfg.TiDBMaxReuseColumn) if len(cfg.Instance.TiDBServiceScope) > 0 { - variable.ServiceScope.Store(strings.ToLower(cfg.Instance.TiDBServiceScope)) + vardef.ServiceScope.Store(strings.ToLower(cfg.Instance.TiDBServiceScope)) } } diff --git a/cmd/tidb-server/main_test.go b/cmd/tidb-server/main_test.go index dad9e00373a13..1ff8866c2e253 100644 --- a/cmd/tidb-server/main_test.go +++ b/cmd/tidb-server/main_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit/testsetup" "github.com/stretchr/testify/require" @@ -51,9 +52,9 @@ func TestRunMain(t *testing.T) { func TestSetGlobalVars(t *testing.T) { defer view.Stop() - require.Equal(t, "tikv,tiflash,tidb", variable.GetSysVar(variable.TiDBIsolationReadEngines).Value) - require.Equal(t, "1073741824", variable.GetSysVar(variable.TiDBMemQuotaQuery).Value) - require.NotEqual(t, "test", variable.GetSysVar(variable.Version).Value) + require.Equal(t, "tikv,tiflash,tidb", variable.GetSysVar(vardef.TiDBIsolationReadEngines).Value) + require.Equal(t, "1073741824", variable.GetSysVar(vardef.TiDBMemQuotaQuery).Value) + require.NotEqual(t, "test", variable.GetSysVar(vardef.Version).Value) config.UpdateGlobal(func(conf *config.Config) { conf.IsolationRead.Engines = []string{"tikv", "tidb"} @@ -61,9 +62,9 @@ func TestSetGlobalVars(t *testing.T) { }) setGlobalVars() - require.Equal(t, "tikv,tidb", variable.GetSysVar(variable.TiDBIsolationReadEngines).Value) - require.Equal(t, "test", variable.GetSysVar(variable.Version).Value) - require.Equal(t, variable.GetSysVar(variable.Version).Value, mysql.ServerVersion) + require.Equal(t, "tikv,tidb", variable.GetSysVar(vardef.TiDBIsolationReadEngines).Value) + require.Equal(t, "test", variable.GetSysVar(vardef.Version).Value) + require.Equal(t, variable.GetSysVar(vardef.Version).Value, mysql.ServerVersion) config.UpdateGlobal(func(conf *config.Config) { conf.ServerVersion = "" @@ -71,13 +72,13 @@ func TestSetGlobalVars(t *testing.T) { setGlobalVars() // variable.Version won't change when len(conf.ServerVersion) == 0 - require.Equal(t, "test", variable.GetSysVar(variable.Version).Value) - require.Equal(t, variable.GetSysVar(variable.Version).Value, mysql.ServerVersion) + require.Equal(t, "test", variable.GetSysVar(vardef.Version).Value) + require.Equal(t, variable.GetSysVar(vardef.Version).Value, mysql.ServerVersion) cfg := config.GetGlobalConfig() - require.Equal(t, cfg.Socket, variable.GetSysVar(variable.Socket).Value) + require.Equal(t, cfg.Socket, variable.GetSysVar(vardef.Socket).Value) if hostname, err := os.Hostname(); err == nil { - require.Equal(t, variable.GetSysVar(variable.Hostname).Value, hostname) + require.Equal(t, variable.GetSysVar(vardef.Hostname).Value, hostname) } } diff --git a/go.mod b/go.mod index bc38296352a31..4e50aefed258a 100644 --- a/go.mod +++ b/go.mod @@ -311,7 +311,7 @@ require ( google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291 // indirect google.golang.org/protobuf v1.36.1 gopkg.in/inf.v0 v0.9.1 // indirect - gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect + gopkg.in/natefinch/lumberjack.v2 v2.2.1 gopkg.in/yaml.v3 v3.0.1 // indirect k8s.io/apimachinery v0.29.11 // indirect k8s.io/klog/v2 v2.120.1 // indirect diff --git a/lightning/pkg/importer/BUILD.bazel b/lightning/pkg/importer/BUILD.bazel index 6677b7b66d5ac..50baddbc915ee 100644 --- a/lightning/pkg/importer/BUILD.bazel +++ b/lightning/pkg/importer/BUILD.bazel @@ -59,7 +59,7 @@ go_library( "//pkg/parser/mysql", "//pkg/planner/core", "//pkg/session", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/driver", "//pkg/store/driver/txn", "//pkg/table", diff --git a/lightning/pkg/importer/import.go b/lightning/pkg/importer/import.go index 2dfb86afadcaf..7b830ffac3420 100644 --- a/lightning/pkg/importer/import.go +++ b/lightning/pkg/importer/import.go @@ -58,7 +58,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/driver" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/collate" @@ -403,7 +403,7 @@ func NewImportControllerWithPauser( taskType, err := common.GetExplicitRequestSourceTypeFromDB(ctx, db) if err != nil { - return nil, errors.Annotatef(err, "get system variable '%s' failed", variable.TiDBExplicitRequestSourceType) + return nil, errors.Annotatef(err, "get system variable '%s' failed", vardef.TiDBExplicitRequestSourceType) } if taskType == "" { taskType = kvutil.ExplicitTypeLightning diff --git a/lightning/pkg/importer/tidb.go b/lightning/pkg/importer/tidb.go index c14747f37cd4c..1b07c6a6a965f 100644 --- a/lightning/pkg/importer/tidb.go +++ b/lightning/pkg/importer/tidb.go @@ -33,7 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/tikv/client-go/v2/util" "go.uber.org/zap" ) @@ -64,23 +64,23 @@ func DBFromConfig(ctx context.Context, dsn config.DBStore) (*sql.DB, error) { } vars := map[string]string{ - variable.TiDBBuildStatsConcurrency: strconv.Itoa(dsn.BuildStatsConcurrency), - variable.TiDBDistSQLScanConcurrency: strconv.Itoa(dsn.DistSQLScanConcurrency), - variable.TiDBIndexSerialScanConcurrency: strconv.Itoa(dsn.IndexSerialScanConcurrency), - variable.TiDBChecksumTableConcurrency: strconv.Itoa(dsn.ChecksumTableConcurrency), + vardef.TiDBBuildStatsConcurrency: strconv.Itoa(dsn.BuildStatsConcurrency), + vardef.TiDBDistSQLScanConcurrency: strconv.Itoa(dsn.DistSQLScanConcurrency), + vardef.TiDBIndexSerialScanConcurrency: strconv.Itoa(dsn.IndexSerialScanConcurrency), + vardef.TiDBChecksumTableConcurrency: strconv.Itoa(dsn.ChecksumTableConcurrency), // after https://github.com/pingcap/tidb/pull/17102 merge, // we need set session to true for insert auto_random value in TiDB Backend - variable.TiDBAllowAutoRandExplicitInsert: "1", + vardef.TiDBAllowAutoRandExplicitInsert: "1", // allow use _tidb_rowid in sql statement - variable.TiDBOptWriteRowID: "1", + vardef.TiDBOptWriteRowID: "1", // always set auto-commit to ON - variable.AutoCommit: "1", + vardef.AutoCommit: "1", // always set transaction mode to optimistic - variable.TiDBTxnMode: "optimistic", + vardef.TiDBTxnMode: "optimistic", // disable foreign key checks - variable.ForeignKeyChecks: "0", - variable.TiDBExplicitRequestSourceType: util.ExplicitTypeLightning, + vardef.ForeignKeyChecks: "0", + vardef.TiDBExplicitRequestSourceType: util.ExplicitTypeLightning, } if dsn.Vars != nil { diff --git a/pkg/bindinfo/BUILD.bazel b/pkg/bindinfo/BUILD.bazel index 504cc88a10250..5ace8c1221ef6 100644 --- a/pkg/bindinfo/BUILD.bazel +++ b/pkg/bindinfo/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "//pkg/planner/core/resolve", "//pkg/sessionctx", "//pkg/sessionctx/sessionstates", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/types", "//pkg/types/parser_driver", @@ -60,7 +61,7 @@ go_test( "//pkg/parser/terror", "//pkg/server", "//pkg/session/types", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/testkit/testsetup", "//pkg/util/parser", diff --git a/pkg/bindinfo/binding_cache.go b/pkg/bindinfo/binding_cache.go index ea07f75f9ee73..1ce82f558ab43 100644 --- a/pkg/bindinfo/binding_cache.go +++ b/pkg/bindinfo/binding_cache.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" ) // digestBiMap represents a bidirectional map between noDBDigest and sqlDigest, used to support cross-db binding. @@ -152,7 +152,7 @@ type bindingCache struct { func newBindCache() BindingCache { cache, _ := ristretto.NewCache(&ristretto.Config{ NumCounters: 1e6, - MaxCost: variable.MemQuotaBindingCache.Load(), + MaxCost: vardef.MemQuotaBindingCache.Load(), BufferItems: 64, Cost: func(value any) int64 { return int64(value.(*Binding).size()) diff --git a/pkg/bindinfo/binding_cache_test.go b/pkg/bindinfo/binding_cache_test.go index 1af7bffd3c2e3..0d858ec36273f 100644 --- a/pkg/bindinfo/binding_cache_test.go +++ b/pkg/bindinfo/binding_cache_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/stretchr/testify/require" ) @@ -75,9 +75,9 @@ func TestBindCache(t *testing.T) { binding := &Binding{BindSQL: "SELECT * FROM t1"} kvSize := int(binding.size()) defer func(v int64) { - variable.MemQuotaBindingCache.Store(v) - }(variable.MemQuotaBindingCache.Load()) - variable.MemQuotaBindingCache.Store(int64(kvSize*3) - 1) + vardef.MemQuotaBindingCache.Store(v) + }(vardef.MemQuotaBindingCache.Load()) + vardef.MemQuotaBindingCache.Store(int64(kvSize*3) - 1) bindCache := newBindCache() defer bindCache.Close() diff --git a/pkg/bindinfo/global_handle.go b/pkg/bindinfo/global_handle.go index 3f6e3145808ee..3414dd325ad76 100644 --- a/pkg/bindinfo/global_handle.go +++ b/pkg/bindinfo/global_handle.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -509,8 +510,8 @@ var ( ) // GetScope gets the status variables scope. -func (*globalBindingHandle) GetScope(_ string) variable.ScopeFlag { - return variable.ScopeSession +func (*globalBindingHandle) GetScope(_ string) vardef.ScopeFlag { + return vardef.ScopeSession } // Stats returns the server statistics. diff --git a/pkg/config/BUILD.bazel b/pkg/config/BUILD.bazel index f3bf88df15e7a..b891e42732e9e 100644 --- a/pkg/config/BUILD.bazel +++ b/pkg/config/BUILD.bazel @@ -7,13 +7,13 @@ go_library( "config_util.go", "const.go", "store.go", + "tiflash.go", ], importpath = "github.com/pingcap/tidb/pkg/config", visibility = ["//visibility:public"], deps = [ "//pkg/parser/terror", "//pkg/util/logutil", - "//pkg/util/tiflashcompute", "//pkg/util/tikvutil", "//pkg/util/versioninfo", "@com_github_burntsushi_toml//:toml", diff --git a/pkg/config/config.go b/pkg/config/config.go index ac3bf7927b631..34523e8bf28d7 100644 --- a/pkg/config/config.go +++ b/pkg/config/config.go @@ -35,7 +35,6 @@ import ( zaplog "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/util/logutil" - "github.com/pingcap/tidb/pkg/util/tiflashcompute" "github.com/pingcap/tidb/pkg/util/tikvutil" "github.com/pingcap/tidb/pkg/util/versioninfo" tikvcfg "github.com/tikv/client-go/v2/config" @@ -1053,8 +1052,8 @@ var defaultConf = Config{ Enable32BitsConnectionID: true, TrxSummary: DefaultTrxSummary(), DisaggregatedTiFlash: false, - TiFlashComputeAutoScalerType: tiflashcompute.DefASStr, - TiFlashComputeAutoScalerAddr: tiflashcompute.DefAWSAutoScalerAddr, + TiFlashComputeAutoScalerType: DefASStr, + TiFlashComputeAutoScalerAddr: DefAWSAutoScalerAddr, IsTiFlashComputeFixedPool: false, AutoScalerClusterID: "", UseAutoScaler: false, @@ -1377,9 +1376,9 @@ func (c *Config) Valid() error { // Check tiflash_compute topo fetch is valid. if c.DisaggregatedTiFlash && c.UseAutoScaler { - if !tiflashcompute.IsValidAutoScalerConfig(c.TiFlashComputeAutoScalerType) { + if !IsValidAutoScalerConfig(c.TiFlashComputeAutoScalerType) { return fmt.Errorf("invalid AutoScaler type, expect %s, %s or %s, got %s", - tiflashcompute.MockASStr, tiflashcompute.AWSASStr, tiflashcompute.GCPASStr, c.TiFlashComputeAutoScalerType) + MockASStr, AWSASStr, GCPASStr, c.TiFlashComputeAutoScalerType) } if c.TiFlashComputeAutoScalerAddr == "" { return fmt.Errorf("autoscaler-addr cannot be empty when disaggregated-tiflash mode is true") diff --git a/pkg/config/tiflash.go b/pkg/config/tiflash.go new file mode 100644 index 0000000000000..1219636573d72 --- /dev/null +++ b/pkg/config/tiflash.go @@ -0,0 +1,70 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package config + +const ( + // MockASStr is string value for mock AutoScaler. + MockASStr = "mock" + // AWSASStr is string value for aws AutoScaler. + AWSASStr = "aws" + // GCPASStr is string value for gcp AutoScaler. + GCPASStr = "gcp" + // TestASStr is string value for test AutoScaler. + TestASStr = "test" + // InvalidASStr is string value for invalid AutoScaler. + InvalidASStr = "invalid" +) + +const ( + // DefAWSAutoScalerAddr is default address for aws AutoScaler. + DefAWSAutoScalerAddr = "tiflash-autoscale-lb.tiflash-autoscale.svc.cluster.local:8081" + // DefASStr is default AutoScaler. + DefASStr = AWSASStr +) + +const ( + // MockASType is int value for mock AutoScaler. + MockASType int = iota + // AWSASType is int value for aws AutoScaler. + AWSASType + // GCPASType is int value for gcp AutoScaler. + GCPASType + // TestASType is for local tidb test AutoScaler. + TestASType + // InvalidASType is int value for invalid check. + InvalidASType +) + +// IsValidAutoScalerConfig return true if user config of autoscaler type is valid. +func IsValidAutoScalerConfig(typ string) bool { + t := GetAutoScalerType(typ) + return t == MockASType || t == AWSASType || t == GCPASType +} + +// GetAutoScalerType return topo fetcher type. +func GetAutoScalerType(typ string) int { + switch typ { + case MockASStr: + return MockASType + case AWSASStr: + return AWSASType + case GCPASStr: + return GCPASType + case TestASStr: + return TestASType + default: + return InvalidASType + } +} diff --git a/pkg/ddl/BUILD.bazel b/pkg/ddl/BUILD.bazel index 04a8a9677e77a..2dc23911008ce 100644 --- a/pkg/ddl/BUILD.bazel +++ b/pkg/ddl/BUILD.bazel @@ -130,6 +130,7 @@ go_library( "//pkg/resourcemanager/util", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/statistics", @@ -179,6 +180,7 @@ go_library( "//pkg/util/timeutil", "//pkg/util/topsql", "//pkg/util/topsql/state", + "//pkg/util/tracing", "@com_github_coreos_go_semver//semver", "@com_github_docker_go_units//:go-units", "@com_github_google_uuid//:uuid", @@ -322,6 +324,7 @@ go_test( "//pkg/session", "//pkg/session/types", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/store/gcworker", diff --git a/pkg/ddl/add_column.go b/pkg/ddl/add_column.go index 5244d3d825f09..e1ed88b82c8e1 100644 --- a/pkg/ddl/add_column.go +++ b/pkg/ddl/add_column.go @@ -40,7 +40,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" field_types "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" driver "github.com/pingcap/tidb/pkg/types/parser_driver" @@ -554,7 +554,7 @@ func columnDefToCol(ctx *metabuild.Context, offset int, colDef *ast.ColumnDef, o case ast.ColumnOptionFulltext: ctx.AppendWarning(dbterror.ErrTableCantHandleFt.FastGenByArgs()) case ast.ColumnOptionCheck: - if !variable.EnableCheckConstraint.Load() { + if !vardef.EnableCheckConstraint.Load() { ctx.AppendWarning(errCheckConstraintIsOff) } else { // Check the column CHECK constraint dependency lazily, after fill all the name. diff --git a/pkg/ddl/backfilling.go b/pkg/ddl/backfilling.go index 702ff7e595d50..bff0afbd1011b 100644 --- a/pkg/ddl/backfilling.go +++ b/pkg/ddl/backfilling.go @@ -40,7 +40,7 @@ import ( "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/terror" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/util" @@ -211,7 +211,7 @@ func newBackfillCtx(id int, rInfo *reorgInfo, } } - batchCnt := rInfo.ReorgMeta.GetBatchSizeOrDefault(int(variable.GetDDLReorgBatchSize())) + batchCnt := rInfo.ReorgMeta.GetBatchSize() return &backfillCtx{ id: id, ddlCtx: rInfo.jobCtx.oldDDLCtx, @@ -239,7 +239,7 @@ func getIdxNamesFromArgs(args *model.ModifyIndexArgs) string { } func updateTxnEntrySizeLimitIfNeeded(txn kv.Transaction) { - if entrySizeLimit := variable.TxnEntrySizeLimit.Load(); entrySizeLimit > 0 { + if entrySizeLimit := vardef.TxnEntrySizeLimit.Load(); entrySizeLimit > 0 { txn.SetOption(kv.SizeLimits, kv.TxnSizeLimits{ Entry: entrySizeLimit, Total: kv.TxnTotalSizeLimit.Load(), @@ -458,7 +458,7 @@ func (w *backfillWorker) run(d *ddlCtx, bf backfiller, job *model.Job) { // Change the batch size dynamically. currentBatchCnt := w.GetCtx().batchCnt - targetBatchSize := job.ReorgMeta.GetBatchSizeOrDefault(int(variable.GetDDLReorgBatchSize())) + targetBatchSize := job.ReorgMeta.GetBatchSize() if targetBatchSize != currentBatchCnt { w.GetCtx().batchCnt = targetBatchSize logger.Info("adjust ddl job config success", @@ -664,7 +664,7 @@ var ( // TestCheckWorkerNumCh use for test adjust backfill worker. TestCheckWorkerNumCh = make(chan *sync.WaitGroup) // TestCheckWorkerNumber use for test adjust backfill worker. - TestCheckWorkerNumber = int32(variable.DefTiDBDDLReorgWorkerCount) + TestCheckWorkerNumber = int32(vardef.DefTiDBDDLReorgWorkerCount) // TestCheckReorgTimeout is used to mock timeout when reorg data. TestCheckReorgTimeout = int32(0) ) @@ -765,7 +765,7 @@ func (dc *ddlCtx) addIndexWithLocalIngest( zap.Int64s("index IDs", indexIDs)) return errors.Trace(err) } - importConc := job.ReorgMeta.GetConcurrencyOrDefault(int(variable.GetDDLReorgWorkerCounter())) + importConc := job.ReorgMeta.GetConcurrency() pipe, err := NewAddIndexIngestPipeline( opCtx, dc.store, @@ -823,7 +823,7 @@ func adjustWorkerCntAndMaxWriteSpeed(ctx context.Context, pipe *operator.AsyncPi case <-ctx.Done(): return case <-ticker.C: - maxWriteSpeed := job.ReorgMeta.GetMaxWriteSpeedOrDefault() + maxWriteSpeed := job.ReorgMeta.GetMaxWriteSpeed() if maxWriteSpeed != bcCtx.GetLocalBackend().GetWriteSpeedLimit() { bcCtx.GetLocalBackend().UpdateWriteSpeedLimit(maxWriteSpeed) logutil.DDLIngestLogger().Info("adjust ddl job config success", @@ -831,7 +831,7 @@ func adjustWorkerCntAndMaxWriteSpeed(ctx context.Context, pipe *operator.AsyncPi zap.Int("max write speed", bcCtx.GetLocalBackend().GetWriteSpeedLimit())) } - concurrency := job.ReorgMeta.GetConcurrencyOrDefault(int(variable.GetDDLReorgWorkerCounter())) + concurrency := job.ReorgMeta.GetConcurrency() targetReaderCnt, targetWriterCnt := expectedIngestWorkerCnt(concurrency, avgRowSize) currentReaderCnt, currentWriterCnt := reader.GetWorkerPoolSize(), writer.GetWorkerPoolSize() if int32(targetReaderCnt) != currentReaderCnt || int32(targetWriterCnt) != currentWriterCnt { @@ -1060,7 +1060,7 @@ func (dc *ddlCtx) writePhysicalTableRecord( break outer case <-ticker.C: currentWorkerCnt := exec.currentWorkerSize() - targetWorkerCnt := reorgInfo.ReorgMeta.GetConcurrencyOrDefault(int(variable.GetDDLReorgWorkerCounter())) + targetWorkerCnt := reorgInfo.ReorgMeta.GetConcurrency() if currentWorkerCnt != targetWorkerCnt { err := exec.adjustWorkerSize() if err != nil { @@ -1209,7 +1209,7 @@ func mergeWarningsAndWarningsCount(partWarnings, totalWarnings map[errors.ErrorI func logSlowOperations(elapsed time.Duration, slowMsg string, threshold uint32) { if threshold == 0 { - threshold = atomic.LoadUint32(&variable.DDLSlowOprThreshold) + threshold = atomic.LoadUint32(&vardef.DDLSlowOprThreshold) } if elapsed >= time.Duration(threshold)*time.Millisecond { diff --git a/pkg/ddl/backfilling_operators.go b/pkg/ddl/backfilling_operators.go index fd71382a4f159..03cff6dbdc749 100644 --- a/pkg/ddl/backfilling_operators.go +++ b/pkg/ddl/backfilling_operators.go @@ -42,7 +42,6 @@ import ( "github.com/pingcap/tidb/pkg/resourcemanager/pool/workerpool" "github.com/pingcap/tidb/pkg/resourcemanager/util" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/tablecodec" @@ -181,7 +180,7 @@ func NewAddIndexIngestPipeline( srcOp := NewTableScanTaskSource(ctx, store, tbl, startKey, endKey, backendCtx) scanOp := NewTableScanOperator(ctx, sessPool, copCtx, srcChkPool, readerCnt, - reorgMeta.GetBatchSizeOrDefault(int(variable.GetDDLReorgBatchSize())), rm, backendCtx) + reorgMeta.GetBatchSize(), rm, backendCtx) ingestOp := NewIndexIngestOperator(ctx, copCtx, backendCtx, sessPool, tbl, indexes, engines, srcChkPool, writerCnt, reorgMeta, rowCntListener) sinkOp := newIndexWriteResultSink(ctx, backendCtx, tbl, indexes, rowCntListener) @@ -247,7 +246,7 @@ func NewWriteIndexToExternalStoragePipeline( srcOp := NewTableScanTaskSource(ctx, store, tbl, startKey, endKey, nil) scanOp := NewTableScanOperator(ctx, sessPool, copCtx, srcChkPool, readerCnt, - reorgMeta.GetBatchSizeOrDefault(int(variable.GetDDLReorgBatchSize())), nil, nil) + reorgMeta.GetBatchSize(), nil, nil) writeOp := NewWriteExternalStoreOperator( ctx, copCtx, sessPool, jobID, subtaskID, tbl, indexes, extStore, srcChkPool, writerCnt, @@ -276,7 +275,7 @@ func createChunkPool(copCtx copr.CopContext, reorgMeta *model.DDLReorgMeta) *syn return &sync.Pool{ New: func() any { return chunk.NewChunkWithCapacity(copCtx.GetBase().FieldTypes, - reorgMeta.GetBatchSizeOrDefault(int(variable.GetDDLReorgBatchSize()))) + reorgMeta.GetBatchSize()) }, } } @@ -601,7 +600,7 @@ func (w *tableScanWorker) scanRecords(task TableScanTask, sender func(IndexRecor func (w *tableScanWorker) getChunk() *chunk.Chunk { targetCap := ingest.CopReadBatchSize(w.hintBatchSize) if w.reorgMeta != nil { - targetCap = ingest.CopReadBatchSize(w.reorgMeta.GetBatchSizeOrDefault(int(variable.GetDDLReorgBatchSize()))) + targetCap = ingest.CopReadBatchSize(w.reorgMeta.GetBatchSize()) } chk := w.srcChkPool.Get().(*chunk.Chunk) if chk.Capacity() != targetCap { diff --git a/pkg/ddl/backfilling_test.go b/pkg/ddl/backfilling_test.go index aedf19e7dd814..800d11c940b04 100644 --- a/pkg/ddl/backfilling_test.go +++ b/pkg/ddl/backfilling_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" @@ -269,8 +270,8 @@ func TestReorgExprContext(t *testing.T) { } func TestReorgTableMutateContext(t *testing.T) { - originalRowFmt := variable.GetDDLReorgRowFormat() - defer variable.SetDDLReorgRowFormat(originalRowFmt) + originalRowFmt := vardef.GetDDLReorgRowFormat() + defer vardef.SetDDLReorgRowFormat(originalRowFmt) exprCtx := exprstatic.NewExprContext() @@ -285,14 +286,14 @@ func TestReorgTableMutateContext(t *testing.T) { require.Equal(t, variable.AssertionLevelOff, ctx.TxnAssertionLevel()) require.Equal(t, sctxTblCtx.TxnAssertionLevel(), ctx.TxnAssertionLevel()) - require.Equal(t, variable.GetDDLReorgRowFormat() != variable.DefTiDBRowFormatV1, ctx.GetRowEncodingConfig().IsRowLevelChecksumEnabled) - require.Equal(t, variable.GetDDLReorgRowFormat() != variable.DefTiDBRowFormatV1, ctx.GetRowEncodingConfig().RowEncoder.Enable) + require.Equal(t, vardef.GetDDLReorgRowFormat() != vardef.DefTiDBRowFormatV1, ctx.GetRowEncodingConfig().IsRowLevelChecksumEnabled) + require.Equal(t, vardef.GetDDLReorgRowFormat() != vardef.DefTiDBRowFormatV1, ctx.GetRowEncodingConfig().RowEncoder.Enable) require.Equal(t, sctxTblCtx.GetRowEncodingConfig(), ctx.GetRowEncodingConfig()) require.NotNil(t, ctx.GetMutateBuffers()) require.Equal(t, sctxTblCtx.GetMutateBuffers(), ctx.GetMutateBuffers()) - require.Equal(t, variable.DefTiDBShardAllocateStep, ctx.GetRowIDShardGenerator().GetShardStep()) + require.Equal(t, vardef.DefTiDBShardAllocateStep, ctx.GetRowIDShardGenerator().GetShardStep()) sctx.GetSessionVars().TxnCtx.StartTS = 123 // make sure GetRowIDShardGenerator() pass assert require.Equal(t, sctxTblCtx.GetRowIDShardGenerator().GetShardStep(), ctx.GetRowIDShardGenerator().GetShardStep()) require.GreaterOrEqual(t, ctx.GetRowIDShardGenerator().GetCurrentShard(1), int64(0)) @@ -319,7 +320,7 @@ func TestReorgTableMutateContext(t *testing.T) { } // test when the row format is v1 - variable.SetDDLReorgRowFormat(variable.DefTiDBRowFormatV1) + vardef.SetDDLReorgRowFormat(vardef.DefTiDBRowFormatV1) sctx := newMockReorgSessCtx(&mockStorage{client: &mock.Client{}}) require.NoError(t, initSessCtx(sctx, &model.DDLReorgMeta{})) ctx := newReorgTableMutateContext(exprCtx) diff --git a/pkg/ddl/backfilling_txn_executor.go b/pkg/ddl/backfilling_txn_executor.go index a7d9552a72c05..49f5efb414bf9 100644 --- a/pkg/ddl/backfilling_txn_executor.go +++ b/pkg/ddl/backfilling_txn_executor.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/resourcegroup" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" contextutil "github.com/pingcap/tidb/pkg/util/context" "github.com/pingcap/tidb/pkg/util/execdetails" @@ -88,7 +88,7 @@ func newTxnBackfillExecutor(ctx context.Context, info *reorgInfo, sessPool *sess if err != nil { return nil, err } - workerCnt := info.ReorgMeta.GetConcurrencyOrDefault(int(variable.GetDDLReorgWorkerCounter())) + workerCnt := info.ReorgMeta.GetConcurrency() return &txnBackfillExecutor{ ctx: ctx, reorgInfo: info, @@ -163,20 +163,20 @@ func newDefaultReorgDistSQLCtx(kvClient kv.Client, warnHandler contextutil.WarnA WarnHandler: warnHandler, Client: kvClient, EnableChunkRPC: true, - EnabledRateLimitAction: variable.DefTiDBEnableRateLimitAction, + EnabledRateLimitAction: vardef.DefTiDBEnableRateLimitAction, KVVars: tikvstore.NewVariables(&sqlKiller.Signal), SessionMemTracker: memory.NewTracker(memory.LabelForSession, -1), Location: time.UTC, SQLKiller: &sqlKiller, CPUUsage: &cpuUsages, ErrCtx: errctx.NewContextWithLevels(stmtctx.DefaultStmtErrLevels, warnHandler), - TiFlashReplicaRead: tiflash.GetTiFlashReplicaReadByStr(variable.DefTiFlashReplicaRead), - TiFlashMaxThreads: variable.DefTiFlashMaxThreads, - TiFlashMaxBytesBeforeExternalJoin: variable.DefTiFlashMaxBytesBeforeExternalJoin, - TiFlashMaxBytesBeforeExternalGroupBy: variable.DefTiFlashMaxBytesBeforeExternalGroupBy, - TiFlashMaxBytesBeforeExternalSort: variable.DefTiFlashMaxBytesBeforeExternalSort, - TiFlashMaxQueryMemoryPerNode: variable.DefTiFlashMemQuotaQueryPerNode, - TiFlashQuerySpillRatio: variable.DefTiFlashQuerySpillRatio, + TiFlashReplicaRead: tiflash.GetTiFlashReplicaReadByStr(vardef.DefTiFlashReplicaRead), + TiFlashMaxThreads: vardef.DefTiFlashMaxThreads, + TiFlashMaxBytesBeforeExternalJoin: vardef.DefTiFlashMaxBytesBeforeExternalJoin, + TiFlashMaxBytesBeforeExternalGroupBy: vardef.DefTiFlashMaxBytesBeforeExternalGroupBy, + TiFlashMaxBytesBeforeExternalSort: vardef.DefTiFlashMaxBytesBeforeExternalSort, + TiFlashMaxQueryMemoryPerNode: vardef.DefTiFlashMemQuotaQueryPerNode, + TiFlashQuerySpillRatio: vardef.DefTiFlashQuerySpillRatio, ResourceGroupName: resourcegroup.DefaultResourceGroupName, ExecDetails: &execDetails, } @@ -202,8 +202,8 @@ func initSessCtx(sessCtx sessionctx.Context, reorgMeta *model.DDLReorgMeta) erro sessCtx.GetSessionVars().StmtCtx.SetTimeZone(&tz) // Set the row encode format version. - rowFormat := variable.GetDDLReorgRowFormat() - sessCtx.GetSessionVars().RowEncoder.Enable = rowFormat != variable.DefTiDBRowFormatV1 + rowFormat := vardef.GetDDLReorgRowFormat() + sessCtx.GetSessionVars().RowEncoder.Enable = rowFormat != vardef.DefTiDBRowFormatV1 // Simulate the sql mode environment in the worker sessionCtx. sqlMode := reorgMeta.SQLMode sessCtx.GetSessionVars().SQLMode = sqlMode @@ -248,7 +248,7 @@ func restoreSessCtx(sessCtx sessionctx.Context) func(sessCtx sessionctx.Context) } func (b *txnBackfillExecutor) expectedWorkerSize() (size int) { - workerCnt := b.reorgInfo.ReorgMeta.GetConcurrencyOrDefault(int(variable.GetDDLReorgWorkerCounter())) + workerCnt := b.reorgInfo.ReorgMeta.GetConcurrency() return min(workerCnt, maxBackfillWorkerSize) } diff --git a/pkg/ddl/cluster.go b/pkg/ddl/cluster.go index e38326cd18d43..342d68cbe4a77 100644 --- a/pkg/ddl/cluster.go +++ b/pkg/ddl/cluster.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" @@ -162,9 +163,9 @@ func getGlobalSysVarAsBool(sess sessionctx.Context, name string) (bool, error) { } func setGlobalSysVarFromBool(ctx context.Context, sess sessionctx.Context, name string, value bool) error { - sv := variable.On + sv := vardef.On if !value { - sv = variable.Off + sv = vardef.Off } return sess.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(ctx, name, sv) @@ -207,13 +208,13 @@ func checkAndSetFlashbackClusterInfo(ctx context.Context, se sessionctx.Context, if err = closePDSchedule(ctx); err != nil { return err } - if err = setGlobalSysVarFromBool(ctx, se, variable.TiDBEnableAutoAnalyze, false); err != nil { + if err = setGlobalSysVarFromBool(ctx, se, vardef.TiDBEnableAutoAnalyze, false); err != nil { return err } - if err = setGlobalSysVarFromBool(ctx, se, variable.TiDBSuperReadOnly, true); err != nil { + if err = setGlobalSysVarFromBool(ctx, se, vardef.TiDBSuperReadOnly, true); err != nil { return err } - if err = setGlobalSysVarFromBool(ctx, se, variable.TiDBTTLJobEnable, false); err != nil { + if err = setGlobalSysVarFromBool(ctx, se, vardef.TiDBTTLJobEnable, false); err != nil { return err } @@ -651,7 +652,7 @@ func flashbackToVersion( return rangetask.NewRangeTaskRunner( "flashback-to-version-runner", store.(tikv.Storage), - int(variable.GetDDLFlashbackConcurrency()), + int(vardef.GetDDLFlashbackConcurrency()), handler, ).RunOnRange(ctx, startKey, endKey) } @@ -718,19 +719,19 @@ func (w *worker) onFlashbackCluster(jobCtx *jobContext, job *model.Job) (ver int return ver, errors.Trace(err) } - args.EnableAutoAnalyze, err = getGlobalSysVarAsBool(sess, variable.TiDBEnableAutoAnalyze) + args.EnableAutoAnalyze, err = getGlobalSysVarAsBool(sess, vardef.TiDBEnableAutoAnalyze) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } - args.SuperReadOnly, err = getGlobalSysVarAsBool(sess, variable.TiDBSuperReadOnly) + args.SuperReadOnly, err = getGlobalSysVarAsBool(sess, vardef.TiDBSuperReadOnly) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } - args.EnableTTLJob, err = getGlobalSysVarAsBool(sess, variable.TiDBTTLJobEnable) + args.EnableTTLJob, err = getGlobalSysVarAsBool(sess, vardef.TiDBTTLJobEnable) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -866,18 +867,18 @@ func finishFlashbackCluster(w *worker, job *model.Job) error { } } - if err = setGlobalSysVarFromBool(w.workCtx, sess, variable.TiDBSuperReadOnly, args.SuperReadOnly); err != nil { + if err = setGlobalSysVarFromBool(w.workCtx, sess, vardef.TiDBSuperReadOnly, args.SuperReadOnly); err != nil { return errors.Trace(err) } if job.IsCancelled() { // only restore `tidb_ttl_job_enable` when flashback failed - if err = setGlobalSysVarFromBool(w.workCtx, sess, variable.TiDBTTLJobEnable, args.EnableTTLJob); err != nil { + if err = setGlobalSysVarFromBool(w.workCtx, sess, vardef.TiDBTTLJobEnable, args.EnableTTLJob); err != nil { return errors.Trace(err) } } - if err := setGlobalSysVarFromBool(w.workCtx, sess, variable.TiDBEnableAutoAnalyze, args.EnableAutoAnalyze); err != nil { + if err := setGlobalSysVarFromBool(w.workCtx, sess, vardef.TiDBEnableAutoAnalyze, args.EnableAutoAnalyze); err != nil { return errors.Trace(err) } diff --git a/pkg/ddl/cluster_test.go b/pkg/ddl/cluster_test.go index 1e77dfa08dc19..75012b9ac8203 100644 --- a/pkg/ddl/cluster_test.go +++ b/pkg/ddl/cluster_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" "github.com/pingcap/tidb/pkg/types" @@ -140,16 +140,16 @@ func TestGlobalVariablesOnFlashback(t *testing.T) { if job.SchemaState == model.StateWriteReorganization { rs, err := tk.Exec("show variables like 'tidb_gc_enable'") assert.NoError(t, err) - assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) + assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], vardef.Off) rs, err = tk.Exec("show variables like 'tidb_enable_auto_analyze'") assert.NoError(t, err) - assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) + assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], vardef.Off) rs, err = tk.Exec("show variables like 'tidb_super_read_only'") assert.NoError(t, err) - assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.On) + assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], vardef.On) rs, err = tk.Exec("show variables like 'tidb_ttl_job_enable'") assert.NoError(t, err) - assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) + assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], vardef.Off) } }) // first try with `tidb_gc_enable` = on and `tidb_super_read_only` = off and `tidb_ttl_job_enable` = on @@ -161,13 +161,13 @@ func TestGlobalVariablesOnFlashback(t *testing.T) { rs, err := tk.Exec("show variables like 'tidb_super_read_only'") require.NoError(t, err) - require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) + require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], vardef.Off) rs, err = tk.Exec("show variables like 'tidb_gc_enable'") require.NoError(t, err) - require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.On) + require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], vardef.On) rs, err = tk.Exec("show variables like 'tidb_ttl_job_enable'") require.NoError(t, err) - require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) + require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], vardef.Off) // second try with `tidb_gc_enable` = off and `tidb_super_read_only` = on and `tidb_ttl_job_enable` = off tk.MustExec("set global tidb_gc_enable = off") @@ -179,13 +179,13 @@ func TestGlobalVariablesOnFlashback(t *testing.T) { tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts).Format(types.TimeFSPFormat))) rs, err = tk.Exec("show variables like 'tidb_super_read_only'") require.NoError(t, err) - require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.On) + require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], vardef.On) rs, err = tk.Exec("show variables like 'tidb_gc_enable'") require.NoError(t, err) - require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) + require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], vardef.Off) rs, err = tk.Exec("show variables like 'tidb_ttl_job_enable'") assert.NoError(t, err) - assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) + assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], vardef.Off) testfailpoint.Disable(t, "github.com/pingcap/tidb/pkg/ddl/beforeRunOneJobStep") require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/mockFlashbackTest")) @@ -220,7 +220,7 @@ func TestCancelFlashbackCluster(t *testing.T) { rs, err := tk.Exec("show variables like 'tidb_ttl_job_enable'") assert.NoError(t, err) - assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.On) + assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], vardef.On) // Try canceled on StateWriteReorganization, cancel failed hook = newCancelJobHook(t, store, func(job *model.Job) bool { @@ -232,7 +232,7 @@ func TestCancelFlashbackCluster(t *testing.T) { rs, err = tk.Exec("show variables like 'tidb_ttl_job_enable'") assert.NoError(t, err) - assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) + assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], vardef.Off) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/mockFlashbackTest")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/injectSafeTS")) diff --git a/pkg/ddl/column.go b/pkg/ddl/column.go index e609e6511c957..5410b4d437f1c 100644 --- a/pkg/ddl/column.go +++ b/pkg/ddl/column.go @@ -39,7 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" @@ -637,9 +637,9 @@ func newUpdateColumnWorker(id int, t table.PhysicalTable, decodeColMap map[int64 oldCol, newCol := getOldAndNewColumnsForUpdateColumn(t, reorgInfo.currElement.ID) rowDecoder := decoder.NewRowDecoder(t, t.WritableCols(), decodeColMap) failpoint.Inject("forceRowLevelChecksumOnUpdateColumnBackfill", func() { - orig := variable.EnableRowLevelChecksum.Load() - defer variable.EnableRowLevelChecksum.Store(orig) - variable.EnableRowLevelChecksum.Store(true) + orig := vardef.EnableRowLevelChecksum.Load() + defer vardef.EnableRowLevelChecksum.Store(orig) + vardef.EnableRowLevelChecksum.Store(true) }) return &updateColumnWorker{ backfillCtx: bCtx, @@ -647,7 +647,7 @@ func newUpdateColumnWorker(id int, t table.PhysicalTable, decodeColMap map[int64 newColInfo: newCol, rowDecoder: rowDecoder, rowMap: make(map[int64]types.Datum, len(decodeColMap)), - checksumNeeded: variable.EnableRowLevelChecksum.Load(), + checksumNeeded: vardef.EnableRowLevelChecksum.Load(), }, nil } diff --git a/pkg/ddl/create_table.go b/pkg/ddl/create_table.go index 688fd9df895f1..8223ba43bf069 100644 --- a/pkg/ddl/create_table.go +++ b/pkg/ddl/create_table.go @@ -42,7 +42,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/format" "github.com/pingcap/tidb/pkg/parser/mysql" field_types "github.com/pingcap/tidb/pkg/parser/types" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" @@ -874,8 +874,8 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err return dbterror.ErrUnsupportedShardRowIDBits } tbInfo.ShardRowIDBits = op.UintValue - if tbInfo.ShardRowIDBits > variable.MaxShardRowIDBits { - tbInfo.ShardRowIDBits = variable.MaxShardRowIDBits + if tbInfo.ShardRowIDBits > vardef.MaxShardRowIDBits { + tbInfo.ShardRowIDBits = vardef.MaxShardRowIDBits } tbInfo.MaxShardRowIDBits = tbInfo.ShardRowIDBits case ast.TableOptionPreSplitRegion: @@ -1326,7 +1326,7 @@ func BuildTableInfo( // check constraint if constr.Tp == ast.ConstraintCheck { - if !variable.EnableCheckConstraint.Load() { + if !vardef.EnableCheckConstraint.Load() { ctx.AppendWarning(errCheckConstraintIsOff) continue } @@ -1582,12 +1582,12 @@ func isSingleIntPK(constr *ast.Constraint, lastCol *model.ColumnInfo) bool { } // ShouldBuildClusteredIndex is used to determine whether the CREATE TABLE statement should build a clustered index table. -func ShouldBuildClusteredIndex(mode variable.ClusteredIndexDefMode, opt *ast.IndexOption, isSingleIntPK bool) bool { +func ShouldBuildClusteredIndex(mode vardef.ClusteredIndexDefMode, opt *ast.IndexOption, isSingleIntPK bool) bool { if opt == nil || opt.PrimaryKeyTp == ast.PrimaryKeyTypeDefault { switch mode { - case variable.ClusteredIndexDefModeOn: + case vardef.ClusteredIndexDefModeOn: return true - case variable.ClusteredIndexDefModeIntOnly: + case vardef.ClusteredIndexDefModeIntOnly: return !config.GetGlobalConfig().AlterPrimaryKey && isSingleIntPK default: return false diff --git a/pkg/ddl/db_integration_test.go b/pkg/ddl/db_integration_test.go index 89a367aac28eb..46aba2e75f6d0 100644 --- a/pkg/ddl/db_integration_test.go +++ b/pkg/ddl/db_integration_test.go @@ -42,7 +42,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/tablecodec" @@ -2379,7 +2379,7 @@ func TestDuplicateErrorMessage(t *testing.T) { for _, newCollate := range []bool{false, true} { collate.SetNewCollationEnabledForTest(newCollate) - for _, clusteredIndex := range []variable.ClusteredIndexDefMode{variable.ClusteredIndexDefModeOn, variable.ClusteredIndexDefModeOff, variable.ClusteredIndexDefModeIntOnly} { + for _, clusteredIndex := range []vardef.ClusteredIndexDefMode{vardef.ClusteredIndexDefModeOn, vardef.ClusteredIndexDefModeOff, vardef.ClusteredIndexDefModeIntOnly} { tk.Session().GetSessionVars().EnableClusteredIndex = clusteredIndex for _, t := range tests { tk.MustExec("drop table if exists t;") diff --git a/pkg/ddl/db_test.go b/pkg/ddl/db_test.go index 2327afd517d08..0e6b8ba249c43 100644 --- a/pkg/ddl/db_test.go +++ b/pkg/ddl/db_test.go @@ -41,7 +41,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" parsertypes "github.com/pingcap/tidb/pkg/parser/types" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/external" @@ -408,7 +408,7 @@ func TestAddIndexFailOnCaseWhenCanExit(t *testing.T) { }() store := testkit.CreateMockStoreWithSchemaLease(t, dbTestLease) tk := testkit.NewTestKit(t, store) - originalVal := variable.GetDDLErrorCountLimit() + originalVal := vardef.GetDDLErrorCountLimit() tk.MustExec("set @@global.tidb_ddl_error_count_limit = 1") defer tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_error_count_limit = %d", originalVal)) @@ -1169,16 +1169,16 @@ func TestAdminAlterDDLJobUpdateSysTable(t *testing.T) { insertMockJob2Table(tk, &job) tk.MustExec(fmt.Sprintf("admin alter ddl jobs %d thread = 8;", job.ID)) j := getJobMetaByID(t, tk, job.ID) - require.Equal(t, j.ReorgMeta.GetConcurrencyOrDefault(int(variable.GetDDLReorgWorkerCounter())), 8) + require.Equal(t, 8, j.ReorgMeta.GetConcurrency()) tk.MustExec(fmt.Sprintf("admin alter ddl jobs %d batch_size = 256;", job.ID)) j = getJobMetaByID(t, tk, job.ID) - require.Equal(t, j.ReorgMeta.GetBatchSizeOrDefault(int(variable.GetDDLReorgBatchSize())), 256) + require.Equal(t, 256, j.ReorgMeta.GetBatchSize()) tk.MustExec(fmt.Sprintf("admin alter ddl jobs %d thread = 16, batch_size = 512;", job.ID)) j = getJobMetaByID(t, tk, job.ID) - require.Equal(t, j.ReorgMeta.GetConcurrencyOrDefault(int(variable.GetDDLReorgWorkerCounter())), 16) - require.Equal(t, j.ReorgMeta.GetBatchSizeOrDefault(int(variable.GetDDLReorgBatchSize())), 512) + require.Equal(t, 16, j.ReorgMeta.GetConcurrency()) + require.Equal(t, 512, j.ReorgMeta.GetBatchSize()) deleteJobMetaByID(tk, job.ID) } diff --git a/pkg/ddl/ddl.go b/pkg/ddl/ddl.go index 645f1bc40246c..f4bebca51f832 100644 --- a/pkg/ddl/ddl.go +++ b/pkg/ddl/ddl.go @@ -55,6 +55,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics/handle" tidbutil "github.com/pingcap/tidb/pkg/util" @@ -180,7 +181,7 @@ type DDL interface { // Stats returns the DDL statistics. Stats(vars *variable.SessionVars) (map[string]any, error) // GetScope gets the status variables scope. - GetScope(status string) variable.ScopeFlag + GetScope(status string) vardef.ScopeFlag // Stop stops DDL worker. Stop() error // RegisterStatsHandle registers statistics handle and its corresponding event channel for ddl. @@ -1130,7 +1131,7 @@ func (d *ddl) cleanDeadTableLock(unlockTables []model.TableLockTpInfo, se model. // SwitchMDL enables MDL or disable MDL. func (d *ddl) SwitchMDL(enable bool) error { - isEnableBefore := variable.EnableMDL.Load() + isEnableBefore := vardef.EnableMDL.Load() if isEnableBefore == enable { return nil } @@ -1154,7 +1155,7 @@ func (d *ddl) SwitchMDL(enable bool) error { return errors.New("please wait for all jobs done") } - variable.EnableMDL.Store(enable) + vardef.EnableMDL.Store(enable) err = kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), d.store, true, func(_ context.Context, txn kv.Transaction) error { m := meta.NewMutator(txn) oldEnable, _, err := m.GetMetadataLock() @@ -1178,7 +1179,7 @@ func (d *ddl) SwitchMDL(enable bool) error { // It should be called before any DDL that could break data consistency. // This provides a safe window for async commit and 1PC to commit with an old schema. func delayForAsyncCommit() { - if variable.EnableMDL.Load() { + if vardef.EnableMDL.Load() { // If metadata lock is enabled. The transaction of DDL must begin after // pre-write of the async commit transaction, then the commit ts of DDL // must be greater than the async commit transaction. In this case, the diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index 8b45f286483b0..61bcb0bc556e2 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -55,6 +55,7 @@ import ( "github.com/pingcap/tidb/pkg/privilege" rg "github.com/pingcap/tidb/pkg/resourcegroup" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/statistics/handle" @@ -69,6 +70,7 @@ import ( "github.com/pingcap/tidb/pkg/util/domainutil" "github.com/pingcap/tidb/pkg/util/generic" "github.com/pingcap/tidb/pkg/util/stringutil" + "github.com/pingcap/tidb/pkg/util/tracing" "github.com/tikv/client-go/v2/oracle" pdhttp "github.com/tikv/pd/client/http" "go.uber.org/zap" @@ -92,7 +94,7 @@ const ( tiflashCheckPendingTablesRetry = 7 ) -var errCheckConstraintIsOff = errors.NewNoStackError(variable.TiDBEnableCheckConstraint + " is off") +var errCheckConstraintIsOff = errors.NewNoStackError(vardef.TiDBEnableCheckConstraint + " is off") // Executor is the interface for executing DDL statements. // it's mostly called by SQL executor. @@ -199,11 +201,11 @@ func (e *executor) CreateSchema(ctx sessionctx.Context, stmt *ast.CreateDatabase // If no charset and/or collation is specified use collation_server and character_set_server charsetOpt := ast.CharsetOpt{} if sessionVars.GlobalVarsAccessor != nil { - charsetOpt.Col, err = sessionVars.GetSessionOrGlobalSystemVar(context.Background(), variable.CollationServer) + charsetOpt.Col, err = sessionVars.GetSessionOrGlobalSystemVar(context.Background(), vardef.CollationServer) if err != nil { return err } - charsetOpt.Chs, err = sessionVars.GetSessionOrGlobalSystemVar(context.Background(), variable.CharacterSetServer) + charsetOpt.Chs, err = sessionVars.GetSessionOrGlobalSystemVar(context.Background(), vardef.CharacterSetServer) if err != nil { return err } @@ -1373,7 +1375,7 @@ func preSplitAndScatter(ctx sessionctx.Context, store kv.Storage, tbInfo *model. preSplit func() scatterScope string ) - val, ok := ctx.GetSessionVars().GetSystemVar(variable.TiDBScatterRegion) + val, ok := ctx.GetSessionVars().GetSystemVar(vardef.TiDBScatterRegion) if !ok { logutil.DDLLogger().Warn("get system variable met problem, won't scatter region") } else { @@ -1384,7 +1386,7 @@ func preSplitAndScatter(ctx sessionctx.Context, store kv.Storage, tbInfo *model. } else { preSplit = func() { splitTableRegion(ctx, sp, tbInfo, scatterScope) } } - if scatterScope != variable.ScatterOff { + if scatterScope != vardef.ScatterOff { preSplit() } else { go preSplit() @@ -1492,10 +1494,10 @@ func (e *executor) CreateView(ctx sessionctx.Context, s *ast.CreateViewStmt) (er tblCharset := "" tblCollate := "" - if v, ok := ctx.GetSessionVars().GetSystemVar(variable.CharacterSetConnection); ok { + if v, ok := ctx.GetSessionVars().GetSystemVar(vardef.CharacterSetConnection); ok { tblCharset = v } - if v, ok := ctx.GetSessionVars().GetSystemVar(variable.CollationConnection); ok { + if v, ok := ctx.GetSessionVars().GetSystemVar(vardef.CollationConnection); ok { tblCollate = v } @@ -1710,7 +1712,7 @@ func (e *executor) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Alter Table") } } - if isMultiSchemaChanges(validSpecs) && (sctx.GetSessionVars().EnableRowLevelChecksum || variable.EnableRowLevelChecksum.Load()) { + if isMultiSchemaChanges(validSpecs) && (sctx.GetSessionVars().EnableRowLevelChecksum || vardef.EnableRowLevelChecksum.Load()) { return dbterror.ErrRunMultiSchemaChanges.GenWithStack("Unsupported multi schema change when row level checksum is enabled") } // set name for anonymous foreign key. @@ -1804,7 +1806,7 @@ func (e *executor) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt case ast.ConstraintFulltext: sctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrTableCantHandleFt) case ast.ConstraintCheck: - if !variable.EnableCheckConstraint.Load() { + if !vardef.EnableCheckConstraint.Load() { sctx.GetSessionVars().StmtCtx.AppendWarning(errCheckConstraintIsOff) } else { err = e.CreateCheckConstraint(sctx, ident, ast.NewCIStr(constr.Name), spec.Constraint) @@ -1836,8 +1838,8 @@ func (e *executor) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt for i, opt := range spec.Options { switch opt.Tp { case ast.TableOptionShardRowID: - if opt.UintValue > variable.MaxShardRowIDBits { - opt.UintValue = variable.MaxShardRowIDBits + if opt.UintValue > vardef.MaxShardRowIDBits { + opt.UintValue = vardef.MaxShardRowIDBits } err = e.ShardRowID(sctx, ident, opt.UintValue) case ast.TableOptionAutoIncrement: @@ -1907,13 +1909,13 @@ func (e *executor) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt case ast.AlterTableIndexInvisible: err = e.AlterIndexVisibility(sctx, ident, spec.IndexName, spec.Visibility) case ast.AlterTableAlterCheck: - if !variable.EnableCheckConstraint.Load() { + if !vardef.EnableCheckConstraint.Load() { sctx.GetSessionVars().StmtCtx.AppendWarning(errCheckConstraintIsOff) } else { err = e.AlterCheckConstraint(sctx, ident, ast.NewCIStr(spec.Constraint.Name), spec.Constraint.Enforced) } case ast.AlterTableDropCheck: - if !variable.EnableCheckConstraint.Load() { + if !vardef.EnableCheckConstraint.Load() { sctx.GetSessionVars().StmtCtx.AppendWarning(errCheckConstraintIsOff) } else { err = e.DropCheckConstraint(sctx, ident, ast.NewCIStr(spec.Constraint.Name)) @@ -4930,18 +4932,18 @@ func (e *executor) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast func initJobReorgMetaFromVariables(job *model.Job, sctx sessionctx.Context) error { m := NewDDLReorgMeta(sctx) setReorgParam := func() { - if sv, ok := sctx.GetSessionVars().GetSystemVar(variable.TiDBDDLReorgWorkerCount); ok { + if sv, ok := sctx.GetSessionVars().GetSystemVar(vardef.TiDBDDLReorgWorkerCount); ok { m.SetConcurrency(variable.TidbOptInt(sv, 0)) } - if sv, ok := sctx.GetSessionVars().GetSystemVar(variable.TiDBDDLReorgBatchSize); ok { + if sv, ok := sctx.GetSessionVars().GetSystemVar(vardef.TiDBDDLReorgBatchSize); ok { m.SetBatchSize(variable.TidbOptInt(sv, 0)) } - m.SetMaxWriteSpeed(int(variable.DDLReorgMaxWriteSpeed.Load())) + m.SetMaxWriteSpeed(int(vardef.DDLReorgMaxWriteSpeed.Load())) } setDistTaskParam := func() error { - m.IsDistReorg = variable.EnableDistTask.Load() - m.IsFastReorg = variable.EnableFastReorg.Load() - m.TargetScope = variable.ServiceScope.Load() + m.IsDistReorg = vardef.EnableDistTask.Load() + m.IsFastReorg = vardef.EnableFastReorg.Load() + m.TargetScope = vardef.ServiceScope.Load() if hasSysDB(job) { if m.IsDistReorg { logutil.DDLLogger().Info("cannot use distributed task execution on system DB", @@ -4998,8 +5000,8 @@ func initJobReorgMetaFromVariables(job *model.Job, sctx sessionctx.Context) erro zap.Bool("enableDistTask", m.IsDistReorg), zap.Bool("enableFastReorg", m.IsFastReorg), zap.String("targetScope", m.TargetScope), - zap.Int("concurrency", m.GetConcurrencyOrDefault(int(variable.GetDDLReorgWorkerCounter()))), - zap.Int("batchSize", m.GetBatchSizeOrDefault(int(variable.GetDDLReorgBatchSize()))), + zap.Int("concurrency", m.GetConcurrency()), + zap.Int("batchSize", m.GetBatchSize()), ) return nil } @@ -5099,7 +5101,7 @@ func buildFKInfo(fkName ast.CIStr, keys []*ast.IndexPartSpecification, refer *as RefTable: refer.Table.Name, Cols: make([]ast.CIStr, len(keys)), } - if variable.EnableForeignKey.Load() { + if vardef.EnableForeignKey.Load() { fkInfo.Version = model.FKVersion1 } @@ -6629,7 +6631,7 @@ func (e *executor) doDDLJob2(ctx sessionctx.Context, job *model.Job, args model. // depend on job.ID, use JobID from jobSubmitResult. func (e *executor) DoDDLJobWrapper(ctx sessionctx.Context, jobW *JobWrapper) (resErr error) { job := jobW.Job - job.TraceInfo = &model.TraceInfo{ + job.TraceInfo = &tracing.TraceInfo{ ConnectionID: ctx.GetSessionVars().ConnectionID, SessionAlias: ctx.GetSessionVars().SessionAlias, } diff --git a/pkg/ddl/foreign_key.go b/pkg/ddl/foreign_key.go index 249a4d8977591..041edb16cde9c 100644 --- a/pkg/ddl/foreign_key.go +++ b/pkg/ddl/foreign_key.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/sqlexec" ) @@ -149,7 +149,7 @@ func allocateFKIndexID(tblInfo *model.TableInfo) int64 { } func checkTableForeignKeysValid(sctx sessionctx.Context, is infoschema.InfoSchema, schema string, tbInfo *model.TableInfo) error { - if !variable.EnableForeignKey.Load() { + if !vardef.EnableForeignKey.Load() { return nil } fkCheck := sctx.GetSessionVars().ForeignKeyChecks @@ -211,7 +211,7 @@ func checkTableForeignKeyValid(is infoschema.InfoSchema, schema string, tbInfo * } func checkTableForeignKeyValidInOwner(jobCtx *jobContext, job *model.Job, tbInfo *model.TableInfo, fkCheck bool) (retryable bool, _ error) { - if !variable.EnableForeignKey.Load() { + if !vardef.EnableForeignKey.Load() { return false, nil } is := jobCtx.infoCache.GetLatest() @@ -404,7 +404,7 @@ func checkTableHasForeignKeyReferred(is infoschemactx.MetaOnlyInfoSchema, schema } func checkDropTableHasForeignKeyReferredInOwner(infoCache *infoschema.InfoCache, job *model.Job, args *model.DropTableArgs) error { - if !variable.EnableForeignKey.Load() { + if !vardef.EnableForeignKey.Load() { return nil } objectIdents, fkCheck := args.Identifiers, args.FKCheck @@ -434,7 +434,7 @@ func checkTruncateTableHasForeignKeyReferredInOwner(infoCache *infoschema.InfoCa } func checkTableHasForeignKeyReferredInOwner(infoCache *infoschema.InfoCache, schema, tbl string, ignoreTables []ast.Ident, fkCheck bool) (_ *model.ReferredFKInfo, _ error) { - if !variable.EnableForeignKey.Load() { + if !vardef.EnableForeignKey.Load() { return nil, nil } is := infoCache.GetLatest() @@ -490,7 +490,7 @@ func checkIndexNeededInForeignKey(is infoschema.InfoSchema, dbName string, tbInf } func checkIndexNeededInForeignKeyInOwner(infoCache *infoschema.InfoCache, job *model.Job, dbName string, tbInfo *model.TableInfo, idxInfo *model.IndexInfo) error { - if !variable.EnableForeignKey.Load() { + if !vardef.EnableForeignKey.Load() { return nil } is := infoCache.GetLatest() @@ -522,7 +522,7 @@ func checkDropColumnWithForeignKeyConstraint(is infoschema.InfoSchema, dbName st } func checkDropColumnWithForeignKeyConstraintInOwner(infoCache *infoschema.InfoCache, job *model.Job, tbInfo *model.TableInfo, colName string) error { - if !variable.EnableForeignKey.Load() { + if !vardef.EnableForeignKey.Load() { return nil } is := infoCache.GetLatest() @@ -603,7 +603,7 @@ func checkDatabaseHasForeignKeyReferred(ctx context.Context, is infoschema.InfoS } func checkDatabaseHasForeignKeyReferredInOwner(jobCtx *jobContext, job *model.Job) error { - if !variable.EnableForeignKey.Load() { + if !vardef.EnableForeignKey.Load() { return nil } args, err := model.GetDropSchemaArgs(job) @@ -633,7 +633,7 @@ func checkFKDupName(tbInfo *model.TableInfo, fkName ast.CIStr) error { } func checkAddForeignKeyValid(is infoschema.InfoSchema, schema string, tbInfo *model.TableInfo, fk *model.FKInfo, fkCheck bool) error { - if !variable.EnableForeignKey.Load() { + if !vardef.EnableForeignKey.Load() { return nil } err := checkTableForeignKeyValid(is, schema, tbInfo, fk, fkCheck) @@ -648,7 +648,7 @@ func checkAddForeignKeyValidInOwner(infoCache *infoschema.InfoCache, schema stri if err != nil { return err } - if !variable.EnableForeignKey.Load() { + if !vardef.EnableForeignKey.Load() { return nil } is := infoCache.GetLatest() diff --git a/pkg/ddl/index.go b/pkg/ddl/index.go index b3bd6b8c0da4e..5fb415510a1d5 100644 --- a/pkg/ddl/index.go +++ b/pkg/ddl/index.go @@ -56,6 +56,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/store/helper" "github.com/pingcap/tidb/pkg/table" @@ -1302,7 +1303,7 @@ func pickBackfillType(job *model.Job) (model.ReorgType, error) { func loadCloudStorageURI(w *worker, job *model.Job) { jc := w.jobContext(job.ID, job.ReorgMeta) - jc.cloudStorageURI = variable.CloudStorageURI.Load() + jc.cloudStorageURI = vardef.CloudStorageURI.Load() job.ReorgMeta.UseCloudStorage = len(jc.cloudStorageURI) > 0 && job.ReorgMeta.IsDistReorg } @@ -1451,7 +1452,7 @@ func runIngestReorgJob(w *worker, jobCtx *jobContext, job *model.Job, } func isRetryableJobError(err error, jobErrCnt int64) bool { - if jobErrCnt+1 >= variable.GetDDLErrorCountLimit() { + if jobErrCnt+1 >= vardef.GetDDLErrorCountLimit() { return false } return isRetryableError(err) @@ -2528,7 +2529,7 @@ func (w *worker) executeDistTask(stepCtx context.Context, t table.Table, reorgIn }) } else { job := reorgInfo.Job - workerCntLimit := job.ReorgMeta.GetConcurrencyOrDefault(int(variable.GetDDLReorgWorkerCounter())) + workerCntLimit := job.ReorgMeta.GetConcurrency() cpuCount, err := handle.GetCPUCountOfNode(ctx) if err != nil { return err diff --git a/pkg/ddl/index_change_test.go b/pkg/ddl/index_change_test.go index 2c5b327931579..74d89c21e0e14 100644 --- a/pkg/ddl/index_change_test.go +++ b/pkg/ddl/index_change_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/testkit" @@ -231,7 +231,7 @@ func checkAddPublicForAddIndex(ctx sessionctx.Context, writeTbl, publicTbl table return errors.Trace(err) } err = checkIndexExists(ctx, publicTbl, 6, 6, true) - if variable.EnableFastReorg.Load() { + if vardef.EnableFastReorg.Load() { // Need check temp index also. err1 = checkIndexExists(ctx, writeTbl, 6, 6, true) } @@ -254,14 +254,14 @@ func checkAddPublicForAddIndex(ctx sessionctx.Context, writeTbl, publicTbl table return errors.Trace(err) } err = checkIndexExists(ctx, publicTbl, 5, 7, true) - if variable.EnableFastReorg.Load() { + if vardef.EnableFastReorg.Load() { // Need check temp index also. err1 = checkIndexExists(ctx, writeTbl, 5, 7, true) } if err != nil && err1 != nil { return errors.Trace(err) } - if variable.EnableFastReorg.Load() { + if vardef.EnableFastReorg.Load() { err = checkIndexExists(ctx, writeTbl, 7, 7, false) } else { err = checkIndexExists(ctx, publicTbl, 7, 7, false) @@ -295,7 +295,7 @@ func checkAddPublicForAddIndex(ctx sessionctx.Context, writeTbl, publicTbl table idxVal := row[1].GetInt64() handle := row[0].GetInt64() err = checkIndexExists(ctx, publicTbl, idxVal, handle, true) - if variable.EnableFastReorg.Load() { + if vardef.EnableFastReorg.Load() { // Need check temp index also. err1 = checkIndexExists(ctx, writeTbl, idxVal, handle, true) } diff --git a/pkg/ddl/index_modify_test.go b/pkg/ddl/index_modify_test.go index 6ad42279fe3f9..b51eff27d5fc1 100644 --- a/pkg/ddl/index_modify_test.go +++ b/pkg/ddl/index_modify_test.go @@ -38,7 +38,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/table" @@ -197,7 +197,7 @@ func testAddIndex(t *testing.T, tp testAddIndexType, createTableSQL, idxTp strin }() } if (testClusteredIndex & tp) > 0 { - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn } tk.MustExec("drop table if exists test_add_index") tk.MustExec(createTableSQL) @@ -642,15 +642,15 @@ func TestAddIndexWithPK(t *testing.T) { tests := []struct { name string - mode variable.ClusteredIndexDefMode + mode vardef.ClusteredIndexDefMode }{ { "ClusteredIndexDefModeIntOnly", - variable.ClusteredIndexDefModeIntOnly, + vardef.ClusteredIndexDefModeIntOnly, }, { "ClusteredIndexDefModeOn", - variable.ClusteredIndexDefModeOn, + vardef.ClusteredIndexDefModeOn, }, } @@ -1380,10 +1380,10 @@ func TestAddVectorIndexRollback(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") - limit := variable.GetDDLErrorCountLimit() - variable.SetDDLErrorCountLimit(5) + limit := vardef.GetDDLErrorCountLimit() + vardef.SetDDLErrorCountLimit(5) defer func() { - variable.SetDDLErrorCountLimit(limit) + vardef.SetDDLErrorCountLimit(limit) }() // mock TiFlash replicas diff --git a/pkg/ddl/ingest/BUILD.bazel b/pkg/ddl/ingest/BUILD.bazel index 9c7b28ce6de6e..852c2322889c7 100644 --- a/pkg/ddl/ingest/BUILD.bazel +++ b/pkg/ddl/ingest/BUILD.bazel @@ -38,7 +38,7 @@ go_library( "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/sessiontxn", "//pkg/table", "//pkg/util", diff --git a/pkg/ddl/ingest/backend_mgr.go b/pkg/ddl/ingest/backend_mgr.go index 391a5aa157c9c..be883d5f693a4 100644 --- a/pkg/ddl/ingest/backend_mgr.go +++ b/pkg/ddl/ingest/backend_mgr.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/pkg/lightning/backend/local" "github.com/pingcap/tidb/pkg/lightning/common" "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/tikv/client-go/v2/tikv" @@ -107,8 +106,8 @@ func (b *BackendCtxBuilder) Build() (BackendCtx, error) { intest.Assert(job.ReorgMeta != nil) resGroupName := job.ReorgMeta.ResourceGroupName - concurrency := job.ReorgMeta.GetConcurrencyOrDefault(int(variable.GetDDLReorgWorkerCounter())) - maxWriteSpeed := job.ReorgMeta.GetMaxWriteSpeedOrDefault() + concurrency := job.ReorgMeta.GetConcurrency() + maxWriteSpeed := job.ReorgMeta.GetMaxWriteSpeed() hasUnique, err := hasUniqueIndex(job) if err != nil { return nil, err diff --git a/pkg/ddl/ingest/config.go b/pkg/ddl/ingest/config.go index a97cb81d9cd03..9fd21bbc8fecf 100644 --- a/pkg/ddl/ingest/config.go +++ b/pkg/ddl/ingest/config.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/checkpoints" "github.com/pingcap/tidb/pkg/lightning/common" lightning "github.com/pingcap/tidb/pkg/lightning/config" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/size" kvutil "github.com/tikv/client-go/v2/util" @@ -93,7 +93,7 @@ func CopReadBatchSize(hintSize int) int { if hintSize > 0 { return hintSize } - return 10 * int(variable.GetDDLReorgBatchSize()) + return 10 * int(vardef.GetDDLReorgBatchSize()) } // CopReadChunkPoolSize is the size of chunk pool, which @@ -103,7 +103,7 @@ func CopReadChunkPoolSize(hintConc int) int { if hintConc > 0 { return 10 * hintConc } - return 10 * int(variable.GetDDLReorgWorkerCounter()) + return 10 * int(vardef.GetDDLReorgWorkerCounter()) } // NewDDLTLS creates a common.TLS from the tidb config for DDL. diff --git a/pkg/ddl/ingest/disk_root.go b/pkg/ddl/ingest/disk_root.go index 620bb7f66f91c..5f4c5b06ade60 100644 --- a/pkg/ddl/ingest/disk_root.go +++ b/pkg/ddl/ingest/disk_root.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/ddl/logutil" lcom "github.com/pingcap/tidb/pkg/lightning/common" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util/dbterror" "go.uber.org/zap" ) @@ -122,9 +122,9 @@ func (d *diskRootImpl) UpdateUsage() { func (d *diskRootImpl) ShouldImport() bool { d.mu.RLock() defer d.mu.RUnlock() - if d.bcUsed > variable.DDLDiskQuota.Load() { + if d.bcUsed > vardef.DDLDiskQuota.Load() { logutil.DDLIngestLogger().Info("disk usage is over quota", - zap.Uint64("quota", variable.DDLDiskQuota.Load()), + zap.Uint64("quota", vardef.DDLDiskQuota.Load()), zap.String("usage", d.usageInfo())) return true } @@ -181,7 +181,7 @@ func (d *diskRootImpl) StartupCheck() error { if err != nil { return errors.Trace(err) } - quota := variable.DDLDiskQuota.Load() + quota := vardef.DDLDiskQuota.Load() if sz.Available < quota { return errors.Errorf("the available disk space(%d) in %s should be greater than @@tidb_ddl_disk_quota(%d)", sz.Available, d.path, quota) diff --git a/pkg/ddl/job_scheduler.go b/pkg/ddl/job_scheduler.go index f6119a8aa0232..67b87363387a7 100644 --- a/pkg/ddl/job_scheduler.go +++ b/pkg/ddl/job_scheduler.go @@ -43,13 +43,14 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/owner" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" tidbutil "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/generic" "github.com/pingcap/tidb/pkg/util/intest" tidblogutil "github.com/pingcap/tidb/pkg/util/logutil" + "github.com/pingcap/tidb/pkg/util/tracing" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -530,7 +531,7 @@ func (s *jobScheduler) deliveryJob(wk *worker, pool *workerPool, jobW *model.Job }) } -func (s *jobScheduler) getJobRunCtx(jobID int64, traceInfo *model.TraceInfo) *jobContext { +func (s *jobScheduler) getJobRunCtx(jobID int64, traceInfo *tracing.TraceInfo) *jobContext { ch, _ := s.ddlJobDoneChMap.Load(jobID) return &jobContext{ ctx: s.schCtx, @@ -564,7 +565,7 @@ func (s *jobScheduler) transitOneJobStepAndWaitSync(wk *worker, jobCtx *jobConte // current owner. job := jobW.Job if jobCtx.isUnSynced(job.ID) || (job.Started() && !jobCtx.maybeAlreadyRunOnce(job.ID)) { - if variable.EnableMDL.Load() { + if vardef.EnableMDL.Load() { version, err := s.sysTblMgr.GetMDLVer(s.schCtx, job.ID) if err == nil { jobCtx.logger.Info("the job have schema version un-synced", @@ -622,7 +623,7 @@ func (s *jobScheduler) cleanMDLInfo(job *model.Job, ownerID string) { defer func() { metrics.DDLCleanMDLInfoHist.Observe(time.Since(start).Seconds()) }() - if !variable.EnableMDL.Load() { + if !vardef.EnableMDL.Load() { return } var sql string diff --git a/pkg/ddl/job_submitter.go b/pkg/ddl/job_submitter.go index 7c41669f3c670..570174c4357ed 100644 --- a/pkg/ddl/job_submitter.go +++ b/pkg/ddl/job_submitter.go @@ -40,7 +40,7 @@ import ( "github.com/pingcap/tidb/pkg/owner" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/terror" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/generic" @@ -104,7 +104,7 @@ func (s *JobSubmitter) addBatchDDLJobs(jobWs []*JobWrapper) { err error newWs []*JobWrapper ) - fastCreate := variable.EnableFastCreateTable.Load() + fastCreate := vardef.EnableFastCreateTable.Load() if fastCreate { newWs, err = mergeCreateTableJobs(jobWs) if err != nil { diff --git a/pkg/ddl/job_worker.go b/pkg/ddl/job_worker.go index 4d30700cb3b37..45e135d99c32e 100644 --- a/pkg/ddl/job_worker.go +++ b/pkg/ddl/job_worker.go @@ -42,7 +42,7 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" tidbutil "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/topsql" @@ -277,7 +277,7 @@ func (w *worker) updateDDLJob(jobCtx *jobContext, job *model.Job, updateRawArgs // registerMDLInfo registers metadata lock info. func (w *worker) registerMDLInfo(job *model.Job, ver int64) error { - if !variable.EnableMDL.Load() { + if !vardef.EnableMDL.Load() { return nil } if ver == 0 { @@ -715,7 +715,7 @@ func (w *worker) countForPanic(jobCtx *jobContext, job *model.Job) { if err1 := loadDDLVars(w); err1 != nil { logger.Error("load DDL global variable failed", zap.Error(err1)) } - errorCount := variable.GetDDLErrorCountLimit() + errorCount := vardef.GetDDLErrorCountLimit() if job.ErrorCount > errorCount { msg := fmt.Sprintf("panic in handling DDL logic and error count beyond the limitation %d, cancelled", errorCount) @@ -743,8 +743,8 @@ func (w *worker) countForError(jobCtx *jobContext, job *model.Job, err error) er logger.Error("load DDL global variable failed", zap.Error(err1)) } // Check error limit to avoid falling into an infinite loop. - if job.ErrorCount > variable.GetDDLErrorCountLimit() && job.State == model.JobStateRunning && job.IsRollbackable() { - logger.Warn("DDL job error count exceed the limit, cancelling it now", zap.Int64("errorCountLimit", variable.GetDDLErrorCountLimit())) + if job.ErrorCount > vardef.GetDDLErrorCountLimit() && job.State == model.JobStateRunning && job.IsRollbackable() { + logger.Warn("DDL job error count exceed the limit, cancelling it now", zap.Int64("errorCountLimit", vardef.GetDDLErrorCountLimit())) job.State = model.JobStateCancelling } return err @@ -884,9 +884,9 @@ func (w *worker) runOneJobStep( return case model.JobStateRunning: if latestJob.IsAlterable() { - job.ReorgMeta.SetConcurrency(latestJob.ReorgMeta.GetConcurrencyOrDefault(int(variable.GetDDLReorgWorkerCounter()))) - job.ReorgMeta.SetBatchSize(latestJob.ReorgMeta.GetBatchSizeOrDefault(int(variable.GetDDLReorgBatchSize()))) - job.ReorgMeta.SetMaxWriteSpeed(latestJob.ReorgMeta.GetMaxWriteSpeedOrDefault()) + job.ReorgMeta.SetConcurrency(latestJob.ReorgMeta.GetConcurrency()) + job.ReorgMeta.SetBatchSize(latestJob.ReorgMeta.GetBatchSize()) + job.ReorgMeta.SetMaxWriteSpeed(latestJob.ReorgMeta.GetMaxWriteSpeed()) } } } @@ -1101,7 +1101,7 @@ func updateGlobalVersionAndWaitSynced( err = jobCtx.schemaVerSyncer.OwnerUpdateGlobalVersion(ctx, latestSchemaVersion) if err != nil { logutil.DDLLogger().Info("update latest schema version failed", zap.Int64("ver", latestSchemaVersion), zap.Error(err)) - if variable.EnableMDL.Load() { + if vardef.EnableMDL.Load() { return err } if terror.ErrorEqual(err, context.DeadlineExceeded) { diff --git a/pkg/ddl/metabuild_test.go b/pkg/ddl/metabuild_test.go index 2f43896176360..8d2937b45dd58 100644 --- a/pkg/ddl/metabuild_test.go +++ b/pkg/ddl/metabuild_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/meta/metabuild" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" contextutil "github.com/pingcap/tidb/pkg/util/context" "github.com/pingcap/tidb/pkg/util/deeptest" "github.com/pingcap/tidb/pkg/util/mock" @@ -86,12 +86,12 @@ func TestNewMetaBuildContextWithSctx(t *testing.T) { { field: "clusteredIndexDefMode", setSctx: func(val any) { - sessVars.EnableClusteredIndex = val.(variable.ClusteredIndexDefMode) + sessVars.EnableClusteredIndex = val.(vardef.ClusteredIndexDefMode) }, testVals: []any{ - variable.ClusteredIndexDefModeIntOnly, - variable.ClusteredIndexDefModeOff, - variable.ClusteredIndexDefModeOn, + vardef.ClusteredIndexDefModeIntOnly, + vardef.ClusteredIndexDefModeOff, + vardef.ClusteredIndexDefModeOn, }, getter: func(ctx *metabuild.Context) any { return ctx.GetClusteredIndexDefMode() @@ -102,7 +102,7 @@ func TestNewMetaBuildContextWithSctx(t *testing.T) { setSctx: func(val any) { sessVars.ShardRowIDBits = val.(uint64) }, - testVals: []any{uint64(variable.DefShardRowIDBits), uint64(6)}, + testVals: []any{uint64(vardef.DefShardRowIDBits), uint64(6)}, getter: func(ctx *metabuild.Context) any { return ctx.GetShardRowIDBits() }, @@ -112,7 +112,7 @@ func TestNewMetaBuildContextWithSctx(t *testing.T) { setSctx: func(val any) { sessVars.PreSplitRegions = val.(uint64) }, - testVals: []any{uint64(variable.DefPreSplitRegions), uint64(123)}, + testVals: []any{uint64(vardef.DefPreSplitRegions), uint64(123)}, getter: func(ctx *metabuild.Context) any { return ctx.GetPreSplitRegions() }, diff --git a/pkg/ddl/modify_column.go b/pkg/ddl/modify_column.go index ae7ba0641d2d6..12a9583ab5444 100644 --- a/pkg/ddl/modify_column.go +++ b/pkg/ddl/modify_column.go @@ -41,7 +41,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/format" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" @@ -400,7 +400,7 @@ func updateTTLInfoWhenModifyColumn(tblInfo *model.TableInfo, oldCol, newCol ast. } func adjustForeignKeyChildTableInfoAfterModifyColumn(infoCache *infoschema.InfoCache, t *meta.Mutator, job *model.Job, tblInfo *model.TableInfo, newCol, oldCol *model.ColumnInfo) ([]schemaIDAndTableInfo, error) { - if !variable.EnableForeignKey.Load() || newCol.Name.L == oldCol.Name.L { + if !vardef.EnableForeignKey.Load() || newCol.Name.L == oldCol.Name.L { return nil, nil } is := infoCache.GetLatest() diff --git a/pkg/ddl/modify_column_test.go b/pkg/ddl/modify_column_test.go index 977c02829836f..ecde0a215b8ab 100644 --- a/pkg/ddl/modify_column_test.go +++ b/pkg/ddl/modify_column_test.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/external" @@ -50,10 +50,10 @@ func batchInsert(tk *testkit.TestKit, tbl string, start, end int) { func TestModifyColumnReorgInfo(t *testing.T) { store := testkit.CreateMockStore(t) - limit := variable.GetDDLErrorCountLimit() - variable.SetDDLErrorCountLimit(5) + limit := vardef.GetDDLErrorCountLimit() + vardef.SetDDLErrorCountLimit(5) defer func() { - variable.SetDDLErrorCountLimit(limit) + vardef.SetDDLErrorCountLimit(limit) }() tk := testkit.NewTestKit(t, store) tk.MustExec("use test") diff --git a/pkg/ddl/multi_schema_change_test.go b/pkg/ddl/multi_schema_change_test.go index 8bae5ed1c2abd..9e396198852b4 100644 --- a/pkg/ddl/multi_schema_change_test.go +++ b/pkg/ddl/multi_schema_change_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" "github.com/stretchr/testify/assert" @@ -779,19 +779,19 @@ func TestMultiSchemaChangeMixedWithUpdate(t *testing.T) { func TestMultiSchemaChangeBlockedByRowLevelChecksum(t *testing.T) { store := testkit.CreateMockStore(t) - orig := variable.EnableRowLevelChecksum.Load() - defer variable.EnableRowLevelChecksum.Store(orig) + orig := vardef.EnableRowLevelChecksum.Load() + defer vardef.EnableRowLevelChecksum.Store(orig) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t (c int)") - variable.EnableRowLevelChecksum.Store(true) + vardef.EnableRowLevelChecksum.Store(true) tk.Session().GetSessionVars().EnableRowLevelChecksum = false tk.MustGetErrCode("alter table t add column c1 int, add column c2 int", errno.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t add (c1 int, c2 int)", errno.ErrUnsupportedDDLOperation) - variable.EnableRowLevelChecksum.Store(false) + vardef.EnableRowLevelChecksum.Store(false) tk.Session().GetSessionVars().EnableRowLevelChecksum = true tk.MustGetErrCode("alter table t add column c1 int, add column c2 int", errno.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t add (c1 int, c2 int)", errno.ErrUnsupportedDDLOperation) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 1198b932c4375..f476984b306dc 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -48,7 +48,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/opcode" field_types "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/tablecodec" @@ -2562,7 +2562,7 @@ func (w *worker) onTruncateTablePartition(jobCtx *jobContext, job *model.Job) (i preSplitAndScatter(w.sess.Context, jobCtx.store, tblInfo, newDefinitions) failpoint.Inject("truncatePartFail1", func(val failpoint.Value) { if val.(bool) { - job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 + job.ErrorCount += vardef.GetDDLErrorCountLimit() / 2 err = errors.New("Injected error by truncatePartFail1") failpoint.Return(ver, err) } @@ -2596,7 +2596,7 @@ func (w *worker) onTruncateTablePartition(jobCtx *jobContext, job *model.Job) (i } failpoint.Inject("truncatePartFail2", func(val failpoint.Value) { if val.(bool) { - job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 + job.ErrorCount += vardef.GetDDLErrorCountLimit() / 2 err = errors.New("Injected error by truncatePartFail2") failpoint.Return(ver, err) } @@ -2617,7 +2617,7 @@ func (w *worker) onTruncateTablePartition(jobCtx *jobContext, job *model.Job) (i failpoint.Inject("truncatePartFail3", func(val failpoint.Value) { if val.(bool) { - job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 + job.ErrorCount += vardef.GetDDLErrorCountLimit() / 2 err = errors.New("Injected error by truncatePartFail3") failpoint.Return(ver, err) } @@ -3312,7 +3312,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver if s, ok := jobCtx.store.(kv.SplittableStore); ok && s != nil { // 1. partInfo only contains the AddingPartitions // 2. ScatterTable control all new split region need waiting for scatter region finish at table level. - splitPartitionTableRegion(w.sess.Context, s, tblInfo, partInfo.Definitions, variable.ScatterTable) + splitPartitionTableRegion(w.sess.Context, s, tblInfo, partInfo.Definitions, vardef.ScatterTable) } if job.Type == model.ActionReorganizePartition { @@ -3416,7 +3416,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver failpoint.Inject("reorgPartFail1", func(val failpoint.Value) { // Failures will retry, then do rollback if val.(bool) { - job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 + job.ErrorCount += vardef.GetDDLErrorCountLimit() / 2 failpoint.Return(ver, errors.New("Injected error by reorgPartFail1")) } }) @@ -3472,7 +3472,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver failpoint.Inject("reorgPartFail2", func(val failpoint.Value) { if val.(bool) { - job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 + job.ErrorCount += vardef.GetDDLErrorCountLimit() / 2 failpoint.Return(ver, errors.New("Injected error by reorgPartFail2")) } }) @@ -3507,7 +3507,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver } failpoint.Inject("reorgPartFail3", func(val failpoint.Value) { if val.(bool) { - job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 + job.ErrorCount += vardef.GetDDLErrorCountLimit() / 2 failpoint.Return(ver, errors.New("Injected error by reorgPartFail3")) } }) @@ -3558,7 +3558,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver } failpoint.Inject("reorgPartFail4", func(val failpoint.Value) { if val.(bool) { - job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 + job.ErrorCount += vardef.GetDDLErrorCountLimit() / 2 failpoint.Return(ver, errors.New("Injected error by reorgPartFail4")) } }) @@ -3610,7 +3610,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver failpoint.Inject("reorgPartFail5", func(val failpoint.Value) { if val.(bool) { - job.ErrorCount += variable.GetDDLErrorCountLimit() / 2 + job.ErrorCount += vardef.GetDDLErrorCountLimit() / 2 failpoint.Return(ver, errors.New("Injected error by reorgPartFail5")) } }) @@ -4271,7 +4271,7 @@ func checkExchangePartitionRecordValidation( return dbterror.ErrUnsupportedPartitionType.GenWithStackByArgs(pt.Name.O) } - if variable.EnableCheckConstraint.Load() { + if vardef.EnableCheckConstraint.Load() { pcc, ok := ptbl.(CheckConstraintTable) if !ok { return errors.Errorf("exchange partition process assert table partition failed") @@ -4296,7 +4296,7 @@ func checkExchangePartitionRecordValidation( } // Check partition table records. - if variable.EnableCheckConstraint.Load() { + if vardef.EnableCheckConstraint.Load() { ncc, ok := ntbl.(CheckConstraintTable) if !ok { return errors.Errorf("exchange partition process assert table partition failed") diff --git a/pkg/ddl/placement_policy.go b/pkg/ddl/placement_policy.go index c758586afcd87..23dd9b43cfe36 100644 --- a/pkg/ddl/placement_policy.go +++ b/pkg/ddl/placement_policy.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/util/dbterror" ) @@ -553,10 +553,10 @@ func handleDatabasePlacement(ctx sessionctx.Context, dbInfo *model.DBInfo) error } sessVars := ctx.GetSessionVars() - if sessVars.PlacementMode == variable.PlacementModeIgnore { + if sessVars.PlacementMode == vardef.PlacementModeIgnore { dbInfo.PlacementPolicyRef = nil sessVars.StmtCtx.AppendNote( - errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", variable.PlacementModeIgnore), + errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", vardef.PlacementModeIgnore), ) return nil } @@ -568,9 +568,9 @@ func handleDatabasePlacement(ctx sessionctx.Context, dbInfo *model.DBInfo) error func handleTablePlacement(ctx sessionctx.Context, tbInfo *model.TableInfo) error { sessVars := ctx.GetSessionVars() - if sessVars.PlacementMode == variable.PlacementModeIgnore && removeTablePlacement(tbInfo) { + if sessVars.PlacementMode == vardef.PlacementModeIgnore && removeTablePlacement(tbInfo) { sessVars.StmtCtx.AppendNote( - errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", variable.PlacementModeIgnore), + errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", vardef.PlacementModeIgnore), ) return nil } @@ -595,9 +595,9 @@ func handleTablePlacement(ctx sessionctx.Context, tbInfo *model.TableInfo) error func handlePartitionPlacement(ctx sessionctx.Context, partInfo *model.PartitionInfo) error { sessVars := ctx.GetSessionVars() - if sessVars.PlacementMode == variable.PlacementModeIgnore && removePartitionPlacement(partInfo) { + if sessVars.PlacementMode == vardef.PlacementModeIgnore && removePartitionPlacement(partInfo) { sessVars.StmtCtx.AppendNote( - errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", variable.PlacementModeIgnore), + errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", vardef.PlacementModeIgnore), ) return nil } @@ -634,9 +634,9 @@ func checkAndNormalizePlacementPolicy(ctx sessionctx.Context, placementPolicyRef func checkIgnorePlacementDDL(ctx sessionctx.Context) bool { sessVars := ctx.GetSessionVars() - if sessVars.PlacementMode == variable.PlacementModeIgnore { + if sessVars.PlacementMode == vardef.PlacementModeIgnore { sessVars.StmtCtx.AppendNote( - errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", variable.PlacementModeIgnore), + errors.NewNoStackErrorf("Placement is ignored when TIDB_PLACEMENT_MODE is '%s'", vardef.PlacementModeIgnore), ) return true } diff --git a/pkg/ddl/placement_policy_ddl_test.go b/pkg/ddl/placement_policy_ddl_test.go index ded9710e3af59..a7eb3aae0e7cc 100644 --- a/pkg/ddl/placement_policy_ddl_test.go +++ b/pkg/ddl/placement_policy_ddl_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/stretchr/testify/require" @@ -125,7 +125,7 @@ func TestPlacementPolicyInUse(t *testing.T) { t4.State = model.StatePublic db1.Deprecated.Tables = append(db1.Deprecated.Tables, t4) - builder := infoschema.NewBuilder(dom, nil, infoschema.NewData(), variable.SchemaCacheSize.Load() > 0) + builder := infoschema.NewBuilder(dom, nil, infoschema.NewData(), vardef.SchemaCacheSize.Load() > 0) err = builder.InitWithDBInfos( []*model.DBInfo{db1, db2, dbP}, []*model.PolicyInfo{p1, p2, p3, p4, p5}, diff --git a/pkg/ddl/primary_key_handle_test.go b/pkg/ddl/primary_key_handle_test.go index d86e0db9f9c08..0f485600c42ac 100644 --- a/pkg/ddl/primary_key_handle_test.go +++ b/pkg/ddl/primary_key_handle_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/tablecodec" @@ -188,7 +188,7 @@ func TestMultiRegionGetTableEndCommonHandle(t *testing.T) { })) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("create table t(a varchar(20), b int, c float, d bigint, primary key (a, b, c))") var builder strings.Builder @@ -221,7 +221,7 @@ func TestGetTableEndCommonHandle(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("create table t(a varchar(15), b bigint, c int, primary key (a, b))") tk.MustExec("create table t1(a varchar(15), b bigint, c int, primary key (a(2), b))") @@ -256,7 +256,7 @@ func TestCreateClusteredIndex(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("CREATE TABLE t1 (a int primary key, b int)") tk.MustExec("CREATE TABLE t2 (a varchar(255) primary key, b int)") tk.MustExec("CREATE TABLE t3 (a int, b int, c int, primary key (a, b))") @@ -296,7 +296,7 @@ func TestCreateClusteredIndex(t *testing.T) { require.NoError(t, err) require.True(t, tbl.Meta().IsCommonHandle) - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly tk.MustExec("CREATE TABLE t7 (a varchar(255) primary key, b int)") is = domain.GetDomain(tk.Session()).InfoSchema() tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t7")) diff --git a/pkg/ddl/reorg.go b/pkg/ddl/reorg.go index c8efcd8342764..1228861aa3354 100644 --- a/pkg/ddl/reorg.go +++ b/pkg/ddl/reorg.go @@ -45,6 +45,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/table" @@ -225,7 +226,7 @@ func (*reorgTableMutateContext) GetExchangePartitionDMLSupport() (tblctx.Exchang // newReorgTableMutateContext creates a new table.MutateContext for reorganization. func newReorgTableMutateContext(exprCtx exprctx.ExprContext) table.MutateContext { rowEncoder := &rowcodec.Encoder{ - Enable: variable.GetDDLReorgRowFormat() != variable.DefTiDBRowFormatV1, + Enable: vardef.GetDDLReorgRowFormat() != vardef.DefTiDBRowFormatV1, } encodingConfig := tblctx.RowEncodingConfig{ @@ -241,7 +242,7 @@ func newReorgTableMutateContext(exprCtx exprctx.ExprContext) table.MutateContext // we still provide a valid one to keep the context complete and to avoid panic if it is used in the future. shardID: variable.NewRowIDShardGenerator( rand.New(rand.NewSource(time.Now().UnixNano())), // #nosec G404 - variable.DefTiDBShardAllocateStep, + vardef.DefTiDBShardAllocateStep, ), } } diff --git a/pkg/ddl/rollingback.go b/pkg/ddl/rollingback.go index 25ce4dd8ce539..6b3ab5e37f4dd 100644 --- a/pkg/ddl/rollingback.go +++ b/pkg/ddl/rollingback.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util/dbterror" "go.uber.org/zap" ) @@ -671,7 +671,7 @@ func convertJob2RollbackJob(w *worker, jobCtx *jobContext, job *model.Job) (ver if err1 := loadDDLVars(w); err1 != nil { logger.Error("load DDL global variable failed", zap.Error(err1)) } - errorCount := variable.GetDDLErrorCountLimit() + errorCount := vardef.GetDDLErrorCountLimit() if job.ErrorCount > errorCount { logger.Warn("rollback DDL job error count exceed the limit, cancelled it now", zap.Int64("errorCountLimit", errorCount)) job.Error = toTError(errors.Errorf("rollback DDL job error count exceed the limit %d, cancelled it now", errorCount)) diff --git a/pkg/ddl/schematracker/BUILD.bazel b/pkg/ddl/schematracker/BUILD.bazel index 5eda9f6bf3987..2c427f1b9776c 100644 --- a/pkg/ddl/schematracker/BUILD.bazel +++ b/pkg/ddl/schematracker/BUILD.bazel @@ -25,6 +25,7 @@ go_library( "//pkg/parser/mysql", "//pkg/parser/types", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics/handle", "//pkg/store/helper", diff --git a/pkg/ddl/schematracker/checker.go b/pkg/ddl/schematracker/checker.go index 936f874b4f831..5aac03ebcc6dd 100644 --- a/pkg/ddl/schematracker/checker.go +++ b/pkg/ddl/schematracker/checker.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics/handle" "github.com/pingcap/tidb/pkg/store/helper" @@ -511,7 +512,7 @@ func (d *Checker) Stats(vars *variable.SessionVars) (map[string]any, error) { } // GetScope implements the DDL interface. -func (d *Checker) GetScope(status string) variable.ScopeFlag { +func (d *Checker) GetScope(status string) vardef.ScopeFlag { return d.realDDL.GetScope(status) } diff --git a/pkg/ddl/schematracker/dm_tracker.go b/pkg/ddl/schematracker/dm_tracker.go index e029e7495d2cf..d44f68fcb47d1 100644 --- a/pkg/ddl/schematracker/dm_tracker.go +++ b/pkg/ddl/schematracker/dm_tracker.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" field_types "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/util/collate" @@ -199,7 +199,7 @@ func (d *SchemaTracker) CreateTable(ctx sessionctx.Context, s *ast.CreateTableSt // suppress ErrTooLongKey metabuild.WithSuppressTooLongIndexErr(true), // support drop PK - metabuild.WithClusteredIndexDefMode(variable.ClusteredIndexDefModeOff), + metabuild.WithClusteredIndexDefMode(vardef.ClusteredIndexDefModeOff), ) // build tableInfo var ( diff --git a/pkg/ddl/schemaver/BUILD.bazel b/pkg/ddl/schemaver/BUILD.bazel index eae7a867057cf..027ba5816c697 100644 --- a/pkg/ddl/schemaver/BUILD.bazel +++ b/pkg/ddl/schemaver/BUILD.bazel @@ -13,7 +13,7 @@ go_library( "//pkg/ddl/util", "//pkg/domain/infosync", "//pkg/metrics", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/util", "//pkg/util/disttask", "@com_github_pingcap_errors//:errors", @@ -39,7 +39,7 @@ go_test( "//pkg/ddl/util", "//pkg/domain/infosync", "//pkg/parser/terror", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/util", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", diff --git a/pkg/ddl/schemaver/mem_syncer.go b/pkg/ddl/schemaver/mem_syncer.go index a245b49a55e9d..f6999a8719238 100644 --- a/pkg/ddl/schemaver/mem_syncer.go +++ b/pkg/ddl/schemaver/mem_syncer.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" clientv3 "go.etcd.io/etcd/client/v3" ) @@ -68,7 +68,7 @@ func (s *MemSyncer) UpdateSelfVersion(_ context.Context, jobID int64, version in failpoint.Return(errors.New("mock update mdl to etcd error")) } }) - if variable.EnableMDL.Load() { + if vardef.EnableMDL.Load() { s.mdlSchemaVersions.Store(jobID, version) } else { atomic.StoreInt64(&s.selfSchemaVersion, version) @@ -117,7 +117,7 @@ func (s *MemSyncer) WaitVersionSynced(ctx context.Context, jobID int64, latestVe case <-ctx.Done(): return errors.Trace(ctx.Err()) case <-ticker.C: - if variable.EnableMDL.Load() { + if vardef.EnableMDL.Load() { ver, ok := s.mdlSchemaVersions.Load(jobID) if ok && ver.(int64) >= latestVer { return nil diff --git a/pkg/ddl/schemaver/syncer.go b/pkg/ddl/schemaver/syncer.go index 0ce5258a1030b..f5c2ca738e4ff 100644 --- a/pkg/ddl/schemaver/syncer.go +++ b/pkg/ddl/schemaver/syncer.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/util" "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/metrics" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" tidbutil "github.com/pingcap/tidb/pkg/util" disttaskutil "github.com/pingcap/tidb/pkg/util/disttask" "go.etcd.io/etcd/api/v3/mvccpb" @@ -279,7 +279,7 @@ func (s *etcdSyncer) UpdateSelfVersion(ctx context.Context, jobID int64, version ver := strconv.FormatInt(version, 10) var err error var path string - if variable.EnableMDL.Load() { + if vardef.EnableMDL.Load() { // If jobID is 0, it doesn't need to put into etcd `DDLAllSchemaVersionsByJob` key. if jobID == 0 { return nil @@ -322,7 +322,7 @@ func (s *etcdSyncer) removeSelfVersionPath() error { // WaitVersionSynced implements Syncer.WaitVersionSynced interface. func (s *etcdSyncer) WaitVersionSynced(ctx context.Context, jobID int64, latestVer int64) error { startTime := time.Now() - if !variable.EnableMDL.Load() { + if !vardef.EnableMDL.Load() { time.Sleep(CheckVersFirstWaitTime) } notMatchVerCnt := 0 @@ -345,7 +345,7 @@ func (s *etcdSyncer) WaitVersionSynced(ctx context.Context, jobID int64, latestV return errors.Trace(err) } - if variable.EnableMDL.Load() { + if vardef.EnableMDL.Load() { serverInfos, err := infosync.GetAllServerInfo(ctx) if err != nil { return err @@ -372,7 +372,7 @@ func (s *etcdSyncer) WaitVersionSynced(ctx context.Context, jobID int64, latestV } // Check all schema versions. - if variable.EnableMDL.Load() { + if vardef.EnableMDL.Load() { notifyCh := make(chan struct{}) var unmatchedNodeInfo atomic.Pointer[string] matchFn := func(nodeVersions map[string]int64) bool { diff --git a/pkg/ddl/schemaver/syncer_nokit_test.go b/pkg/ddl/schemaver/syncer_nokit_test.go index 019cb5c79a7e7..aec80b1c7f6b0 100644 --- a/pkg/ddl/schemaver/syncer_nokit_test.go +++ b/pkg/ddl/schemaver/syncer_nokit_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/ddl/util" "github.com/pingcap/tidb/pkg/domain/infosync" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/stretchr/testify/require" "go.etcd.io/etcd/api/v3/mvccpb" "go.etcd.io/etcd/tests/v3/integration" @@ -174,7 +174,7 @@ func TestSyncJobSchemaVerLoop(t *testing.T) { require.NoError(t, err) // job 4 is matched using WaitVersionSynced - variable.EnableMDL.Store(true) + vardef.EnableMDL.Store(true) serverInfos := map[string]*infosync.ServerInfo{"aa": {ID: "aa", IP: "test", Port: 4000}} bytes, err := json.Marshal(serverInfos) require.NoError(t, err) diff --git a/pkg/ddl/schemaver/syncer_test.go b/pkg/ddl/schemaver/syncer_test.go index a62bd24250b13..afbb663916856 100644 --- a/pkg/ddl/schemaver/syncer_test.go +++ b/pkg/ddl/schemaver/syncer_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/schemaver" util2 "github.com/pingcap/tidb/pkg/ddl/util" "github.com/pingcap/tidb/pkg/parser/terror" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" "go.etcd.io/etcd/api/v3/mvccpb" @@ -39,7 +39,7 @@ import ( const minInterval = 10 * time.Nanosecond // It's used to test timeout. func TestSyncerSimple(t *testing.T) { - variable.EnableMDL.Store(false) + vardef.EnableMDL.Store(false) if runtime.GOOS == "windows" { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } diff --git a/pkg/ddl/serverstate/BUILD.bazel b/pkg/ddl/serverstate/BUILD.bazel index e9e1cf420e802..4614fe956675a 100644 --- a/pkg/ddl/serverstate/BUILD.bazel +++ b/pkg/ddl/serverstate/BUILD.bazel @@ -32,7 +32,7 @@ go_test( ":serverstate", "//pkg/ddl/schemaver", "//pkg/ddl/util", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/util", "@com_github_stretchr_testify//require", "@io_etcd_go_etcd_api_v3//mvccpb", diff --git a/pkg/ddl/serverstate/syncer_test.go b/pkg/ddl/serverstate/syncer_test.go index a6c181f2754b5..a699449290768 100644 --- a/pkg/ddl/serverstate/syncer_test.go +++ b/pkg/ddl/serverstate/syncer_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/schemaver" "github.com/pingcap/tidb/pkg/ddl/serverstate" util2 "github.com/pingcap/tidb/pkg/ddl/util" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" "go.etcd.io/etcd/api/v3/mvccpb" @@ -43,7 +43,7 @@ func checkRespKV(t *testing.T, kvCount int, key, val string, kvs ...*mvccpb.KeyV } func TestStateSyncerSimple(t *testing.T) { - variable.EnableMDL.Store(false) + vardef.EnableMDL.Store(false) if runtime.GOOS == "windows" { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } diff --git a/pkg/ddl/split_region.go b/pkg/ddl/split_region.go index affa168b91eeb..305dd2fef2a3b 100644 --- a/pkg/ddl/split_region.go +++ b/pkg/ddl/split_region.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" tikverr "github.com/tikv/client-go/v2/error" @@ -46,7 +46,7 @@ func splitPartitionTableRegion(ctx sessionctx.Context, store kv.SplittableStore, regionIDs = append(regionIDs, SplitRecordRegion(ctxWithTimeout, store, def.ID, tbInfo.ID, scatterScope)) } } - if scatterScope != variable.ScatterOff { + if scatterScope != vardef.ScatterOff { WaitScatterRegionFinish(ctxWithTimeout, store, regionIDs...) } } @@ -61,7 +61,7 @@ func splitTableRegion(ctx sessionctx.Context, store kv.SplittableStore, tbInfo * } else { regionIDs = append(regionIDs, SplitRecordRegion(ctxWithTimeout, store, tbInfo.ID, tbInfo.ID, scatterScope)) } - if scatterScope != variable.ScatterOff { + if scatterScope != vardef.ScatterOff { WaitScatterRegionFinish(ctxWithTimeout, store, regionIDs...) } } @@ -70,9 +70,9 @@ func splitTableRegion(ctx sessionctx.Context, store kv.SplittableStore, tbInfo * // If it is `ScatterGlobal`, the scatter configured at global level uniformly use -1 as `tID`. func getScatterConfig(scope string, tableID int64) (scatter bool, tID int64) { switch scope { - case variable.ScatterTable: + case vardef.ScatterTable: return true, tableID - case variable.ScatterGlobal: + case vardef.ScatterGlobal: return true, -1 default: return false, tableID diff --git a/pkg/ddl/stat.go b/pkg/ddl/stat.go index 15a2227327b0a..8432926125d0f 100644 --- a/pkg/ddl/stat.go +++ b/pkg/ddl/stat.go @@ -16,6 +16,7 @@ package ddl import ( "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" ) @@ -25,7 +26,7 @@ var ( ) // GetScope gets the status variables scope. -func (*ddl) GetScope(_ string) variable.ScopeFlag { +func (*ddl) GetScope(_ string) vardef.ScopeFlag { // Now ddl status variables scope are all default scope. return variable.DefaultStatusVarScopeFlag } diff --git a/pkg/ddl/table.go b/pkg/ddl/table.go index 1b00f53862631..1691212fda278 100644 --- a/pkg/ddl/table.go +++ b/pkg/ddl/table.go @@ -36,7 +36,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" field_types "github.com/pingcap/tidb/pkg/parser/types" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/tablecodec" @@ -897,7 +897,7 @@ func adjustForeignKeyChildTableInfoAfterRenameTable( infoCache *infoschema.InfoCache, t *meta.Mutator, job *model.Job, fkh *foreignKeyHelper, tblInfo *model.TableInfo, oldSchemaName, oldTableName, newTableName ast.CIStr, newSchemaID int64) error { - if !variable.EnableForeignKey.Load() || newTableName.L == oldTableName.L { + if !vardef.EnableForeignKey.Load() || newTableName.L == oldTableName.L { return nil } is := infoCache.GetLatest() diff --git a/pkg/ddl/tests/fail/BUILD.bazel b/pkg/ddl/tests/fail/BUILD.bazel index f62a676b23f49..4ef791eeb51a8 100644 --- a/pkg/ddl/tests/fail/BUILD.bazel +++ b/pkg/ddl/tests/fail/BUILD.bazel @@ -21,7 +21,7 @@ go_test( "//pkg/kv", "//pkg/parser/ast", "//pkg/session", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/mockstore", "//pkg/tablecodec", "//pkg/testkit", diff --git a/pkg/ddl/tests/fail/fail_db_test.go b/pkg/ddl/tests/fail/fail_db_test.go index dd1265e3e3c7b..ec1bd9872f859 100644 --- a/pkg/ddl/tests/fail/fail_db_test.go +++ b/pkg/ddl/tests/fail/fail_db_test.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/testkit" @@ -128,10 +128,10 @@ func TestHalfwayCancelOperations(t *testing.T) { tk.MustExec("use cancel_job_db") tk.MustExec("select * from tx") // test for exchanging partition - limit := variable.GetDDLErrorCountLimit() - variable.SetDDLErrorCountLimit(3) + limit := vardef.GetDDLErrorCountLimit() + vardef.SetDDLErrorCountLimit(3) defer func() { - variable.SetDDLErrorCountLimit(limit) + vardef.SetDDLErrorCountLimit(limit) }() require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/exchangePartitionErr", `return(true)`)) defer func() { @@ -321,7 +321,7 @@ func TestGenGlobalIDFail(t *testing.T) { func TestRunDDLJobPanicEnableClusteredIndex(t *testing.T) { s := createFailDBSuite(t) testAddIndexWorkerNum(t, s, func(tk *testkit.TestKit) { - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("create table test_add_index (c1 bigint, c2 bigint, c3 bigint, primary key(c1, c3))") }) } @@ -348,7 +348,7 @@ func TestRunDDLJobPanicEnableFastCreateTable(t *testing.T) { } func testAddIndexWorkerNum(t *testing.T, s *failedSuite, test func(*testkit.TestKit)) { - if variable.EnableDistTask.Load() { + if vardef.EnableDistTask.Load() { t.Skip("dist reorg didn't support checkBackfillWorkerNum, skip this test") } @@ -388,7 +388,7 @@ func testAddIndexWorkerNum(t *testing.T, s *failedSuite, test func(*testkit.Test err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) require.NoError(t, err) - originDDLAddIndexWorkerCnt := variable.GetDDLReorgWorkerCounter() + originDDLAddIndexWorkerCnt := vardef.GetDDLReorgWorkerCounter() lastSetWorkerCnt := originDDLAddIndexWorkerCnt atomic.StoreInt32(&ddl.TestCheckWorkerNumber, lastSetWorkerCnt) ddl.TestCheckWorkerNumber = lastSetWorkerCnt @@ -532,7 +532,7 @@ func TestModifyColumn(t *testing.T) { maxBatch := 20 batchCnt := 100 // Make sure there are no duplicate keys. - defaultBatchSize := variable.DefTiDBDDLReorgBatchSize * variable.DefTiDBDDLReorgWorkerCount + defaultBatchSize := vardef.DefTiDBDDLReorgBatchSize * vardef.DefTiDBDDLReorgWorkerCount base := defaultBatchSize * 20 for i := 1; i < batchCnt; i++ { n := base + i*defaultBatchSize + i diff --git a/pkg/ddl/tests/partition/BUILD.bazel b/pkg/ddl/tests/partition/BUILD.bazel index 25a8e3a23c3ef..8d55cd1fff863 100644 --- a/pkg/ddl/tests/partition/BUILD.bazel +++ b/pkg/ddl/tests/partition/BUILD.bazel @@ -30,7 +30,7 @@ go_test( "//pkg/parser/terror", "//pkg/session", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/sessiontxn", "//pkg/statistics/handle/ddl/testutil", "//pkg/store/gcworker", diff --git a/pkg/ddl/tests/partition/db_partition_test.go b/pkg/ddl/tests/partition/db_partition_test.go index 399d9325ac684..7dc7523f3c8b1 100644 --- a/pkg/ddl/tests/partition/db_partition_test.go +++ b/pkg/ddl/tests/partition/db_partition_test.go @@ -39,7 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" "github.com/pingcap/tidb/pkg/store/mockstore" @@ -2753,7 +2753,7 @@ func TestAddPartitionReplicaBiggerThanTiFlashStores(t *testing.T) { require.Error(t, err) require.EqualError(t, err, "[ddl:-1][ddl] the tiflash replica count: 1 should be less than the total tiflash server count: 0") // Test `add partition` waiting TiFlash replica can exit when its retry count is beyond the limitation. - originErrCountLimit := variable.GetDDLErrorCountLimit() + originErrCountLimit := vardef.GetDDLErrorCountLimit() tk.MustExec("set @@global.tidb_ddl_error_count_limit = 3") defer func() { tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_error_count_limit = %v", originErrCountLimit)) diff --git a/pkg/ddl/tests/serial/BUILD.bazel b/pkg/ddl/tests/serial/BUILD.bazel index 9fdfed5608c2a..4f301d2d9f459 100644 --- a/pkg/ddl/tests/serial/BUILD.bazel +++ b/pkg/ddl/tests/serial/BUILD.bazel @@ -25,7 +25,7 @@ go_test( "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/session", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/sessiontxn", "//pkg/store/mockstore", "//pkg/table", diff --git a/pkg/ddl/tests/serial/serial_test.go b/pkg/ddl/tests/serial/serial_test.go index ec753f0d958c4..88ffc7d002e05 100644 --- a/pkg/ddl/tests/serial/serial_test.go +++ b/pkg/ddl/tests/serial/serial_test.go @@ -38,7 +38,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/table" @@ -774,7 +774,7 @@ func TestCancelJobByErrorCountLimit(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t") - limit := variable.GetDDLErrorCountLimit() + limit := vardef.GetDDLErrorCountLimit() tk.MustExec("set @@global.tidb_ddl_error_count_limit = 16") err := util.LoadDDLVars(tk.Session()) require.NoError(t, err) @@ -791,7 +791,7 @@ func TestTruncateTableUpdateSchemaVersionErr(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t") - limit := variable.GetDDLErrorCountLimit() + limit := vardef.GetDDLErrorCountLimit() tk.MustExec("set @@global.tidb_ddl_error_count_limit = 5") err := util.LoadDDLVars(tk.Session()) require.NoError(t, err) @@ -1177,7 +1177,7 @@ func TestCreateTableNoBlock(t *testing.T) { defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/checkOwnerCheckAllVersionsWaitTime")) }() - save := variable.GetDDLErrorCountLimit() + save := vardef.GetDDLErrorCountLimit() tk.MustExec("set @@global.tidb_ddl_error_count_limit = 1") defer func() { tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_error_count_limit = %v", save)) diff --git a/pkg/ddl/util/BUILD.bazel b/pkg/ddl/util/BUILD.bazel index 7d20ab229c60a..b21beec04f1a8 100644 --- a/pkg/ddl/util/BUILD.bazel +++ b/pkg/ddl/util/BUILD.bazel @@ -18,7 +18,7 @@ go_library( "//pkg/metrics", "//pkg/parser/terror", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/table/tables", "//pkg/util/chunk", "//pkg/util/sqlexec", diff --git a/pkg/ddl/util/util.go b/pkg/ddl/util/util.go index ed992e67b33b0..4c1a0e708bac6 100644 --- a/pkg/ddl/util/util.go +++ b/pkg/ddl/util/util.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/sqlexec" @@ -187,12 +187,12 @@ func UpdateDeleteRange(sctx sessionctx.Context, dr DelRangeTask, newStartKey, ol func LoadDDLReorgVars(ctx context.Context, sctx sessionctx.Context) error { // close issue #21391 // variable.TiDBRowFormatVersion is used to encode the new row for column type change. - return loadGlobalVars(ctx, sctx, []string{variable.TiDBDDLReorgWorkerCount, variable.TiDBDDLReorgBatchSize, variable.TiDBRowFormatVersion}) + return loadGlobalVars(ctx, sctx, []string{vardef.TiDBDDLReorgWorkerCount, vardef.TiDBDDLReorgBatchSize, vardef.TiDBRowFormatVersion}) } // LoadDDLVars loads ddl variable from mysql.global_variables. func LoadDDLVars(ctx sessionctx.Context) error { - return loadGlobalVars(context.Background(), ctx, []string{variable.TiDBDDLErrorCountLimit}) + return loadGlobalVars(context.Background(), ctx, []string{vardef.TiDBDDLErrorCountLimit}) } // loadGlobalVars loads global variable from mysql.global_variables. diff --git a/pkg/distsql/BUILD.bazel b/pkg/distsql/BUILD.bazel index d97f39d2ce1dd..29f9ce8a27b44 100644 --- a/pkg/distsql/BUILD.bazel +++ b/pkg/distsql/BUILD.bazel @@ -23,7 +23,7 @@ go_library( "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner/util", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/copr", "//pkg/tablecodec", "//pkg/types", @@ -76,7 +76,7 @@ go_test( "//pkg/resourcegroup", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/copr", "//pkg/tablecodec", "//pkg/testkit/testsetup", diff --git a/pkg/distsql/context_test.go b/pkg/distsql/context_test.go index 9cb394fccaf3e..7a3ba7542b5f1 100644 --- a/pkg/distsql/context_test.go +++ b/pkg/distsql/context_test.go @@ -17,7 +17,7 @@ package distsql import ( distsqlctx "github.com/pingcap/tidb/pkg/distsql/context" "github.com/pingcap/tidb/pkg/errctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" contextutil "github.com/pingcap/tidb/pkg/util/context" ) @@ -25,16 +25,16 @@ import ( func NewDistSQLContextForTest() *distsqlctx.DistSQLContext { return &distsqlctx.DistSQLContext{ WarnHandler: contextutil.NewFuncWarnAppenderForTest(func(level string, err error) {}), - TiFlashMaxThreads: variable.DefTiFlashMaxThreads, - TiFlashMaxBytesBeforeExternalJoin: variable.DefTiFlashMaxBytesBeforeExternalJoin, - TiFlashMaxBytesBeforeExternalGroupBy: variable.DefTiFlashMaxBytesBeforeExternalGroupBy, - TiFlashMaxBytesBeforeExternalSort: variable.DefTiFlashMaxBytesBeforeExternalSort, - TiFlashMaxQueryMemoryPerNode: variable.DefTiFlashMemQuotaQueryPerNode, - TiFlashQuerySpillRatio: variable.DefTiFlashQuerySpillRatio, + TiFlashMaxThreads: vardef.DefTiFlashMaxThreads, + TiFlashMaxBytesBeforeExternalJoin: vardef.DefTiFlashMaxBytesBeforeExternalJoin, + TiFlashMaxBytesBeforeExternalGroupBy: vardef.DefTiFlashMaxBytesBeforeExternalGroupBy, + TiFlashMaxBytesBeforeExternalSort: vardef.DefTiFlashMaxBytesBeforeExternalSort, + TiFlashMaxQueryMemoryPerNode: vardef.DefTiFlashMemQuotaQueryPerNode, + TiFlashQuerySpillRatio: vardef.DefTiFlashQuerySpillRatio, - DistSQLConcurrency: variable.DefDistSQLScanConcurrency, - MinPagingSize: variable.DefMinPagingSize, - MaxPagingSize: variable.DefMaxPagingSize, + DistSQLConcurrency: vardef.DefDistSQLScanConcurrency, + MinPagingSize: vardef.DefMinPagingSize, + MaxPagingSize: vardef.DefMaxPagingSize, ResourceGroupName: "default", ErrCtx: errctx.NewContext(contextutil.IgnoreWarn), diff --git a/pkg/distsql/distsql.go b/pkg/distsql/distsql.go index 2404c1fade1b3..812df6ef93f5e 100644 --- a/pkg/distsql/distsql.go +++ b/pkg/distsql/distsql.go @@ -24,7 +24,7 @@ import ( distsqlctx "github.com/pingcap/tidb/pkg/distsql/context" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/metrics" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/topsql/stmtstats" @@ -119,23 +119,23 @@ func Select(ctx context.Context, dctx *distsqlctx.DistSQLContext, kvReq *kv.Requ // SetTiFlashConfVarsInContext set some TiFlash config variables in context. func SetTiFlashConfVarsInContext(ctx context.Context, dctx *distsqlctx.DistSQLContext) context.Context { if dctx.TiFlashMaxThreads != -1 { - ctx = metadata.AppendToOutgoingContext(ctx, variable.TiDBMaxTiFlashThreads, strconv.FormatInt(dctx.TiFlashMaxThreads, 10)) + ctx = metadata.AppendToOutgoingContext(ctx, vardef.TiDBMaxTiFlashThreads, strconv.FormatInt(dctx.TiFlashMaxThreads, 10)) } if dctx.TiFlashMaxBytesBeforeExternalJoin != -1 { - ctx = metadata.AppendToOutgoingContext(ctx, variable.TiDBMaxBytesBeforeTiFlashExternalJoin, strconv.FormatInt(dctx.TiFlashMaxBytesBeforeExternalJoin, 10)) + ctx = metadata.AppendToOutgoingContext(ctx, vardef.TiDBMaxBytesBeforeTiFlashExternalJoin, strconv.FormatInt(dctx.TiFlashMaxBytesBeforeExternalJoin, 10)) } if dctx.TiFlashMaxBytesBeforeExternalGroupBy != -1 { - ctx = metadata.AppendToOutgoingContext(ctx, variable.TiDBMaxBytesBeforeTiFlashExternalGroupBy, strconv.FormatInt(dctx.TiFlashMaxBytesBeforeExternalGroupBy, 10)) + ctx = metadata.AppendToOutgoingContext(ctx, vardef.TiDBMaxBytesBeforeTiFlashExternalGroupBy, strconv.FormatInt(dctx.TiFlashMaxBytesBeforeExternalGroupBy, 10)) } if dctx.TiFlashMaxBytesBeforeExternalSort != -1 { - ctx = metadata.AppendToOutgoingContext(ctx, variable.TiDBMaxBytesBeforeTiFlashExternalSort, strconv.FormatInt(dctx.TiFlashMaxBytesBeforeExternalSort, 10)) + ctx = metadata.AppendToOutgoingContext(ctx, vardef.TiDBMaxBytesBeforeTiFlashExternalSort, strconv.FormatInt(dctx.TiFlashMaxBytesBeforeExternalSort, 10)) } if dctx.TiFlashMaxQueryMemoryPerNode <= 0 { - ctx = metadata.AppendToOutgoingContext(ctx, variable.TiFlashMemQuotaQueryPerNode, "0") + ctx = metadata.AppendToOutgoingContext(ctx, vardef.TiFlashMemQuotaQueryPerNode, "0") } else { - ctx = metadata.AppendToOutgoingContext(ctx, variable.TiFlashMemQuotaQueryPerNode, strconv.FormatInt(dctx.TiFlashMaxQueryMemoryPerNode, 10)) + ctx = metadata.AppendToOutgoingContext(ctx, vardef.TiFlashMemQuotaQueryPerNode, strconv.FormatInt(dctx.TiFlashMaxQueryMemoryPerNode, 10)) } - ctx = metadata.AppendToOutgoingContext(ctx, variable.TiFlashQuerySpillRatio, strconv.FormatFloat(dctx.TiFlashQuerySpillRatio, 'f', -1, 64)) + ctx = metadata.AppendToOutgoingContext(ctx, vardef.TiFlashQuerySpillRatio, strconv.FormatFloat(dctx.TiFlashQuerySpillRatio, 'f', -1, 64)) return ctx } diff --git a/pkg/distsql/request_builder_test.go b/pkg/distsql/request_builder_test.go index 7dbad068608dc..65872577ee840 100644 --- a/pkg/distsql/request_builder_test.go +++ b/pkg/distsql/request_builder_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/resourcegroup" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/codec" @@ -316,7 +316,7 @@ func TestRequestBuilder1(t *testing.T) { Cacheable: true, KeepOrder: false, Desc: false, - Concurrency: variable.DefDistSQLScanConcurrency, + Concurrency: vardef.DefDistSQLScanConcurrency, IsolationLevel: 0, Priority: 0, NotFillCache: false, @@ -400,7 +400,7 @@ func TestRequestBuilder2(t *testing.T) { Cacheable: true, KeepOrder: false, Desc: false, - Concurrency: variable.DefDistSQLScanConcurrency, + Concurrency: vardef.DefDistSQLScanConcurrency, IsolationLevel: 0, Priority: 0, NotFillCache: false, @@ -450,7 +450,7 @@ func TestRequestBuilder3(t *testing.T) { Cacheable: true, KeepOrder: false, Desc: false, - Concurrency: variable.DefDistSQLScanConcurrency, + Concurrency: vardef.DefDistSQLScanConcurrency, IsolationLevel: 0, Priority: 0, NotFillCache: false, @@ -499,7 +499,7 @@ func TestRequestBuilder4(t *testing.T) { Cacheable: true, KeepOrder: false, Desc: false, - Concurrency: variable.DefDistSQLScanConcurrency, + Concurrency: vardef.DefDistSQLScanConcurrency, IsolationLevel: 0, Priority: 0, NotFillCache: false, @@ -637,7 +637,7 @@ func TestRequestBuilder8(t *testing.T) { StartTs: 0x0, Data: []uint8(nil), KeyRanges: kv.NewNonPartitionedKeyRanges(nil), - Concurrency: variable.DefDistSQLScanConcurrency, + Concurrency: vardef.DefDistSQLScanConcurrency, IsolationLevel: 0, Priority: 0, MemTracker: (*memory.Tracker)(nil), @@ -663,7 +663,7 @@ func TestRequestBuilderTiKVClientReadTimeout(t *testing.T) { StartTs: 0x0, Data: []uint8(nil), KeyRanges: kv.NewNonPartitionedKeyRanges(nil), - Concurrency: variable.DefDistSQLScanConcurrency, + Concurrency: vardef.DefDistSQLScanConcurrency, IsolationLevel: 0, Priority: 0, MemTracker: (*memory.Tracker)(nil), @@ -690,7 +690,7 @@ func TestRequestBuilderMaxExecutionTime(t *testing.T) { StartTs: 0x0, Data: []uint8(nil), KeyRanges: kv.NewNonPartitionedKeyRanges(nil), - Concurrency: variable.DefDistSQLScanConcurrency, + Concurrency: vardef.DefDistSQLScanConcurrency, IsolationLevel: 0, Priority: 0, MemTracker: (*memory.Tracker)(nil), diff --git a/pkg/disttask/framework/storage/BUILD.bazel b/pkg/disttask/framework/storage/BUILD.bazel index 04481c4993c58..adc9c4efb8f46 100644 --- a/pkg/disttask/framework/storage/BUILD.bazel +++ b/pkg/disttask/framework/storage/BUILD.bazel @@ -16,7 +16,7 @@ go_library( "//pkg/disttask/framework/proto", "//pkg/kv", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/util", "//pkg/util/chunk", "//pkg/util/cpu", @@ -49,7 +49,7 @@ go_test( "//pkg/disttask/framework/testutil", "//pkg/kv", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/testkit/testfailpoint", "//pkg/testkit/testsetup", diff --git a/pkg/disttask/framework/storage/table_test.go b/pkg/disttask/framework/storage/table_test.go index 13cc884163838..ad63255edb4f3 100644 --- a/pkg/disttask/framework/storage/table_test.go +++ b/pkg/disttask/framework/storage/table_test.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/disttask/framework/testutil" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" "github.com/pingcap/tidb/pkg/util/sqlexec" @@ -262,9 +262,9 @@ func TestSwitchTaskStep(t *testing.T) { subtasksStepTwo[i] = proto.NewSubtask(proto.StepTwo, taskID, proto.TaskTypeExample, ":4000", 11, []byte(fmt.Sprintf("%d", i)), i+1) } - require.NoError(t, tk.Session().GetSessionVars().SetSystemVar(variable.TiDBMemQuotaQuery, "1024")) + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar(vardef.TiDBMemQuotaQuery, "1024")) require.NoError(t, tm.SwitchTaskStep(ctx, task, proto.TaskStateRunning, proto.StepTwo, subtasksStepTwo)) - value, ok := tk.Session().GetSessionVars().GetSystemVar(variable.TiDBMemQuotaQuery) + value, ok := tk.Session().GetSessionVars().GetSystemVar(vardef.TiDBMemQuotaQuery) require.True(t, ok) require.Equal(t, "1024", value) task, err = tm.GetTaskByID(ctx, taskID) diff --git a/pkg/disttask/framework/storage/task_table.go b/pkg/disttask/framework/storage/task_table.go index 38a601a31a6d1..86fb027cb7b9d 100644 --- a/pkg/disttask/framework/storage/task_table.go +++ b/pkg/disttask/framework/storage/task_table.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/disttask/framework/proto" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/sqlexec" @@ -599,14 +599,14 @@ func (mgr *TaskManager) SwitchTaskStep( ) error { return mgr.WithNewTxn(ctx, func(se sessionctx.Context) error { vars := se.GetSessionVars() - if vars.MemQuotaQuery < variable.DefTiDBMemQuotaQuery { + if vars.MemQuotaQuery < vardef.DefTiDBMemQuotaQuery { bak := vars.MemQuotaQuery - if err := vars.SetSystemVar(variable.TiDBMemQuotaQuery, - strconv.Itoa(variable.DefTiDBMemQuotaQuery)); err != nil { + if err := vars.SetSystemVar(vardef.TiDBMemQuotaQuery, + strconv.Itoa(vardef.DefTiDBMemQuotaQuery)); err != nil { return err } defer func() { - _ = vars.SetSystemVar(variable.TiDBMemQuotaQuery, strconv.Itoa(int(bak))) + _ = vars.SetSystemVar(vardef.TiDBMemQuotaQuery, strconv.Itoa(int(bak))) }() } err := mgr.updateTaskStateStep(ctx, se, task, nextState, nextStep) diff --git a/pkg/disttask/framework/taskexecutor/BUILD.bazel b/pkg/disttask/framework/taskexecutor/BUILD.bazel index 54cb568ac67ef..4559a5c04fd72 100644 --- a/pkg/disttask/framework/taskexecutor/BUILD.bazel +++ b/pkg/disttask/framework/taskexecutor/BUILD.bazel @@ -21,7 +21,7 @@ go_library( "//pkg/disttask/framework/taskexecutor/execute", "//pkg/lightning/log", "//pkg/metrics", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/util", "//pkg/util/backoff", "//pkg/util/cgroup", diff --git a/pkg/disttask/framework/taskexecutor/manager.go b/pkg/disttask/framework/taskexecutor/manager.go index 87f6a1841a7b6..b24c85d8fac94 100644 --- a/pkg/disttask/framework/taskexecutor/manager.go +++ b/pkg/disttask/framework/taskexecutor/manager.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/disttask/framework/storage" "github.com/pingcap/tidb/pkg/metrics" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" tidbutil "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/backoff" "github.com/pingcap/tidb/pkg/util/cgroup" @@ -184,7 +184,7 @@ func (m *Manager) handleTasksLoop() { m.handleTasks() // service scope might change, so we call WithLabelValues every time. - metrics.DistTaskUsedSlotsGauge.WithLabelValues(variable.ServiceScope.Load()). + metrics.DistTaskUsedSlotsGauge.WithLabelValues(vardef.ServiceScope.Load()). Set(float64(m.slotManager.usedSlots())) metrics.GlobalSortUploadWorkerCount.Set(float64(litstorage.GetActiveUploadWorkerCount())) } diff --git a/pkg/domain/BUILD.bazel b/pkg/domain/BUILD.bazel index 8164e05d09181..f8adc64e3efc9 100644 --- a/pkg/domain/BUILD.bazel +++ b/pkg/domain/BUILD.bazel @@ -59,6 +59,7 @@ go_library( "//pkg/sessionctx", "//pkg/sessionctx/sessionstates", "//pkg/sessionctx/sysproctrack", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/statistics/handle", @@ -155,6 +156,7 @@ go_test( "//pkg/parser/terror", "//pkg/server", "//pkg/session", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/store/mockstore", "//pkg/testkit", diff --git a/pkg/domain/db_test.go b/pkg/domain/db_test.go index 9b083f062b6e5..03de9325db3a7 100644 --- a/pkg/domain/db_test.go +++ b/pkg/domain/db_test.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/server" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/mathutil" @@ -187,7 +187,7 @@ func TestFetchAllSchemasWithTablesWithFailpoint(t *testing.T) { dbName := fmt.Sprintf("test_%d", i) tk.MustExec("create database " + dbName) } - variable.SchemaCacheSize.Store(1000000) + vardef.SchemaCacheSize.Store(1000000) dbs, err = domain.FetchAllSchemasWithTables(m) require.NoError(t, err) diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index 7eacee96175af..edc60301e4c53 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -69,6 +69,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" "github.com/pingcap/tidb/pkg/sessionctx/sysproctrack" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics/handle" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze" @@ -310,7 +311,7 @@ func (do *Domain) loadInfoSchema(startTS uint64, isSnapshot bool) (infoschema.In schemaTs = 0 } - enableV2 := variable.SchemaCacheSize.Load() > 0 + enableV2 := vardef.SchemaCacheSize.Load() > 0 currentSchemaVersion := int64(0) oldInfoSchema := do.infoCache.GetLatest() if oldInfoSchema != nil { @@ -503,7 +504,7 @@ const fetchSchemaConcurrency = 1 func (*Domain) splitForConcurrentFetch(schemas []*model.DBInfo) [][]*model.DBInfo { groupCnt := fetchSchemaConcurrency schemaCnt := len(schemas) - if variable.SchemaCacheSize.Load() > 0 && schemaCnt > 1000 { + if vardef.SchemaCacheSize.Load() > 0 && schemaCnt > 1000 { // TODO: Temporary solution to speed up when too many databases, will refactor it later. groupCnt = 8 } @@ -532,7 +533,7 @@ func (*Domain) fetchSchemasWithTables(ctx context.Context, schemas []*model.DBIn } var tables []*model.TableInfo var err error - if variable.SchemaCacheSize.Load() > 0 && !infoschema.IsSpecialDB(di.Name.L) { + if vardef.SchemaCacheSize.Load() > 0 && !infoschema.IsSpecialDB(di.Name.L) { name2ID, specialTableInfos, err := m.GetAllNameToIDAndTheMustLoadedTableInfo(di.ID) if err != nil { return err @@ -746,7 +747,7 @@ func (do *Domain) Store() kv.Storage { } // GetScope gets the status variables scope. -func (*Domain) GetScope(string) variable.ScopeFlag { +func (*Domain) GetScope(string) vardef.ScopeFlag { // Now domain status variables scope are all default scope. return variable.DefaultStatusVarScopeFlag } @@ -1056,7 +1057,7 @@ func (do *Domain) mdlCheckLoop() { return } - if !variable.EnableMDL.Load() { + if !vardef.EnableMDL.Load() { continue } @@ -1357,7 +1358,7 @@ func NewDomainWithEtcdClient(store kv.Storage, schemaLease time.Duration, statsL mdlCheckCh: make(chan struct{}), } - do.infoCache = infoschema.NewCache(do, int(variable.SchemaVersionCacheLimit.Load())) + do.infoCache = infoschema.NewCache(do, int(vardef.SchemaVersionCacheLimit.Load())) do.stopAutoAnalyze.Store(false) do.wg = util.NewWaitGroupEnhancedWrapper("domain", do.exit, config.GetGlobalConfig().TiDBEnableExitCheck) do.SchemaValidator = NewSchemaValidator(schemaLease, do) @@ -1628,7 +1629,7 @@ func (do *Domain) closestReplicaReadCheckLoop(ctx context.Context, pdClient pd.C // - The AZ if this tidb contains more tidb than other AZ and this tidb's id is the bigger one. func (do *Domain) checkReplicaRead(ctx context.Context, pdClient pd.Client) error { do.sysVarCache.RLock() - replicaRead := do.sysVarCache.global[variable.TiDBReplicaRead] + replicaRead := do.sysVarCache.global[vardef.TiDBReplicaRead] do.sysVarCache.RUnlock() if !strings.EqualFold(replicaRead, "closest-adaptive") { @@ -2741,7 +2742,7 @@ func (do *Domain) autoAnalyzeWorker() { // the probability of this happening is very high that the ticker condition and exist condition will be met // at the same time. // This causes the auto analyze task to be triggered all the time and block the shutdown of tidb. - if variable.RunAutoAnalyze.Load() && !do.stopAutoAnalyze.Load() && do.statsOwner.IsOwner() { + if vardef.RunAutoAnalyze.Load() && !do.stopAutoAnalyze.Load() && do.statsOwner.IsOwner() { statsHandle.HandleAutoAnalyze() } case <-do.exit: @@ -3273,8 +3274,8 @@ func (do *Domain) StopAutoAnalyze() { // InitInstancePlanCache initializes the instance level plan cache for this Domain. func (do *Domain) InitInstancePlanCache() { - hardLimit := variable.InstancePlanCacheMaxMemSize.Load() - softLimit := float64(hardLimit) * (1 - variable.InstancePlanCacheReservedPercentage.Load()) + hardLimit := vardef.InstancePlanCacheMaxMemSize.Load() + softLimit := float64(hardLimit) * (1 - vardef.InstancePlanCacheReservedPercentage.Load()) do.instancePlanCache = NewInstancePlanCache(int64(softLimit), hardLimit) // use a separate goroutine to avoid the eviction blocking other operations. do.wg.Run(do.planCacheEvictTrigger, "planCacheEvictTrigger") @@ -3299,8 +3300,8 @@ func (do *Domain) planCacheMetricsAndVars() { select { case <-ticker.C: // update limits - hardLimit := variable.InstancePlanCacheMaxMemSize.Load() - softLimit := int64(float64(hardLimit) * (1 - variable.InstancePlanCacheReservedPercentage.Load())) + hardLimit := vardef.InstancePlanCacheMaxMemSize.Load() + softLimit := int64(float64(hardLimit) * (1 - vardef.InstancePlanCacheReservedPercentage.Load())) curSoft, curHard := do.instancePlanCache.GetLimits() if curSoft != softLimit || curHard != hardLimit { do.instancePlanCache.SetLimits(softLimit, hardLimit) @@ -3331,7 +3332,7 @@ func (do *Domain) planCacheEvictTrigger() { case <-ticker.C: // trigger the eviction begin := time.Now() - enabled := variable.EnableInstancePlanCache.Load() + enabled := vardef.EnableInstancePlanCache.Load() detailInfo, numEvicted := do.instancePlanCache.Evict(!enabled) // evict all if the plan cache is disabled metrics2.GetPlanCacheInstanceEvict().Set(float64(numEvicted)) if numEvicted > 0 { @@ -3363,7 +3364,7 @@ func (do *Domain) SetupWorkloadBasedLearningWorker() { func (do *Domain) readTableCostWorker(wbLearningHandle *workloadbasedlearning.Handle) { // Recover the panic and log the error when worker exit. defer util.Recover(metrics.LabelDomain, "readTableCostWorker", nil, false) - readTableCostTicker := time.NewTicker(variable.WorkloadBasedLearningInterval.Load()) + readTableCostTicker := time.NewTicker(vardef.WorkloadBasedLearningInterval.Load()) defer func() { readTableCostTicker.Stop() logutil.BgLogger().Info("readTableCostWorker exited.") @@ -3371,7 +3372,7 @@ func (do *Domain) readTableCostWorker(wbLearningHandle *workloadbasedlearning.Ha for { select { case <-readTableCostTicker.C: - if variable.EnableWorkloadBasedLearning.Load() && do.statsOwner.IsOwner() { + if vardef.EnableWorkloadBasedLearning.Load() && do.statsOwner.IsOwner() { wbLearningHandle.HandleReadTableCost() } case <-do.exit: diff --git a/pkg/domain/domain_sysvars.go b/pkg/domain/domain_sysvars.go index 866e79e080ae3..86a5b193f10a4 100644 --- a/pkg/domain/domain_sysvars.go +++ b/pkg/domain/domain_sysvars.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/opt" @@ -58,7 +59,7 @@ func (do *Domain) setStatsCacheCapacity(c int64) { func (do *Domain) setPDClientDynamicOption(name, sVal string) error { switch name { - case variable.TiDBTSOClientBatchMaxWaitTime: + case vardef.TiDBTSOClientBatchMaxWaitTime: val, err := strconv.ParseFloat(sVal, 64) if err != nil { return err @@ -67,15 +68,15 @@ func (do *Domain) setPDClientDynamicOption(name, sVal string) error { if err != nil { return err } - variable.MaxTSOBatchWaitInterval.Store(val) - case variable.TiDBEnableTSOFollowerProxy: + vardef.MaxTSOBatchWaitInterval.Store(val) + case vardef.TiDBEnableTSOFollowerProxy: val := variable.TiDBOptOn(sVal) err := do.updatePDClient(opt.EnableTSOFollowerProxy, val) if err != nil { return err } - variable.EnableTSOFollowerProxy.Store(val) - case variable.PDEnableFollowerHandleRegion: + vardef.EnableTSOFollowerProxy.Store(val) + case vardef.PDEnableFollowerHandleRegion: val := variable.TiDBOptOn(sVal) // Note: EnableFollowerHandle is only used for region API now. // If pd support more APIs in follower, the pd option may be changed. @@ -83,16 +84,16 @@ func (do *Domain) setPDClientDynamicOption(name, sVal string) error { if err != nil { return err } - variable.EnablePDFollowerHandleRegion.Store(val) - case variable.TiDBTSOClientRPCMode: + vardef.EnablePDFollowerHandleRegion.Store(val) + case vardef.TiDBTSOClientRPCMode: var concurrency int switch sVal { - case variable.TSOClientRPCModeDefault: + case vardef.TSOClientRPCModeDefault: concurrency = 1 - case variable.TSOClientRPCModeParallel: + case vardef.TSOClientRPCModeParallel: concurrency = 2 - case variable.TSOClientRPCModeParallelFast: + case vardef.TSOClientRPCModeParallelFast: concurrency = 4 default: return variable.ErrWrongValueForVar.GenWithStackByArgs(name, sVal) diff --git a/pkg/domain/domain_test.go b/pkg/domain/domain_test.go index 3a4665c376eaa..f048ac6d6152d 100644 --- a/pkg/domain/domain_test.go +++ b/pkg/domain/domain_test.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/types" @@ -261,7 +262,7 @@ func TestClosestReplicaReadChecker(t *testing.T) { }() dom.sysVarCache.Lock() dom.sysVarCache.global = map[string]string{ - variable.TiDBReplicaRead: "closest-adaptive", + vardef.TiDBReplicaRead: "closest-adaptive", } dom.sysVarCache.Unlock() diff --git a/pkg/domain/infosync/BUILD.bazel b/pkg/domain/infosync/BUILD.bazel index edccd8a52482f..827cb35eb53e3 100644 --- a/pkg/domain/infosync/BUILD.bazel +++ b/pkg/domain/infosync/BUILD.bazel @@ -28,7 +28,7 @@ go_library( "//pkg/parser/terror", "//pkg/resourcegroup", "//pkg/session/cursor", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/helper", "//pkg/tablecodec", "//pkg/util", diff --git a/pkg/domain/infosync/info.go b/pkg/domain/infosync/info.go index cca9d6ad2d869..9b858055160e2 100644 --- a/pkg/domain/infosync/info.go +++ b/pkg/domain/infosync/info.go @@ -45,7 +45,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/resourcegroup" "github.com/pingcap/tidb/pkg/session/cursor" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" util2 "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/engine" @@ -780,7 +780,7 @@ func (is *InfoSyncer) ReportMinStartTS(store kv.Storage) { } now := oracle.GetTimeFromTS(currentVer.Ver) // GCMaxWaitTime is in seconds, GCMaxWaitTime * 1000 converts it to milliseconds. - startTSLowerLimit := oracle.GoTimeToLowerLimitStartTS(now, variable.GCMaxWaitTime.Load()*1000) + startTSLowerLimit := oracle.GoTimeToLowerLimitStartTS(now, vardef.GCMaxWaitTime.Load()*1000) minStartTS := oracle.GoTimeToTS(now) logutil.BgLogger().Debug("ReportMinStartTS", zap.Uint64("initial minStartTS", minStartTS), zap.Uint64("StartTSLowerLimit", startTSLowerLimit)) diff --git a/pkg/domain/plan_replayer.go b/pkg/domain/plan_replayer.go index fd4784aefdfc0..2a0a2c4b3da20 100644 --- a/pkg/domain/plan_replayer.go +++ b/pkg/domain/plan_replayer.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/chunk" @@ -464,7 +465,7 @@ func (w *planReplayerTaskDumpWorker) HandleTask(task *PlanReplayerDumpTask) (suc return true } - file, fileName, err := replayer.GeneratePlanReplayerFile(task.IsCapture, task.IsContinuesCapture, variable.EnableHistoricalStatsForCapture.Load()) + file, fileName, err := replayer.GeneratePlanReplayerFile(task.IsCapture, task.IsContinuesCapture, vardef.EnableHistoricalStatsForCapture.Load()) if err != nil { logutil.BgLogger().Warn("generate task file failed", zap.String("category", "plan-replayer-capture"), zap.String("sqlDigest", taskKey.SQLDigest), diff --git a/pkg/domain/plan_replayer_dump.go b/pkg/domain/plan_replayer_dump.go index 5d0f1fa1b4492..c671b781cd46c 100644 --- a/pkg/domain/plan_replayer_dump.go +++ b/pkg/domain/plan_replayer_dump.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/util" @@ -322,7 +323,7 @@ func DumpPlanReplayerInfo(ctx context.Context, sctx sessionctx.Context, // For continuous capture task, we dump stats in storage only if EnableHistoricalStatsForCapture is disabled. // For manual plan replayer dump command or capture, we directly dump stats in storage if task.IsCapture && task.IsContinuesCapture { - if !variable.EnableHistoricalStatsForCapture.Load() { + if !vardef.EnableHistoricalStatsForCapture.Load() { // Dump stats fallbackMsg, err := dumpStats(zw, pairs, do, 0) if err != nil { @@ -431,7 +432,7 @@ func dumpSQLMeta(zw *zip.Writer, task *PlanReplayerDumpTask) error { varMap[PlanReplayerTaskMetaIsContinues] = strconv.FormatBool(task.IsContinuesCapture) varMap[PlanReplayerTaskMetaSQLDigest] = task.SQLDigest varMap[PlanReplayerTaskMetaPlanDigest] = task.PlanDigest - varMap[PlanReplayerTaskEnableHistoricalStats] = strconv.FormatBool(variable.EnableHistoricalStatsForCapture.Load()) + varMap[PlanReplayerTaskEnableHistoricalStats] = strconv.FormatBool(vardef.EnableHistoricalStatsForCapture.Load()) if task.HistoricalStatsTS > 0 { varMap[PlanReplayerHistoricalStatsTS] = strconv.FormatUint(task.HistoricalStatsTS, 10) } @@ -600,7 +601,7 @@ func dumpSQLs(execStmts []ast.StmtNode, zw *zip.Writer) error { func dumpVariables(sctx sessionctx.Context, sessionVars *variable.SessionVars, zw *zip.Writer) error { varMap := make(map[string]string) for _, v := range variable.GetSysVars() { - if v.IsNoop && !variable.EnableNoopVariables.Load() { + if v.IsNoop && !vardef.EnableNoopVariables.Load() { continue } if infoschema.SysVarHiddenForSem(sctx, v.Name) { diff --git a/pkg/domain/schema_validator.go b/pkg/domain/schema_validator.go index 14d81b74dc727..f4d82d211b0d6 100644 --- a/pkg/domain/schema_validator.go +++ b/pkg/domain/schema_validator.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/metrics" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/tikv/client-go/v2/oracle" @@ -85,7 +85,7 @@ func NewSchemaValidator(lease time.Duration, do *Domain) SchemaValidator { return &schemaValidator{ isStarted: true, lease: lease, - deltaSchemaInfos: make([]deltaSchemaInfo, 0, variable.DefTiDBMaxDeltaSchemaCount), + deltaSchemaInfos: make([]deltaSchemaInfo, 0, vardef.DefTiDBMaxDeltaSchemaCount), do: do, } } @@ -261,7 +261,7 @@ func (s *schemaValidator) Check(txnTS uint64, schemaVer int64, relatedPhysicalTa // When disabling MDL -> enabling MDL, the old transaction's needCheckSchema is true, we need to check it. // When enabling MDL -> disabling MDL, the old transaction's needCheckSchema is false, so still need to check it, and variable EnableMDL is false now. - if needCheckSchema || !variable.EnableMDL.Load() { + if needCheckSchema || !vardef.EnableMDL.Load() { changed := s.isRelatedTablesChanged(schemaVer, relatedPhysicalTableIDs) if changed { return nil, ResultFail @@ -279,7 +279,7 @@ func (s *schemaValidator) Check(txnTS uint64, schemaVer int64, relatedPhysicalTa } func (s *schemaValidator) enqueue(schemaVersion int64, change *transaction.RelatedSchemaChange) { - maxCnt := int(variable.GetMaxDeltaSchemaCount()) + maxCnt := int(vardef.GetMaxDeltaSchemaCount()) if maxCnt <= 0 { logutil.BgLogger().Info("the schema validator enqueue", zap.Int("delta max count", maxCnt)) return diff --git a/pkg/domain/schema_validator_test.go b/pkg/domain/schema_validator_test.go index e1c241d3b4f37..83773b93f659e 100644 --- a/pkg/domain/schema_validator_test.go +++ b/pkg/domain/schema_validator_test.go @@ -18,7 +18,7 @@ import ( "testing" "time" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" @@ -111,19 +111,19 @@ func subTestSchemaValidatorGeneral(t *testing.T) { // subTestEnqueue is batched in TestSchemaValidator func subTestEnqueue(t *testing.T) { lease := 10 * time.Millisecond - originalCnt := variable.GetMaxDeltaSchemaCount() - defer variable.SetMaxDeltaSchemaCount(originalCnt) + originalCnt := vardef.GetMaxDeltaSchemaCount() + defer vardef.SetMaxDeltaSchemaCount(originalCnt) validator := NewSchemaValidator(lease, nil).(*schemaValidator) require.True(t, validator.IsStarted()) // maxCnt is 0. - variable.SetMaxDeltaSchemaCount(0) + vardef.SetMaxDeltaSchemaCount(0) validator.enqueue(1, &transaction.RelatedSchemaChange{PhyTblIDS: []int64{11}, ActionTypes: []uint64{11}}) require.Len(t, validator.deltaSchemaInfos, 0) // maxCnt is 10. - variable.SetMaxDeltaSchemaCount(10) + vardef.SetMaxDeltaSchemaCount(10) ds := []deltaSchemaInfo{ {0, []int64{1}, []uint64{1}}, {1, []int64{1}, []uint64{1}}, @@ -171,19 +171,19 @@ func subTestEnqueue(t *testing.T) { // subTestEnqueueActionType is batched in TestSchemaValidator func subTestEnqueueActionType(t *testing.T) { lease := 10 * time.Millisecond - originalCnt := variable.GetMaxDeltaSchemaCount() - defer variable.SetMaxDeltaSchemaCount(originalCnt) + originalCnt := vardef.GetMaxDeltaSchemaCount() + defer vardef.SetMaxDeltaSchemaCount(originalCnt) validator := NewSchemaValidator(lease, nil).(*schemaValidator) require.True(t, validator.IsStarted()) // maxCnt is 0. - variable.SetMaxDeltaSchemaCount(0) + vardef.SetMaxDeltaSchemaCount(0) validator.enqueue(1, &transaction.RelatedSchemaChange{PhyTblIDS: []int64{11}, ActionTypes: []uint64{11}}) require.Len(t, validator.deltaSchemaInfos, 0) // maxCnt is 10. - variable.SetMaxDeltaSchemaCount(10) + vardef.SetMaxDeltaSchemaCount(10) ds := []deltaSchemaInfo{ {0, []int64{1}, []uint64{1}}, {1, []int64{1}, []uint64{1}}, diff --git a/pkg/domain/sysvar_cache.go b/pkg/domain/sysvar_cache.go index 6294aa8260080..03c320f91c8ae 100644 --- a/pkg/domain/sysvar_cache.go +++ b/pkg/domain/sysvar_cache.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/syncutil" @@ -139,7 +140,7 @@ func (do *Domain) rebuildSysVarCache(ctx sessionctx.Context) error { // This ensures it is run on all tidb servers. // This does not apply to INSTANCE scoped vars (HasGlobalScope() is false) if sv.SetGlobal != nil && !sv.SkipSysvarCache() { - sVal = sv.ValidateWithRelaxedValidation(ctx.GetSessionVars(), sVal, variable.ScopeGlobal) + sVal = sv.ValidateWithRelaxedValidation(ctx.GetSessionVars(), sVal, vardef.ScopeGlobal) err = sv.SetGlobal(context.Background(), ctx.GetSessionVars(), sVal) if err != nil { logutil.BgLogger().Error(fmt.Sprintf("load global variable %s error", sv.Name), zap.Error(err)) @@ -154,6 +155,6 @@ func (do *Domain) rebuildSysVarCache(ctx sessionctx.Context) error { defer do.sysVarCache.Unlock() do.sysVarCache.session = newSessionCache do.sysVarCache.global = newGlobalCache - do.infoCache.ReSize(int(variable.SchemaVersionCacheLimit.Load())) + do.infoCache.ReSize(int(vardef.SchemaVersionCacheLimit.Load())) return nil } diff --git a/pkg/executor/BUILD.bazel b/pkg/executor/BUILD.bazel index db46b514bda37..607def3fc7ce6 100644 --- a/pkg/executor/BUILD.bazel +++ b/pkg/executor/BUILD.bazel @@ -184,6 +184,7 @@ go_library( "//pkg/sessionctx", "//pkg/sessionctx/sessionstates", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/sessiontxn/staleread", @@ -434,6 +435,7 @@ go_test( "//pkg/session/types", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/sessiontxn/staleread", diff --git a/pkg/executor/adapter.go b/pkg/executor/adapter.go index ddc0aa599d852..aebb4f6520a24 100644 --- a/pkg/executor/adapter.go +++ b/pkg/executor/adapter.go @@ -55,6 +55,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/sessiontxn/staleread" @@ -509,21 +510,21 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { sctx := a.Ctx ctx = util.SetSessionID(ctx, sctx.GetSessionVars().ConnectionID) if _, ok := a.Plan.(*plannercore.Analyze); ok && sctx.GetSessionVars().InRestrictedSQL { - oriStats, ok := sctx.GetSessionVars().GetSystemVar(variable.TiDBBuildStatsConcurrency) + oriStats, ok := sctx.GetSessionVars().GetSystemVar(vardef.TiDBBuildStatsConcurrency) if !ok { - oriStats = strconv.Itoa(variable.DefBuildStatsConcurrency) + oriStats = strconv.Itoa(vardef.DefBuildStatsConcurrency) } oriScan := sctx.GetSessionVars().AnalyzeDistSQLScanConcurrency() - oriIso, ok := sctx.GetSessionVars().GetSystemVar(variable.TxnIsolation) + oriIso, ok := sctx.GetSessionVars().GetSystemVar(vardef.TxnIsolation) if !ok { oriIso = "REPEATABLE-READ" } - autoConcurrency, err1 := sctx.GetSessionVars().GetSessionOrGlobalSystemVar(ctx, variable.TiDBAutoBuildStatsConcurrency) + autoConcurrency, err1 := sctx.GetSessionVars().GetSessionOrGlobalSystemVar(ctx, vardef.TiDBAutoBuildStatsConcurrency) terror.Log(err1) if err1 == nil { - terror.Log(sctx.GetSessionVars().SetSystemVar(variable.TiDBBuildStatsConcurrency, autoConcurrency)) + terror.Log(sctx.GetSessionVars().SetSystemVar(vardef.TiDBBuildStatsConcurrency, autoConcurrency)) } - sVal, err2 := sctx.GetSessionVars().GetSessionOrGlobalSystemVar(ctx, variable.TiDBSysProcScanConcurrency) + sVal, err2 := sctx.GetSessionVars().GetSessionOrGlobalSystemVar(ctx, vardef.TiDBSysProcScanConcurrency) terror.Log(err2) if err2 == nil { concurrency, err3 := strconv.ParseInt(sVal, 10, 64) @@ -532,11 +533,11 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { sctx.GetSessionVars().SetAnalyzeDistSQLScanConcurrency(int(concurrency)) } } - terror.Log(sctx.GetSessionVars().SetSystemVar(variable.TxnIsolation, ast.ReadCommitted)) + terror.Log(sctx.GetSessionVars().SetSystemVar(vardef.TxnIsolation, ast.ReadCommitted)) defer func() { - terror.Log(sctx.GetSessionVars().SetSystemVar(variable.TiDBBuildStatsConcurrency, oriStats)) + terror.Log(sctx.GetSessionVars().SetSystemVar(vardef.TiDBBuildStatsConcurrency, oriStats)) sctx.GetSessionVars().SetAnalyzeDistSQLScanConcurrency(oriScan) - terror.Log(sctx.GetSessionVars().SetSystemVar(variable.TxnIsolation, oriIso)) + terror.Log(sctx.GetSessionVars().SetSystemVar(vardef.TxnIsolation, oriIso)) }() } @@ -546,7 +547,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { // must set plan according to the `Execute` plan before getting planDigest a.inheritContextFromExecuteStmt() - if rm := domain.GetDomain(sctx).RunawayManager(); variable.EnableResourceControl.Load() && rm != nil { + if rm := domain.GetDomain(sctx).RunawayManager(); vardef.EnableResourceControl.Load() && rm != nil { sessionVars := sctx.GetSessionVars() stmtCtx := sessionVars.StmtCtx _, planDigest := GetPlanDigest(stmtCtx) @@ -1311,7 +1312,7 @@ func FormatSQL(sql string) stringutil.StringerFunc { func formatSQL(sql string) string { length := len(sql) - maxQueryLen := variable.QueryLogMaxLen.Load() + maxQueryLen := vardef.QueryLogMaxLen.Load() if maxQueryLen <= 0 { return QueryReplacer.Replace(sql) // no limit } @@ -2185,7 +2186,7 @@ func (a *ExecStmt) observeStmtBeginForTopSQL(ctx context.Context) context.Contex // UpdatePlanCacheRuntimeInfo updates the runtime information of the plan in the plan cache. func (a *ExecStmt) UpdatePlanCacheRuntimeInfo() { - if !variable.EnableInstancePlanCache.Load() { + if !vardef.EnableInstancePlanCache.Load() { return // only record for Instance Plan Cache } v := a.Ctx.GetSessionVars().PlanCacheValue diff --git a/pkg/executor/adapter_test.go b/pkg/executor/adapter_test.go index 4310fefe17a14..c678162ad93f6 100644 --- a/pkg/executor/adapter_test.go +++ b/pkg/executor/adapter_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/executor" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/util" @@ -31,10 +31,10 @@ import ( func TestFormatSQL(t *testing.T) { val := executor.FormatSQL("aaaa") require.Equal(t, "aaaa", val.String()) - variable.QueryLogMaxLen.Store(0) + vardef.QueryLogMaxLen.Store(0) val = executor.FormatSQL("aaaaaaaaaaaaaaaaaaaa") require.Equal(t, "aaaaaaaaaaaaaaaaaaaa", val.String()) - variable.QueryLogMaxLen.Store(5) + vardef.QueryLogMaxLen.Store(5) val = executor.FormatSQL("aaaaaaaaaaaaaaaaaaaa") require.Equal(t, "aaaaa(len:20)", val.String()) } diff --git a/pkg/executor/aggfuncs/BUILD.bazel b/pkg/executor/aggfuncs/BUILD.bazel index e2733d08c409e..44ce225857f41 100644 --- a/pkg/executor/aggfuncs/BUILD.bazel +++ b/pkg/executor/aggfuncs/BUILD.bazel @@ -100,7 +100,7 @@ go_test( "//pkg/parser/mysql", "//pkg/planner/util", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/testkit/testsetup", "//pkg/types", diff --git a/pkg/executor/aggfuncs/func_group_concat_test.go b/pkg/executor/aggfuncs/func_group_concat_test.go index d8c4d8c4082a5..bb0f07785a78f 100644 --- a/pkg/executor/aggfuncs/func_group_concat_test.go +++ b/pkg/executor/aggfuncs/func_group_concat_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/codec" @@ -50,12 +50,12 @@ func TestGroupConcat(t *testing.T) { testMultiArgsAggFunc(t, ctx, test2) defer func() { - err := ctx.GetSessionVars().SetSystemVar(variable.GroupConcatMaxLen, "1024") + err := ctx.GetSessionVars().SetSystemVar(vardef.GroupConcatMaxLen, "1024") require.NoError(t, err) }() // minimum GroupConcatMaxLen is 4 for i := 4; i <= 7; i++ { - err := ctx.GetSessionVars().SetSystemVar(variable.GroupConcatMaxLen, fmt.Sprint(i)) + err := ctx.GetSessionVars().SetSystemVar(vardef.GroupConcatMaxLen, fmt.Sprint(i)) require.NoError(t, err) test2 = buildMultiArgsAggTester(ast.AggFuncGroupConcat, []byte{mysql.TypeString, mysql.TypeString}, mysql.TypeString, 5, nil, "44 33 22 11 00"[:i]) test2.orderBy = true diff --git a/pkg/executor/aggregate/BUILD.bazel b/pkg/executor/aggregate/BUILD.bazel index aea4b636a98da..44d30cc04cdb1 100644 --- a/pkg/executor/aggregate/BUILD.bazel +++ b/pkg/executor/aggregate/BUILD.bazel @@ -22,7 +22,7 @@ go_library( "//pkg/parser/terror", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/types", "//pkg/util", "//pkg/util/channel", @@ -57,7 +57,7 @@ go_test( "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/types", "//pkg/util/chunk", "//pkg/util/memory", diff --git a/pkg/executor/aggregate/agg_hash_executor.go b/pkg/executor/aggregate/agg_hash_executor.go index a9083004c0d21..407003ee0ed1e 100644 --- a/pkg/executor/aggregate/agg_hash_executor.go +++ b/pkg/executor/aggregate/agg_hash_executor.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/channel" "github.com/pingcap/tidb/pkg/util/chunk" @@ -284,7 +284,7 @@ func (e *HashAggExec) initForUnparallelExec() { e.dataInDisk = chunk.NewDataInDiskByChunks(exec.RetTypes(e.Children(0))) e.tmpChkForSpill = exec.TryNewCacheChunk(e.Children(0)) - if vars := e.Ctx().GetSessionVars(); vars.TrackAggregateMemoryUsage && variable.EnableTmpStorageOnOOM.Load() { + if vars := e.Ctx().GetSessionVars(); vars.TrackAggregateMemoryUsage && vardef.EnableTmpStorageOnOOM.Load() { if e.diskTracker != nil { e.diskTracker.Reset() } else { @@ -398,7 +398,7 @@ func (e *HashAggExec) initForParallelExec(ctx sessionctx.Context) error { e.inflightChunkSync = &sync.WaitGroup{} - isTrackerEnabled := e.Ctx().GetSessionVars().TrackAggregateMemoryUsage && variable.EnableTmpStorageOnOOM.Load() + isTrackerEnabled := e.Ctx().GetSessionVars().TrackAggregateMemoryUsage && vardef.EnableTmpStorageOnOOM.Load() isParallelHashAggSpillEnabled := e.Ctx().GetSessionVars().EnableParallelHashaggSpill baseRetTypeNum := len(e.RetFieldTypes()) diff --git a/pkg/executor/aggregate/agg_spill_test.go b/pkg/executor/aggregate/agg_spill_test.go index 0b262c0cdfc86..132cb4329809c 100644 --- a/pkg/executor/aggregate/agg_spill_test.go +++ b/pkg/executor/aggregate/agg_spill_test.go @@ -34,7 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/memory" @@ -191,10 +191,10 @@ func getMockDataSourceParameters(ctx sessionctx.Context) testutil.MockDataSource } func buildHashAggExecutor(t *testing.T, ctx sessionctx.Context, child exec.Executor) *aggregate.HashAggExec { - if err := ctx.GetSessionVars().SetSystemVar(variable.TiDBHashAggFinalConcurrency, fmt.Sprintf("%v", 5)); err != nil { + if err := ctx.GetSessionVars().SetSystemVar(vardef.TiDBHashAggFinalConcurrency, fmt.Sprintf("%v", 5)); err != nil { t.Fatal(err) } - if err := ctx.GetSessionVars().SetSystemVar(variable.TiDBHashAggPartialConcurrency, fmt.Sprintf("%v", 5)); err != nil { + if err := ctx.GetSessionVars().SetSystemVar(vardef.TiDBHashAggPartialConcurrency, fmt.Sprintf("%v", 5)); err != nil { t.Fatal(err) } diff --git a/pkg/executor/analyze.go b/pkg/executor/analyze.go index 674b49a383d1e..172fdde5468f8 100644 --- a/pkg/executor/analyze.go +++ b/pkg/executor/analyze.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/statistics" @@ -327,7 +328,7 @@ func getTableIDFromTask(task *analyzeTask) statistics.AnalyzeTableID { } func (e *AnalyzeExec) saveV2AnalyzeOpts() error { - if !variable.PersistAnalyzeOptions.Load() || len(e.OptionsMap) == 0 { + if !vardef.PersistAnalyzeOptions.Load() || len(e.OptionsMap) == 0 { return nil } // only to save table options if dynamic prune mode diff --git a/pkg/executor/analyze_utils.go b/pkg/executor/analyze_utils.go index 13fd3d508c07d..b1b0b45ac2854 100644 --- a/pkg/executor/analyze_utils.go +++ b/pkg/executor/analyze_utils.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/store/helper" "github.com/pingcap/tidb/pkg/util" @@ -40,7 +40,7 @@ func adaptiveAnlayzeDistSQLConcurrency(ctx context.Context, sctx sessionctx.Cont tikvStore, ok := sctx.GetStore().(helper.Storage) if !ok { logutil.BgLogger().Warn("Information about TiKV store status can be gotten only when the storage is TiKV") - return variable.DefAnalyzeDistSQLScanConcurrency + return vardef.DefAnalyzeDistSQLScanConcurrency } tikvHelper := &helper.Helper{ Store: tikvStore, @@ -49,15 +49,15 @@ func adaptiveAnlayzeDistSQLConcurrency(ctx context.Context, sctx sessionctx.Cont pdCli, err := tikvHelper.TryGetPDHTTPClient() if err != nil { logutil.BgLogger().Warn("fail to TryGetPDHTTPClient", zap.Error(err)) - return variable.DefAnalyzeDistSQLScanConcurrency + return vardef.DefAnalyzeDistSQLScanConcurrency } storesStat, err := pdCli.GetStores(ctx) if err != nil { logutil.BgLogger().Warn("fail to get stores info", zap.Error(err)) - return variable.DefAnalyzeDistSQLScanConcurrency + return vardef.DefAnalyzeDistSQLScanConcurrency } if storesStat.Count <= 5 { - return variable.DefAnalyzeDistSQLScanConcurrency + return vardef.DefAnalyzeDistSQLScanConcurrency } else if storesStat.Count <= 10 { return storesStat.Count } else if storesStat.Count <= 20 { @@ -79,11 +79,11 @@ func getIntFromSessionVars(ctx sessionctx.Context, name string) (int, error) { } func getBuildStatsConcurrency(ctx sessionctx.Context) (int, error) { - return getIntFromSessionVars(ctx, variable.TiDBBuildStatsConcurrency) + return getIntFromSessionVars(ctx, vardef.TiDBBuildStatsConcurrency) } func getBuildSamplingStatsConcurrency(ctx sessionctx.Context) (int, error) { - return getIntFromSessionVars(ctx, variable.TiDBBuildSamplingStatsConcurrency) + return getIntFromSessionVars(ctx, vardef.TiDBBuildSamplingStatsConcurrency) } var errAnalyzeWorkerPanic = errors.New("analyze worker panic") diff --git a/pkg/executor/batch_point_get_test.go b/pkg/executor/batch_point_get_test.go index b756662ba4580..b523ce86a9e47 100644 --- a/pkg/executor/batch_point_get_test.go +++ b/pkg/executor/batch_point_get_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/executor" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/tikv" @@ -42,7 +42,7 @@ func TestBatchPointGetLockExistKey(t *testing.T) { errCh <- tk1.ExecToErr("use test") errCh <- tk2.ExecToErr("use test") - tk1.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk1.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly errCh <- tk1.ExecToErr(fmt.Sprintf("drop table if exists %s", tableName)) errCh <- tk1.ExecToErr(fmt.Sprintf("create table %s(id int, v int, k int, %s key0(id, v))", tableName, key)) diff --git a/pkg/executor/benchmark_test.go b/pkg/executor/benchmark_test.go index 8bf66afc77e6b..b67f89b885572 100644 --- a/pkg/executor/benchmark_test.go +++ b/pkg/executor/benchmark_test.go @@ -44,7 +44,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/disk" @@ -126,14 +126,14 @@ func buildStreamAggExecutor(ctx sessionctx.Context, srcExec exec.Executor, schem func buildAggExecutor(b *testing.B, testCase *testutil.AggTestCase, child exec.Executor) exec.Executor { ctx := testCase.Ctx if testCase.ExecType == "stream" { - if err := ctx.GetSessionVars().SetSystemVar(variable.TiDBStreamAggConcurrency, fmt.Sprintf("%v", testCase.Concurrency)); err != nil { + if err := ctx.GetSessionVars().SetSystemVar(vardef.TiDBStreamAggConcurrency, fmt.Sprintf("%v", testCase.Concurrency)); err != nil { b.Fatal(err) } } else { - if err := ctx.GetSessionVars().SetSystemVar(variable.TiDBHashAggFinalConcurrency, fmt.Sprintf("%v", testCase.Concurrency)); err != nil { + if err := ctx.GetSessionVars().SetSystemVar(vardef.TiDBHashAggFinalConcurrency, fmt.Sprintf("%v", testCase.Concurrency)); err != nil { b.Fatal(err) } - if err := ctx.GetSessionVars().SetSystemVar(variable.TiDBHashAggPartialConcurrency, fmt.Sprintf("%v", testCase.Concurrency)); err != nil { + if err := ctx.GetSessionVars().SetSystemVar(vardef.TiDBHashAggPartialConcurrency, fmt.Sprintf("%v", testCase.Concurrency)); err != nil { b.Fatal(err) } } @@ -160,7 +160,7 @@ func buildAggExecutor(b *testing.B, testCase *testutil.AggTestCase, child exec.E } func benchmarkAggExecWithCase(b *testing.B, casTest *testutil.AggTestCase) { - if err := casTest.Ctx.GetSessionVars().SetSystemVar(variable.TiDBStreamAggConcurrency, fmt.Sprintf("%v", casTest.Concurrency)); err != nil { + if err := casTest.Ctx.GetSessionVars().SetSystemVar(vardef.TiDBStreamAggConcurrency, fmt.Sprintf("%v", casTest.Concurrency)); err != nil { b.Fatal(err) } @@ -359,10 +359,10 @@ func buildWindowExecutor(ctx sessionctx.Context, windowFunc string, funcs int, f func benchmarkWindowExecWithCase(b *testing.B, casTest *testutil.WindowTestCase) { ctx := casTest.Ctx - if err := ctx.GetSessionVars().SetSystemVar(variable.TiDBWindowConcurrency, fmt.Sprintf("%v", casTest.Concurrency)); err != nil { + if err := ctx.GetSessionVars().SetSystemVar(vardef.TiDBWindowConcurrency, fmt.Sprintf("%v", casTest.Concurrency)); err != nil { b.Fatal(err) } - if err := ctx.GetSessionVars().SetSystemVar(variable.TiDBEnablePipelinedWindowFunction, fmt.Sprintf("%v", casTest.Pipelined)); err != nil { + if err := ctx.GetSessionVars().SetSystemVar(vardef.TiDBEnablePipelinedWindowFunction, fmt.Sprintf("%v", casTest.Pipelined)); err != nil { b.Fatal(err) } @@ -609,8 +609,8 @@ func (tc hashJoinTestCase) String() string { func defaultHashJoinTestCase(cols []*types.FieldType, joinType logicalop.JoinType, useOuterToBuild bool) *hashJoinTestCase { ctx := mock.NewContext() - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().InitChunkSize = vardef.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = vardef.DefMaxChunkSize ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(-1, -1) ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(-1, -1) ctx.GetSessionVars().SetIndexLookupJoinConcurrency(4) @@ -1194,14 +1194,14 @@ func (tc IndexJoinTestCase) Columns() []*expression.Column { func defaultIndexJoinTestCase() *IndexJoinTestCase { ctx := mock.NewContext() - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().InitChunkSize = vardef.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = vardef.DefMaxChunkSize ctx.GetSessionVars().SnapshotTS = 1 ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(-1, -1) ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(-1, -1) tc := &IndexJoinTestCase{ OuterRows: 100000, - InnerRows: variable.DefMaxChunkSize * 100, + InnerRows: vardef.DefMaxChunkSize * 100, Concurrency: 4, Ctx: ctx, OuterJoinKeyIdx: []int{0, 1}, @@ -1627,8 +1627,8 @@ func prepare4MergeJoin(tc *mergeJoinTestCase, innerDS, outerDS *testutil.MockDat func newMergeJoinBenchmark(numOuterRows, numInnerDup, numInnerRedundant int) (tc *mergeJoinTestCase, innerDS, outerDS *testutil.MockDataSource) { ctx := mock.NewContext() - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().InitChunkSize = vardef.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = vardef.DefMaxChunkSize ctx.GetSessionVars().SnapshotTS = 1 ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(-1, -1) ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(-1, -1) @@ -1877,8 +1877,8 @@ func (tc topNTestCase) String() string { func defaultTopNTestCase() *topNTestCase { ctx := mock.NewContext() - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().InitChunkSize = vardef.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = vardef.DefMaxChunkSize ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(-1, -1) return &topNTestCase{ rows: 100000, diff --git a/pkg/executor/builder.go b/pkg/executor/builder.go index 0cc412e3f875e..0c1de28efbae4 100644 --- a/pkg/executor/builder.go +++ b/pkg/executor/builder.go @@ -68,6 +68,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/sessiontxn/staleread" @@ -1453,7 +1454,7 @@ func (us *UnionScanExec) handleCachedTable(b *executorBuilder, x bypassDataSourc if tbl.Meta().TableCacheStatusType == model.TableCacheStatusEnable { cachedTable := tbl.(table.CachedTable) // Determine whether the cache can be used. - leaseDuration := time.Duration(variable.TableCacheLease.Load()) * time.Second + leaseDuration := time.Duration(vardef.TableCacheLease.Load()) * time.Second cacheData, loading := cachedTable.TryReadFromCache(startTS, leaseDuration) if cacheData != nil { vars.StmtCtx.ReadFromTableCache = true @@ -5735,7 +5736,7 @@ func (b *executorBuilder) getCacheTable(tblInfo *model.TableInfo, startTS uint64 return nil } sessVars := b.ctx.GetSessionVars() - leaseDuration := time.Duration(variable.TableCacheLease.Load()) * time.Second + leaseDuration := time.Duration(vardef.TableCacheLease.Load()) * time.Second cacheData, loading := tbl.(table.CachedTable).TryReadFromCache(startTS, leaseDuration) if cacheData != nil { sessVars.StmtCtx.ReadFromTableCache = true diff --git a/pkg/executor/checksum.go b/pkg/executor/checksum.go index f53c0b4fcbe4a..bccbef01f1561 100644 --- a/pkg/executor/checksum.go +++ b/pkg/executor/checksum.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/ranger" @@ -321,7 +321,7 @@ func (c *checksumContext) handleResponse(update *tipb.ChecksumResponse) { func getChecksumTableConcurrency(ctx sessionctx.Context) (int, error) { sessionVars := ctx.GetSessionVars() - concurrency, err := sessionVars.GetSessionOrGlobalSystemVar(context.Background(), variable.TiDBChecksumTableConcurrency) + concurrency, err := sessionVars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBChecksumTableConcurrency) if err != nil { return 0, err } diff --git a/pkg/executor/coprocessor.go b/pkg/executor/coprocessor.go index e3fc1179e0379..51425f8997d04 100644 --- a/pkg/executor/coprocessor.go +++ b/pkg/executor/coprocessor.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/pkg/executor/internal/exec" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/privilege" @@ -44,7 +43,7 @@ func copHandlerCtx(ctx context.Context, req *coprocessor.Request) context.Contex return ctx } - traceInfo := &model.TraceInfo{ + traceInfo := &tracing.TraceInfo{ ConnectionID: source.ConnectionId, SessionAlias: source.SessionAlias, } diff --git a/pkg/executor/cte.go b/pkg/executor/cte.go index d9ac97b943b38..c8aabd6a5e8f6 100644 --- a/pkg/executor/cte.go +++ b/pkg/executor/cte.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/join" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/codec" @@ -364,7 +364,7 @@ func (p *cteProducer) genCTEResult(ctx context.Context) (err error) { } failpoint.Inject("testCTEStorageSpill", func(val failpoint.Value) { - if val.(bool) && variable.EnableTmpStorageOnOOM.Load() { + if val.(bool) && vardef.EnableTmpStorageOnOOM.Load() { defer resAction.WaitForTest() defer iterInAction.WaitForTest() if iterOutAction != nil { @@ -585,7 +585,7 @@ func setupCTEStorageTracker(tbl cteutil.Storage, ctx sessionctx.Context, parentM diskTracker.SetLabel(memory.LabelForCTEStorage) diskTracker.AttachTo(parentDiskTracker) - if variable.EnableTmpStorageOnOOM.Load() { + if vardef.EnableTmpStorageOnOOM.Load() { actionSpill = tbl.ActionSpill() failpoint.Inject("testCTEStorageSpill", func(val failpoint.Value) { if val.(bool) { diff --git a/pkg/executor/ddl.go b/pkg/executor/ddl.go index f7a979a6b93a7..2d5bcb072ccfd 100644 --- a/pkg/executor/ddl.go +++ b/pkg/executor/ddl.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/resolve" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/sessiontxn/staleread" @@ -360,11 +361,11 @@ func (e *DDLExec) executeDropDatabase(s *ast.DropDatabaseStmt) error { sessionVars := e.Ctx().GetSessionVars() if err == nil && strings.ToLower(sessionVars.CurrentDB) == dbName.L { sessionVars.CurrentDB = "" - err = sessionVars.SetSystemVar(variable.CharsetDatabase, mysql.DefaultCharset) + err = sessionVars.SetSystemVar(vardef.CharsetDatabase, mysql.DefaultCharset) if err != nil { return err } - err = sessionVars.SetSystemVar(variable.CollationDatabase, mysql.DefaultCollationName) + err = sessionVars.SetSystemVar(vardef.CollationDatabase, mysql.DefaultCollationName) if err != nil { return err } @@ -781,21 +782,21 @@ func (e *DDLExec) executeAlterPlacementPolicy(s *ast.AlterPlacementPolicyStmt) e } func (e *DDLExec) executeCreateResourceGroup(s *ast.CreateResourceGroupStmt) error { - if !variable.EnableResourceControl.Load() && !e.Ctx().GetSessionVars().InRestrictedSQL { + if !vardef.EnableResourceControl.Load() && !e.Ctx().GetSessionVars().InRestrictedSQL { return infoschema.ErrResourceGroupSupportDisabled } return e.ddlExecutor.AddResourceGroup(e.Ctx(), s) } func (e *DDLExec) executeAlterResourceGroup(s *ast.AlterResourceGroupStmt) error { - if !variable.EnableResourceControl.Load() && !e.Ctx().GetSessionVars().InRestrictedSQL { + if !vardef.EnableResourceControl.Load() && !e.Ctx().GetSessionVars().InRestrictedSQL { return infoschema.ErrResourceGroupSupportDisabled } return e.ddlExecutor.AlterResourceGroup(e.Ctx(), s) } func (e *DDLExec) executeDropResourceGroup(s *ast.DropResourceGroupStmt) error { - if !variable.EnableResourceControl.Load() && !e.Ctx().GetSessionVars().InRestrictedSQL { + if !vardef.EnableResourceControl.Load() && !e.Ctx().GetSessionVars().InRestrictedSQL { return infoschema.ErrResourceGroupSupportDisabled } return e.ddlExecutor.DropResourceGroup(e.Ctx(), s) diff --git a/pkg/executor/delete.go b/pkg/executor/delete.go index 04c6a8efa2c51..12fee7aad76e2 100644 --- a/pkg/executor/delete.go +++ b/pkg/executor/delete.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" @@ -88,7 +88,7 @@ func (e *DeleteExec) deleteSingleTableByChunk(ctx context.Context) error { batchDMLSize := e.Ctx().GetSessionVars().DMLBatchSize // If tidb_batch_delete is ON and not in a transaction, we could use BatchDelete mode. batchDelete := e.Ctx().GetSessionVars().BatchDelete && !e.Ctx().GetSessionVars().InTxn() && - variable.EnableBatchDML.Load() && batchDMLSize > 0 + vardef.EnableBatchDML.Load() && batchDMLSize > 0 fields := exec.RetTypes(e.Children(0)) datumRow := make([]types.Datum, 0, len(fields)) chk := exec.TryNewCacheChunk(e.Children(0)) diff --git a/pkg/executor/delete_test.go b/pkg/executor/delete_test.go index 8c3eb1ade31b9..09c5a211e1b05 100644 --- a/pkg/executor/delete_test.go +++ b/pkg/executor/delete_test.go @@ -19,7 +19,7 @@ import ( "testing" "time" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -87,7 +87,7 @@ func TestDeleteLockKey(t *testing.T) { tk1, tk2 := testkit.NewTestKit(t, store), testkit.NewTestKit(t, store) tk1.MustExec("use test") tk2.MustExec("use test") - tk1.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk1.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly tk1.MustExec(testCase.ddl) tk1.MustExec(testCase.pre) tk1.MustExec("begin pessimistic") diff --git a/pkg/executor/executor_required_rows_test.go b/pkg/executor/executor_required_rows_test.go index ec32ea1f6cd7f..b18fc01da4824 100644 --- a/pkg/executor/executor_required_rows_test.go +++ b/pkg/executor/executor_required_rows_test.go @@ -34,7 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/disk" @@ -210,8 +210,8 @@ func buildLimitExec(ctx sessionctx.Context, src exec.Executor, offset, count int func defaultCtx() sessionctx.Context { ctx := mock.NewContext() - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().InitChunkSize = vardef.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = vardef.DefMaxChunkSize ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(-1, ctx.GetSessionVars().MemQuotaQuery) ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(-1, -1) ctx.GetSessionVars().SnapshotTS = uint64(1) diff --git a/pkg/executor/explain_unit_test.go b/pkg/executor/explain_unit_test.go index 0233bdd453bf5..38811bb666116 100644 --- a/pkg/executor/explain_unit_test.go +++ b/pkg/executor/explain_unit_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/mock" @@ -65,8 +65,8 @@ func getColumns() []*expression.Column { // close() must be called after next() to avoid goroutines leak func TestExplainAnalyzeInvokeNextAndClose(t *testing.T) { ctx := mock.NewContext() - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().InitChunkSize = vardef.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = vardef.DefMaxChunkSize schema := expression.NewSchema(getColumns()...) baseExec := exec.NewBaseExecutor(ctx, schema, 0) explainExec := &ExplainExec{ diff --git a/pkg/executor/explainfor_test.go b/pkg/executor/explainfor_test.go index 27ecff8db29c1..494b6a5d5d000 100644 --- a/pkg/executor/explainfor_test.go +++ b/pkg/executor/explainfor_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/planner/core" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" @@ -280,7 +280,7 @@ func TestPointGetUserVarPlanCache(t *testing.T) { tk.MustExec("use test") tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set @@tidb_enable_collect_execution_info=0;") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("drop table if exists t1") tk.MustExec("CREATE TABLE t1 (a BIGINT, b VARCHAR(40), PRIMARY KEY (a, b))") tk.MustExec("INSERT INTO t1 VALUES (1,'3'),(2,'4')") diff --git a/pkg/executor/grant.go b/pkg/executor/grant.go index b25d6a769b51c..dbdcd7d2ca814 100644 --- a/pkg/executor/grant.go +++ b/pkg/executor/grant.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/privilege/privileges" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/util" @@ -154,7 +154,7 @@ func (e *GrantExec) Next(ctx context.Context, _ *chunk.Chunk) error { return err } - defaultAuthPlugin, err := e.Ctx().GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.DefaultAuthPlugin) + defaultAuthPlugin, err := e.Ctx().GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.DefaultAuthPlugin) if err != nil { return err } diff --git a/pkg/executor/historical_stats_test.go b/pkg/executor/historical_stats_test.go index ce2575af64070..3ce70768dff8f 100644 --- a/pkg/executor/historical_stats_test.go +++ b/pkg/executor/historical_stats_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics/handle/storage" "github.com/pingcap/tidb/pkg/statistics/util" "github.com/pingcap/tidb/pkg/testkit" @@ -269,7 +269,7 @@ func TestGCOutdatedHistoryStats(t *testing.T) { tableInfo.Meta().ID)).Check(testkit.Rows("1")) tk.MustExec("set @@global.tidb_historical_stats_duration = '1s'") - duration := variable.HistoricalStatsDuration.Load() + duration := vardef.HistoricalStatsDuration.Load() fmt.Println(duration.String()) time.Sleep(2 * time.Second) err = dom.StatsHandle().ClearOutdatedHistoryStats() diff --git a/pkg/executor/import_into.go b/pkg/executor/import_into.go index b82b15585ab4f..1c0fa35f47b31 100644 --- a/pkg/executor/import_into.go +++ b/pkg/executor/import_into.go @@ -38,7 +38,7 @@ import ( plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" @@ -212,7 +212,7 @@ func (e *ImportIntoExec) submitTask(ctx context.Context) (int64, *proto.TaskBase return 0, nil, exeerrors.ErrLoadDataInvalidURI.FastGenByArgs(plannercore.ImportIntoDataSource, err.Error()) } logutil.Logger(ctx).Info("get job importer", zap.Stringer("param", e.controller.Parameters), - zap.Bool("dist-task-enabled", variable.EnableDistTask.Load())) + zap.Bool("dist-task-enabled", vardef.EnableDistTask.Load())) if importFromServer { ecp, err2 := e.controller.PopulateChunks(ctx) if err2 != nil { @@ -221,7 +221,7 @@ func (e *ImportIntoExec) submitTask(ctx context.Context) (int64, *proto.TaskBase return importinto.SubmitStandaloneTask(ctx, e.controller.Plan, e.stmt, ecp) } // if tidb_enable_dist_task=true, we import distributively, otherwise we import on current node. - if variable.EnableDistTask.Load() { + if vardef.EnableDistTask.Load() { return importinto.SubmitTask(ctx, e.controller.Plan, e.stmt) } return importinto.SubmitStandaloneTask(ctx, e.controller.Plan, e.stmt, nil) diff --git a/pkg/executor/importer/BUILD.bazel b/pkg/executor/importer/BUILD.bazel index c593c60eb23fa..c8b586e17b615 100644 --- a/pkg/executor/importer/BUILD.bazel +++ b/pkg/executor/importer/BUILD.bazel @@ -47,7 +47,7 @@ go_library( "//pkg/planner/planctx", "//pkg/planner/util", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/sessiontxn", "//pkg/table", "//pkg/table/tables", @@ -132,7 +132,7 @@ go_test( "//pkg/planner/core/resolve", "//pkg/planner/util", "//pkg/session", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/testkit/testfailpoint", "//pkg/testkit/testsetup", diff --git a/pkg/executor/importer/import.go b/pkg/executor/importer/import.go index b36598715309b..cdf413ba71401 100644 --- a/pkg/executor/importer/import.go +++ b/pkg/executor/importer/import.go @@ -50,7 +50,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/planctx" plannerutil "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" tidbutil "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/chunk" @@ -326,7 +326,7 @@ func NewPlanFromLoadDataPlan(userSctx sessionctx.Context, plan *plannercore.Load if charset == nil { // https://dev.mysql.com/doc/refman/8.0/en/load-data.html#load-data-character-set d, err2 := userSctx.GetSessionVars().GetSessionOrGlobalSystemVar( - context.Background(), variable.CharsetDatabase) + context.Background(), vardef.CharsetDatabase) if err2 != nil { logger.Error("LOAD DATA get charset failed", zap.Error(err2)) } else { @@ -548,7 +548,7 @@ func (p *Plan) initDefaultOptions(targetNodeCPUCnt int) { p.Detached = false p.DisableTiKVImportMode = false p.MaxEngineSize = config.ByteSize(defaultMaxEngineSize) - p.CloudStorageURI = variable.CloudStorageURI.Load() + p.CloudStorageURI = vardef.CloudStorageURI.Load() v := "utf8mb4" p.Charset = &v @@ -1433,7 +1433,7 @@ func GetTargetNodeCPUCnt(ctx context.Context, sourceType DataSourceType, path st } serverDiskImport := storage.IsLocal(u) - if serverDiskImport || !variable.EnableDistTask.Load() { + if serverDiskImport || !vardef.EnableDistTask.Load() { return cpu.GetCPUCount(), nil } return handle.GetCPUCountOfNode(ctx) diff --git a/pkg/executor/importer/import_test.go b/pkg/executor/importer/import_test.go index 9ebd90a276f73..7decbfa932269 100644 --- a/pkg/executor/importer/import_test.go +++ b/pkg/executor/importer/import_test.go @@ -38,7 +38,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" plannercore "github.com/pingcap/tidb/pkg/planner/core" plannerutil "github.com/pingcap/tidb/pkg/planner/util" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" "github.com/pingcap/tidb/pkg/util/logutil" @@ -58,9 +58,9 @@ func TestInitDefaultOptions(t *testing.T) { plan = &Plan{ DataSourceType: DataSourceTypeFile, } - variable.CloudStorageURI.Store("s3://bucket/path") + vardef.CloudStorageURI.Store("s3://bucket/path") t.Cleanup(func() { - variable.CloudStorageURI.Store("") + vardef.CloudStorageURI.Store("") }) plan.initDefaultOptions(1) require.Equal(t, config.ByteSize(0), plan.DiskQuota) @@ -144,9 +144,9 @@ func TestInitOptionsPositiveCase(t *testing.T) { require.True(t, plan.DisablePrecheck, sql) // set cloud storage uri - variable.CloudStorageURI.Store("s3://bucket/path") + vardef.CloudStorageURI.Store("s3://bucket/path") t.Cleanup(func() { - variable.CloudStorageURI.Store("") + vardef.CloudStorageURI.Store("") }) plan = &Plan{Format: DataFormatCSV} err = plan.initOptions(ctx, sctx, convertOptions(stmt.(*ast.ImportIntoStmt).Options)) diff --git a/pkg/executor/importer/importer_testkit_test.go b/pkg/executor/importer/importer_testkit_test.go index 6d8ff16109128..a97c0e93fbae9 100644 --- a/pkg/executor/importer/importer_testkit_test.go +++ b/pkg/executor/importer/importer_testkit_test.go @@ -42,7 +42,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" "github.com/pingcap/tidb/pkg/types" @@ -78,7 +78,7 @@ func TestVerifyChecksum(t *testing.T) { // admin checksum table always return 1, 1, 1 for memory store // Checksum = required backupDistScanCon := tk.Session().GetSessionVars().DistSQLScanConcurrency() - require.Equal(t, variable.DefDistSQLScanConcurrency, backupDistScanCon) + require.Equal(t, vardef.DefDistSQLScanConcurrency, backupDistScanCon) localChecksum := verify.MakeKVChecksum(1, 1, 1) err := importer.VerifyChecksum(ctx, plan, localChecksum, tk.Session(), logutil.BgLogger()) require.NoError(t, err) @@ -110,9 +110,9 @@ func TestVerifyChecksum(t *testing.T) { index idx10(id) )`) tk.MustExec("insert into db.tb2 values(1)") - backup, err := tk.Session().GetSessionVars().GetSessionOrGlobalSystemVar(ctx, variable.TiDBChecksumTableConcurrency) + backup, err := tk.Session().GetSessionVars().GetSessionOrGlobalSystemVar(ctx, vardef.TiDBChecksumTableConcurrency) require.NoError(t, err) - err = tk.Session().GetSessionVars().SetSystemVar(variable.TiDBChecksumTableConcurrency, "1") + err = tk.Session().GetSessionVars().SetSystemVar(vardef.TiDBChecksumTableConcurrency, "1") require.NoError(t, err) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/afterHandleChecksumRequest", `sleep(1000)`)) @@ -121,7 +121,7 @@ func TestVerifyChecksum(t *testing.T) { err = importer.VerifyChecksum(ctx2, plan2, localChecksum, tk.Session(), logutil.BgLogger()) require.ErrorContains(t, err, "Query execution was interrupted") - err = tk.Session().GetSessionVars().SetSystemVar(variable.TiDBChecksumTableConcurrency, backup) + err = tk.Session().GetSessionVars().SetSystemVar(vardef.TiDBChecksumTableConcurrency, backup) require.NoError(t, err) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/afterHandleChecksumRequest")) diff --git a/pkg/executor/importer/table_import.go b/pkg/executor/importer/table_import.go index 6d20d0df1c33c..e2606b7c3c607 100644 --- a/pkg/executor/importer/table_import.go +++ b/pkg/executor/importer/table_import.go @@ -49,7 +49,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" tidbmetrics "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -936,13 +936,13 @@ func checksumTable(ctx context.Context, se sessionctx.Context, plan *Plan, logge func setBackoffWeight(se sessionctx.Context, plan *Plan, logger *zap.Logger) error { backoffWeight := local.DefaultBackoffWeight - if val, ok := plan.ImportantSysVars[variable.TiDBBackOffWeight]; ok { + if val, ok := plan.ImportantSysVars[vardef.TiDBBackOffWeight]; ok { if weight, err := strconv.Atoi(val); err == nil && weight > backoffWeight { backoffWeight = weight } } logger.Info("set backoff weight", zap.Int("weight", backoffWeight)) - return se.GetSessionVars().SetSystemVar(variable.TiDBBackOffWeight, strconv.Itoa(backoffWeight)) + return se.GetSessionVars().SetSystemVar(vardef.TiDBBackOffWeight, strconv.Itoa(backoffWeight)) } // GetImportRootDir returns the root directory for import. diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index 30e47113a70ae..eb79689b57a9a 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -56,6 +56,7 @@ import ( "github.com/pingcap/tidb/pkg/resourcegroup/runaway" "github.com/pingcap/tidb/pkg/session/txninfo" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/statistics" @@ -333,11 +334,11 @@ func (e *memtableRetriever) setDataForVariablesInfo(ctx sessionctx.Context) erro isNoop, // IS_NOOP ) // min and max value is only supported for numeric types - if !(sv.Type == variable.TypeUnsigned || sv.Type == variable.TypeInt || sv.Type == variable.TypeFloat) { + if !(sv.Type == vardef.TypeUnsigned || sv.Type == vardef.TypeInt || sv.Type == vardef.TypeFloat) { row[4].SetNull() row[5].SetNull() } - if sv.Type == variable.TypeEnum { + if sv.Type == vardef.TypeEnum { possibleValues := strings.Join(sv.PossibleValues, ",") row[6].SetString(possibleValues, mysql.DefaultCollationName) } diff --git a/pkg/executor/insert_common.go b/pkg/executor/insert_common.go index cd46554d6b659..4cbfdf1edebfa 100644 --- a/pkg/executor/insert_common.go +++ b/pkg/executor/insert_common.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -204,7 +204,7 @@ func insertRows(ctx context.Context, base insertCommon) (err error) { e := base.insertCommon() sessVars := e.Ctx().GetSessionVars() batchSize := sessVars.DMLBatchSize - batchInsert := sessVars.BatchInsert && !sessVars.InTxn() && variable.EnableBatchDML.Load() && batchSize > 0 + batchInsert := sessVars.BatchInsert && !sessVars.InTxn() && vardef.EnableBatchDML.Load() && batchSize > 0 e.lazyFillAutoID = true evalRowFunc := e.fastEvalRow @@ -458,7 +458,7 @@ func insertRowsFromSelect(ctx context.Context, base insertCommon) error { sessVars := e.Ctx().GetSessionVars() batchSize := sessVars.DMLBatchSize - batchInsert := sessVars.BatchInsert && !sessVars.InTxn() && variable.EnableBatchDML.Load() && batchSize > 0 + batchInsert := sessVars.BatchInsert && !sessVars.InTxn() && vardef.EnableBatchDML.Load() && batchSize > 0 memUsageOfRows := int64(0) memUsageOfExtraCols := int64(0) memTracker := e.memTracker diff --git a/pkg/executor/insert_test.go b/pkg/executor/insert_test.go index c27a6e2052d04..05cec1f6b5124 100644 --- a/pkg/executor/insert_test.go +++ b/pkg/executor/insert_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/executor" "github.com/pingcap/tidb/pkg/meta/autoid" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util" @@ -435,7 +435,7 @@ func TestDuplicateEntryMessage(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") - for _, enable := range []variable.ClusteredIndexDefMode{variable.ClusteredIndexDefModeOn, variable.ClusteredIndexDefModeOff, variable.ClusteredIndexDefModeIntOnly} { + for _, enable := range []vardef.ClusteredIndexDefMode{vardef.ClusteredIndexDefModeOn, vardef.ClusteredIndexDefModeOff, vardef.ClusteredIndexDefModeIntOnly} { tk.Session().GetSessionVars().EnableClusteredIndex = enable tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int, b char(10), unique key(b)) collate utf8mb4_general_ci;") diff --git a/pkg/executor/internal/builder/BUILD.bazel b/pkg/executor/internal/builder/BUILD.bazel index a27321acc7309..ca7a82c43ad3e 100644 --- a/pkg/executor/internal/builder/BUILD.bazel +++ b/pkg/executor/internal/builder/BUILD.bazel @@ -11,7 +11,7 @@ go_library( "//pkg/planner/core/base", "//pkg/planner/planctx", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/util/timeutil", "@com_github_pingcap_tipb//go-tipb", ], diff --git a/pkg/executor/internal/builder/builder_utils.go b/pkg/executor/internal/builder/builder_utils.go index eb7adaa2943a2..2c15cc51a3847 100644 --- a/pkg/executor/internal/builder/builder_utils.go +++ b/pkg/executor/internal/builder/builder_utils.go @@ -20,7 +20,7 @@ import ( plannercore "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/planctx" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util/timeutil" "github.com/pingcap/tipb/go-tipb" ) @@ -54,7 +54,7 @@ func ConstructDAGReq(ctx sessionctx.Context, plans []plannercore.PhysicalPlan, s dagReq.CollectExecutionSummaries = &collExec } dagReq.Flags = sc.PushDownFlags() - if ctx.GetSessionVars().GetDivPrecisionIncrement() != variable.DefDivPrecisionIncrement { + if ctx.GetSessionVars().GetDivPrecisionIncrement() != vardef.DefDivPrecisionIncrement { var divPrecIncr uint32 = uint32(ctx.GetSessionVars().GetDivPrecisionIncrement()) dagReq.DivPrecisionIncrement = &divPrecIncr } diff --git a/pkg/executor/internal/calibrateresource/BUILD.bazel b/pkg/executor/internal/calibrateresource/BUILD.bazel index 0f6b557ffd183..a07c95db4a440 100644 --- a/pkg/executor/internal/calibrateresource/BUILD.bazel +++ b/pkg/executor/internal/calibrateresource/BUILD.bazel @@ -13,7 +13,7 @@ go_library( "//pkg/parser/ast", "//pkg/parser/duration", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/sessiontxn/staleread", "//pkg/util", "//pkg/util/chunk", diff --git a/pkg/executor/internal/calibrateresource/calibrate_resource.go b/pkg/executor/internal/calibrateresource/calibrate_resource.go index e47f9a519f3c8..e0b586b225f45 100644 --- a/pkg/executor/internal/calibrateresource/calibrate_resource.go +++ b/pkg/executor/internal/calibrateresource/calibrate_resource.go @@ -38,7 +38,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/duration" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn/staleread" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/chunk" @@ -251,7 +251,7 @@ func (e *Executor) Next(ctx context.Context, req *chunk.Chunk) error { return nil } e.done = true - if !variable.EnableResourceControl.Load() { + if !vardef.EnableResourceControl.Load() { return infoschema.ErrResourceGroupSupportDisabled } ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnOthers) diff --git a/pkg/executor/internal/mpp/BUILD.bazel b/pkg/executor/internal/mpp/BUILD.bazel index 840d594317922..e559fb26fa1fe 100644 --- a/pkg/executor/internal/mpp/BUILD.bazel +++ b/pkg/executor/internal/mpp/BUILD.bazel @@ -22,7 +22,7 @@ go_library( "//pkg/planner/core", "//pkg/planner/core/base", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/copr", "//pkg/store/driver/backoff", "//pkg/store/driver/error", diff --git a/pkg/executor/internal/mpp/local_mpp_coordinator.go b/pkg/executor/internal/mpp/local_mpp_coordinator.go index c0465f951252a..0f69134f1fe22 100644 --- a/pkg/executor/internal/mpp/local_mpp_coordinator.go +++ b/pkg/executor/internal/mpp/local_mpp_coordinator.go @@ -37,7 +37,7 @@ import ( plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/copr" "github.com/pingcap/tidb/pkg/store/driver/backoff" derr "github.com/pingcap/tidb/pkg/store/driver/error" @@ -231,7 +231,7 @@ func (c *localMppCoordinator) appendMPPDispatchReq(pf *plannercore.Fragment, all } rgName := c.sessionCtx.GetSessionVars().StmtCtx.ResourceGroupName - if !variable.EnableResourceControl.Load() { + if !vardef.EnableResourceControl.Load() { rgName = "" } logutil.BgLogger().Info("Dispatch mpp task", zap.Uint64("timestamp", mppTask.StartTs), diff --git a/pkg/executor/internal/querywatch/BUILD.bazel b/pkg/executor/internal/querywatch/BUILD.bazel index 88b8ddce61cc5..e943edb1fcf08 100644 --- a/pkg/executor/internal/querywatch/BUILD.bazel +++ b/pkg/executor/internal/querywatch/BUILD.bazel @@ -36,7 +36,7 @@ go_test( "//pkg/errno", "//pkg/kv", "//pkg/meta/autoid", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/testkit/testsetup", "@com_github_pingcap_failpoint//:failpoint", diff --git a/pkg/executor/internal/querywatch/query_watch_test.go b/pkg/executor/internal/querywatch/query_watch_test.go index ead75f0f63877..6919a0dd8298c 100644 --- a/pkg/executor/internal/querywatch/query_watch_test.go +++ b/pkg/executor/internal/querywatch/query_watch_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/failpoint" mysql "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -35,7 +35,7 @@ func TestQueryWatch(t *testing.T) { }() store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) - if variable.SchemaCacheSize.Load() != 0 { + if vardef.SchemaCacheSize.Load() != 0 { t.Skip("skip this test because the schema cache is enabled") } tk.MustExec("use test") diff --git a/pkg/executor/internal/testutil/BUILD.bazel b/pkg/executor/internal/testutil/BUILD.bazel index edd9bf8f8246c..d71b7935f737c 100644 --- a/pkg/executor/internal/testutil/BUILD.bazel +++ b/pkg/executor/internal/testutil/BUILD.bazel @@ -20,7 +20,7 @@ go_library( "//pkg/planner/core/operator/logicalop", "//pkg/planner/property", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/types", "//pkg/util/chunk", "//pkg/util/memory", diff --git a/pkg/executor/internal/testutil/agg.go b/pkg/executor/internal/testutil/agg.go index d4143bc2eddda..1810f3249ca75 100644 --- a/pkg/executor/internal/testutil/agg.go +++ b/pkg/executor/internal/testutil/agg.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" ) @@ -53,8 +53,8 @@ func (a AggTestCase) String() string { // DefaultAggTestCase returns default agg test case func DefaultAggTestCase(ctx sessionctx.Context, exec string) *AggTestCase { - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().InitChunkSize = vardef.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = vardef.DefMaxChunkSize // return &AggTestCase{exec, ast.AggFuncSum, 1000, false, 10000000, 4, true, ctx} return &AggTestCase{ ExecType: exec, diff --git a/pkg/executor/internal/testutil/limit.go b/pkg/executor/internal/testutil/limit.go index 84d072a1b885f..330bcc45732ba 100644 --- a/pkg/executor/internal/testutil/limit.go +++ b/pkg/executor/internal/testutil/limit.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/memory" ) @@ -51,8 +51,8 @@ func (tc LimitCase) String() string { // DefaultLimitTestCase returns default limit test case func DefaultLimitTestCase(ctx sessionctx.Context) *LimitCase { - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().InitChunkSize = vardef.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = vardef.DefMaxChunkSize ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(-1, -1) tc := &LimitCase{ Rows: 30000, diff --git a/pkg/executor/internal/testutil/sort.go b/pkg/executor/internal/testutil/sort.go index 5d7633310fdf2..3dc20db9d1955 100644 --- a/pkg/executor/internal/testutil/sort.go +++ b/pkg/executor/internal/testutil/sort.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/memory" ) @@ -48,8 +48,8 @@ func (tc SortCase) String() string { // DefaultSortTestCase returns default sort test case func DefaultSortTestCase(ctx sessionctx.Context) *SortCase { - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().InitChunkSize = vardef.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = vardef.DefMaxChunkSize ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(-1, -1) tc := &SortCase{Rows: 300000, OrderByIdx: []int{0, 1}, Ndvs: []int{0, 0}, Ctx: ctx} return tc @@ -57,8 +57,8 @@ func DefaultSortTestCase(ctx sessionctx.Context) *SortCase { // SortTestCaseWithMemoryLimit returns sort test case func SortTestCaseWithMemoryLimit(ctx sessionctx.Context, bytesLimit int64) *SortCase { - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().InitChunkSize = vardef.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = vardef.DefMaxChunkSize ctx.GetSessionVars().MemTracker = memory.NewTracker(-1, bytesLimit) ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(-1, bytesLimit) ctx.GetSessionVars().StmtCtx.MemTracker.AttachTo(ctx.GetSessionVars().MemTracker) diff --git a/pkg/executor/internal/testutil/window.go b/pkg/executor/internal/testutil/window.go index b205bb4eb0128..d80029bd49bea 100644 --- a/pkg/executor/internal/testutil/window.go +++ b/pkg/executor/internal/testutil/window.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" ) @@ -50,8 +50,8 @@ func (a WindowTestCase) String() string { // DefaultWindowTestCase returns default window test case func DefaultWindowTestCase(ctx sessionctx.Context) *WindowTestCase { - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().InitChunkSize = vardef.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = vardef.DefMaxChunkSize return &WindowTestCase{ WindowFunc: ast.WindowFuncRowNumber, NumFunc: 1, diff --git a/pkg/executor/join/BUILD.bazel b/pkg/executor/join/BUILD.bazel index be8d78d7a1bea..467c6ab5747e5 100644 --- a/pkg/executor/join/BUILD.bazel +++ b/pkg/executor/join/BUILD.bazel @@ -47,7 +47,7 @@ go_library( "//pkg/planner/core/operator/logicalop", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/types", "//pkg/util", "//pkg/util/bitmap", @@ -115,7 +115,7 @@ go_test( "//pkg/planner/core/operator/logicalop", "//pkg/session", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/types", "//pkg/util", diff --git a/pkg/executor/join/hash_join_v1.go b/pkg/executor/join/hash_join_v1.go index 69a5bd9b4b268..066e07fe9c01a 100644 --- a/pkg/executor/join/hash_join_v1.go +++ b/pkg/executor/join/hash_join_v1.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/bitmap" @@ -1078,7 +1078,7 @@ func (w *BuildWorkerV1) BuildHashTableForList(buildSideResultCh <-chan *chunk.Ch rowContainer.GetMemTracker().SetLabel(memory.LabelForBuildSideResult) rowContainer.GetDiskTracker().AttachTo(w.HashJoinCtx.diskTracker) rowContainer.GetDiskTracker().SetLabel(memory.LabelForBuildSideResult) - if variable.EnableTmpStorageOnOOM.Load() { + if vardef.EnableTmpStorageOnOOM.Load() { actionSpill := rowContainer.ActionSpill() failpoint.Inject("testRowContainerSpill", func(val failpoint.Value) { if val.(bool) { diff --git a/pkg/executor/join/hash_join_v2.go b/pkg/executor/join/hash_join_v2.go index 56a067b4bcbc5..f81acb5850ce2 100644 --- a/pkg/executor/join/hash_join_v2.go +++ b/pkg/executor/join/hash_join_v2.go @@ -33,7 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/channel" @@ -729,7 +729,7 @@ func (e *HashJoinV2Exec) Open(ctx context.Context) error { e.spillHelper = newHashJoinSpillHelper(e, int(e.partitionNumber), e.ProbeSideTupleFetcher.ProbeSideExec.RetFieldTypes()) e.maxSpillRound = 1 - if variable.EnableTmpStorageOnOOM.Load() && e.partitionNumber > 1 { + if vardef.EnableTmpStorageOnOOM.Load() && e.partitionNumber > 1 { e.initMaxSpillRound() e.spillAction = newHashJoinSpillAction(e.spillHelper) e.Ctx().GetSessionVars().MemTracker.FallbackOldAndSetNewAction(e.spillAction) diff --git a/pkg/executor/join/joiner_test.go b/pkg/executor/join/joiner_test.go index 92a3aab004deb..b08838e08ec64 100644 --- a/pkg/executor/join/joiner_test.go +++ b/pkg/executor/join/joiner_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/disk" @@ -33,8 +33,8 @@ import ( func defaultCtx() sessionctx.Context { ctx := mock.NewContext() - ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize - ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().InitChunkSize = vardef.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = vardef.DefMaxChunkSize ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(-1, ctx.GetSessionVars().MemQuotaQuery) ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(-1, -1) ctx.GetSessionVars().SnapshotTS = uint64(1) diff --git a/pkg/executor/join/merge_join.go b/pkg/executor/join/merge_join.go index 7dc647ce6c592..11c3ba894b15d 100644 --- a/pkg/executor/join/merge_join.go +++ b/pkg/executor/join/merge_join.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/internal/vecgroupchecker" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/disk" "github.com/pingcap/tidb/pkg/util/memory" @@ -101,7 +101,7 @@ func (t *MergeJoinTable) init(executor *MergeJoinExec) { t.rowContainer.GetMemTracker().SetLabel(memory.LabelForInnerTable) t.rowContainer.GetDiskTracker().AttachTo(executor.diskTracker) t.rowContainer.GetDiskTracker().SetLabel(memory.LabelForInnerTable) - if variable.EnableTmpStorageOnOOM.Load() { + if vardef.EnableTmpStorageOnOOM.Load() { actionSpill := t.rowContainer.ActionSpill() failpoint.Inject("testMergeJoinRowContainerSpill", func(val failpoint.Value) { if val.(bool) { diff --git a/pkg/executor/join/merge_join_test.go b/pkg/executor/join/merge_join_test.go index e314fd98b3928..fa05c0e1f65c9 100644 --- a/pkg/executor/join/merge_join_test.go +++ b/pkg/executor/join/merge_join_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/config" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" @@ -204,7 +204,7 @@ func TestVectorizedMergeJoin(t *testing.T) { } } - tk.Session().GetSessionVars().MaxChunkSize = variable.DefInitChunkSize + tk.Session().GetSessionVars().MaxChunkSize = vardef.DefInitChunkSize chunkSize := tk.Session().GetSessionVars().MaxChunkSize cases := []struct { t1 []int @@ -327,7 +327,7 @@ func TestVectorizedShuffleMergeJoin(t *testing.T) { } } - tk.Session().GetSessionVars().MaxChunkSize = variable.DefInitChunkSize + tk.Session().GetSessionVars().MaxChunkSize = vardef.DefInitChunkSize chunkSize := tk.Session().GetSessionVars().MaxChunkSize cases := []struct { t1 []int diff --git a/pkg/executor/main_test.go b/pkg/executor/main_test.go index 30b387cf7c219..035d8e134f1b1 100644 --- a/pkg/executor/main_test.go +++ b/pkg/executor/main_test.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/meta/autoid" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/pingcap/tidb/pkg/testkit/testmain" "github.com/pingcap/tidb/pkg/testkit/testsetup" @@ -46,7 +46,7 @@ func TestMain(m *testing.M) { conf.Experimental.AllowsExpressionIndex = true }) tikv.EnableFailpoints() - variable.StatsCacheMemQuota.Store(5000) + vardef.StatsCacheMemQuota.Store(5000) opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"), diff --git a/pkg/executor/plan_replayer.go b/pkg/executor/plan_replayer.go index 75fcd3897037e..3aceca018c657 100644 --- a/pkg/executor/plan_replayer.go +++ b/pkg/executor/plan_replayer.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/statistics/util" @@ -386,7 +387,7 @@ func loadVariables(ctx sessionctx.Context, z *zip.Reader) error { logutil.BgLogger().Warn(fmt.Sprintf("skip set variable %s:%s", name, value), zap.Error(err)) continue } - sVal, err := sysVar.Validate(vars, value, variable.ScopeSession) + sVal, err := sysVar.Validate(vars, value, vardef.ScopeSession) if err != nil { unLoadVars = append(unLoadVars, name) logutil.BgLogger().Warn(fmt.Sprintf("skip variable %s:%s", name, value), zap.Error(err)) @@ -529,7 +530,7 @@ func (e *PlanReplayerLoadInfo) createTable(z *zip.Reader) error { // We need to disable foreign key check when we create schema and tables. // because the order of creating schema and tables is not guaranteed. e.Ctx.GetSessionVars().ForeignKeyChecks = false - e.Ctx.GetSessionVars().PlacementMode = variable.PlacementModeIgnore + e.Ctx.GetSessionVars().PlacementMode = vardef.PlacementModeIgnore defer func() { e.Ctx.GetSessionVars().ForeignKeyChecks = originForeignKeyChecks e.Ctx.GetSessionVars().PlacementMode = originPlacementMode diff --git a/pkg/executor/point_get_test.go b/pkg/executor/point_get_test.go index d4058f3e16f78..310d67b2e44d5 100644 --- a/pkg/executor/point_get_test.go +++ b/pkg/executor/point_get_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" storeerr "github.com/pingcap/tidb/pkg/store/driver/error" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/testkit" @@ -76,7 +76,7 @@ func TestReturnValues(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly tk.MustExec("create table t (a varchar(64) primary key, b int)") tk.MustExec("insert t values ('a', 1), ('b', 2), ('c', 3)") tk.MustExec("begin pessimistic") @@ -235,7 +235,7 @@ func TestPointGetLockExistKey(t *testing.T) { tk1.MustExec("use test") tk2.MustExec("use test") - tk1.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk1.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly tk1.MustExec(fmt.Sprintf("drop table if exists %s", tableName)) tk1.MustExec(fmt.Sprintf("create table %s(id int, v int, k int, %s key0(id, v))", tableName, key)) diff --git a/pkg/executor/prepared_test.go b/pkg/executor/prepared_test.go index 85aec7f63b868..45c7c10c792b8 100644 --- a/pkg/executor/prepared_test.go +++ b/pkg/executor/prepared_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" @@ -244,7 +245,7 @@ func TestPlanCacheClusterIndex(t *testing.T) { tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) tk.MustExec("use test") tk.MustExec("drop table if exists t1") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("set @@tidb_enable_collect_execution_info=0;") tk.MustExec("create table t1(a varchar(20), b varchar(20), c varchar(20), primary key(a, b))") tk.MustExec("insert into t1 values('1','1','111'),('2','2','222'),('3','3','333')") @@ -329,7 +330,7 @@ func TestPlanCacheClusterIndex(t *testing.T) { tk.MustQuery(`execute stmt2 using @v2, @v2, @v3, @v3`).Check(testkit.Rows("b b 2 2 2", "c c 3 3 3")) // For issue 19002 - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec(`drop table if exists t1`) tk.MustExec(`create table t1(a int, b int, c int, primary key(a, b))`) tk.MustExec(`insert into t1 values(1,1,111),(2,2,222),(3,3,333)`) diff --git a/pkg/executor/sample_test.go b/pkg/executor/sample_test.go index 86727861aafdf..1bba0ca281ba0 100644 --- a/pkg/executor/sample_test.go +++ b/pkg/executor/sample_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -42,7 +42,7 @@ func TestTableSampleBasic(t *testing.T) { store := testkit.CreateMockStore(t) tk := createSampleTestkit(t, store) tk.MustExec("create table t (a int);") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustQuery("select * from t tablesample regions();").Check(testkit.Rows()) tk.MustExec("insert into t values (0), (1000), (2000);") diff --git a/pkg/executor/select.go b/pkg/executor/select.go index 843183abad8ea..f5d66e03ba226 100644 --- a/pkg/executor/select.go +++ b/pkg/executor/select.go @@ -49,6 +49,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/table" @@ -976,12 +977,12 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.InitMemTracker(memory.LabelForSQLText, -1) } logOnQueryExceedMemQuota := domain.GetDomain(ctx).ExpensiveQueryHandle().LogOnQueryExceedMemQuota - switch variable.OOMAction.Load() { - case variable.OOMActionCancel: + switch vardef.OOMAction.Load() { + case vardef.OOMActionCancel: action := &memory.PanicOnExceed{ConnID: vars.ConnectionID, Killer: vars.MemTracker.Killer} action.SetLogHook(logOnQueryExceedMemQuota) vars.MemTracker.SetActionOnExceed(action) - case variable.OOMActionLog: + case vardef.OOMActionLog: fallthrough default: action := &memory.LogOnExceed{ConnID: vars.ConnectionID} @@ -992,7 +993,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.MemTracker.AttachTo(vars.MemTracker) sc.InitDiskTracker(memory.LabelForSQLText, -1) globalConfig := config.GetGlobalConfig() - if variable.EnableTmpStorageOnOOM.Load() && sc.DiskTracker != nil { + if vardef.EnableTmpStorageOnOOM.Load() && sc.DiskTracker != nil { sc.DiskTracker.AttachTo(vars.DiskTracker) if GlobalDiskUsageTracker != nil { vars.DiskTracker.AttachTo(GlobalDiskUsageTracker) @@ -1007,7 +1008,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.InitSQLDigest(prepareStmt.NormalizedSQL, prepareStmt.SQLDigest) // For `execute stmt` SQL, should reset the SQL digest with the prepare SQL digest. goCtx := context.Background() - if variable.EnablePProfSQLCPU.Load() && len(prepareStmt.NormalizedSQL) > 0 { + if vardef.EnablePProfSQLCPU.Load() && len(prepareStmt.NormalizedSQL) > 0 { goCtx = pprof.WithLabels(goCtx, pprof.Labels("sql", FormatSQL(prepareStmt.NormalizedSQL).String())) pprof.SetGoroutineLabels(goCtx) } @@ -1156,7 +1157,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { ) vars.PlanCacheParams.Reset() - if priority := mysql.PriorityEnum(atomic.LoadInt32(&variable.ForcePriority)); priority != mysql.NoPriority { + if priority := mysql.PriorityEnum(atomic.LoadInt32(&vardef.ForcePriority)); priority != mysql.NoPriority { sc.Priority = priority } if vars.StmtCtx.LastInsertIDSet { diff --git a/pkg/executor/set.go b/pkg/executor/set.go index 11320becbca46..272ed05b72921 100644 --- a/pkg/executor/set.go +++ b/pkg/executor/set.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/plugin" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table/temptable" "github.com/pingcap/tidb/pkg/util/chunk" @@ -134,7 +135,7 @@ func (e *SetExecutor) setSysVariable(ctx context.Context, name string, v *expres } } - if sysVar.IsNoop && !variable.EnableNoopVariables.Load() { + if sysVar.IsNoop && !vardef.EnableNoopVariables.Load() { // The variable is a noop. For compatibility we allow it to still // be changed, but we append a warning since users might be expecting // something that's not going to happen. @@ -164,11 +165,11 @@ func (e *SetExecutor) setSysVariable(ctx context.Context, name string, v *expres return nil }) showValStr := valStr - if name == variable.TiDBCloudStorageURI { + if name == vardef.TiDBCloudStorageURI { showValStr = ast.RedactURL(showValStr) } logutil.BgLogger().Info("set global var", zap.Uint64("conn", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", showValStr)) - if name == variable.TiDBServiceScope { + if name == vardef.TiDBServiceScope { dom := domain.GetDomain(e.Ctx()) oldConfig := config.GetGlobalConfig() if oldConfig.Instance.TiDBServiceScope != valStr { @@ -191,27 +192,27 @@ func (e *SetExecutor) setSysVariable(ctx context.Context, name string, v *expres return err } getSnapshotTSByName := func() uint64 { - if name == variable.TiDBSnapshot { + if name == vardef.TiDBSnapshot { return sessionVars.SnapshotTS - } else if name == variable.TiDBTxnReadTS { + } else if name == vardef.TiDBTxnReadTS { return sessionVars.TxnReadTS.PeakTxnReadTS() } return 0 } oldSnapshotTS := getSnapshotTSByName() fallbackOldSnapshotTS := func() { - if name == variable.TiDBSnapshot { + if name == vardef.TiDBSnapshot { sessionVars.SnapshotTS = oldSnapshotTS - } else if name == variable.TiDBTxnReadTS { + } else if name == vardef.TiDBTxnReadTS { sessionVars.TxnReadTS.SetTxnReadTS(oldSnapshotTS) } } if sessionVars.InTxn() { - if name == variable.TxnIsolationOneShot || - name == variable.TiDBTxnReadTS { + if name == vardef.TxnIsolationOneShot || + name == vardef.TiDBTxnReadTS { return errors.Trace(exeerrors.ErrCantChangeTxCharacteristics) } - if name == variable.TiDBSnapshot && sessionVars.TxnCtx.IsStaleness { + if name == vardef.TiDBSnapshot && sessionVars.TxnCtx.IsStaleness { return errors.Trace(exeerrors.ErrCantChangeTxCharacteristics) } } @@ -222,9 +223,9 @@ func (e *SetExecutor) setSysVariable(ctx context.Context, name string, v *expres newSnapshotTS := getSnapshotTSByName() newSnapshotIsSet := newSnapshotTS > 0 && newSnapshotTS != oldSnapshotTS if newSnapshotIsSet { - isStaleRead := name == variable.TiDBTxnReadTS + isStaleRead := name == vardef.TiDBTxnReadTS err = sessionctx.ValidateSnapshotReadTS(ctx, e.Ctx().GetStore(), newSnapshotTS, isStaleRead) - if name != variable.TiDBTxnReadTS { + if name != vardef.TiDBTxnReadTS { // Also check gc safe point for snapshot read. // We don't check snapshot with gc safe point for read_ts // Client-go will automatically check the snapshotTS with gc safe point. It's unnecessary to check gc safe point during set executor. @@ -268,32 +269,32 @@ func (e *SetExecutor) setCharset(cs, co string, isSetName bool) error { } } if isSetName { - for _, v := range variable.SetNamesVariables { + for _, v := range vardef.SetNamesVariables { if err = sessionVars.SetSystemVar(v, cs); err != nil { return errors.Trace(err) } } - return errors.Trace(sessionVars.SetSystemVar(variable.CollationConnection, co)) + return errors.Trace(sessionVars.SetSystemVar(vardef.CollationConnection, co)) } // Set charset statement, see also https://dev.mysql.com/doc/refman/8.0/en/set-character-set.html. - for _, v := range variable.SetCharsetVariables { + for _, v := range vardef.SetCharsetVariables { if err = sessionVars.SetSystemVar(v, cs); err != nil { return errors.Trace(err) } } - csDB, err := sessionVars.GlobalVarsAccessor.GetGlobalSysVar(variable.CharsetDatabase) + csDB, err := sessionVars.GlobalVarsAccessor.GetGlobalSysVar(vardef.CharsetDatabase) if err != nil { return err } - coDB, err := sessionVars.GlobalVarsAccessor.GetGlobalSysVar(variable.CollationDatabase) + coDB, err := sessionVars.GlobalVarsAccessor.GetGlobalSysVar(vardef.CollationDatabase) if err != nil { return err } - err = sessionVars.SetSystemVar(variable.CharacterSetConnection, csDB) + err = sessionVars.SetSystemVar(vardef.CharacterSetConnection, csDB) if err != nil { return errors.Trace(err) } - return errors.Trace(sessionVars.SetSystemVar(variable.CollationConnection, coDB)) + return errors.Trace(sessionVars.SetSystemVar(vardef.CollationConnection, coDB)) } func (e *SetExecutor) getVarValue(ctx context.Context, v *expression.VarAssignment, sysVar *variable.SysVar) (value string, err error) { @@ -324,7 +325,7 @@ func (e *SetExecutor) getVarValue(ctx context.Context, v *expression.VarAssignme } func (e *SetExecutor) loadSnapshotInfoSchemaIfNeeded(name string, snapshotTS uint64) error { - if name != variable.TiDBSnapshot && name != variable.TiDBTxnReadTS { + if name != vardef.TiDBSnapshot && name != vardef.TiDBTxnReadTS { return nil } return loadSnapshotInfoSchemaIfNeeded(e.Ctx(), snapshotTS) diff --git a/pkg/executor/set_test.go b/pkg/executor/set_test.go index 44c3e0e224ec4..2bf815e02a080 100644 --- a/pkg/executor/set_test.go +++ b/pkg/executor/set_test.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/types" @@ -136,16 +137,16 @@ func TestSetVar(t *testing.T) { tk.MustExec("set @@global.ddl_slow_threshold=12345") tk.MustQuery("select @@global.ddl_slow_threshold").Check(testkit.Rows("12345")) - require.Equal(t, uint32(12345), variable.DDLSlowOprThreshold) + require.Equal(t, uint32(12345), vardef.DDLSlowOprThreshold) tk.MustExec("set session ddl_slow_threshold=\"54321\"") tk.MustQuery("show variables like 'ddl_slow_threshold'").Check(testkit.Rows("ddl_slow_threshold 54321")) - require.Equal(t, uint32(54321), variable.DDLSlowOprThreshold) + require.Equal(t, uint32(54321), vardef.DDLSlowOprThreshold) tk.MustExec("set @@global.ddl_slow_threshold=-1") - tk.MustQuery("select @@global.ddl_slow_threshold").Check(testkit.Rows(strconv.Itoa(variable.DefTiDBDDLSlowOprThreshold))) - require.Equal(t, uint32(variable.DefTiDBDDLSlowOprThreshold), variable.DDLSlowOprThreshold) + tk.MustQuery("select @@global.ddl_slow_threshold").Check(testkit.Rows(strconv.Itoa(vardef.DefTiDBDDLSlowOprThreshold))) + require.Equal(t, uint32(vardef.DefTiDBDDLSlowOprThreshold), vardef.DDLSlowOprThreshold) require.Error(t, tk.ExecToErr("set @@global.ddl_slow_threshold=abc")) - tk.MustQuery("select @@global.ddl_slow_threshold").Check(testkit.Rows(strconv.Itoa(variable.DefTiDBDDLSlowOprThreshold))) - require.Equal(t, uint32(variable.DefTiDBDDLSlowOprThreshold), variable.DDLSlowOprThreshold) + tk.MustQuery("select @@global.ddl_slow_threshold").Check(testkit.Rows(strconv.Itoa(vardef.DefTiDBDDLSlowOprThreshold))) + require.Equal(t, uint32(vardef.DefTiDBDDLSlowOprThreshold), vardef.DDLSlowOprThreshold) // Test set transaction isolation level, which is equivalent to setting variable "tx_isolation". tk.MustExec("SET SESSION TRANSACTION ISOLATION LEVEL READ COMMITTED") @@ -279,7 +280,7 @@ func TestSetVar(t *testing.T) { require.Error(t, tk.ExecToErr("set tidb_checksum_table_concurrency = 'abc'")) tk.MustQuery(`select @@tidb_checksum_table_concurrency;`).Check(testkit.Rows("42")) tk.MustExec("set tidb_checksum_table_concurrency = 257") - tk.MustQuery(`select @@tidb_checksum_table_concurrency;`).Check(testkit.Rows(strconv.Itoa(variable.MaxConfigurableConcurrency))) + tk.MustQuery(`select @@tidb_checksum_table_concurrency;`).Check(testkit.Rows(strconv.Itoa(vardef.MaxConfigurableConcurrency))) tk.MustExec("set tidb_build_stats_concurrency = 42") tk.MustQuery(`select @@tidb_build_stats_concurrency;`).Check(testkit.Rows("42")) @@ -289,9 +290,9 @@ func TestSetVar(t *testing.T) { require.Error(t, tk.ExecToErr("set tidb_build_stats_concurrency = 'abc'")) tk.MustQuery(`select @@tidb_build_stats_concurrency;`).Check(testkit.Rows("42")) tk.MustExec("set tidb_build_stats_concurrency = 257") - tk.MustQuery(`select @@tidb_build_stats_concurrency;`).Check(testkit.Rows(strconv.Itoa(variable.MaxConfigurableConcurrency))) + tk.MustQuery(`select @@tidb_build_stats_concurrency;`).Check(testkit.Rows(strconv.Itoa(vardef.MaxConfigurableConcurrency))) tk.MustExec("set tidb_build_sampling_stats_concurrency = 257") - tk.MustQuery(`select @@tidb_build_sampling_stats_concurrency;`).Check(testkit.Rows(strconv.Itoa(variable.MaxConfigurableConcurrency))) + tk.MustQuery(`select @@tidb_build_sampling_stats_concurrency;`).Check(testkit.Rows(strconv.Itoa(vardef.MaxConfigurableConcurrency))) tk.MustExec(`set tidb_partition_prune_mode = "static"`) tk.MustQuery(`select @@tidb_partition_prune_mode;`).Check(testkit.Rows("static")) @@ -382,7 +383,7 @@ func TestSetVar(t *testing.T) { require.Error(t, tk.ExecToErr("set session tidb_scatter_region = 'test'")) // test for tidb_wait_split_region_timeout - tk.MustQuery(`select @@session.tidb_wait_split_region_timeout;`).Check(testkit.Rows(strconv.Itoa(variable.DefWaitSplitRegionTimeout))) + tk.MustQuery(`select @@session.tidb_wait_split_region_timeout;`).Check(testkit.Rows(strconv.Itoa(vardef.DefWaitSplitRegionTimeout))) tk.MustExec("set tidb_wait_split_region_timeout = 1") tk.MustQuery(`select @@session.tidb_wait_split_region_timeout;`).Check(testkit.Rows("1")) tk.MustExec("set tidb_wait_split_region_timeout = 0") @@ -484,7 +485,7 @@ func TestSetVar(t *testing.T) { require.True(t, terror.ErrorEqual(err, variable.ErrWrongValueForVar)) // test for tidb_mem_quota_apply_cache - defVal := fmt.Sprintf("%v", variable.DefTiDBMemQuotaApplyCache) + defVal := fmt.Sprintf("%v", vardef.DefTiDBMemQuotaApplyCache) tk.MustQuery(`select @@tidb_mem_quota_apply_cache`).Check(testkit.Rows(defVal)) tk.MustExec(`set global tidb_mem_quota_apply_cache = 1`) tk.MustQuery(`select @@global.tidb_mem_quota_apply_cache`).Check(testkit.Rows("1")) @@ -495,7 +496,7 @@ func TestSetVar(t *testing.T) { tk.MustQuery(`select @@tidb_mem_quota_apply_cache`).Check(testkit.Rows("123")) // test for tidb_mem_quota_bind_cache - defVal = fmt.Sprintf("%v", variable.DefTiDBMemQuotaBindingCache) + defVal = fmt.Sprintf("%v", vardef.DefTiDBMemQuotaBindingCache) tk.MustQuery(`select @@tidb_mem_quota_binding_cache`).Check(testkit.Rows(defVal)) tk.MustExec(`set global tidb_mem_quota_binding_cache = 1`) tk.MustQuery(`select @@global.tidb_mem_quota_binding_cache`).Check(testkit.Rows("1")) @@ -757,7 +758,7 @@ func TestSetVar(t *testing.T) { tk.MustGetErrCode("set @@global.plugin_dir = ''", errno.ErrIncorrectGlobalLocalVar) // test for tidb_max_auto_analyze_time - tk.MustQuery("select @@tidb_max_auto_analyze_time").Check(testkit.Rows(strconv.Itoa(variable.DefTiDBMaxAutoAnalyzeTime))) + tk.MustQuery("select @@tidb_max_auto_analyze_time").Check(testkit.Rows(strconv.Itoa(vardef.DefTiDBMaxAutoAnalyzeTime))) tk.MustExec("set global tidb_max_auto_analyze_time = 60") tk.MustQuery("select @@tidb_max_auto_analyze_time").Check(testkit.Rows("60")) tk.MustExec("set global tidb_max_auto_analyze_time = -1") @@ -785,7 +786,7 @@ func TestSetVar(t *testing.T) { tk.MustQuery("select @@global.tidb_instance_plan_cache_reserved_percentage").Check(testkit.Rows("0.5")) // test variables for cost model ver2 - tk.MustQuery("select @@tidb_cost_model_version").Check(testkit.Rows(fmt.Sprintf("%v", variable.DefTiDBCostModelVer))) + tk.MustQuery("select @@tidb_cost_model_version").Check(testkit.Rows(fmt.Sprintf("%v", vardef.DefTiDBCostModelVer))) tk.MustExec("set tidb_cost_model_version=3") tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect tidb_cost_model_version value: '3'")) tk.MustExec("set tidb_cost_model_version=0") @@ -1010,9 +1011,9 @@ func TestSetCollationAndCharset(t *testing.T) { expectCharset string expectCollation string }{ - {variable.CharacterSetConnection, variable.CollationConnection, "utf8", "utf8_bin"}, - {variable.CharsetDatabase, variable.CollationDatabase, "utf8", "utf8_bin"}, - {variable.CharacterSetServer, variable.CollationServer, "utf8", "utf8_bin"}, + {vardef.CharacterSetConnection, vardef.CollationConnection, "utf8", "utf8_bin"}, + {vardef.CharsetDatabase, vardef.CollationDatabase, "utf8", "utf8_bin"}, + {vardef.CharacterSetServer, vardef.CollationServer, "utf8", "utf8_bin"}, } for _, c := range cases { @@ -1440,33 +1441,33 @@ func TestSetConcurrency(t *testing.T) { tk := testkit.NewTestKit(t, store) // test default value - tk.MustQuery("select @@tidb_executor_concurrency;").Check(testkit.Rows(strconv.Itoa(variable.DefExecutorConcurrency))) + tk.MustQuery("select @@tidb_executor_concurrency;").Check(testkit.Rows(strconv.Itoa(vardef.DefExecutorConcurrency))) - tk.MustQuery("select @@tidb_index_lookup_concurrency;").Check(testkit.Rows(strconv.Itoa(variable.ConcurrencyUnset))) - tk.MustQuery("select @@tidb_index_lookup_join_concurrency;").Check(testkit.Rows(strconv.Itoa(variable.ConcurrencyUnset))) - tk.MustQuery("select @@tidb_hash_join_concurrency;").Check(testkit.Rows(strconv.Itoa(variable.ConcurrencyUnset))) - tk.MustQuery("select @@tidb_hashagg_partial_concurrency;").Check(testkit.Rows(strconv.Itoa(variable.ConcurrencyUnset))) - tk.MustQuery("select @@tidb_hashagg_final_concurrency;").Check(testkit.Rows(strconv.Itoa(variable.ConcurrencyUnset))) - tk.MustQuery("select @@tidb_window_concurrency;").Check(testkit.Rows(strconv.Itoa(variable.ConcurrencyUnset))) - tk.MustQuery("select @@tidb_streamagg_concurrency;").Check(testkit.Rows(strconv.Itoa(variable.DefTiDBStreamAggConcurrency))) - tk.MustQuery("select @@tidb_projection_concurrency;").Check(testkit.Rows(strconv.Itoa(variable.ConcurrencyUnset))) - tk.MustQuery("select @@tidb_distsql_scan_concurrency;").Check(testkit.Rows(strconv.Itoa(variable.DefDistSQLScanConcurrency))) + tk.MustQuery("select @@tidb_index_lookup_concurrency;").Check(testkit.Rows(strconv.Itoa(vardef.ConcurrencyUnset))) + tk.MustQuery("select @@tidb_index_lookup_join_concurrency;").Check(testkit.Rows(strconv.Itoa(vardef.ConcurrencyUnset))) + tk.MustQuery("select @@tidb_hash_join_concurrency;").Check(testkit.Rows(strconv.Itoa(vardef.ConcurrencyUnset))) + tk.MustQuery("select @@tidb_hashagg_partial_concurrency;").Check(testkit.Rows(strconv.Itoa(vardef.ConcurrencyUnset))) + tk.MustQuery("select @@tidb_hashagg_final_concurrency;").Check(testkit.Rows(strconv.Itoa(vardef.ConcurrencyUnset))) + tk.MustQuery("select @@tidb_window_concurrency;").Check(testkit.Rows(strconv.Itoa(vardef.ConcurrencyUnset))) + tk.MustQuery("select @@tidb_streamagg_concurrency;").Check(testkit.Rows(strconv.Itoa(vardef.DefTiDBStreamAggConcurrency))) + tk.MustQuery("select @@tidb_projection_concurrency;").Check(testkit.Rows(strconv.Itoa(vardef.ConcurrencyUnset))) + tk.MustQuery("select @@tidb_distsql_scan_concurrency;").Check(testkit.Rows(strconv.Itoa(vardef.DefDistSQLScanConcurrency))) - tk.MustQuery("select @@tidb_index_serial_scan_concurrency;").Check(testkit.Rows(strconv.Itoa(variable.DefIndexSerialScanConcurrency))) + tk.MustQuery("select @@tidb_index_serial_scan_concurrency;").Check(testkit.Rows(strconv.Itoa(vardef.DefIndexSerialScanConcurrency))) vars := tk.Session().GetSessionVars() - require.Equal(t, variable.DefExecutorConcurrency, vars.ExecutorConcurrency) - require.Equal(t, variable.DefExecutorConcurrency, vars.IndexLookupConcurrency()) - require.Equal(t, variable.DefExecutorConcurrency, vars.IndexLookupJoinConcurrency()) - require.Equal(t, variable.DefExecutorConcurrency, vars.HashJoinConcurrency()) - require.Equal(t, variable.DefExecutorConcurrency, vars.HashAggPartialConcurrency()) - require.Equal(t, variable.DefExecutorConcurrency, vars.HashAggFinalConcurrency()) - require.Equal(t, variable.DefExecutorConcurrency, vars.WindowConcurrency()) - require.Equal(t, variable.DefTiDBStreamAggConcurrency, vars.StreamAggConcurrency()) - require.Equal(t, variable.DefExecutorConcurrency, vars.ProjectionConcurrency()) - require.Equal(t, variable.DefDistSQLScanConcurrency, vars.DistSQLScanConcurrency()) - - require.Equal(t, variable.DefIndexSerialScanConcurrency, vars.IndexSerialScanConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.ExecutorConcurrency) + require.Equal(t, vardef.DefExecutorConcurrency, vars.IndexLookupConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.IndexLookupJoinConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.HashJoinConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.HashAggPartialConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.HashAggFinalConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.WindowConcurrency()) + require.Equal(t, vardef.DefTiDBStreamAggConcurrency, vars.StreamAggConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.ProjectionConcurrency()) + require.Equal(t, vardef.DefDistSQLScanConcurrency, vars.DistSQLScanConcurrency()) + + require.Equal(t, vardef.DefIndexSerialScanConcurrency, vars.IndexSerialScanConcurrency()) // test setting deprecated variables warnTpl := "Warning 1287 '%s' is deprecated and will be removed in a future release. Please use tidb_executor_concurrency instead" @@ -1477,32 +1478,32 @@ func TestSetConcurrency(t *testing.T) { tk.MustQuery(fmt.Sprintf("select @@%s;", v)).Check(testkit.Rows("1")) } - checkSet(variable.TiDBIndexLookupConcurrency) + checkSet(vardef.TiDBIndexLookupConcurrency) require.Equal(t, 1, vars.IndexLookupConcurrency()) - checkSet(variable.TiDBIndexLookupJoinConcurrency) + checkSet(vardef.TiDBIndexLookupJoinConcurrency) require.Equal(t, 1, vars.IndexLookupJoinConcurrency()) - checkSet(variable.TiDBHashJoinConcurrency) + checkSet(vardef.TiDBHashJoinConcurrency) require.Equal(t, 1, vars.HashJoinConcurrency()) - checkSet(variable.TiDBHashAggPartialConcurrency) + checkSet(vardef.TiDBHashAggPartialConcurrency) require.Equal(t, 1, vars.HashAggPartialConcurrency()) - checkSet(variable.TiDBHashAggFinalConcurrency) + checkSet(vardef.TiDBHashAggFinalConcurrency) require.Equal(t, 1, vars.HashAggFinalConcurrency()) - checkSet(variable.TiDBProjectionConcurrency) + checkSet(vardef.TiDBProjectionConcurrency) require.Equal(t, 1, vars.ProjectionConcurrency()) - checkSet(variable.TiDBWindowConcurrency) + checkSet(vardef.TiDBWindowConcurrency) require.Equal(t, 1, vars.WindowConcurrency()) - checkSet(variable.TiDBStreamAggConcurrency) + checkSet(vardef.TiDBStreamAggConcurrency) require.Equal(t, 1, vars.StreamAggConcurrency()) - tk.MustExec(fmt.Sprintf("set @@%s=1;", variable.TiDBDistSQLScanConcurrency)) - tk.MustQuery(fmt.Sprintf("select @@%s;", variable.TiDBDistSQLScanConcurrency)).Check(testkit.Rows("1")) + tk.MustExec(fmt.Sprintf("set @@%s=1;", vardef.TiDBDistSQLScanConcurrency)) + tk.MustQuery(fmt.Sprintf("select @@%s;", vardef.TiDBDistSQLScanConcurrency)).Check(testkit.Rows("1")) require.Equal(t, 1, vars.DistSQLScanConcurrency()) tk.MustExec("set @@tidb_index_serial_scan_concurrency=4") @@ -1520,14 +1521,14 @@ func TestSetConcurrency(t *testing.T) { tk.MustExec("set @@tidb_streamagg_concurrency=-1;") tk.MustExec("set @@tidb_projection_concurrency=-1;") - require.Equal(t, variable.DefExecutorConcurrency, vars.IndexLookupConcurrency()) - require.Equal(t, variable.DefExecutorConcurrency, vars.IndexLookupJoinConcurrency()) - require.Equal(t, variable.DefExecutorConcurrency, vars.HashJoinConcurrency()) - require.Equal(t, variable.DefExecutorConcurrency, vars.HashAggPartialConcurrency()) - require.Equal(t, variable.DefExecutorConcurrency, vars.HashAggFinalConcurrency()) - require.Equal(t, variable.DefExecutorConcurrency, vars.WindowConcurrency()) - require.Equal(t, variable.DefExecutorConcurrency, vars.StreamAggConcurrency()) - require.Equal(t, variable.DefExecutorConcurrency, vars.ProjectionConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.IndexLookupConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.IndexLookupJoinConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.HashJoinConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.HashAggPartialConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.HashAggFinalConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.WindowConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.StreamAggConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.ProjectionConcurrency()) tk.MustExec("set @@tidb_executor_concurrency=-1;") tk.MustQuery(`show warnings`).Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_executor_concurrency value: '-1'")) diff --git a/pkg/executor/show.go b/pkg/executor/show.go index d2931285aa1a8..e06386a0fb9a7 100644 --- a/pkg/executor/show.go +++ b/pkg/executor/show.go @@ -59,6 +59,7 @@ import ( "github.com/pingcap/tidb/pkg/privilege/privileges" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/store/helper" @@ -898,7 +899,7 @@ func (e *ShowExec) fetchShowCharset() error { defaultCollation := desc.DefaultCollation if desc.Name == charset.CharsetUTF8MB4 { var err error - defaultCollation, err = sessVars.GetSessionOrGlobalSystemVar(context.Background(), variable.DefaultCollationForUTF8MB4) + defaultCollation, err = sessVars.GetSessionOrGlobalSystemVar(context.Background(), vardef.DefaultCollationForUTF8MB4) if err != nil { return err } @@ -939,8 +940,8 @@ func (e *ShowExec) fetchShowVariables(ctx context.Context) (err error) { // 2. If the variable is ScopeNone, it's a read-only variable, return the default value of it, // otherwise, fetch the value from table `mysql.Global_Variables`. for _, v := range variable.GetSysVars() { - if v.Scope != variable.ScopeSession { - if v.IsNoop && !variable.EnableNoopVariables.Load() { + if v.Scope != vardef.ScopeSession { + if v.IsNoop && !vardef.EnableNoopVariables.Load() { continue } if fieldFilter != "" && v.Name != fieldFilter { @@ -965,7 +966,7 @@ func (e *ShowExec) fetchShowVariables(ctx context.Context) (err error) { // If it is a session only variable, use the default value defined in code, // otherwise, fetch the value from table `mysql.Global_Variables`. for _, v := range variable.GetSysVars() { - if v.IsNoop && !variable.EnableNoopVariables.Load() { + if v.IsNoop && !vardef.EnableNoopVariables.Load() { continue } if fieldFilter != "" && v.Name != fieldFilter { @@ -993,7 +994,7 @@ func (e *ShowExec) fetchShowStatus() error { } checker := privilege.GetPrivilegeManager(e.Ctx()) for status, v := range statusVars { - if e.GlobalScope && v.Scope == variable.ScopeSession { + if e.GlobalScope && v.Scope == vardef.ScopeSession { continue } // Skip invisible status vars if permission fails. @@ -1671,7 +1672,7 @@ func isUTF8MB4AndDefaultCollation(sessVars *variable.SessionVars, cs, co string) if cs != charset.CharsetUTF8MB4 { return false, false, nil } - defaultCollation, err := sessVars.GetSessionOrGlobalSystemVar(context.Background(), variable.DefaultCollationForUTF8MB4) + defaultCollation, err := sessVars.GetSessionOrGlobalSystemVar(context.Background(), vardef.DefaultCollationForUTF8MB4) if err != nil { return false, false, err } @@ -1763,7 +1764,7 @@ func (e *ShowExec) fetchShowCreateUser(ctx context.Context) error { fmt.Sprintf("'%s'@'%s'", e.User.Username, e.User.Hostname)) } - authPlugin, err := e.Ctx().GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.DefaultAuthPlugin) + authPlugin, err := e.Ctx().GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.DefaultAuthPlugin) if err != nil { return errors.Trace(err) } diff --git a/pkg/executor/show_ddl_jobs.go b/pkg/executor/show_ddl_jobs.go index a885d5d5e18c9..c0b3205de06a0 100644 --- a/pkg/executor/show_ddl_jobs.go +++ b/pkg/executor/show_ddl_jobs.go @@ -33,7 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -315,16 +315,16 @@ func showCommentsFromJob(job *model.Job) string { } } if job.MayNeedReorg() { - concurrency := m.GetConcurrencyOrDefault(int(variable.GetDDLReorgWorkerCounter())) - batchSize := m.GetBatchSizeOrDefault(int(variable.GetDDLReorgBatchSize())) - maxWriteSpeed := m.GetMaxWriteSpeedOrDefault() - if concurrency != variable.DefTiDBDDLReorgWorkerCount { + concurrency := m.GetConcurrency() + batchSize := m.GetBatchSize() + maxWriteSpeed := m.GetMaxWriteSpeed() + if concurrency != vardef.DefTiDBDDLReorgWorkerCount { labels = append(labels, fmt.Sprintf("thread=%d", concurrency)) } - if batchSize != variable.DefTiDBDDLReorgBatchSize { + if batchSize != vardef.DefTiDBDDLReorgBatchSize { labels = append(labels, fmt.Sprintf("batch_size=%d", batchSize)) } - if maxWriteSpeed != variable.DefTiDBDDLReorgMaxWriteSpeed { + if maxWriteSpeed != vardef.DefTiDBDDLReorgMaxWriteSpeed { labels = append(labels, fmt.Sprintf("max_write_speed=%d", maxWriteSpeed)) } if m.TargetScope != "" { diff --git a/pkg/executor/show_ddl_jobs_test.go b/pkg/executor/show_ddl_jobs_test.go index a5105f0588841..87183e1d30342 100644 --- a/pkg/executor/show_ddl_jobs_test.go +++ b/pkg/executor/show_ddl_jobs_test.go @@ -18,7 +18,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/stretchr/testify/require" ) @@ -80,9 +80,9 @@ func TestShowCommentsFromJob(t *testing.T) { IsDistReorg: true, UseCloudStorage: true, } - job.ReorgMeta.Concurrency.Store(variable.DefTiDBDDLReorgWorkerCount) - job.ReorgMeta.BatchSize.Store(variable.DefTiDBDDLReorgBatchSize) - job.ReorgMeta.MaxWriteSpeed.Store(variable.DefTiDBDDLReorgMaxWriteSpeed) + job.ReorgMeta.Concurrency.Store(vardef.DefTiDBDDLReorgWorkerCount) + job.ReorgMeta.BatchSize.Store(vardef.DefTiDBDDLReorgBatchSize) + job.ReorgMeta.MaxWriteSpeed.Store(vardef.DefTiDBDDLReorgMaxWriteSpeed) res = showCommentsFromJob(job) require.Equal(t, "ingest, DXF, cloud", res) @@ -92,9 +92,9 @@ func TestShowCommentsFromJob(t *testing.T) { UseCloudStorage: true, TargetScope: "background", } - job.ReorgMeta.Concurrency.Store(variable.DefTiDBDDLReorgWorkerCount) - job.ReorgMeta.BatchSize.Store(variable.DefTiDBDDLReorgBatchSize) - job.ReorgMeta.MaxWriteSpeed.Store(variable.DefTiDBDDLReorgMaxWriteSpeed) + job.ReorgMeta.Concurrency.Store(vardef.DefTiDBDDLReorgWorkerCount) + job.ReorgMeta.BatchSize.Store(vardef.DefTiDBDDLReorgBatchSize) + job.ReorgMeta.MaxWriteSpeed.Store(vardef.DefTiDBDDLReorgMaxWriteSpeed) res = showCommentsFromJob(job) require.Equal(t, "ingest, DXF, cloud, service_scope=background", res) } diff --git a/pkg/executor/simple.go b/pkg/executor/simple.go index 9af2206a967c8..203297df7e02b 100644 --- a/pkg/executor/simple.go +++ b/pkg/executor/simple.go @@ -51,6 +51,7 @@ import ( "github.com/pingcap/tidb/pkg/resourcegroup" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/types" @@ -608,7 +609,7 @@ func (e *SimpleExec) executeUse(s *ast.UseStmt) error { // collation if this one is not supported. // The SetSystemVar will also update the CharsetDatabase dbCollate = collate.SubstituteMissingCollationToDefault(dbCollate) - return sessionVars.SetSystemVarWithoutValidation(variable.CollationDatabase, dbCollate) + return sessionVars.SetSystemVarWithoutValidation(vardef.CollationDatabase, dbCollate) } func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { @@ -822,12 +823,12 @@ func whetherSavePasswordHistory(plOptions *passwordOrLockOptionsInfo) bool { if plOptions.passwordHistoryChange && plOptions.passwordHistory != notSpecified { passwdSaveNum = plOptions.passwordHistory } else { - passwdSaveNum = variable.PasswordHistory.Load() + passwdSaveNum = vardef.PasswordHistory.Load() } if plOptions.passwordReuseIntervalChange && plOptions.passwordReuseInterval != notSpecified { passwdSaveTime = plOptions.passwordReuseInterval } else { - passwdSaveTime = variable.PasswordReuseInterval.Load() + passwdSaveTime = vardef.PasswordReuseInterval.Load() } return passwdSaveTime > 0 || passwdSaveNum > 0 } @@ -1011,7 +1012,7 @@ func deletePasswordLockingAttribute(ctx context.Context, sqlExecutor sqlexec.SQL } func (e *SimpleExec) isValidatePasswordEnabled() bool { - validatePwdEnable, err := e.Ctx().GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.ValidatePasswordEnable) + validatePwdEnable, err := e.Ctx().GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.ValidatePasswordEnable) if err != nil { return false } @@ -1076,7 +1077,7 @@ func (e *SimpleExec) executeCreateUser(ctx context.Context, s *ast.CreateUserStm } if s.ResourceGroupNameOption != nil { - if !variable.EnableResourceControl.Load() { + if !vardef.EnableResourceControl.Load() { return infoschema.ErrResourceGroupSupportDisabled } if s.IsCreateRole { @@ -1120,7 +1121,7 @@ func (e *SimpleExec) executeCreateUser(ctx context.Context, s *ast.CreateUserStm if savePasswdHistory { sqlescape.MustFormatSQL(sqlPasswordHistory, `INSERT INTO %n.%n (Host, User, Password) VALUES `, mysql.SystemDB, mysql.PasswordHistoryTable) } - defaultAuthPlugin, err := e.Ctx().GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.DefaultAuthPlugin) + defaultAuthPlugin, err := e.Ctx().GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.DefaultAuthPlugin) if err != nil { return errors.Trace(err) } @@ -1326,12 +1327,12 @@ func getUserPasswordLimit(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, if !row.IsNull(0) { res.passwordHistory = int64(row.GetUint64(0)) } else { - res.passwordHistory = variable.PasswordHistory.Load() + res.passwordHistory = vardef.PasswordHistory.Load() } if !row.IsNull(1) { res.passwordReuseInterval = int64(row.GetUint64(1)) } else { - res.passwordReuseInterval = variable.PasswordReuseInterval.Load() + res.passwordReuseInterval = vardef.PasswordReuseInterval.Load() } } if plOptions.passwordHistoryChange { @@ -1339,7 +1340,7 @@ func getUserPasswordLimit(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, if plOptions.passwordHistory != notSpecified { res.passwordHistory = plOptions.passwordHistory } else { - res.passwordHistory = variable.PasswordHistory.Load() + res.passwordHistory = vardef.PasswordHistory.Load() } } if plOptions.passwordReuseIntervalChange { @@ -1347,7 +1348,7 @@ func getUserPasswordLimit(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, if plOptions.passwordReuseInterval != notSpecified { res.passwordReuseInterval = plOptions.passwordReuseInterval } else { - res.passwordReuseInterval = variable.PasswordReuseInterval.Load() + res.passwordReuseInterval = vardef.PasswordReuseInterval.Load() } } return res, nil @@ -1489,7 +1490,7 @@ func fullRecordCheck(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, userD err = closeErr } }() - rows, err := sqlexec.DrainRecordSet(ctx, recordSet, variable.DefMaxChunkSize) + rows, err := sqlexec.DrainRecordSet(ctx, recordSet, vardef.DefMaxChunkSize) if err != nil { return false, err } @@ -1539,7 +1540,7 @@ func checkPasswordHistoryRule(ctx context.Context, sqlExecutor sqlexec.SQLExecut err = closeErr } }() - rows, err := sqlexec.DrainRecordSet(ctx, recordSet, variable.DefMaxChunkSize) + rows, err := sqlexec.DrainRecordSet(ctx, recordSet, vardef.DefMaxChunkSize) if err != nil { return false, err } @@ -1585,7 +1586,7 @@ func checkPasswordTimeRule(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, err = closeErr } }() - rows, err := sqlexec.DrainRecordSet(ctx, recordSet, variable.DefMaxChunkSize) + rows, err := sqlexec.DrainRecordSet(ctx, recordSet, vardef.DefMaxChunkSize) if err != nil { return false, err } @@ -1932,7 +1933,7 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt) } } if s.ResourceGroupNameOption != nil { - if !variable.EnableResourceControl.Load() { + if !vardef.EnableResourceControl.Load() { return infoschema.ErrResourceGroupSupportDisabled } is, err := isRole(ctx, sqlExecutor, spec.User.Username, spec.User.Hostname) diff --git a/pkg/executor/slow_query.go b/pkg/executor/slow_query.go index 4573539924b36..572d83e6b7947 100644 --- a/pkg/executor/slow_query.go +++ b/pkg/executor/slow_query.go @@ -40,6 +40,7 @@ import ( plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" @@ -296,7 +297,7 @@ func (sc *slowLogChecker) isTimeValid(t types.Time) bool { } func getOneLine(reader *bufio.Reader) ([]byte, error) { - return util.ReadLine(reader, int(variable.MaxOfMaxAllowedPacket)) + return util.ReadLine(reader, int(vardef.MaxOfMaxAllowedPacket)) } type offset struct { diff --git a/pkg/executor/slow_query_test.go b/pkg/executor/slow_query_test.go index 801bc86237e1e..74be371cbb151 100644 --- a/pkg/executor/slow_query_test.go +++ b/pkg/executor/slow_query_test.go @@ -34,7 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/logutil" @@ -59,7 +59,7 @@ func parseLog(retriever *slowQueryRetriever, sctx sessionctx.Context, reader *bu func newSlowQueryRetriever() (*slowQueryRetriever, error) { data := infoschema.NewData() - newISBuilder := infoschema.NewBuilder(nil, nil, data, variable.SchemaCacheSize.Load() > 0) + newISBuilder := infoschema.NewBuilder(nil, nil, data, vardef.SchemaCacheSize.Load() > 0) err := newISBuilder.InitWithDBInfos(nil, nil, nil, 0) if err != nil { return nil, err @@ -272,16 +272,16 @@ func TestParseSlowLogFileSerial(t *testing.T) { select * from t; # Time: 2019-04-24-19:41:21.716221 +0800 `) - originValue := variable.MaxOfMaxAllowedPacket - variable.MaxOfMaxAllowedPacket = 65536 - sql := strings.Repeat("x", int(variable.MaxOfMaxAllowedPacket+1)) + originValue := vardef.MaxOfMaxAllowedPacket + vardef.MaxOfMaxAllowedPacket = 65536 + sql := strings.Repeat("x", int(vardef.MaxOfMaxAllowedPacket+1)) slowLog.WriteString(sql) reader := bufio.NewReader(slowLog) _, err = parseSlowLog(ctx, reader) require.Error(t, err) require.EqualError(t, err, "single line length exceeds limit: 65536") - variable.MaxOfMaxAllowedPacket = originValue + vardef.MaxOfMaxAllowedPacket = originValue reader = bufio.NewReader(slowLog) _, err = parseSlowLog(ctx, reader) require.NoError(t, err) diff --git a/pkg/executor/sortexec/BUILD.bazel b/pkg/executor/sortexec/BUILD.bazel index 9800055c4ebbc..6da8cce164453 100644 --- a/pkg/executor/sortexec/BUILD.bazel +++ b/pkg/executor/sortexec/BUILD.bazel @@ -22,7 +22,7 @@ go_library( "//pkg/expression", "//pkg/planner/core", "//pkg/planner/util", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/types", "//pkg/util", "//pkg/util/channel", @@ -47,6 +47,7 @@ go_test( shard_count = 17, deps = [ "//pkg/config", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/testkit", "//pkg/util", @@ -79,7 +80,7 @@ go_test( "//pkg/parser/mysql", "//pkg/planner/core", "//pkg/planner/util", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/types", "//pkg/util", diff --git a/pkg/executor/sortexec/benchmark_test.go b/pkg/executor/sortexec/benchmark_test.go index db9436fa2d49c..c64f98ea4b483 100644 --- a/pkg/executor/sortexec/benchmark_test.go +++ b/pkg/executor/sortexec/benchmark_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/sortexec" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/planner/util" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util/mock" ) @@ -107,9 +107,9 @@ func BenchmarkSortExec(b *testing.B) { } func BenchmarkSortExecSpillToDisk(b *testing.B) { - enableTmpStorageOnOOMCurrentVal := variable.EnableTmpStorageOnOOM.Load() - variable.EnableTmpStorageOnOOM.Store(true) - defer variable.EnableTmpStorageOnOOM.Store(enableTmpStorageOnOOMCurrentVal) + enableTmpStorageOnOOMCurrentVal := vardef.EnableTmpStorageOnOOM.Load() + vardef.EnableTmpStorageOnOOM.Store(true) + defer vardef.EnableTmpStorageOnOOM.Store(enableTmpStorageOnOOMCurrentVal) b.ReportAllocs() cas := testutil.SortTestCaseWithMemoryLimit(mock.NewContext(), 1) diff --git a/pkg/executor/sortexec/sort.go b/pkg/executor/sortexec/sort.go index edfb02c6ed2c3..403ef78785f23 100644 --- a/pkg/executor/sortexec/sort.go +++ b/pkg/executor/sortexec/sort.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/internal/exec" "github.com/pingcap/tidb/pkg/expression" plannerutil "github.com/pingcap/tidb/pkg/planner/util" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/channel" @@ -155,7 +155,7 @@ func (e *SortExec) Close() error { func (e *SortExec) Open(ctx context.Context) error { e.fetched = &atomic.Bool{} e.fetched.Store(false) - e.enableTmpStorageOnOOM = variable.EnableTmpStorageOnOOM.Load() + e.enableTmpStorageOnOOM = vardef.EnableTmpStorageOnOOM.Load() e.finishCh = make(chan struct{}, 1) // To avoid duplicated initialization for TopNExec. diff --git a/pkg/executor/sortexec/sort_test.go b/pkg/executor/sortexec/sort_test.go index d36e3ed071b6a..1c2149fda1652 100644 --- a/pkg/executor/sortexec/sort_test.go +++ b/pkg/executor/sortexec/sort_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/config" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" @@ -103,9 +103,9 @@ func TestIssue16696(t *testing.T) { conf.TempStoragePath = t.TempDir() conf.Performance.EnableStatsCacheMemQuota = true }) - alarmRatio := variable.MemoryUsageAlarmRatio.Load() - variable.MemoryUsageAlarmRatio.Store(0.0) - defer variable.MemoryUsageAlarmRatio.Store(alarmRatio) + alarmRatio := vardef.MemoryUsageAlarmRatio.Load() + vardef.MemoryUsageAlarmRatio.Store(0.0) + defer vardef.MemoryUsageAlarmRatio.Store(alarmRatio) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/sortexec/testSortedRowContainerSpill", "return(true)")) defer require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/sortexec/testSortedRowContainerSpill")) diff --git a/pkg/executor/sortexec/topn.go b/pkg/executor/sortexec/topn.go index 8e024831a7082..b5a69390991b3 100644 --- a/pkg/executor/sortexec/topn.go +++ b/pkg/executor/sortexec/topn.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/executor/internal/exec" plannercore "github.com/pingcap/tidb/pkg/planner/core" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/channel" "github.com/pingcap/tidb/pkg/util/chunk" @@ -87,7 +87,7 @@ func (e *TopNExec) Open(ctx context.Context) error { e.isSpillTriggeredInStage1ForTest = false e.isSpillTriggeredInStage2ForTest = false - if variable.EnableTmpStorageOnOOM.Load() { + if vardef.EnableTmpStorageOnOOM.Load() { e.diskTracker = disk.NewTracker(e.ID(), -1) diskTracker := e.Ctx().GetSessionVars().StmtCtx.DiskTracker if diskTracker != nil { diff --git a/pkg/executor/stmtsummary_test.go b/pkg/executor/stmtsummary_test.go index c8f504f41c179..e29bc36ee77d3 100644 --- a/pkg/executor/stmtsummary_test.go +++ b/pkg/executor/stmtsummary_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/mock" @@ -33,7 +33,7 @@ import ( func TestStmtSummaryRetriverV2_TableStatementsSummary(t *testing.T) { data := infoschema.NewData() - infoSchemaBuilder := infoschema.NewBuilder(nil, nil, data, variable.SchemaCacheSize.Load() > 0) + infoSchemaBuilder := infoschema.NewBuilder(nil, nil, data, vardef.SchemaCacheSize.Load() > 0) err := infoSchemaBuilder.InitWithDBInfos(nil, nil, nil, 0) require.NoError(t, err) infoSchema := infoSchemaBuilder.Build(math.MaxUint64) @@ -78,7 +78,7 @@ func TestStmtSummaryRetriverV2_TableStatementsSummary(t *testing.T) { func TestStmtSummaryRetriverV2_TableStatementsSummaryEvicted(t *testing.T) { data := infoschema.NewData() - infoSchemaBuilder := infoschema.NewBuilder(nil, nil, data, variable.SchemaCacheSize.Load() > 0) + infoSchemaBuilder := infoschema.NewBuilder(nil, nil, data, vardef.SchemaCacheSize.Load() > 0) err := infoSchemaBuilder.InitWithDBInfos(nil, nil, nil, 0) require.NoError(t, err) infoSchema := infoSchemaBuilder.Build(math.MaxUint64) @@ -158,7 +158,7 @@ func TestStmtSummaryRetriverV2_TableStatementsSummaryHistory(t *testing.T) { stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) data := infoschema.NewData() - infoSchemaBuilder := infoschema.NewBuilder(nil, nil, data, variable.SchemaCacheSize.Load() > 0) + infoSchemaBuilder := infoschema.NewBuilder(nil, nil, data, vardef.SchemaCacheSize.Load() > 0) err = infoSchemaBuilder.InitWithDBInfos(nil, nil, nil, 0) require.NoError(t, err) infoSchema := infoSchemaBuilder.Build(math.MaxUint64) diff --git a/pkg/executor/test/admintest/BUILD.bazel b/pkg/executor/test/admintest/BUILD.bazel index 214217f6113bd..b987e190e6e3c 100644 --- a/pkg/executor/test/admintest/BUILD.bazel +++ b/pkg/executor/test/admintest/BUILD.bazel @@ -21,7 +21,7 @@ go_test( "//pkg/parser/ast", "//pkg/session", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/table", "//pkg/table/tables", "//pkg/tablecodec", diff --git a/pkg/executor/test/admintest/admin_test.go b/pkg/executor/test/admintest/admin_test.go index f79e28500d8cf..4ed02a9fe3b9c 100644 --- a/pkg/executor/test/admintest/admin_test.go +++ b/pkg/executor/test/admintest/admin_test.go @@ -34,7 +34,7 @@ import ( ast "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/tablecodec" @@ -302,7 +302,7 @@ func TestClusteredIndexAdminRecoverIndex(t *testing.T) { tk.MustExec("drop database if exists test_cluster_index_admin_recover;") tk.MustExec("create database test_cluster_index_admin_recover;") tk.MustExec("use test_cluster_index_admin_recover;") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn dbName := ast.NewCIStr("test_cluster_index_admin_recover") tblName := ast.NewCIStr("t") @@ -440,7 +440,7 @@ func TestAdminRecoverIndex1(t *testing.T) { tblName := ast.NewCIStr("admin_test") tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly tk.MustExec("create table admin_test (c1 varchar(255), c2 int, c3 int default 1, primary key(c1), unique key(c2))") tk.MustExec("insert admin_test (c1, c2) values ('1', 1), ('2', 2), ('3', 3), ('10', 10), ('20', 20)") @@ -654,7 +654,7 @@ func TestAdminCleanupIndexPKNotHandle(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly tk.MustExec("create table admin_test (c1 int, c2 int, c3 int, primary key (c1, c2))") tk.MustExec("insert admin_test (c1, c2) values (1, 2), (3, 4), (-5, 5)") @@ -772,7 +772,7 @@ func TestClusteredAdminCleanupIndex(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("create table admin_test (c1 varchar(255), c2 int, c3 char(10) default 'c3', primary key (c1, c3), unique key(c2), key (c3))") tk.MustExec("insert admin_test (c1, c2) values ('c1_1', 2), ('c1_2', 4), ('c1_3', NULL)") tk.MustExec("insert admin_test (c1, c3) values ('c1_4', 'c3_4'), ('c1_5', 'c3_5'), ('c1_6', default)") diff --git a/pkg/executor/test/analyzetest/BUILD.bazel b/pkg/executor/test/analyzetest/BUILD.bazel index 38edb3ef02f25..912abbb8ee70c 100644 --- a/pkg/executor/test/analyzetest/BUILD.bazel +++ b/pkg/executor/test/analyzetest/BUILD.bazel @@ -25,6 +25,7 @@ go_test( "//pkg/planner/core", "//pkg/session", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/statistics/handle/ddl/testutil", diff --git a/pkg/executor/test/analyzetest/analyze_test.go b/pkg/executor/test/analyzetest/analyze_test.go index c6c1d7c9dfdbb..c121ccb01aad6 100644 --- a/pkg/executor/test/analyzetest/analyze_test.go +++ b/pkg/executor/test/analyzetest/analyze_test.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" @@ -294,7 +295,7 @@ func TestNormalAnalyzeOnCommonHandle(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2, t3, t4") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("CREATE TABLE t1 (a int primary key, b int)") tk.MustExec("insert into t1 values(1,1), (2,2), (3,3)") tk.MustExec("CREATE TABLE t2 (a varchar(255) primary key, b int)") diff --git a/pkg/executor/test/analyzetest/memorycontrol/BUILD.bazel b/pkg/executor/test/analyzetest/memorycontrol/BUILD.bazel index 83de820219af5..abd01a8783e3c 100644 --- a/pkg/executor/test/analyzetest/memorycontrol/BUILD.bazel +++ b/pkg/executor/test/analyzetest/memorycontrol/BUILD.bazel @@ -13,7 +13,7 @@ go_test( deps = [ "//pkg/config", "//pkg/executor", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/statistics", "//pkg/statistics/handle/ddl/testutil", "//pkg/testkit", diff --git a/pkg/executor/test/analyzetest/memorycontrol/main_test.go b/pkg/executor/test/analyzetest/memorycontrol/main_test.go index d0c329bee4e63..77886a68d65d6 100644 --- a/pkg/executor/test/analyzetest/memorycontrol/main_test.go +++ b/pkg/executor/test/analyzetest/memorycontrol/main_test.go @@ -18,7 +18,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/config" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "go.uber.org/goleak" ) @@ -26,7 +26,7 @@ func TestMain(m *testing.M) { config.UpdateGlobal(func(conf *config.Config) { conf.Performance.EnableStatsCacheMemQuota = true }) - variable.StatsCacheMemQuota.Store(1000000) + vardef.StatsCacheMemQuota.Store(1000000) opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"), goleak.IgnoreTopFunction("github.com/bazelbuild/rules_go/go/tools/bzltestutil.RegisterTimeoutHandler.func1"), diff --git a/pkg/executor/test/ddl/BUILD.bazel b/pkg/executor/test/ddl/BUILD.bazel index aa5dc958422ae..4a000e98b73e3 100644 --- a/pkg/executor/test/ddl/BUILD.bazel +++ b/pkg/executor/test/ddl/BUILD.bazel @@ -21,6 +21,7 @@ go_test( "//pkg/meta/autoid", "//pkg/parser/ast", "//pkg/parser/mysql", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/store/mockstore", diff --git a/pkg/executor/test/ddl/ddl_test.go b/pkg/executor/test/ddl/ddl_test.go index c8fc4c812e6b3..6185d7c924dc0 100644 --- a/pkg/executor/test/ddl/ddl_test.go +++ b/pkg/executor/test/ddl/ddl_test.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/store/mockstore" @@ -783,20 +784,20 @@ func TestSetDDLReorgWorkerCnt(t *testing.T) { tk.MustExec("use test") err := ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) require.NoError(t, err) - require.Equal(t, int32(variable.DefTiDBDDLReorgWorkerCount), variable.GetDDLReorgWorkerCounter()) + require.Equal(t, int32(vardef.DefTiDBDDLReorgWorkerCount), vardef.GetDDLReorgWorkerCounter()) tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 1") err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) require.NoError(t, err) - require.Equal(t, int32(1), variable.GetDDLReorgWorkerCounter()) + require.Equal(t, int32(1), vardef.GetDDLReorgWorkerCounter()) tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 100") err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) require.NoError(t, err) - require.Equal(t, int32(100), variable.GetDDLReorgWorkerCounter()) + require.Equal(t, int32(100), vardef.GetDDLReorgWorkerCounter()) tk.MustGetDBError("set @@global.tidb_ddl_reorg_worker_cnt = invalid_val", variable.ErrWrongTypeForVar) tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 100") err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) require.NoError(t, err) - require.Equal(t, int32(100), variable.GetDDLReorgWorkerCounter()) + require.Equal(t, int32(100), vardef.GetDDLReorgWorkerCounter()) tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = -1") tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_reorg_worker_cnt value: '-1'")) tk.MustQuery("select @@global.tidb_ddl_reorg_worker_cnt").Check(testkit.Rows("1")) @@ -818,7 +819,7 @@ func TestSetDDLReorgWorkerCnt(t *testing.T) { tk.MustExec("set @@tidb_ddl_reorg_worker_cnt = 10;") tk.MustQuery("select @@tidb_ddl_reorg_worker_cnt;").Check(testkit.Rows("10")) tk.MustQuery("select @@global.tidb_ddl_reorg_worker_cnt;").Check(testkit.Rows("256")) - require.Equal(t, int32(256), variable.GetDDLReorgWorkerCounter()) + require.Equal(t, int32(256), vardef.GetDDLReorgWorkerCounter()) } func TestSetDDLReorgBatchSize(t *testing.T) { @@ -827,23 +828,23 @@ func TestSetDDLReorgBatchSize(t *testing.T) { tk.MustExec("use test") err := ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) require.NoError(t, err) - require.Equal(t, int32(variable.DefTiDBDDLReorgBatchSize), variable.GetDDLReorgBatchSize()) + require.Equal(t, int32(vardef.DefTiDBDDLReorgBatchSize), vardef.GetDDLReorgBatchSize()) tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = 1") tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_reorg_batch_size value: '1'")) err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) require.NoError(t, err) - require.Equal(t, variable.MinDDLReorgBatchSize, variable.GetDDLReorgBatchSize()) - tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_batch_size = %v", variable.MaxDDLReorgBatchSize+1)) - tk.MustQuery("show warnings;").Check(testkit.Rows(fmt.Sprintf("Warning 1292 Truncated incorrect tidb_ddl_reorg_batch_size value: '%d'", variable.MaxDDLReorgBatchSize+1))) + require.Equal(t, vardef.MinDDLReorgBatchSize, vardef.GetDDLReorgBatchSize()) + tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_batch_size = %v", vardef.MaxDDLReorgBatchSize+1)) + tk.MustQuery("show warnings;").Check(testkit.Rows(fmt.Sprintf("Warning 1292 Truncated incorrect tidb_ddl_reorg_batch_size value: '%d'", vardef.MaxDDLReorgBatchSize+1))) err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) require.NoError(t, err) - require.Equal(t, variable.MaxDDLReorgBatchSize, variable.GetDDLReorgBatchSize()) + require.Equal(t, vardef.MaxDDLReorgBatchSize, vardef.GetDDLReorgBatchSize()) tk.MustGetDBError("set @@global.tidb_ddl_reorg_batch_size = invalid_val", variable.ErrWrongTypeForVar) tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = 100") err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) require.NoError(t, err) - require.Equal(t, int32(100), variable.GetDDLReorgBatchSize()) + require.Equal(t, int32(100), vardef.GetDDLReorgBatchSize()) tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = -1") tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_reorg_batch_size value: '-1'")) @@ -868,23 +869,23 @@ func TestSetDDLErrorCountLimit(t *testing.T) { tk.MustExec("use test") err := ddlutil.LoadDDLVars(tk.Session()) require.NoError(t, err) - require.Equal(t, int64(variable.DefTiDBDDLErrorCountLimit), variable.GetDDLErrorCountLimit()) + require.Equal(t, int64(vardef.DefTiDBDDLErrorCountLimit), vardef.GetDDLErrorCountLimit()) tk.MustExec("set @@global.tidb_ddl_error_count_limit = -1") tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_error_count_limit value: '-1'")) err = ddlutil.LoadDDLVars(tk.Session()) require.NoError(t, err) - require.Equal(t, int64(0), variable.GetDDLErrorCountLimit()) + require.Equal(t, int64(0), vardef.GetDDLErrorCountLimit()) tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_error_count_limit = %v", uint64(math.MaxInt64)+1)) tk.MustQuery("show warnings;").Check(testkit.Rows(fmt.Sprintf("Warning 1292 Truncated incorrect tidb_ddl_error_count_limit value: '%d'", uint64(math.MaxInt64)+1))) err = ddlutil.LoadDDLVars(tk.Session()) require.NoError(t, err) - require.Equal(t, int64(math.MaxInt64), variable.GetDDLErrorCountLimit()) + require.Equal(t, int64(math.MaxInt64), vardef.GetDDLErrorCountLimit()) tk.MustGetDBError("set @@global.tidb_ddl_error_count_limit = invalid_val", variable.ErrWrongTypeForVar) tk.MustExec("set @@global.tidb_ddl_error_count_limit = 100") err = ddlutil.LoadDDLVars(tk.Session()) require.NoError(t, err) - require.Equal(t, int64(100), variable.GetDDLErrorCountLimit()) + require.Equal(t, int64(100), vardef.GetDDLErrorCountLimit()) res := tk.MustQuery("select @@global.tidb_ddl_error_count_limit") res.Check(testkit.Rows("100")) } @@ -892,19 +893,19 @@ func TestSetDDLErrorCountLimit(t *testing.T) { func TestSetDDLReorgMaxWriteSpeed(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) - require.Equal(t, int64(variable.DefTiDBDDLReorgMaxWriteSpeed), variable.DDLReorgMaxWriteSpeed.Load()) + require.Equal(t, int64(vardef.DefTiDBDDLReorgMaxWriteSpeed), vardef.DDLReorgMaxWriteSpeed.Load()) // valid values for _, val := range []int64{1, 0, 100, 1024 * 1024, 2147483647, units.PiB} { tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_max_write_speed = %d", val)) - require.Equal(t, val, variable.DDLReorgMaxWriteSpeed.Load()) + require.Equal(t, val, vardef.DDLReorgMaxWriteSpeed.Load()) tk.MustQuery("select @@global.tidb_ddl_reorg_max_write_speed").Check(testkit.Rows(strconv.FormatInt(val, 10))) } for _, val := range []string{"1", "0", "100", "2KB", "3MiB", "4 gb", "2147483647", "1125899906842624" /* 1PiB */} { tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_max_write_speed = '%s'", val)) expected, err := units.RAMInBytes(val) require.NoError(t, err) - require.Equal(t, expected, variable.DDLReorgMaxWriteSpeed.Load()) + require.Equal(t, expected, vardef.DDLReorgMaxWriteSpeed.Load()) tk.MustQuery("select @@global.tidb_ddl_reorg_max_write_speed").Check(testkit.Rows(strconv.FormatInt(expected, 10))) } @@ -919,13 +920,13 @@ func TestLoadDDLDistributeVars(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - require.Equal(t, variable.DefTiDBEnableDistTask, variable.EnableDistTask.Load()) + require.Equal(t, vardef.DefTiDBEnableDistTask, vardef.EnableDistTask.Load()) tk.MustGetDBError("set @@global.tidb_enable_dist_task = invalid_val", variable.ErrWrongValueForVar) - require.Equal(t, variable.DefTiDBEnableDistTask, variable.EnableDistTask.Load()) + require.Equal(t, vardef.DefTiDBEnableDistTask, vardef.EnableDistTask.Load()) tk.MustExec("set @@global.tidb_enable_dist_task = 'on'") - require.Equal(t, true, variable.EnableDistTask.Load()) + require.Equal(t, true, vardef.EnableDistTask.Load()) tk.MustExec(fmt.Sprintf("set @@global.tidb_enable_dist_task = %v", false)) - require.Equal(t, false, variable.EnableDistTask.Load()) + require.Equal(t, false, vardef.EnableDistTask.Load()) } func forceFullReload(t *testing.T, store kv.Storage, dom *domain.Domain) { diff --git a/pkg/executor/test/distsqltest/BUILD.bazel b/pkg/executor/test/distsqltest/BUILD.bazel index e6ae1cc596001..1c7e5fb554ef0 100644 --- a/pkg/executor/test/distsqltest/BUILD.bazel +++ b/pkg/executor/test/distsqltest/BUILD.bazel @@ -13,7 +13,7 @@ go_test( "//pkg/config", "//pkg/kv", "//pkg/meta/autoid", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//tikv", diff --git a/pkg/executor/test/distsqltest/distsql_test.go b/pkg/executor/test/distsqltest/distsql_test.go index 74440d94136b3..ea00bcc3f0e9e 100644 --- a/pkg/executor/test/distsqltest/distsql_test.go +++ b/pkg/executor/test/distsqltest/distsql_test.go @@ -21,7 +21,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -62,7 +62,7 @@ func TestDistsqlPartitionTableConcurrency(t *testing.T) { // 20-ranges-partitioned table checker ctx3 := context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) { require.Equal(t, req.KeyRanges.PartitionNum(), 20) - require.Equal(t, req.Concurrency, variable.DefDistSQLScanConcurrency) + require.Equal(t, req.Concurrency, vardef.DefDistSQLScanConcurrency) }) ctxs := []context.Context{ctx1, ctx2, ctx3} for i, tbl := range []string{"t1", "t2", "t3"} { diff --git a/pkg/executor/test/executor/BUILD.bazel b/pkg/executor/test/executor/BUILD.bazel index b8ab704ebe3e7..7713ba83cda28 100644 --- a/pkg/executor/test/executor/BUILD.bazel +++ b/pkg/executor/test/executor/BUILD.bazel @@ -33,6 +33,7 @@ go_test( "//pkg/session", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/store/mockstore", diff --git a/pkg/executor/test/executor/executor_test.go b/pkg/executor/test/executor/executor_test.go index 8d53c1a212da2..2ffd002b22ba8 100644 --- a/pkg/executor/test/executor/executor_test.go +++ b/pkg/executor/test/executor/executor_test.go @@ -55,6 +55,7 @@ import ( "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/store/mockstore" @@ -664,8 +665,8 @@ func TestPrevStmtDesensitization(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") - tk.MustExec(fmt.Sprintf("set @@session.%v=1", variable.TiDBRedactLog)) - defer tk.MustExec(fmt.Sprintf("set @@session.%v=0", variable.TiDBRedactLog)) + tk.MustExec(fmt.Sprintf("set @@session.%v=1", vardef.TiDBRedactLog)) + defer tk.MustExec(fmt.Sprintf("set @@session.%v=0", vardef.TiDBRedactLog)) tk.MustExec("create table t (a int, unique key (a))") tk.MustExec("begin") tk.MustExec("insert into t values (1),(2)") @@ -2027,7 +2028,7 @@ func TestClusteredIndexIsPointGet(t *testing.T) { tk.MustExec("create database test_cluster_index_is_point_get;") tk.MustExec("use test_cluster_index_is_point_get;") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a varchar(255), b int, c char(10), primary key (c, a));") ctx := tk.Session().(sessionctx.Context) diff --git a/pkg/executor/test/jointest/hashjoin/BUILD.bazel b/pkg/executor/test/jointest/hashjoin/BUILD.bazel index c01e95fd769b2..c2727f669fbb5 100644 --- a/pkg/executor/test/jointest/hashjoin/BUILD.bazel +++ b/pkg/executor/test/jointest/hashjoin/BUILD.bazel @@ -15,7 +15,7 @@ go_test( "//pkg/executor/join", "//pkg/meta/autoid", "//pkg/session", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/util/dbterror/exeerrors", "@com_github_pingcap_failpoint//:failpoint", diff --git a/pkg/executor/test/jointest/hashjoin/hash_join_test.go b/pkg/executor/test/jointest/hashjoin/hash_join_test.go index ccb369c84140e..074eaa8f11518 100644 --- a/pkg/executor/test/jointest/hashjoin/hash_join_test.go +++ b/pkg/executor/test/jointest/hashjoin/hash_join_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/executor/join" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" "github.com/stretchr/testify/require" @@ -38,7 +38,7 @@ func TestIndexNestedLoopHashJoin(t *testing.T) { tk.MustExec("set @@tidb_init_chunk_size=2") tk.MustExec("set @@tidb_index_join_batch_size=10") tk.MustExec("DROP TABLE IF EXISTS t, s") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly tk.MustExec("create table t(pk int primary key, a int)") for i := 0; i < 100; i++ { tk.MustExec(fmt.Sprintf("insert into t values(%d, %d)", i, i)) @@ -432,7 +432,7 @@ func TestIssue31129(t *testing.T) { tk.MustExec("set @@tidb_init_chunk_size=2") tk.MustExec("set @@tidb_index_join_batch_size=10") tk.MustExec("DROP TABLE IF EXISTS t, s") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly tk.MustExec("create table t(pk int primary key, a int)") for i := 0; i < 100; i++ { tk.MustExec(fmt.Sprintf("insert into t values(%d, %d)", i, i)) diff --git a/pkg/executor/test/passwordtest/BUILD.bazel b/pkg/executor/test/passwordtest/BUILD.bazel index 8b165cd8ce842..a8376fedcbb98 100644 --- a/pkg/executor/test/passwordtest/BUILD.bazel +++ b/pkg/executor/test/passwordtest/BUILD.bazel @@ -16,7 +16,7 @@ go_test( "//pkg/parser/auth", "//pkg/parser/mysql", "//pkg/privilege/privileges", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/util/sqlescape", "@com_github_stretchr_testify//require", diff --git a/pkg/executor/test/passwordtest/password_management_test.go b/pkg/executor/test/passwordtest/password_management_test.go index 1d66178004d54..61fc5bd548033 100644 --- a/pkg/executor/test/passwordtest/password_management_test.go +++ b/pkg/executor/test/passwordtest/password_management_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/privilege/privileges" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/sqlescape" "github.com/stretchr/testify/require" @@ -223,7 +223,7 @@ func TestPasswordManagement(t *testing.T) { // Password expires and takes effect. err = tk.Session().Auth(&auth.UserIdentity{Username: "u2", Hostname: "%"}, sha1Password("Uu3@22222"), nil, nil) require.ErrorContains(t, err, "Your password has expired.") - variable.IsSandBoxModeEnabled.Store(true) + vardef.IsSandBoxModeEnabled.Store(true) err = tk.Session().Auth(&auth.UserIdentity{Username: "u2", Hostname: "%"}, sha1Password("Uu3@22222"), nil, nil) require.NoError(t, err) require.True(t, tk.Session().InSandBoxMode()) diff --git a/pkg/executor/test/seqtest/BUILD.bazel b/pkg/executor/test/seqtest/BUILD.bazel index 1473aaa6cda03..c7337eccaf60d 100644 --- a/pkg/executor/test/seqtest/BUILD.bazel +++ b/pkg/executor/test/seqtest/BUILD.bazel @@ -27,6 +27,7 @@ go_test( "//pkg/planner/core", "//pkg/server", "//pkg/session", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/store/copr", "//pkg/store/mockstore", diff --git a/pkg/executor/test/seqtest/seq_executor_test.go b/pkg/executor/test/seqtest/seq_executor_test.go index edec2ef512c27..9bc86ca748d24 100644 --- a/pkg/executor/test/seqtest/seq_executor_test.go +++ b/pkg/executor/test/seqtest/seq_executor_test.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/store/copr" "github.com/pingcap/tidb/pkg/store/mockstore" @@ -114,7 +115,7 @@ func TestEarlyClose(t *testing.T) { type stats struct { } -func (s stats) GetScope(_ string) variable.ScopeFlag { return variable.DefaultStatusVarScopeFlag } +func (s stats) GetScope(_ string) vardef.ScopeFlag { return variable.DefaultStatusVarScopeFlag } func (s stats) Stats(_ *variable.SessionVars) (map[string]any, error) { m := make(map[string]any) diff --git a/pkg/executor/test/showtest/BUILD.bazel b/pkg/executor/test/showtest/BUILD.bazel index 1a9311a054a79..4469d0050fc39 100644 --- a/pkg/executor/test/showtest/BUILD.bazel +++ b/pkg/executor/test/showtest/BUILD.bazel @@ -22,6 +22,7 @@ go_test( "//pkg/parser/types", "//pkg/privilege/privileges", "//pkg/session", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/testkit", "//pkg/types", diff --git a/pkg/executor/test/showtest/show_test.go b/pkg/executor/test/showtest/show_test.go index 0e91999db9c07..e9fdeb5913559 100644 --- a/pkg/executor/test/showtest/show_test.go +++ b/pkg/executor/test/showtest/show_test.go @@ -32,6 +32,7 @@ import ( parsertypes "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/privilege/privileges" "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/types" @@ -995,7 +996,7 @@ func TestShowVar(t *testing.T) { sessionVars := make([]string, 0, len(variable.GetSysVars())) globalVars := make([]string, 0, len(variable.GetSysVars())) for _, v := range variable.GetSysVars() { - if v.Scope == variable.ScopeSession { + if v.Scope == vardef.ScopeSession { sessionVars = append(sessionVars, v.Name) } else { globalVars = append(globalVars, v.Name) @@ -1026,7 +1027,7 @@ func TestShowVar(t *testing.T) { if strings.HasPrefix(line, "version ") { require.Equal(t, mysql.ServerVersion, line[len("version "):]) } else if strings.HasPrefix(line, "version_comment ") { - require.Equal(t, variable.GetSysVar(variable.VersionComment), line[len("version_comment "):]) + require.Equal(t, variable.GetSysVar(vardef.VersionComment), line[len("version_comment "):]) } } diff --git a/pkg/executor/test/splittest/BUILD.bazel b/pkg/executor/test/splittest/BUILD.bazel index 10c20a8fd732d..41dd1ff0ab80f 100644 --- a/pkg/executor/test/splittest/BUILD.bazel +++ b/pkg/executor/test/splittest/BUILD.bazel @@ -15,7 +15,7 @@ go_test( "//pkg/domain/infosync", "//pkg/kv", "//pkg/parser/terror", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/copr", "//pkg/store/driver/backoff", "//pkg/store/helper", diff --git a/pkg/executor/test/splittest/split_table_test.go b/pkg/executor/test/splittest/split_table_test.go index 2a3d577d41f91..6d507478637cc 100644 --- a/pkg/executor/test/splittest/split_table_test.go +++ b/pkg/executor/test/splittest/split_table_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/terror" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/copr" "github.com/pingcap/tidb/pkg/store/driver/backoff" "github.com/pingcap/tidb/pkg/store/helper" @@ -45,7 +45,7 @@ func TestClusterIndexShowTableRegion(t *testing.T) { tk.MustExec("drop database if exists cluster_index_regions;") tk.MustExec("create database cluster_index_regions;") tk.MustExec("use cluster_index_regions;") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("create table t (a int, b int, c int, primary key(a, b));") tk.MustExec("insert t values (1, 1, 1), (2, 2, 2);") tk.MustQuery("split table t between (1, 0) and (2, 3) regions 2;").Check(testkit.Rows("1 1")) diff --git a/pkg/executor/test/tiflashtest/tiflash_test.go b/pkg/executor/test/tiflashtest/tiflash_test.go index a7244d125f865..b69b067d3221f 100644 --- a/pkg/executor/test/tiflashtest/tiflash_test.go +++ b/pkg/executor/test/tiflashtest/tiflash_test.go @@ -1409,7 +1409,7 @@ func TestDisaggregatedTiFlash(t *testing.T) { conf.DisaggregatedTiFlash = false conf.UseAutoScaler = false }) - err := tiflashcompute.InitGlobalTopoFetcher(tiflashcompute.TestASStr, "tmpAddr", "tmpClusterID", false) + err := tiflashcompute.InitGlobalTopoFetcher(config.TestASStr, "tmpAddr", "tmpClusterID", false) require.NoError(t, err) store := testkit.CreateMockStore(t, withMockTiFlash(2)) @@ -1429,7 +1429,7 @@ func TestDisaggregatedTiFlash(t *testing.T) { // Expect error, because TestAutoScaler return empty topo. require.Contains(t, err.Error(), "Cannot find proper topo to dispatch MPPTask: topo from AutoScaler is empty") - err = tiflashcompute.InitGlobalTopoFetcher(tiflashcompute.AWSASStr, "tmpAddr", "tmpClusterID", false) + err = tiflashcompute.InitGlobalTopoFetcher(config.AWSASStr, "tmpAddr", "tmpClusterID", false) require.NoError(t, err) err = tk.ExecToErr("select * from t;") // Expect error, because AWSAutoScaler is not setup, so http request will fail. @@ -1448,7 +1448,7 @@ func TestDisaggregatedTiFlashNonAutoScaler(t *testing.T) { }) // Setting globalTopoFetcher to nil to can make sure cannot fetch topo from AutoScaler. - err := tiflashcompute.InitGlobalTopoFetcher(tiflashcompute.InvalidASStr, "tmpAddr", "tmpClusterID", false) + err := tiflashcompute.InitGlobalTopoFetcher(config.InvalidASStr, "tmpAddr", "tmpClusterID", false) require.Contains(t, err.Error(), "unexpected topo fetch type. expect: mock or aws or gcp, got invalid") store := testkit.CreateMockStore(t, withMockTiFlash(2)) @@ -1589,7 +1589,7 @@ func TestTiFlashComputeDispatchPolicy(t *testing.T) { // unistore does not support later materialization tk.MustExec("set tidb_opt_enable_late_materialization=0") - err = tiflashcompute.InitGlobalTopoFetcher(tiflashcompute.TestASStr, "tmpAddr", "tmpClusterID", false) + err = tiflashcompute.InitGlobalTopoFetcher(config.TestASStr, "tmpAddr", "tmpClusterID", false) require.NoError(t, err) useASs := []bool{true, false} diff --git a/pkg/executor/update_test.go b/pkg/executor/update_test.go index 425a99394bb99..05b21eddae8ab 100644 --- a/pkg/executor/update_test.go +++ b/pkg/executor/update_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" sessiontypes "github.com/pingcap/tidb/pkg/session/types" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -33,12 +33,12 @@ func TestPessimisticUpdatePKLazyCheck(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - testUpdatePKLazyCheck(t, tk, variable.ClusteredIndexDefModeOn) - testUpdatePKLazyCheck(t, tk, variable.ClusteredIndexDefModeOff) - testUpdatePKLazyCheck(t, tk, variable.ClusteredIndexDefModeIntOnly) + testUpdatePKLazyCheck(t, tk, vardef.ClusteredIndexDefModeOn) + testUpdatePKLazyCheck(t, tk, vardef.ClusteredIndexDefModeOff) + testUpdatePKLazyCheck(t, tk, vardef.ClusteredIndexDefModeIntOnly) } -func testUpdatePKLazyCheck(t *testing.T, tk *testkit.TestKit, clusteredIndex variable.ClusteredIndexDefMode) { +func testUpdatePKLazyCheck(t *testing.T, tk *testkit.TestKit, clusteredIndex vardef.ClusteredIndexDefMode) { tk.Session().GetSessionVars().EnableClusteredIndex = clusteredIndex tk.MustExec(`drop table if exists upk`) tk.MustExec(`create table upk (a int, b int, c int, primary key (a, b))`) diff --git a/pkg/executor/write.go b/pkg/executor/write.go index c7aaa8d9462a5..673294a4fb42e 100644 --- a/pkg/executor/write.go +++ b/pkg/executor/write.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" @@ -456,7 +456,7 @@ func checkRowForExchangePartition(sctx sessionctx.Context, row []types.Datum, tb if err != nil { return err } - if variable.EnableCheckConstraint.Load() { + if vardef.EnableCheckConstraint.Load() { if err = table.CheckRowConstraintWithDatum(evalCtx, pt.WritableConstraint(), row); err != nil { // TODO: make error include ExchangePartition info. return err diff --git a/pkg/expression/BUILD.bazel b/pkg/expression/BUILD.bazel index cffed01908018..56539d7e07372 100644 --- a/pkg/expression/BUILD.bazel +++ b/pkg/expression/BUILD.bazel @@ -96,6 +96,7 @@ go_library( "//pkg/parser/types", "//pkg/planner/cascades/base", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/store/helper", "//pkg/tablecodec", @@ -228,6 +229,7 @@ go_test( "//pkg/planner/core/resolve", "//pkg/session", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/tablecodec", diff --git a/pkg/expression/aggregation/BUILD.bazel b/pkg/expression/aggregation/BUILD.bazel index 6c39f3ffe3bdb..ad49ef655096b 100644 --- a/pkg/expression/aggregation/BUILD.bazel +++ b/pkg/expression/aggregation/BUILD.bazel @@ -77,6 +77,7 @@ go_test( "//pkg/parser/mysql", "//pkg/planner/cascades/base", "//pkg/planner/util", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/testkit/testsetup", "//pkg/types", diff --git a/pkg/expression/aggregation/aggregation_test.go b/pkg/expression/aggregation/aggregation_test.go index b8e9ff20a346d..409d12edfaca1 100644 --- a/pkg/expression/aggregation/aggregation_test.go +++ b/pkg/expression/aggregation/aggregation_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cascades/base" "github.com/pingcap/tidb/pkg/planner/util" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -40,7 +41,7 @@ func createAggFuncSuite() (s *mockAggFuncSuite) { s = new(mockAggFuncSuite) s.ctx = mock.NewContext() s.ctx.GetSessionVars().GlobalVarsAccessor = variable.NewMockGlobalAccessor4Tests() - s.ctx.GetSessionVars().DivPrecisionIncrement = variable.DefDivPrecisionIncrement + s.ctx.GetSessionVars().DivPrecisionIncrement = vardef.DefDivPrecisionIncrement s.rows = make([]chunk.Row, 0, 5050) for i := 1; i <= 100; i++ { for j := 0; j < i; j++ { diff --git a/pkg/expression/bench_test.go b/pkg/expression/bench_test.go index 0d259b558a6db..c4dbab5fba54a 100644 --- a/pkg/expression/bench_test.go +++ b/pkg/expression/bench_test.go @@ -35,7 +35,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/benchdaily" "github.com/pingcap/tidb/pkg/util/chunk" @@ -1541,7 +1541,7 @@ func testVectorizedBuiltinFunc(t *testing.T, vecExprCases vecExprBenchCases) { if testCase.aesModes == "" { testCase.aesModes = "aes-128-ecb" } - err := ctx.GetSessionVars().SetSystemVar(variable.BlockEncryptionMode, testCase.aesModes) + err := ctx.GetSessionVars().SetSystemVar(vardef.BlockEncryptionMode, testCase.aesModes) require.NoError(t, err) if funcName == ast.CurrentUser || funcName == ast.User { ctx.GetSessionVars().User = &auth.UserIdentity{ @@ -1782,7 +1782,7 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases if testCase.aesModes == "" { testCase.aesModes = "aes-128-ecb" } - err := ctx.GetSessionVars().SetSystemVar(variable.BlockEncryptionMode, testCase.aesModes) + err := ctx.GetSessionVars().SetSystemVar(vardef.BlockEncryptionMode, testCase.aesModes) if err != nil { panic(err) } diff --git a/pkg/expression/builtin_encryption.go b/pkg/expression/builtin_encryption.go index cf5861179a4d9..3f0c0b5c12dcb 100644 --- a/pkg/expression/builtin_encryption.go +++ b/pkg/expression/builtin_encryption.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/pkg/expression/expropt" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -1063,7 +1064,7 @@ func (b *builtinValidatePasswordStrengthSig) evalInt(ctx EvalContext, row chunk. } else if len([]rune(str)) < 4 { return 0, false, nil } - if validation, err := globalVars.GetGlobalSysVar(variable.ValidatePasswordEnable); err != nil { + if validation, err := globalVars.GetGlobalSysVar(vardef.ValidatePasswordEnable); err != nil { return 0, true, err } else if !variable.TiDBOptOn(validation) { return 0, false, nil diff --git a/pkg/expression/builtin_encryption_test.go b/pkg/expression/builtin_encryption_test.go index 54d7ffd612c14..f3632aada0a26 100644 --- a/pkg/expression/builtin_encryption_test.go +++ b/pkg/expression/builtin_encryption_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -61,9 +62,9 @@ var cryptTests = []struct { func TestSQLDecode(t *testing.T) { ctx := createContext(t) for _, tt := range cryptTests { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, tt.chs) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, tt.chs) require.NoError(t, err) - err = ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CollationConnection, tt.chs) + err = ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CollationConnection, tt.chs) require.NoError(t, err) f, err := newFunctionForTest(ctx, ast.Decode, primitiveValsToConstants(ctx, []any{tt.origin, tt.password})...) require.NoError(t, err) @@ -80,9 +81,9 @@ func TestSQLDecode(t *testing.T) { func TestSQLEncode(t *testing.T) { ctx := createContext(t) for _, test := range cryptTests { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, test.chs) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, test.chs) require.NoError(t, err) - err = ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CollationConnection, test.chs) + err = ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CollationConnection, test.chs) require.NoError(t, err) var h []byte if test.crypt != nil { @@ -150,7 +151,7 @@ func TestAESEncrypt(t *testing.T) { fc := funcs[ast.AesEncrypt] for _, tt := range aesTests { - err := ctx.GetSessionVars().SetSystemVar(variable.BlockEncryptionMode, tt.mode) + err := ctx.GetSessionVars().SetSystemVar(vardef.BlockEncryptionMode, tt.mode) require.NoError(t, err) args := []types.Datum{types.NewDatum(tt.origin)} for _, param := range tt.params { @@ -162,7 +163,7 @@ func TestAESEncrypt(t *testing.T) { require.NoError(t, err) require.Equal(t, types.NewDatum(tt.crypt), toHex(crypt)) } - err := ctx.GetSessionVars().SetSystemVar(variable.BlockEncryptionMode, "aes-128-ecb") + err := ctx.GetSessionVars().SetSystemVar(vardef.BlockEncryptionMode, "aes-128-ecb") require.NoError(t, err) testNullInput(t, ctx, ast.AesEncrypt) testAmbiguousInput(t, ctx, ast.AesEncrypt) @@ -195,9 +196,9 @@ func TestAESEncrypt(t *testing.T) { for _, tt := range gbkTests { msg := fmt.Sprintf("%v", tt) - err := ctx.GetSessionVars().SetSystemVar(variable.CharacterSetConnection, tt.chs) + err := ctx.GetSessionVars().SetSystemVar(vardef.CharacterSetConnection, tt.chs) require.NoError(t, err, msg) - err = ctx.GetSessionVars().SetSystemVar(variable.BlockEncryptionMode, tt.mode) + err = ctx.GetSessionVars().SetSystemVar(vardef.BlockEncryptionMode, tt.mode) require.NoError(t, err, msg) args := primitiveValsToConstants(ctx, []any{tt.origin}) @@ -217,7 +218,7 @@ func TestAESDecrypt(t *testing.T) { fc := funcs[ast.AesDecrypt] for _, tt := range aesTests { msg := fmt.Sprintf("%v", tt) - err := ctx.GetSessionVars().SetSystemVar(variable.BlockEncryptionMode, tt.mode) + err := ctx.GetSessionVars().SetSystemVar(vardef.BlockEncryptionMode, tt.mode) require.NoError(t, err, msg) args := []types.Datum{fromHex(tt.crypt)} for _, param := range tt.params { @@ -233,7 +234,7 @@ func TestAESDecrypt(t *testing.T) { } require.Equal(t, types.NewCollationStringDatum(tt.origin.(string), charset.CollationBin), str, msg) } - err := ctx.GetSessionVars().SetSystemVar(variable.BlockEncryptionMode, "aes-128-ecb") + err := ctx.GetSessionVars().SetSystemVar(vardef.BlockEncryptionMode, "aes-128-ecb") require.NoError(t, err) testNullInput(t, ctx, ast.AesDecrypt) testAmbiguousInput(t, ctx, ast.AesDecrypt) @@ -267,9 +268,9 @@ func TestAESDecrypt(t *testing.T) { for _, tt := range gbkTests { msg := fmt.Sprintf("%v", tt) - err := ctx.GetSessionVars().SetSystemVar(variable.CharacterSetConnection, tt.chs) + err := ctx.GetSessionVars().SetSystemVar(vardef.CharacterSetConnection, tt.chs) require.NoError(t, err, msg) - err = ctx.GetSessionVars().SetSystemVar(variable.BlockEncryptionMode, tt.mode) + err = ctx.GetSessionVars().SetSystemVar(vardef.BlockEncryptionMode, tt.mode) require.NoError(t, err, msg) // Set charset and collate except first argument args := datumsToConstants([]types.Datum{fromHex(tt.crypt)}) @@ -283,7 +284,7 @@ func TestAESDecrypt(t *testing.T) { } func testNullInput(t *testing.T, ctx *mock.Context, fnName string) { - err := ctx.GetSessionVars().SetSystemVar(variable.BlockEncryptionMode, "aes-128-ecb") + err := ctx.GetSessionVars().SetSystemVar(vardef.BlockEncryptionMode, "aes-128-ecb") require.NoError(t, err) fc := funcs[fnName] arg := types.NewStringDatum("str") @@ -305,7 +306,7 @@ func testAmbiguousInput(t *testing.T, ctx *mock.Context, fnName string) { fc := funcs[fnName] arg := types.NewStringDatum("str") // test for modes that require init_vector - err := ctx.GetSessionVars().SetSystemVar(variable.BlockEncryptionMode, "aes-128-cbc") + err := ctx.GetSessionVars().SetSystemVar(vardef.BlockEncryptionMode, "aes-128-cbc") require.NoError(t, err) _, err = fc.getFunction(ctx, datumsToConstants([]types.Datum{arg, arg})) require.Error(t, err) @@ -315,7 +316,7 @@ func testAmbiguousInput(t *testing.T, ctx *mock.Context, fnName string) { require.Error(t, err) // test for modes that do not require init_vector - err = ctx.GetSessionVars().SetSystemVar(variable.BlockEncryptionMode, "aes-128-ecb") + err = ctx.GetSessionVars().SetSystemVar(vardef.BlockEncryptionMode, "aes-128-ecb") require.NoError(t, err) f, err = fc.getFunction(ctx, datumsToConstants([]types.Datum{arg, arg, arg})) require.NoError(t, err) @@ -366,7 +367,7 @@ func TestSha1Hash(t *testing.T) { fc := funcs[ast.SHA] for _, tt := range sha1Tests { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, tt.chs) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, tt.chs) require.NoError(t, err) f, _ := fc.getFunction(ctx, primitiveValsToConstants(ctx, []any{tt.origin})) crypt, err := evalBuiltinFunc(f, ctx, chunk.Row{}) @@ -437,7 +438,7 @@ func TestSha2Hash(t *testing.T) { fc := funcs[ast.SHA2] for _, tt := range sha2Tests { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, tt.chs) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, tt.chs) require.NoError(t, err) f, err := fc.getFunction(ctx, primitiveValsToConstants(ctx, []any{tt.origin, tt.hashLength})) require.NoError(t, err) @@ -479,7 +480,7 @@ func TestMD5Hash(t *testing.T) { {nil, "", "", true, false}, } for _, c := range cases { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, c.charset) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, c.charset) require.NoError(t, err) f, err := newFunctionForTest(ctx, ast.MD5, primitiveValsToConstants(ctx, []any{c.args})...) require.NoError(t, err) @@ -573,7 +574,7 @@ func TestCompress(t *testing.T) { {"gbk", "你好", string(decodeHex("04000000789C3AF278D76140000000FFFF07F40325"))}, } for _, test := range tests { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, test.chs) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, test.chs) require.NoErrorf(t, err, "%v", test) arg := primitiveValsToConstants(ctx, []any{test.in}) f, err := fc.getFunction(ctx, arg) @@ -658,7 +659,7 @@ func TestValidatePasswordStrength(t *testing.T) { ctx.GetSessionVars().User = &auth.UserIdentity{Username: "testuser"} globalVarsAccessor := variable.NewMockGlobalAccessor4Tests() ctx.GetSessionVars().GlobalVarsAccessor = globalVarsAccessor - err := globalVarsAccessor.SetGlobalSysVar(context.Background(), variable.ValidatePasswordDictionary, "1234") + err := globalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordDictionary, "1234") require.NoError(t, err) tests := []struct { @@ -690,7 +691,7 @@ func TestValidatePasswordStrength(t *testing.T) { } } // enable password validation - err = globalVarsAccessor.SetGlobalSysVar(context.Background(), variable.ValidatePasswordEnable, "ON") + err = globalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordEnable, "ON") require.NoError(t, err) for _, test := range tests { arg := types.NewDatum(test.in) @@ -726,7 +727,7 @@ func TestPassword(t *testing.T) { warnCount := len(ctx.GetSessionVars().StmtCtx.GetWarnings()) for _, c := range cases { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, c.charset) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, c.charset) require.NoError(t, err) f, err := newFunctionForTest(ctx, ast.PasswordFunc, primitiveValsToConstants(ctx, []any{c.args})...) require.NoError(t, err) diff --git a/pkg/expression/builtin_encryption_vec.go b/pkg/expression/builtin_encryption_vec.go index 29b19fe596f37..831235458136c 100644 --- a/pkg/expression/builtin_encryption_vec.go +++ b/pkg/expression/builtin_encryption_vec.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/auth" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -900,7 +901,7 @@ func (b *builtinValidatePasswordStrengthSig) vecEvalInt(ctx EvalContext, input * i64s := result.Int64s() globalVars := vars.GlobalVarsAccessor enableValidation := false - validation, err := globalVars.GetGlobalSysVar(variable.ValidatePasswordEnable) + validation, err := globalVars.GetGlobalSysVar(vardef.ValidatePasswordEnable) if err != nil { return err } diff --git a/pkg/expression/builtin_math_test.go b/pkg/expression/builtin_math_test.go index 2cecf2fe49a27..ec54b446fd97d 100644 --- a/pkg/expression/builtin_math_test.go +++ b/pkg/expression/builtin_math_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit/testutil" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -561,7 +561,7 @@ func TestCRC32(t *testing.T) { {[]any{"一"}, "gbk", 2925846374, false}, } for _, c := range tbl { - err := ctx.GetSessionVars().SetSystemVar(variable.CharacterSetConnection, c.chs) + err := ctx.GetSessionVars().SetSystemVar(vardef.CharacterSetConnection, c.chs) require.NoError(t, err) f, err := newFunctionForTest(ctx, ast.CRC32, primitiveValsToConstants(ctx, c.input)...) require.NoError(t, err) diff --git a/pkg/expression/builtin_miscellaneous.go b/pkg/expression/builtin_miscellaneous.go index 5dce7287b7180..35f61806ae9d9 100644 --- a/pkg/expression/builtin_miscellaneous.go +++ b/pkg/expression/builtin_miscellaneous.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/expression/expropt" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -222,7 +223,7 @@ func (b *builtinLockSig) evalInt(ctx EvalContext, row chunk.Row) (int64, bool, e if lockName == "" || utf8.RuneCountInString(lockName) > 64 { return 0, false, errUserLockWrongName.GenWithStackByArgs(lockName) } - maxTimeout := int64(variable.GetSysVar(variable.InnodbLockWaitTimeout).MaxValue) + maxTimeout := int64(variable.GetSysVar(vardef.InnodbLockWaitTimeout).MaxValue) timeout, isNullTimeout, err := b.args[1].EvalInt(ctx, row) if err != nil { return 0, false, err diff --git a/pkg/expression/builtin_string_test.go b/pkg/expression/builtin_string_test.go index b650475bb3aea..ca2e127b50ac3 100644 --- a/pkg/expression/builtin_string_test.go +++ b/pkg/expression/builtin_string_test.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit/testutil" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -93,7 +93,7 @@ func TestLengthAndOctetLength(t *testing.T) { } for _, lengthMethod := range lengthMethods { for _, c := range tbl { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, c.chs) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, c.chs) require.NoError(t, err) f, err := newFunctionForTest(ctx, lengthMethod, primitiveValsToConstants(ctx, []any{c.input})...) require.NoError(t, err) @@ -154,7 +154,7 @@ func TestASCII(t *testing.T) { } for _, c := range tbl { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, c.chs) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, c.chs) require.NoError(t, err) f, err := newFunctionForTest(ctx, ast.ASCII, primitiveValsToConstants(ctx, []any{c.input})...) require.NoError(t, err) @@ -615,7 +615,7 @@ func TestLower(t *testing.T) { {"àáèéêìíòóùúüāēěīńňōūǎǐǒǔǖǘǚǜⅪⅫ", "", "àáèéêìíòóùúüāēěīńňōūǎǐǒǔǖǘǚǜⅺⅻ"}, } for _, c := range tbl { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, c.chs) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, c.chs) require.NoError(t, err) f, err := newFunctionForTest(ctx, ast.Lower, primitiveValsToConstants(ctx, []any{c.input})...) require.NoError(t, err) @@ -674,7 +674,7 @@ func TestUpper(t *testing.T) { {"àáèéêìíòóùúüāēěīńňōūǎǐǒǔǖǘǚǜⅪⅫ", "", "ÀÁÈÉÊÌÍÒÓÙÚÜĀĒĚĪŃŇŌŪǍǏǑǓǕǗǙǛⅪⅫ"}, } for _, c := range tbl { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, c.chs) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, c.chs) require.NoError(t, err) f, err := newFunctionForTest(ctx, ast.Upper, primitiveValsToConstants(ctx, []any{c.input})...) require.NoError(t, err) @@ -1284,7 +1284,7 @@ func TestHexFunc(t *testing.T) { {"一忒(๑•ㅂ•)و✧", "gbk", "", errno.ErrInvalidCharacterString}, } for _, c := range strCases { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, c.chs) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, c.chs) require.NoError(t, err) f, err := newFunctionForTest(ctx, ast.Hex, primitiveValsToConstants(ctx, []any{c.arg})...) require.NoError(t, err) @@ -1365,7 +1365,7 @@ func TestBitLength(t *testing.T) { } for _, c := range cases { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, c.chs) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, c.chs) require.NoError(t, err) f, err := newFunctionForTest(ctx, ast.BitLength, primitiveValsToConstants(ctx, []any{c.args})...) require.NoError(t, err) @@ -2215,7 +2215,7 @@ func TestOrd(t *testing.T) { {"数据库", 51965, "gbk", false, false}, } for _, c := range cases { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, c.chs) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, c.chs) require.NoError(t, err) f, err := newFunctionForTest(ctx, ast.Ord, primitiveValsToConstants(ctx, []any{c.args})...) require.NoError(t, err) @@ -2432,7 +2432,7 @@ func TestToBase64(t *testing.T) { {"一二三!", "", "5LiA5LqM5LiJIQ=="}, } for _, c := range tbl { - err := ctx.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, c.chs) + err := ctx.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, c.chs) require.NoError(t, err) f, err := newFunctionForTest(ctx, ast.ToBase64, primitiveValsToConstants(ctx, []any{c.input})...) require.NoError(t, err) diff --git a/pkg/expression/exprstatic/BUILD.bazel b/pkg/expression/exprstatic/BUILD.bazel index 6c49cf7573064..b5d2e3789bde2 100644 --- a/pkg/expression/exprstatic/BUILD.bazel +++ b/pkg/expression/exprstatic/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/expression/expropt", "//pkg/parser/charset", "//pkg/parser/mysql", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/types", "//pkg/util/context", @@ -40,6 +41,7 @@ go_test( "//pkg/parser/auth", "//pkg/parser/charset", "//pkg/parser/mysql", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/types", "//pkg/util/context", diff --git a/pkg/expression/exprstatic/evalctx.go b/pkg/expression/exprstatic/evalctx.go index a8c3202cd357a..280ca6cd4383f 100644 --- a/pkg/expression/exprstatic/evalctx.go +++ b/pkg/expression/exprstatic/evalctx.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/expression/exprctx" "github.com/pingcap/tidb/pkg/expression/expropt" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" contextutil "github.com/pingcap/tidb/pkg/util/context" @@ -227,10 +228,10 @@ func NewEvalContext(opt ...EvalCtxOption) *EvalContext { evalCtxState: evalCtxState{ currentTime: &timeOnce{}, sqlMode: defaultSQLMode, - maxAllowedPacket: variable.DefMaxAllowedPacket, - enableRedactLog: variable.DefTiDBRedactLog, - defaultWeekFormatMode: variable.DefDefaultWeekFormat, - divPrecisionIncrement: variable.DefDivPrecisionIncrement, + maxAllowedPacket: vardef.DefMaxAllowedPacket, + enableRedactLog: vardef.DefTiDBRedactLog, + defaultWeekFormatMode: vardef.DefDefaultWeekFormat, + divPrecisionIncrement: vardef.DefDivPrecisionIncrement, }, } @@ -418,19 +419,19 @@ func (ctx *EvalContext) loadSessionVarsInternal( for name, val := range sysVars { name = strings.ToLower(name) switch name { - case variable.TimeZone: + case vardef.TimeZone: opts = append(opts, WithLocation(sessionVars.Location())) - case variable.SQLModeVar: + case vardef.SQLModeVar: opts = append(opts, WithSQLMode(sessionVars.SQLMode)) - case variable.Timestamp: + case vardef.Timestamp: opts = append(opts, WithCurrentTime(ctx.currentTimeFuncFromStringVal(val))) - case variable.MaxAllowedPacket: + case vardef.MaxAllowedPacket: opts = append(opts, WithMaxAllowedPacket(sessionVars.MaxAllowedPacket)) - case variable.TiDBRedactLog: + case vardef.TiDBRedactLog: opts = append(opts, WithEnableRedactLog(sessionVars.EnableRedactLog)) - case variable.DefaultWeekFormat: + case vardef.DefaultWeekFormat: opts = append(opts, WithDefaultWeekFormatMode(val)) - case variable.DivPrecisionIncrement: + case vardef.DivPrecisionIncrement: opts = append(opts, WithDivPrecisionIncrement(sessionVars.DivPrecisionIncrement)) } } @@ -439,7 +440,7 @@ func (ctx *EvalContext) loadSessionVarsInternal( func (ctx *EvalContext) currentTimeFuncFromStringVal(val string) func() (time.Time, error) { return func() (time.Time, error) { - if val == variable.DefTimestamp { + if val == vardef.DefTimestamp { return time.Now(), nil } @@ -460,9 +461,9 @@ func newSessionVarsWithSystemVariables(vars map[string]string) (*variable.Sessio // `charset_connection` and `collation_connection` will overwrite each other. // To make the result more determinate, just set them at last step in order: // `charset_connection` first, then `collation_connection`. - case variable.CharacterSetConnection: + case vardef.CharacterSetConnection: cs = []string{name, val} - case variable.CollationConnection: + case vardef.CollationConnection: col = []string{name, val} default: if err := sessionVars.SetSystemVar(name, val); err != nil { diff --git a/pkg/expression/exprstatic/evalctx_test.go b/pkg/expression/exprstatic/evalctx_test.go index 0a792e4ef3e87..3387def59cc41 100644 --- a/pkg/expression/exprstatic/evalctx_test.go +++ b/pkg/expression/exprstatic/evalctx_test.go @@ -27,6 +27,7 @@ import ( infoschema "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" contextutil "github.com/pingcap/tidb/pkg/util/context" @@ -57,9 +58,9 @@ func checkDefaultStaticEvalCtx(t *testing.T, ctx *EvalContext) { require.Equal(t, types.NewContext(types.StrictFlags, time.UTC, ctx), ctx.TypeCtx()) require.Equal(t, errctx.NewContextWithLevels(errctx.LevelMap{}, ctx), ctx.ErrCtx()) require.Equal(t, "", ctx.CurrentDB()) - require.Equal(t, variable.DefMaxAllowedPacket, ctx.GetMaxAllowedPacket()) - require.Equal(t, variable.DefDefaultWeekFormat, ctx.GetDefaultWeekFormatMode()) - require.Equal(t, variable.DefDivPrecisionIncrement, ctx.GetDivPrecisionIncrement()) + require.Equal(t, vardef.DefMaxAllowedPacket, ctx.GetMaxAllowedPacket()) + require.Equal(t, vardef.DefDefaultWeekFormat, ctx.GetDefaultWeekFormatMode()) + require.Equal(t, vardef.DefDivPrecisionIncrement, ctx.GetDivPrecisionIncrement()) require.Empty(t, ctx.AllParamValues()) require.Equal(t, variable.NewUserVars(), ctx.GetUserVarsReader()) require.True(t, ctx.GetOptionalPropSet().IsEmpty()) @@ -571,7 +572,7 @@ func TestEvalCtxLoadSystemVars(t *testing.T) { field: "$.defaultWeekFormatMode", assert: func(ctx *EvalContext, vars *variable.SessionVars) { require.Equal(t, "5", ctx.GetDefaultWeekFormatMode()) - mode, ok := vars.GetSystemVar(variable.DefaultWeekFormat) + mode, ok := vars.GetSystemVar(vardef.DefaultWeekFormat) require.True(t, ok) require.Equal(t, mode, ctx.GetDefaultWeekFormatMode()) }, @@ -661,7 +662,7 @@ func TestEvalCtxLoadSystemVars(t *testing.T) { // additional check about @@timestamp // setting to `variable.DefTimestamp` should return the current timestamp ctx, err = defaultEvalCtx.LoadSystemVars(map[string]string{ - "timestamp": variable.DefTimestamp, + "timestamp": vardef.DefTimestamp, }) require.NoError(t, err) tm, err := ctx.CurrentTime() diff --git a/pkg/expression/exprstatic/exprctx.go b/pkg/expression/exprstatic/exprctx.go index 4c3e2e3d7b043..60b9bb07a2fd2 100644 --- a/pkg/expression/exprstatic/exprctx.go +++ b/pkg/expression/exprstatic/exprctx.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/expression/exprctx" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" contextutil "github.com/pingcap/tidb/pkg/util/context" "github.com/pingcap/tidb/pkg/util/intest" @@ -157,11 +158,11 @@ func NewExprContext(opts ...ExprCtxOption) *ExprContext { charset: cs.Name, collation: cs.DefaultCollation, defaultCollationForUTF8MB4: mysql.DefaultCollationName, - blockEncryptionMode: variable.DefBlockEncryptionMode, - sysDateIsNow: variable.DefSysdateIsNow, - noopFuncsMode: variable.TiDBOptOnOffWarn(variable.DefTiDBEnableNoopFuncs), + blockEncryptionMode: vardef.DefBlockEncryptionMode, + sysDateIsNow: vardef.DefSysdateIsNow, + noopFuncsMode: variable.TiDBOptOnOffWarn(vardef.DefTiDBEnableNoopFuncs), windowingUseHighPrecision: true, - groupConcatMaxLen: variable.DefGroupConcatMaxLen, + groupConcatMaxLen: vardef.DefGroupConcatMaxLen, }, } for _, opt := range opts { @@ -343,23 +344,23 @@ func (ctx *ExprContext) loadSessionVarsInternal( for name := range sysVars { name = strings.ToLower(name) switch name { - case variable.CharacterSetConnection, variable.CollationConnection: + case vardef.CharacterSetConnection, vardef.CollationConnection: opts = append(opts, WithCharset(sessionVars.GetCharsetInfo())) - case variable.DefaultCollationForUTF8MB4: + case vardef.DefaultCollationForUTF8MB4: opts = append(opts, WithDefaultCollationForUTF8MB4(sessionVars.DefaultCollationForUTF8MB4)) - case variable.BlockEncryptionMode: - blockMode, ok := sessionVars.GetSystemVar(variable.BlockEncryptionMode) + case vardef.BlockEncryptionMode: + blockMode, ok := sessionVars.GetSystemVar(vardef.BlockEncryptionMode) intest.Assert(ok) if ok { opts = append(opts, WithBlockEncryptionMode(blockMode)) } - case variable.TiDBSysdateIsNow: + case vardef.TiDBSysdateIsNow: opts = append(opts, WithSysDateIsNow(sessionVars.SysdateIsNow)) - case variable.TiDBEnableNoopFuncs: + case vardef.TiDBEnableNoopFuncs: opts = append(opts, WithNoopFuncsMode(sessionVars.NoopFuncsMode)) - case variable.WindowingUseHighPrecision: + case vardef.WindowingUseHighPrecision: opts = append(opts, WithWindowingUseHighPrecision(sessionVars.WindowingUseHighPrecision)) - case variable.GroupConcatMaxLen: + case vardef.GroupConcatMaxLen: opts = append(opts, WithGroupConcatMaxLen(sessionVars.GroupConcatMaxLen)) } } diff --git a/pkg/expression/exprstatic/exprctx_test.go b/pkg/expression/exprstatic/exprctx_test.go index 0e1f97c7f8ba0..2950c5af703cd 100644 --- a/pkg/expression/exprstatic/exprctx_test.go +++ b/pkg/expression/exprstatic/exprctx_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/expression/exprctx" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" contextutil "github.com/pingcap/tidb/pkg/util/context" "github.com/pingcap/tidb/pkg/util/deeptest" @@ -68,9 +69,9 @@ func checkDefaultStaticExprCtx(t *testing.T, ctx *ExprContext) { require.Equal(t, charsetName, cs.Name) require.Equal(t, cs.DefaultCollation, collation) require.Equal(t, mysql.DefaultCollationName, ctx.GetDefaultCollationForUTF8MB4()) - require.Equal(t, variable.DefBlockEncryptionMode, ctx.GetBlockEncryptionMode()) - require.Equal(t, variable.DefSysdateIsNow, ctx.GetSysdateIsNow()) - require.Equal(t, variable.TiDBOptOnOffWarn(variable.DefTiDBEnableNoopFuncs), ctx.GetNoopFuncsMode()) + require.Equal(t, vardef.DefBlockEncryptionMode, ctx.GetBlockEncryptionMode()) + require.Equal(t, vardef.DefSysdateIsNow, ctx.GetSysdateIsNow()) + require.Equal(t, variable.TiDBOptOnOffWarn(vardef.DefTiDBEnableNoopFuncs), ctx.GetNoopFuncsMode()) require.NotNil(t, ctx.Rng()) require.True(t, ctx.IsUseCache()) require.NotNil(t, ctx.columnIDAllocator) @@ -78,7 +79,7 @@ func checkDefaultStaticExprCtx(t *testing.T, ctx *ExprContext) { require.True(t, ok) require.Equal(t, uint64(0), ctx.ConnectionID()) require.Equal(t, true, ctx.GetWindowingUseHighPrecision()) - require.Equal(t, variable.DefGroupConcatMaxLen, ctx.GetGroupConcatMaxLen()) + require.Equal(t, vardef.DefGroupConcatMaxLen, ctx.GetGroupConcatMaxLen()) } type exprCtxOptionsTestState struct { @@ -263,7 +264,7 @@ func TestExprCtxLoadSystemVars(t *testing.T) { field: "$.blockEncryptionMode", assert: func(ctx *ExprContext, vars *variable.SessionVars) { require.Equal(t, "aes-256-cbc", ctx.GetBlockEncryptionMode()) - blockMode, _ := vars.GetSystemVar(variable.BlockEncryptionMode) + blockMode, _ := vars.GetSystemVar(vardef.BlockEncryptionMode) require.Equal(t, blockMode, ctx.GetBlockEncryptionMode()) }, }, diff --git a/pkg/expression/integration_test/BUILD.bazel b/pkg/expression/integration_test/BUILD.bazel index e101be7024b28..e293617bc7fc2 100644 --- a/pkg/expression/integration_test/BUILD.bazel +++ b/pkg/expression/integration_test/BUILD.bazel @@ -26,6 +26,7 @@ go_test( "//pkg/planner/core/operator/logicalop", "//pkg/planner/core/resolve", "//pkg/session", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/store/mockstore", "//pkg/tablecodec", diff --git a/pkg/expression/integration_test/integration_test.go b/pkg/expression/integration_test/integration_test.go index 7d120515e56c0..040fc9c306d65 100644 --- a/pkg/expression/integration_test/integration_test.go +++ b/pkg/expression/integration_test/integration_test.go @@ -47,6 +47,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/tablecodec" @@ -1920,8 +1921,8 @@ func TestDecodetoChunkReuse(t *testing.T) { tk.MustExec("set tidb_init_chunk_size = 2") tk.MustExec("set tidb_max_chunk_size = 32") defer func() { - tk.MustExec(fmt.Sprintf("set tidb_init_chunk_size = %d", variable.DefInitChunkSize)) - tk.MustExec(fmt.Sprintf("set tidb_max_chunk_size = %d", variable.DefMaxChunkSize)) + tk.MustExec(fmt.Sprintf("set tidb_init_chunk_size = %d", vardef.DefInitChunkSize)) + tk.MustExec(fmt.Sprintf("set tidb_max_chunk_size = %d", vardef.DefMaxChunkSize)) }() rs, err := tk.Exec("select * from chk") require.NoError(t, err) diff --git a/pkg/expression/sessionexpr/BUILD.bazel b/pkg/expression/sessionexpr/BUILD.bazel index 99f5fb0e0a5dd..a1dbf22fe313c 100644 --- a/pkg/expression/sessionexpr/BUILD.bazel +++ b/pkg/expression/sessionexpr/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//pkg/parser/mysql", "//pkg/privilege", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/types", "//pkg/util", diff --git a/pkg/expression/sessionexpr/sessionctx.go b/pkg/expression/sessionexpr/sessionctx.go index 74162eb60ba08..367527a15cfda 100644 --- a/pkg/expression/sessionexpr/sessionctx.go +++ b/pkg/expression/sessionexpr/sessionctx.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -74,7 +75,7 @@ func (ctx *ExprContext) GetDefaultCollationForUTF8MB4() string { // GetBlockEncryptionMode returns the variable block_encryption_mode func (ctx *ExprContext) GetBlockEncryptionMode() string { - blockMode, _ := ctx.sctx.GetSessionVars().GetSystemVar(variable.BlockEncryptionMode) + blockMode, _ := ctx.sctx.GetSessionVars().GetSystemVar(vardef.BlockEncryptionMode) return blockMode } @@ -262,7 +263,7 @@ func (ctx *EvalContext) GetTiDBRedactLog() string { // GetDefaultWeekFormatMode returns the value of the 'default_week_format' system variable. func (ctx *EvalContext) GetDefaultWeekFormatMode() string { - mode, ok := ctx.sctx.GetSessionVars().GetSystemVar(variable.DefaultWeekFormat) + mode, ok := ctx.sctx.GetSessionVars().GetSystemVar(vardef.DefaultWeekFormat) if !ok || mode == "" { return "0" } diff --git a/pkg/expression/typeinfer_test.go b/pkg/expression/typeinfer_test.go index 6264056f8f2da..60f6f22f22cee 100644 --- a/pkg/expression/typeinfer_test.go +++ b/pkg/expression/typeinfer_test.go @@ -26,7 +26,7 @@ import ( plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/types" @@ -112,8 +112,8 @@ func TestInferType(t *testing.T) { tests = append(tests, s.createTestCase4GetVarFunc()...) sctx := testKit.Session() - require.NoError(t, sctx.GetSessionVars().SetSystemVar(variable.CharacterSetConnection, mysql.DefaultCharset)) - require.NoError(t, sctx.GetSessionVars().SetSystemVar(variable.CollationConnection, mysql.DefaultCollationName)) + require.NoError(t, sctx.GetSessionVars().SetSystemVar(vardef.CharacterSetConnection, mysql.DefaultCharset)) + require.NoError(t, sctx.GetSessionVars().SetSystemVar(vardef.CollationConnection, mysql.DefaultCollationName)) ctx := context.Background() par := parser.New() diff --git a/pkg/extension/BUILD.bazel b/pkg/extension/BUILD.bazel index 84060b5a4f5a6..2697935e10bd3 100644 --- a/pkg/extension/BUILD.bazel +++ b/pkg/extension/BUILD.bazel @@ -54,6 +54,7 @@ go_test( "//pkg/sessionctx", "//pkg/sessionctx/sessionstates", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/testkit", "//pkg/testkit/testsetup", diff --git a/pkg/extension/auth_test.go b/pkg/extension/auth_test.go index 4098eae81923d..dad137f5b6574 100644 --- a/pkg/extension/auth_test.go +++ b/pkg/extension/auth_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/extension" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/mock" @@ -188,10 +189,10 @@ func TestAuthPlugin(t *testing.T) { extension.WithCustomAuthPlugins(authChecks), extension.WithCustomSysVariables([]*variable.SysVar{ { - Scope: variable.ScopeGlobal, + Scope: vardef.ScopeGlobal, Name: "extension_authentication_plugin", Value: mysql.AuthNativePassword, - Type: variable.TypeEnum, + Type: vardef.TypeEnum, PossibleValues: []string{p.Name()}, }, }), @@ -343,10 +344,10 @@ func TestAuthPluginSwitchPlugins(t *testing.T) { extension.WithCustomAuthPlugins(authChecks), extension.WithCustomSysVariables([]*variable.SysVar{ { - Scope: variable.ScopeGlobal, + Scope: vardef.ScopeGlobal, Name: "extension_authentication_plugin", Value: mysql.AuthNativePassword, - Type: variable.TypeEnum, + Type: vardef.TypeEnum, PossibleValues: []string{p.Name()}, }, }), @@ -452,10 +453,10 @@ func TestCreateUserWhenGrant(t *testing.T) { extension.WithCustomAuthPlugins(authChecks), extension.WithCustomSysVariables([]*variable.SysVar{ { - Scope: variable.ScopeGlobal, + Scope: vardef.ScopeGlobal, Name: "extension_authentication_plugin", Value: mysql.AuthNativePassword, - Type: variable.TypeEnum, + Type: vardef.TypeEnum, PossibleValues: []string{p.Name()}, }, }), @@ -526,10 +527,10 @@ func TestCreateViewWithPluginUser(t *testing.T) { extension.WithCustomAuthPlugins(authChecks), extension.WithCustomSysVariables([]*variable.SysVar{ { - Scope: variable.ScopeGlobal, + Scope: vardef.ScopeGlobal, Name: "extension_authentication_plugin", Value: mysql.AuthNativePassword, - Type: variable.TypeEnum, + Type: vardef.TypeEnum, PossibleValues: []string{p.Name()}, }, }), @@ -623,10 +624,10 @@ func TestPluginUserModification(t *testing.T) { extension.WithCustomAuthPlugins(authChecks), extension.WithCustomSysVariables([]*variable.SysVar{ { - Scope: variable.ScopeGlobal, + Scope: vardef.ScopeGlobal, Name: "extension_authentication_plugin", Value: mysql.AuthNativePassword, - Type: variable.TypeEnum, + Type: vardef.TypeEnum, PossibleValues: []string{p.Name()}, }, }), diff --git a/pkg/extension/enterprise b/pkg/extension/enterprise index 1aa9cb6276369..cbfb68248994a 160000 --- a/pkg/extension/enterprise +++ b/pkg/extension/enterprise @@ -1 +1 @@ -Subproject commit 1aa9cb62763697d55921c73a8ffa8d8558ab65df +Subproject commit cbfb68248994ad607ae9e186dbaa73873ed8eee2 diff --git a/pkg/extension/registry_test.go b/pkg/extension/registry_test.go index 32ef846bb2dc1..4a1fb85e367df 100644 --- a/pkg/extension/registry_test.go +++ b/pkg/extension/registry_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/privilege/privileges" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/sem" @@ -133,17 +134,17 @@ func TestRegisterExtensionWithSysVars(t *testing.T) { defer extension.Reset() sysVar1 := &variable.SysVar{ - Scope: variable.ScopeGlobal | variable.ScopeSession, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "var1", - Value: variable.On, - Type: variable.TypeBool, + Value: vardef.On, + Type: vardef.TypeBool, } sysVar2 := &variable.SysVar{ - Scope: variable.ScopeSession, + Scope: vardef.ScopeSession, Name: "var2", Value: "val2", - Type: variable.TypeStr, + Type: vardef.TypeStr, } // normal register @@ -156,7 +157,7 @@ func TestRegisterExtensionWithSysVars(t *testing.T) { // test for empty name extension.Reset() require.NoError(t, extension.Register("test", extension.WithCustomSysVariables([]*variable.SysVar{ - {Scope: variable.ScopeGlobal, Name: "", Value: "val3"}, + {Scope: vardef.ScopeGlobal, Name: "", Value: "val3"}, }))) require.EqualError(t, extension.Setup(), "system var name should not be empty") require.Nil(t, variable.GetSysVar("")) @@ -165,12 +166,12 @@ func TestRegisterExtensionWithSysVars(t *testing.T) { extension.Reset() require.NoError(t, extension.Register("test", extension.WithCustomSysVariables([]*variable.SysVar{ sysVar1, - {Scope: variable.ScopeGlobal, Name: variable.TiDBSnapshot, Value: "val3"}, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBSnapshot, Value: "val3"}, }))) require.EqualError(t, extension.Setup(), "system var 'tidb_snapshot' has already registered") require.Nil(t, variable.GetSysVar("var1")) - require.Equal(t, "", variable.GetSysVar(variable.TiDBSnapshot).Value) - require.Equal(t, variable.ScopeSession, variable.GetSysVar(variable.TiDBSnapshot).Scope) + require.Equal(t, "", variable.GetSysVar(vardef.TiDBSnapshot).Value) + require.Equal(t, vardef.ScopeSession, variable.GetSysVar(vardef.TiDBSnapshot).Scope) // test for duplicate name with other extension extension.Reset() @@ -185,12 +186,12 @@ func TestSetVariablePrivilege(t *testing.T) { defer extension.Reset() sysVar1 := &variable.SysVar{ - Scope: variable.ScopeGlobal | variable.ScopeSession, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "var1", Value: "1", MinValue: 0, MaxValue: 100, - Type: variable.TypeInt, + Type: vardef.TypeInt, RequireDynamicPrivileges: func(isGlobal bool, sem bool) []string { privs := []string{"priv1"} if isGlobal { diff --git a/pkg/infoschema/BUILD.bazel b/pkg/infoschema/BUILD.bazel index 8d9a7e70e82ae..451fa4f0a9892 100644 --- a/pkg/infoschema/BUILD.bazel +++ b/pkg/infoschema/BUILD.bazel @@ -42,6 +42,7 @@ go_library( "//pkg/privilege", "//pkg/session/txninfo", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/store/helper", "//pkg/table", @@ -106,7 +107,7 @@ go_test( "//pkg/meta/model", "//pkg/parser/ast", "//pkg/parser/mysql", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/driver", "//pkg/table", "//pkg/testkit", diff --git a/pkg/infoschema/bench_test.go b/pkg/infoschema/bench_test.go index 85f395a008ac9..56f0db9600f56 100644 --- a/pkg/infoschema/bench_test.go +++ b/pkg/infoschema/bench_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/infoschema/internal" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/driver" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/logutil" @@ -58,7 +58,7 @@ func BenchmarkInfoschemaOverhead(b *testing.B) { }() if *version == 2 { - variable.SchemaCacheSize.Store(1000000) + vardef.SchemaCacheSize.Store(1000000) } tc := &infoschemaTestContext{ diff --git a/pkg/infoschema/builder.go b/pkg/infoschema/builder.go index a1826ff5c41a4..2967d61296370 100644 --- a/pkg/infoschema/builder.go +++ b/pkg/infoschema/builder.go @@ -33,7 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/util/domainutil" @@ -1023,7 +1023,7 @@ func NewBuilder(r autoid.Requirement, factory func() (pools.Resource, error), in infoData: infoData, enableV2: useV2, } - schemaCacheSize := variable.SchemaCacheSize.Load() + schemaCacheSize := vardef.SchemaCacheSize.Load() infoData.tableCache.SetCapacity(schemaCacheSize) return builder } diff --git a/pkg/infoschema/infoschema_test.go b/pkg/infoschema/infoschema_test.go index 89a9c78abea9d..0e41934b1968c 100644 --- a/pkg/infoschema/infoschema_test.go +++ b/pkg/infoschema/infoschema_test.go @@ -33,7 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testutil" @@ -104,7 +104,7 @@ func TestBasic(t *testing.T) { internal.AddDB(t, re.Store(), dbInfo) internal.AddTable(t, re.Store(), dbInfo.ID, tblInfo) - builder := infoschema.NewBuilder(re, nil, infoschema.NewData(), variable.SchemaCacheSize.Load() > 0) + builder := infoschema.NewBuilder(re, nil, infoschema.NewData(), vardef.SchemaCacheSize.Load() > 0) err = builder.InitWithDBInfos(dbInfos, nil, nil, 1) require.NoError(t, err) @@ -334,7 +334,7 @@ func TestInfoTables(t *testing.T) { require.NoError(t, err) }() - builder := infoschema.NewBuilder(re, nil, infoschema.NewData(), variable.SchemaCacheSize.Load() > 0) + builder := infoschema.NewBuilder(re, nil, infoschema.NewData(), vardef.SchemaCacheSize.Load() > 0) err := builder.InitWithDBInfos(nil, nil, nil, 0) require.NoError(t, err) is := builder.Build(math.MaxUint64) @@ -395,7 +395,7 @@ func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) { dbInfo.Deprecated.Tables = []*model.TableInfo{} dbInfos := []*model.DBInfo{dbInfo} data := infoschema.NewData() - builder := infoschema.NewBuilder(re, nil, data, variable.SchemaCacheSize.Load() > 0) + builder := infoschema.NewBuilder(re, nil, data, vardef.SchemaCacheSize.Load() > 0) err := builder.InitWithDBInfos(dbInfos, nil, nil, 1) require.NoError(t, err) is := builder.Build(math.MaxUint64) @@ -416,7 +416,7 @@ func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) { err := kv.RunInNewTxn(ctx, re.Store(), true, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMutator(txn) for _, change := range changes { - builder = infoschema.NewBuilder(re, nil, data, variable.SchemaCacheSize.Load() > 0) + builder = infoschema.NewBuilder(re, nil, data, vardef.SchemaCacheSize.Load() > 0) err := builder.InitWithOldInfoSchema(curIs) require.NoError(t, err) change(m, builder) @@ -500,7 +500,7 @@ func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) { require.NoError(t, err) newDB.Deprecated.Tables = tblInfos require.True(t, ok) - builder = infoschema.NewBuilder(re, nil, data, variable.SchemaCacheSize.Load() > 0) + builder = infoschema.NewBuilder(re, nil, data, vardef.SchemaCacheSize.Load() > 0) err = builder.InitWithDBInfos([]*model.DBInfo{newDB}, newIS.AllPlacementPolicies(), newIS.AllResourceGroups(), newIS.SchemaMetaVersion()) require.NoError(t, err) require.True(t, builder.Build(math.MaxUint64).HasTemporaryTable()) @@ -631,7 +631,7 @@ func TestBuildBundle(t *testing.T) { db.Deprecated.Tables, err = is.SchemaTableInfos(context.Background(), db.Name) require.NoError(t, err) } - builder := infoschema.NewBuilder(dom, nil, infoschema.NewData(), variable.SchemaCacheSize.Load() > 0) + builder := infoschema.NewBuilder(dom, nil, infoschema.NewData(), vardef.SchemaCacheSize.Load() > 0) err = builder.InitWithDBInfos([]*model.DBInfo{db}, is.AllPlacementPolicies(), is.AllResourceGroups(), is.SchemaMetaVersion()) require.NoError(t, err) is2 := builder.Build(math.MaxUint64) @@ -1052,13 +1052,13 @@ func TestEnableInfoSchemaV2(t *testing.T) { // Test the @@tidb_enable_infoschema_v2 variable. tk.MustQuery("select @@tidb_schema_cache_size").Check(testkit.Rows("0")) tk.MustQuery("select @@global.tidb_schema_cache_size").Check(testkit.Rows("0")) - require.Equal(t, variable.SchemaCacheSize.Load(), uint64(0)) + require.Equal(t, vardef.SchemaCacheSize.Load(), uint64(0)) // Modify it. tk.MustExec("set @@global.tidb_schema_cache_size = 1073741824") tk.MustQuery("select @@global.tidb_schema_cache_size").Check(testkit.Rows("1073741824")) tk.MustQuery("select @@tidb_schema_cache_size").Check(testkit.Rows("1073741824")) - require.Equal(t, variable.SchemaCacheSize.Load(), uint64(1073741824)) + require.Equal(t, vardef.SchemaCacheSize.Load(), uint64(1073741824)) tk.MustExec("use test") tk.MustExec("create table v2 (id int)") @@ -1081,7 +1081,7 @@ func TestEnableInfoSchemaV2(t *testing.T) { // Change infoschema back to v1 and check again. tk.MustExec("set @@global.tidb_schema_cache_size = 0") tk.MustQuery("select @@global.tidb_schema_cache_size").Check(testkit.Rows("0")) - require.Equal(t, variable.SchemaCacheSize.Load(), uint64(0)) + require.Equal(t, vardef.SchemaCacheSize.Load(), uint64(0)) tk.MustExec("drop table v1") is = domain.GetDomain(tk.Session()).InfoSchema() @@ -1104,7 +1104,7 @@ func (tc *infoschemaTestContext) createSchema() { internal.AddDB(tc.t, tc.re.Store(), dbInfo) tc.dbInfo = dbInfo // init infoschema - builder := infoschema.NewBuilder(tc.re, nil, tc.data, variable.SchemaCacheSize.Load() > 0) + builder := infoschema.NewBuilder(tc.re, nil, tc.data, vardef.SchemaCacheSize.Load() > 0) err := builder.InitWithDBInfos([]*model.DBInfo{}, nil, nil, 1) require.NoError(tc.t, err) tc.is = builder.Build(math.MaxUint64) @@ -1289,7 +1289,7 @@ func (tc *infoschemaTestContext) applyDiffAndCheck(diff *model.SchemaDiff, check txn, err := tc.re.Store().Begin() require.NoError(tc.t, err) - builder := infoschema.NewBuilder(tc.re, nil, tc.data, variable.SchemaCacheSize.Load() > 0) + builder := infoschema.NewBuilder(tc.re, nil, tc.data, vardef.SchemaCacheSize.Load() > 0) err = builder.InitWithOldInfoSchema(tc.is) require.NoError(tc.t, err) // applyDiff @@ -1314,7 +1314,7 @@ func TestApplyDiff(t *testing.T) { for i := 0; i < 2; i++ { if i == 0 { // enable infoschema v2. - variable.SchemaCacheSize.Store(1000000) + vardef.SchemaCacheSize.Store(1000000) } tc := &infoschemaTestContext{ diff --git a/pkg/infoschema/infoschema_v2_test.go b/pkg/infoschema/infoschema_v2_test.go index 498681536349b..4006b39080537 100644 --- a/pkg/infoschema/infoschema_v2_test.go +++ b/pkg/infoschema/infoschema_v2_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/stretchr/testify/require" ) @@ -168,7 +168,7 @@ func TestMisc(t *testing.T) { r.Store().Close() }() - builder := NewBuilder(r, nil, NewData(), variable.SchemaCacheSize.Load() > 0) + builder := NewBuilder(r, nil, NewData(), vardef.SchemaCacheSize.Load() > 0) err := builder.InitWithDBInfos(nil, nil, nil, 1) require.NoError(t, err) is := builder.Build(math.MaxUint64) @@ -291,7 +291,7 @@ func TestBundles(t *testing.T) { schemaName := ast.NewCIStr("testDB") tableName := ast.NewCIStr("test") - builder := NewBuilder(r, nil, NewData(), variable.SchemaCacheSize.Load() > 0) + builder := NewBuilder(r, nil, NewData(), vardef.SchemaCacheSize.Load() > 0) err := builder.InitWithDBInfos(nil, nil, nil, 1) require.NoError(t, err) is := builder.Build(math.MaxUint64) @@ -412,7 +412,7 @@ func TestReferredFKInfo(t *testing.T) { schemaName := ast.NewCIStr("testDB") tableName := ast.NewCIStr("testTable") - builder := NewBuilder(r, nil, NewData(), variable.SchemaCacheSize.Load() > 0) + builder := NewBuilder(r, nil, NewData(), vardef.SchemaCacheSize.Load() > 0) err := builder.InitWithDBInfos(nil, nil, nil, 1) require.NoError(t, err) is := builder.Build(math.MaxUint64) @@ -518,7 +518,7 @@ func TestSpecialAttributeCorrectnessInSchemaChange(t *testing.T) { schemaName := ast.NewCIStr("testDB") tableName := ast.NewCIStr("testTable") - builder := NewBuilder(r, nil, NewData(), variable.SchemaCacheSize.Load() > 0) + builder := NewBuilder(r, nil, NewData(), vardef.SchemaCacheSize.Load() > 0) err := builder.InitWithDBInfos(nil, nil, nil, 1) require.NoError(t, err) is := builder.Build(math.MaxUint64) @@ -625,7 +625,7 @@ func TestDataStructFieldsCorrectnessInSchemaChange(t *testing.T) { schemaName := ast.NewCIStr("testDB") tableName := ast.NewCIStr("testTable") - builder := NewBuilder(r, nil, NewData(), variable.SchemaCacheSize.Load() > 0) + builder := NewBuilder(r, nil, NewData(), vardef.SchemaCacheSize.Load() > 0) err := builder.InitWithDBInfos(nil, nil, nil, 1) require.NoError(t, err) is := builder.Build(math.MaxUint64) diff --git a/pkg/infoschema/test/infoschemav2test/BUILD.bazel b/pkg/infoschema/test/infoschemav2test/BUILD.bazel index 43768aed9468a..affa10dc74873 100644 --- a/pkg/infoschema/test/infoschemav2test/BUILD.bazel +++ b/pkg/infoschema/test/infoschemav2test/BUILD.bazel @@ -17,7 +17,7 @@ go_test( "//pkg/meta/autoid", "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/table", "//pkg/testkit", "//pkg/testkit/testfailpoint", diff --git a/pkg/infoschema/test/infoschemav2test/v2_test.go b/pkg/infoschema/test/infoschemav2test/v2_test.go index 244a65853df0b..f8ba1c967e371 100644 --- a/pkg/infoschema/test/infoschemav2test/v2_test.go +++ b/pkg/infoschema/test/infoschemav2test/v2_test.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" @@ -250,7 +250,7 @@ func TestTiDBSchemaCacheSizeVariable(t *testing.T) { is := dom.InfoSchema() ok, raw := infoschema.IsV2(is) if ok { - val := variable.SchemaCacheSize.Load() + val := vardef.SchemaCacheSize.Load() tk.MustQuery("select @@global.tidb_schema_cache_size").CheckContain(strconv.FormatUint(val, 10)) // On start, the capacity might not be set correctly because infoschema have not load global variable yet. @@ -260,7 +260,7 @@ func TestTiDBSchemaCacheSizeVariable(t *testing.T) { tk.MustExec("set @@global.tidb_schema_cache_size = 1024 * 1024 * 1024") tk.MustQuery("select @@global.tidb_schema_cache_size").CheckContain("1073741824") - require.Equal(t, variable.SchemaCacheSize.Load(), uint64(1073741824)) + require.Equal(t, vardef.SchemaCacheSize.Load(), uint64(1073741824)) tk.MustExec("create table trigger_reload (id int)") // need to trigger infoschema rebuild to reset capacity is = dom.InfoSchema() ok, raw = infoschema.IsV2(is) diff --git a/pkg/kv/BUILD.bazel b/pkg/kv/BUILD.bazel index 196b70fc3203b..f906a33bb8756 100644 --- a/pkg/kv/BUILD.bazel +++ b/pkg/kv/BUILD.bazel @@ -86,6 +86,7 @@ go_test( "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/testkit/testsetup", "//pkg/testkit/testutil", "//pkg/types", diff --git a/pkg/kv/mpp.go b/pkg/kv/mpp.go index 84b6ac1b8c245..519e8362ec326 100644 --- a/pkg/kv/mpp.go +++ b/pkg/kv/mpp.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/tidb/pkg/util/tiflash" "github.com/pingcap/tidb/pkg/util/tiflashcompute" - "github.com/pingcap/tipb/go-tipb" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" ) @@ -261,56 +260,3 @@ func (req *MPPBuildTasksRequest) ToString() string { } return sb.String() } - -// ExchangeCompressionMode means the compress method used in exchange operator -type ExchangeCompressionMode int - -const ( - // ExchangeCompressionModeNONE indicates no compression - ExchangeCompressionModeNONE ExchangeCompressionMode = iota - // ExchangeCompressionModeFast indicates fast compression/decompression speed, compression ratio is lower than HC mode - ExchangeCompressionModeFast - // ExchangeCompressionModeHC indicates high compression (HC) ratio mode - ExchangeCompressionModeHC - // ExchangeCompressionModeUnspecified indicates unspecified compress method, let TiDB choose one - ExchangeCompressionModeUnspecified - - // RecommendedExchangeCompressionMode indicates recommended compression mode - RecommendedExchangeCompressionMode ExchangeCompressionMode = ExchangeCompressionModeFast - - exchangeCompressionModeUnspecifiedName string = "UNSPECIFIED" -) - -// Name returns the name of ExchangeCompressionMode -func (t ExchangeCompressionMode) Name() string { - if t == ExchangeCompressionModeUnspecified { - return exchangeCompressionModeUnspecifiedName - } - return t.ToTipbCompressionMode().String() -} - -// ToExchangeCompressionMode returns the ExchangeCompressionMode from name -func ToExchangeCompressionMode(name string) (ExchangeCompressionMode, bool) { - name = strings.ToUpper(name) - if name == exchangeCompressionModeUnspecifiedName { - return ExchangeCompressionModeUnspecified, true - } - value, ok := tipb.CompressionMode_value[name] - if ok { - return ExchangeCompressionMode(value), true - } - return ExchangeCompressionModeNONE, false -} - -// ToTipbCompressionMode returns tipb.CompressionMode from kv.ExchangeCompressionMode -func (t ExchangeCompressionMode) ToTipbCompressionMode() tipb.CompressionMode { - switch t { - case ExchangeCompressionModeNONE: - return tipb.CompressionMode_NONE - case ExchangeCompressionModeFast: - return tipb.CompressionMode_FAST - case ExchangeCompressionModeHC: - return tipb.CompressionMode_HIGH_COMPRESSION - } - return tipb.CompressionMode_NONE -} diff --git a/pkg/kv/version_test.go b/pkg/kv/version_test.go index d0e780d421f39..019179424522b 100644 --- a/pkg/kv/version_test.go +++ b/pkg/kv/version_test.go @@ -17,6 +17,7 @@ package kv import ( "testing" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/assert" ) @@ -67,31 +68,31 @@ func TestMppVersion(t *testing.T) { } func TestExchangeCompressionMode(t *testing.T) { - assert.Equal(t, "UNSPECIFIED", ExchangeCompressionModeUnspecified.Name()) + assert.Equal(t, "UNSPECIFIED", vardef.ExchangeCompressionModeUnspecified.Name()) { - a, ok := ToExchangeCompressionMode("UNSPECIFIED") - assert.Equal(t, a, ExchangeCompressionModeUnspecified) + a, ok := vardef.ToExchangeCompressionMode("UNSPECIFIED") + assert.Equal(t, a, vardef.ExchangeCompressionModeUnspecified) assert.True(t, ok) } - assert.Equal(t, "NONE", ExchangeCompressionModeNONE.Name()) + assert.Equal(t, "NONE", vardef.ExchangeCompressionModeNONE.Name()) { - a, ok := ToExchangeCompressionMode("NONE") - assert.Equal(t, a, ExchangeCompressionModeNONE) + a, ok := vardef.ToExchangeCompressionMode("NONE") + assert.Equal(t, a, vardef.ExchangeCompressionModeNONE) assert.True(t, ok) } - assert.Equal(t, "FAST", ExchangeCompressionModeFast.Name()) + assert.Equal(t, "FAST", vardef.ExchangeCompressionModeFast.Name()) { - a, ok := ToExchangeCompressionMode("FAST") - assert.Equal(t, a, ExchangeCompressionModeFast) + a, ok := vardef.ToExchangeCompressionMode("FAST") + assert.Equal(t, a, vardef.ExchangeCompressionModeFast) assert.True(t, ok) } - assert.Equal(t, "HIGH_COMPRESSION", ExchangeCompressionModeHC.Name()) + assert.Equal(t, "HIGH_COMPRESSION", vardef.ExchangeCompressionModeHC.Name()) { - a, ok := ToExchangeCompressionMode("HIGH_COMPRESSION") - assert.Equal(t, a, ExchangeCompressionModeHC) + a, ok := vardef.ToExchangeCompressionMode("HIGH_COMPRESSION") + assert.Equal(t, a, vardef.ExchangeCompressionModeHC) assert.True(t, ok) } // default `FAST` - assert.Equal(t, ExchangeCompressionModeFast, RecommendedExchangeCompressionMode) - assert.Equal(t, tipb.CompressionMode_FAST, RecommendedExchangeCompressionMode.ToTipbCompressionMode()) + assert.Equal(t, vardef.ExchangeCompressionModeFast, vardef.RecommendedExchangeCompressionMode) + assert.Equal(t, tipb.CompressionMode_FAST, vardef.RecommendedExchangeCompressionMode.ToTipbCompressionMode()) } diff --git a/pkg/lightning/backend/kv/BUILD.bazel b/pkg/lightning/backend/kv/BUILD.bazel index 97925831f94f3..e4879145d4390 100644 --- a/pkg/lightning/backend/kv/BUILD.bazel +++ b/pkg/lightning/backend/kv/BUILD.bazel @@ -81,6 +81,7 @@ go_test( "//pkg/parser/mysql", "//pkg/planner/core", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/table", "//pkg/table/tables", diff --git a/pkg/lightning/backend/kv/context_test.go b/pkg/lightning/backend/kv/context_test.go index 3e0cb81f0d3ca..6da5fc6f1a5f9 100644 --- a/pkg/lightning/backend/kv/context_test.go +++ b/pkg/lightning/backend/kv/context_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table/tblctx" "github.com/pingcap/tidb/pkg/types" @@ -154,11 +155,11 @@ func TestLitExprContext(t *testing.T) { // system vars timeZone := "SYSTEM" - expectedMaxAllowedPacket := variable.DefMaxAllowedPacket - expectedDivPrecisionInc := variable.DefDivPrecisionIncrement - expectedDefaultWeekFormat := variable.DefDefaultWeekFormat - expectedBlockEncryptionMode := variable.DefBlockEncryptionMode - expectedGroupConcatMaxLen := variable.DefGroupConcatMaxLen + expectedMaxAllowedPacket := vardef.DefMaxAllowedPacket + expectedDivPrecisionInc := vardef.DefDivPrecisionIncrement + expectedDefaultWeekFormat := vardef.DefDefaultWeekFormat + expectedBlockEncryptionMode := vardef.DefBlockEncryptionMode + expectedGroupConcatMaxLen := vardef.DefGroupConcatMaxLen for k, v := range c.sysVars { switch strings.ToLower(k) { case "time_zone": @@ -260,7 +261,7 @@ func TestLitTableMutateContext(t *testing.T) { require.NoError(t, err) checkCommon(t, tblCtx) require.Equal(t, variable.AssertionLevelOff, tblCtx.TxnAssertionLevel()) - require.Equal(t, variable.DefTiDBEnableMutationChecker, tblCtx.EnableMutationChecker()) + require.Equal(t, vardef.DefTiDBEnableMutationChecker, tblCtx.EnableMutationChecker()) require.False(t, tblCtx.EnableMutationChecker()) require.Equal(t, tblctx.RowEncodingConfig{ IsRowLevelChecksumEnabled: false, @@ -268,7 +269,7 @@ func TestLitTableMutateContext(t *testing.T) { }, tblCtx.GetRowEncodingConfig()) g := tblCtx.GetRowIDShardGenerator() require.NotNil(t, g) - require.Equal(t, variable.DefTiDBShardAllocateStep, g.GetShardStep()) + require.Equal(t, vardef.DefTiDBShardAllocateStep, g.GetShardStep()) // test for load vars sysVars := map[string]string{ @@ -288,13 +289,13 @@ func TestLitTableMutateContext(t *testing.T) { }, tblCtx.GetRowEncodingConfig()) g = tblCtx.GetRowIDShardGenerator() require.NotNil(t, g) - require.NotEqual(t, variable.DefTiDBShardAllocateStep, g.GetShardStep()) + require.NotEqual(t, vardef.DefTiDBShardAllocateStep, g.GetShardStep()) require.Equal(t, 1234567, g.GetShardStep()) // test for `RowEncodingConfig.IsRowLevelChecksumEnabled` which should be loaded from global variable. - require.False(t, variable.EnableRowLevelChecksum.Load()) - defer variable.EnableRowLevelChecksum.Store(false) - variable.EnableRowLevelChecksum.Store(true) + require.False(t, vardef.EnableRowLevelChecksum.Load()) + defer vardef.EnableRowLevelChecksum.Store(false) + vardef.EnableRowLevelChecksum.Store(true) sysVars = map[string]string{ "tidb_row_format_version": "2", } diff --git a/pkg/lightning/backend/kv/kv2sql_test.go b/pkg/lightning/backend/kv/kv2sql_test.go index afe076192f0be..c0ed03001ef0a 100644 --- a/pkg/lightning/backend/kv/kv2sql_test.go +++ b/pkg/lightning/backend/kv/kv2sql_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/mock" @@ -37,7 +37,7 @@ func TestIterRawIndexKeysClusteredPK(t *testing.T) { node, _, err := p.ParseSQL("create table t (a varchar(10) primary key, b int, index idx(b));") require.NoError(t, err) mockSctx := mock.NewContext() - mockSctx.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + mockSctx.GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn info, err := ddl.MockTableInfo(mockSctx, node[0].(*ast.CreateTableStmt), 1) require.NoError(t, err) info.State = model.StatePublic @@ -78,7 +78,7 @@ func TestIterRawIndexKeysIntPK(t *testing.T) { node, _, err := p.ParseSQL("create table t (a int primary key, b int, index idx(b));") require.NoError(t, err) mockSctx := mock.NewContext() - mockSctx.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + mockSctx.GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn info, err := ddl.MockTableInfo(mockSctx, node[0].(*ast.CreateTableStmt), 1) require.NoError(t, err) info.State = model.StatePublic diff --git a/pkg/lightning/backend/local/BUILD.bazel b/pkg/lightning/backend/local/BUILD.bazel index dc2e5a19518df..2c728bdea153a 100644 --- a/pkg/lightning/backend/local/BUILD.bazel +++ b/pkg/lightning/backend/local/BUILD.bazel @@ -51,7 +51,7 @@ go_library( "//pkg/metrics", "//pkg/parser/mysql", "//pkg/parser/terror", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/table", "//pkg/table/tables", "//pkg/tablecodec", diff --git a/pkg/lightning/backend/local/checksum.go b/pkg/lightning/backend/local/checksum.go index 967e32d37ea46..8a82e8ae5e8b2 100644 --- a/pkg/lightning/backend/local/checksum.go +++ b/pkg/lightning/backend/local/checksum.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/log" "github.com/pingcap/tidb/pkg/lightning/metric" "github.com/pingcap/tidb/pkg/lightning/verification" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tipb/go-tipb" tikvstore "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/oracle" @@ -135,11 +135,11 @@ func (e *tidbChecksumExecutor) Checksum(ctx context.Context, tableInfo *checkpoi if err == nil && backoffWeight < DefaultBackoffWeight { task.Info("increase tidb_backoff_weight", zap.Int("original", backoffWeight), zap.Int("new", DefaultBackoffWeight)) // increase backoff weight - if _, err := conn.ExecContext(ctx, fmt.Sprintf("SET SESSION %s = '%d';", variable.TiDBBackOffWeight, DefaultBackoffWeight)); err != nil { + if _, err := conn.ExecContext(ctx, fmt.Sprintf("SET SESSION %s = '%d';", vardef.TiDBBackOffWeight, DefaultBackoffWeight)); err != nil { task.Warn("set tidb_backoff_weight failed", zap.Error(err)) } else { defer func() { - if _, err := conn.ExecContext(ctx, fmt.Sprintf("SET SESSION %s = '%d';", variable.TiDBBackOffWeight, backoffWeight)); err != nil { + if _, err := conn.ExecContext(ctx, fmt.Sprintf("SET SESSION %s = '%d';", vardef.TiDBBackOffWeight, backoffWeight)); err != nil { task.Warn("recover tidb_backoff_weight failed", zap.Error(err)) } }() diff --git a/pkg/lightning/common/BUILD.bazel b/pkg/lightning/common/BUILD.bazel index a690ff7ac5591..e9e142c5418f2 100644 --- a/pkg/lightning/common/BUILD.bazel +++ b/pkg/lightning/common/BUILD.bazel @@ -31,7 +31,7 @@ go_library( "//pkg/meta/autoid", "//pkg/meta/model", "//pkg/parser/mysql", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/driver/error", "//pkg/table/tables", "//pkg/types", diff --git a/pkg/lightning/common/util.go b/pkg/lightning/common/util.go index 7017e79a646c2..de97310466a22 100644 --- a/pkg/lightning/common/util.go +++ b/pkg/lightning/common/util.go @@ -39,7 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/log" "github.com/pingcap/tidb/pkg/meta/model" tmysql "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/codec" @@ -626,7 +626,7 @@ func IsDupKeyError(err error) bool { // GetBackoffWeightFromDB gets the backoff weight from database. func GetBackoffWeightFromDB(ctx context.Context, db *sql.DB) (int, error) { - val, err := getSessionVariable(ctx, db, variable.TiDBBackOffWeight) + val, err := getSessionVariable(ctx, db, vardef.TiDBBackOffWeight) if err != nil { return 0, err } @@ -635,7 +635,7 @@ func GetBackoffWeightFromDB(ctx context.Context, db *sql.DB) (int, error) { // GetExplicitRequestSourceTypeFromDB gets the explicit request source type from database. func GetExplicitRequestSourceTypeFromDB(ctx context.Context, db *sql.DB) (string, error) { - return getSessionVariable(ctx, db, variable.TiDBExplicitRequestSourceType) + return getSessionVariable(ctx, db, vardef.TiDBExplicitRequestSourceType) } // copy from dbutil to avoid import cycle diff --git a/pkg/lightning/errormanager/BUILD.bazel b/pkg/lightning/errormanager/BUILD.bazel index f6f4234007c34..cc1a6d245211a 100644 --- a/pkg/lightning/errormanager/BUILD.bazel +++ b/pkg/lightning/errormanager/BUILD.bazel @@ -51,7 +51,7 @@ go_test( "//pkg/parser", "//pkg/parser/ast", "//pkg/parser/mysql", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/table/tables", "//pkg/types", "//pkg/util", diff --git a/pkg/lightning/errormanager/resolveconflict_test.go b/pkg/lightning/errormanager/resolveconflict_test.go index 00dca3d9cfe1e..3f9138943f8a2 100644 --- a/pkg/lightning/errormanager/resolveconflict_test.go +++ b/pkg/lightning/errormanager/resolveconflict_test.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -47,7 +47,7 @@ func TestReplaceConflictMultipleKeysNonclusteredPk(t *testing.T) { node, _, err := p.ParseSQL("create table a (a int primary key nonclustered, b int not null, c int not null, d text, key key_b(b), key key_c(c));") require.NoError(t, err) mockSctx := mock.NewContext() - mockSctx.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOff + mockSctx.GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOff info, err := ddl.MockTableInfo(mockSctx, node[0].(*ast.CreateTableStmt), 108) require.NoError(t, err) info.State = model.StatePublic @@ -268,7 +268,7 @@ func TestReplaceConflictOneKeyNonclusteredPk(t *testing.T) { node, _, err := p.ParseSQL("create table a (a int primary key nonclustered, b int not null, c text, key key_b(b));") require.NoError(t, err) mockSctx := mock.NewContext() - mockSctx.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOff + mockSctx.GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOff info, err := ddl.MockTableInfo(mockSctx, node[0].(*ast.CreateTableStmt), 108) require.NoError(t, err) info.State = model.StatePublic @@ -436,7 +436,7 @@ func TestReplaceConflictOneUniqueKeyNonclusteredPk(t *testing.T) { node, _, err := p.ParseSQL("create table a (a int primary key nonclustered, b int not null, c text, unique key uni_b(b));") require.NoError(t, err) mockSctx := mock.NewContext() - mockSctx.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOff + mockSctx.GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOff info, err := ddl.MockTableInfo(mockSctx, node[0].(*ast.CreateTableStmt), 108) require.NoError(t, err) info.State = model.StatePublic @@ -642,7 +642,7 @@ func TestReplaceConflictOneUniqueKeyNonclusteredVarcharPk(t *testing.T) { node, _, err := p.ParseSQL("create table a (a varchar(20) primary key nonclustered, b int not null, c text, unique key uni_b(b));") require.NoError(t, err) mockSctx := mock.NewContext() - mockSctx.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOff + mockSctx.GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOff info, err := ddl.MockTableInfo(mockSctx, node[0].(*ast.CreateTableStmt), 108) require.NoError(t, err) info.State = model.StatePublic diff --git a/pkg/meta/metabuild/BUILD.bazel b/pkg/meta/metabuild/BUILD.bazel index fde57696100c8..5bcfe6063de25 100644 --- a/pkg/meta/metabuild/BUILD.bazel +++ b/pkg/meta/metabuild/BUILD.bazel @@ -10,7 +10,7 @@ go_library( "//pkg/expression/exprstatic", "//pkg/infoschema/context", "//pkg/parser/mysql", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/util/intest", ], ) @@ -28,6 +28,7 @@ go_test( "//pkg/infoschema/context", "//pkg/parser/charset", "//pkg/parser/mysql", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/util/deeptest", "@com_github_stretchr_testify//require", diff --git a/pkg/meta/metabuild/context.go b/pkg/meta/metabuild/context.go index 972b32d1170cc..a0b98e9950be2 100644 --- a/pkg/meta/metabuild/context.go +++ b/pkg/meta/metabuild/context.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/tidb/pkg/expression/exprstatic" infoschemactx "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util/intest" ) @@ -64,7 +64,7 @@ func WithPrimaryKeyRequired(required bool) Option { } // WithClusteredIndexDefMode sets the clustered index mode. -func WithClusteredIndexDefMode(mode variable.ClusteredIndexDefMode) Option { +func WithClusteredIndexDefMode(mode vardef.ClusteredIndexDefMode) Option { return funcOpt(func(ctx *Context) { ctx.clusteredIndexDefMode = mode }) @@ -103,7 +103,7 @@ type Context struct { exprCtx exprctx.ExprContext enableAutoIncrementInGenerated bool primaryKeyRequired bool - clusteredIndexDefMode variable.ClusteredIndexDefMode + clusteredIndexDefMode vardef.ClusteredIndexDefMode shardRowIDBits uint64 preSplitRegions uint64 suppressTooLongIndexErr bool @@ -113,11 +113,11 @@ type Context struct { // NewContext creates a new context for meta-building. func NewContext(opts ...Option) *Context { ctx := &Context{ - enableAutoIncrementInGenerated: variable.DefTiDBEnableAutoIncrementInGenerated, + enableAutoIncrementInGenerated: vardef.DefTiDBEnableAutoIncrementInGenerated, primaryKeyRequired: false, - clusteredIndexDefMode: variable.DefTiDBEnableClusteredIndex, - shardRowIDBits: variable.DefShardRowIDBits, - preSplitRegions: variable.DefPreSplitRegions, + clusteredIndexDefMode: vardef.DefTiDBEnableClusteredIndex, + shardRowIDBits: vardef.DefShardRowIDBits, + preSplitRegions: vardef.DefPreSplitRegions, suppressTooLongIndexErr: false, } @@ -179,7 +179,7 @@ func (ctx *Context) PrimaryKeyRequired() bool { } // GetClusteredIndexDefMode returns the clustered index mode. -func (ctx *Context) GetClusteredIndexDefMode() variable.ClusteredIndexDefMode { +func (ctx *Context) GetClusteredIndexDefMode() vardef.ClusteredIndexDefMode { return ctx.clusteredIndexDefMode } diff --git a/pkg/meta/metabuild/context_test.go b/pkg/meta/metabuild/context_test.go index 99a20961fe3a8..bcb95d9699cab 100644 --- a/pkg/meta/metabuild/context_test.go +++ b/pkg/meta/metabuild/context_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/metabuild" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util/deeptest" "github.com/stretchr/testify/require" @@ -90,9 +91,9 @@ func TestMetaBuildContext(t *testing.T) { }, checkDefault: defVars.EnableClusteredIndex, option: func(val any) metabuild.Option { - return metabuild.WithClusteredIndexDefMode(val.(variable.ClusteredIndexDefMode)) + return metabuild.WithClusteredIndexDefMode(val.(vardef.ClusteredIndexDefMode)) }, - testVals: []any{variable.ClusteredIndexDefModeOn, variable.ClusteredIndexDefModeOff}, + testVals: []any{vardef.ClusteredIndexDefModeOn, vardef.ClusteredIndexDefModeOff}, }, { name: "shardRowIDBits", diff --git a/pkg/meta/model/BUILD.bazel b/pkg/meta/model/BUILD.bazel index eae2284e24980..3467777891f08 100644 --- a/pkg/meta/model/BUILD.bazel +++ b/pkg/meta/model/BUILD.bazel @@ -26,7 +26,9 @@ go_library( "//pkg/parser/terror", "//pkg/parser/types", "//pkg/planner/cascades/base", + "//pkg/sessionctx/vardef", "//pkg/util/intest", + "//pkg/util/tracing", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", "@com_github_tikv_pd_client//http", diff --git a/pkg/meta/model/job.go b/pkg/meta/model/job.go index 1b0919fe2c01b..b50bdc2846bf2 100644 --- a/pkg/meta/model/job.go +++ b/pkg/meta/model/job.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/util/intest" + "github.com/pingcap/tidb/pkg/util/tracing" ) // ActionType is the type for DDL action. @@ -366,7 +367,7 @@ type Job struct { AdminOperator AdminCommandOperator `json:"admin_operator"` // TraceInfo indicates the information for SQL tracing - TraceInfo *TraceInfo `json:"trace_info"` + TraceInfo *tracing.TraceInfo `json:"trace_info"` // BDRRole indicates the role of BDR cluster when executing this DDL. BDRRole string `json:"bdr_role"` @@ -1185,14 +1186,6 @@ func (tz *TimeZoneLocation) GetLocation() (*time.Location, error) { return tz.location, err } -// TraceInfo is the information for trace. -type TraceInfo struct { - // ConnectionID is the id of the connection - ConnectionID uint64 `json:"connection_id"` - // SessionAlias is the alias of session - SessionAlias string `json:"session_alias"` -} - // JobW is a wrapper of model.Job, it contains the job and the binary representation // of the job. type JobW struct { diff --git a/pkg/meta/model/reorg.go b/pkg/meta/model/reorg.go index 723a3fda3a9b7..a0b2d0b6f810f 100644 --- a/pkg/meta/model/reorg.go +++ b/pkg/meta/model/reorg.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "go.uber.org/atomic" ) @@ -80,12 +81,12 @@ type DDLReorgMeta struct { MaxWriteSpeed atomic.Int64 `json:"max_write_speed"` } -// GetConcurrencyOrDefault gets the concurrency from DDLReorgMeta, -// pass the default value in case of the reorg meta coming from old cluster and Concurrency is 0. -func (dm *DDLReorgMeta) GetConcurrencyOrDefault(defaultVal int) int { +// GetConcurrency gets the concurrency from DDLReorgMeta. +func (dm *DDLReorgMeta) GetConcurrency() int { concurrency := dm.Concurrency.Load() if concurrency == 0 { - return defaultVal + // when the job coming from old cluster, concurrency might not set + return int(vardef.GetDDLReorgWorkerCounter()) } return int(concurrency) } @@ -95,11 +96,12 @@ func (dm *DDLReorgMeta) SetConcurrency(concurrency int) { dm.Concurrency.Store(int64(concurrency)) } -// GetBatchSizeOrDefault gets the batch size from DDLReorgMeta. -func (dm *DDLReorgMeta) GetBatchSizeOrDefault(defaultVal int) int { +// GetBatchSize gets the batch size from DDLReorgMeta. +func (dm *DDLReorgMeta) GetBatchSize() int { batchSize := dm.BatchSize.Load() if batchSize == 0 { - return defaultVal + // when the job coming from old cluster, batch-size might not set + return int(vardef.GetDDLReorgBatchSize()) } return int(batchSize) } @@ -109,9 +111,10 @@ func (dm *DDLReorgMeta) SetBatchSize(batchSize int) { dm.BatchSize.Store(int64(batchSize)) } -// GetMaxWriteSpeedOrDefault gets the max write speed from DDLReorgMeta. +// GetMaxWriteSpeed gets the max write speed from DDLReorgMeta. // 0 means no limit. -func (dm *DDLReorgMeta) GetMaxWriteSpeedOrDefault() int { +func (dm *DDLReorgMeta) GetMaxWriteSpeed() int { + // 0 means no limit, so it's ok even when the job coming from old cluster return int(dm.MaxWriteSpeed.Load()) } diff --git a/pkg/planner/BUILD.bazel b/pkg/planner/BUILD.bazel index 46c1f4afae915..0ad1957973375 100644 --- a/pkg/planner/BUILD.bazel +++ b/pkg/planner/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "//pkg/planner/util/optimizetrace", "//pkg/privilege", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/types", "//pkg/util/dbterror/plannererrors", diff --git a/pkg/planner/cardinality/BUILD.bazel b/pkg/planner/cardinality/BUILD.bazel index c0835161ce35c..f5146ccf9d70e 100644 --- a/pkg/planner/cardinality/BUILD.bazel +++ b/pkg/planner/cardinality/BUILD.bazel @@ -78,7 +78,7 @@ go_test( "//pkg/session", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/statistics", "//pkg/statistics/handle/ddl/testutil", "//pkg/testkit", diff --git a/pkg/planner/cardinality/selectivity_test.go b/pkg/planner/cardinality/selectivity_test.go index f73ffa99baaf2..ae3f2055e64f6 100644 --- a/pkg/planner/cardinality/selectivity_test.go +++ b/pkg/planner/cardinality/selectivity_test.go @@ -39,7 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" "github.com/pingcap/tidb/pkg/testkit" @@ -453,7 +453,7 @@ func TestUniqCompEqualEst(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") - testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + testKit.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn testKit.MustExec("drop table if exists t") testKit.MustExec("create table t(a int, b int, primary key(a, b))") testKit.MustExec("insert into t values(1,1),(1,2),(1,3),(1,4),(1,5),(1,6),(1,7),(1,8),(1,9),(1,10)") diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index 076202f437fa6..cc0ce5bdf26eb 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -144,6 +144,7 @@ go_library( "//pkg/privilege", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/sessiontxn/staleread", @@ -293,6 +294,7 @@ go_test( "//pkg/planner/util/optimizetrace", "//pkg/session", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/statistics", diff --git a/pkg/planner/core/casetest/cbotest/BUILD.bazel b/pkg/planner/core/casetest/cbotest/BUILD.bazel index c9f523e77c84e..cccadcf68a576 100644 --- a/pkg/planner/core/casetest/cbotest/BUILD.bazel +++ b/pkg/planner/core/casetest/cbotest/BUILD.bazel @@ -20,7 +20,7 @@ go_test( "//pkg/planner/core", "//pkg/planner/core/resolve", "//pkg/session", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/statistics", "//pkg/statistics/handle/ddl/testutil", "//pkg/statistics/util", diff --git a/pkg/planner/core/casetest/cbotest/cbo_test.go b/pkg/planner/core/casetest/cbotest/cbo_test.go index 5f9b6baa847e7..c49e1b6da243f 100644 --- a/pkg/planner/core/casetest/cbotest/cbo_test.go +++ b/pkg/planner/core/casetest/cbotest/cbo_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" "github.com/pingcap/tidb/pkg/statistics/util" @@ -596,7 +596,7 @@ func TestIndexEqualUnknown(t *testing.T) { testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t, t1") - testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + testKit.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly testKit.MustExec("CREATE TABLE t(a bigint(20) NOT NULL, b bigint(20) NOT NULL, c bigint(20) NOT NULL, PRIMARY KEY (a,c,b), KEY (b))") require.NoError(t, loadTableStats("analyzeSuiteTestIndexEqualUnknownT.json", dom)) var input []string diff --git a/pkg/planner/core/casetest/partition/BUILD.bazel b/pkg/planner/core/casetest/partition/BUILD.bazel index 571ae2469b6d0..ef4fe48131acb 100644 --- a/pkg/planner/core/casetest/partition/BUILD.bazel +++ b/pkg/planner/core/casetest/partition/BUILD.bazel @@ -14,7 +14,7 @@ go_test( deps = [ "//pkg/config", "//pkg/planner/util/coretestsdk", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/testkit/testdata", "//pkg/testkit/testmain", diff --git a/pkg/planner/core/casetest/partition/partition_pruner_test.go b/pkg/planner/core/casetest/partition/partition_pruner_test.go index 1d6814b7694e2..e34bb0ea8a017 100644 --- a/pkg/planner/core/casetest/partition/partition_pruner_test.go +++ b/pkg/planner/core/casetest/partition/partition_pruner_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/planner/util/coretestsdk" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/stretchr/testify/require" @@ -38,7 +38,7 @@ func TestHashPartitionPruner(t *testing.T) { tk.MustExec("create database test_partition") tk.MustExec("use test_partition") tk.MustExec("drop table if exists t1, t2;") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly tk.MustExec("create table t2(id int, a int, b int, primary key(id, a)) partition by hash(id + a) partitions 10;") tk.MustExec("create table t1(id int primary key, a int, b int) partition by hash(id) partitions 10;") tk.MustExec("create table t3(id int, a int, b int, primary key(id, a)) partition by hash(id) partitions 10;") diff --git a/pkg/planner/core/explain.go b/pkg/planner/core/explain.go index 0ebbfe2fcf73e..9b4703691a8e8 100644 --- a/pkg/planner/core/explain.go +++ b/pkg/planner/core/explain.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/logutil" @@ -404,7 +404,7 @@ func (p *PhysicalSelection) ExplainInfo() string { // ExplainNormalizedInfo implements Plan interface. func (p *PhysicalSelection) ExplainNormalizedInfo() string { - if variable.IgnoreInlistPlanDigest.Load() { + if vardef.IgnoreInlistPlanDigest.Load() { return string(expression.SortedExplainExpressionListIgnoreInlist(p.Conditions)) } return string(expression.SortedExplainNormalizedExpressionList(p.Conditions)) @@ -449,7 +449,7 @@ func (p *PhysicalExpand) explainInfoV2() string { // ExplainNormalizedInfo implements Plan interface. func (p *PhysicalProjection) ExplainNormalizedInfo() string { - if variable.IgnoreInlistPlanDigest.Load() { + if vardef.IgnoreInlistPlanDigest.Load() { return string(expression.SortedExplainExpressionListIgnoreInlist(p.Exprs)) } return string(expression.SortedExplainNormalizedExpressionList(p.Exprs)) @@ -951,7 +951,7 @@ func (p *PhysicalExchangeSender) ExplainInfo() string { case tipb.ExchangeType_Hash: fmt.Fprintf(buffer, "HashPartition") } - if p.CompressionMode != kv.ExchangeCompressionModeNONE { + if p.CompressionMode != vardef.ExchangeCompressionModeNONE { fmt.Fprintf(buffer, ", Compression: %s", p.CompressionMode.Name()) } if p.ExchangeType == tipb.ExchangeType_Hash { diff --git a/pkg/planner/core/expression_rewriter.go b/pkg/planner/core/expression_rewriter.go index 0bcb5fa62ee67..bf299568b1610 100644 --- a/pkg/planner/core/expression_rewriter.go +++ b/pkg/planner/core/expression_rewriter.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/core/rule" "github.com/pingcap/tidb/pkg/planner/util/coreusage" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" @@ -1718,7 +1719,7 @@ func (er *expressionRewriter) rewriteSystemVariable(planCtx *exprRewriterPlanCtx } return } - if sysVar.IsNoop && !variable.EnableNoopVariables.Load() { + if sysVar.IsNoop && !vardef.EnableNoopVariables.Load() { // The variable does nothing, append a warning to the statement output. sessionVars.StmtCtx.AppendWarning(plannererrors.ErrGettingNoopVariable.FastGenByArgs(sysVar.Name)) } @@ -1753,9 +1754,9 @@ func (er *expressionRewriter) rewriteSystemVariable(planCtx *exprRewriterPlanCtx e := expression.DatumToConstant(nativeVal, nativeType, nativeFlag) switch nativeType { case mysql.TypeVarString: - charset, _ := sessionVars.GetSystemVar(variable.CharacterSetConnection) + charset, _ := sessionVars.GetSystemVar(vardef.CharacterSetConnection) e.GetType(er.sctx.GetEvalCtx()).SetCharset(charset) - collate, _ := sessionVars.GetSystemVar(variable.CollationConnection) + collate, _ := sessionVars.GetSystemVar(vardef.CollationConnection) e.GetType(er.sctx.GetEvalCtx()).SetCollate(collate) case mysql.TypeLong, mysql.TypeLonglong: e.GetType(er.sctx.GetEvalCtx()).SetCharset(charset.CharsetBin) diff --git a/pkg/planner/core/integration_test.go b/pkg/planner/core/integration_test.go index 4528bff107fc9..b21e8f5d14294 100644 --- a/pkg/planner/core/integration_test.go +++ b/pkg/planner/core/integration_test.go @@ -34,7 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/testkit" @@ -1035,7 +1035,7 @@ func TestTiFlashFineGrainedShuffleWithMaxTiFlashThreads(t *testing.T) { rows = tk.MustQuery(sql).Rows() streamCount = getStreamCountFromExplain(rows) // require.Equal(t, len(streamCount), 1) - require.Equal(t, uint64(variable.DefStreamCountWhenMaxThreadsNotSet), streamCount[0]) + require.Equal(t, uint64(vardef.DefStreamCountWhenMaxThreadsNotSet), streamCount[0]) // tiflash_fine_grained_shuffle_stream_count should be default value when tidb_max_tiflash_threads is 0. tk.MustExec("set @@tiflash_fine_grained_shuffle_stream_count = 0") @@ -1043,7 +1043,7 @@ func TestTiFlashFineGrainedShuffleWithMaxTiFlashThreads(t *testing.T) { rows = tk.MustQuery(sql).Rows() streamCount = getStreamCountFromExplain(rows) // require.Equal(t, len(streamCount), 1) - require.Equal(t, uint64(variable.DefStreamCountWhenMaxThreadsNotSet), streamCount[0]) + require.Equal(t, uint64(vardef.DefStreamCountWhenMaxThreadsNotSet), streamCount[0]) // Disabled when tiflash_fine_grained_shuffle_stream_count is -1. tk.MustExec("set @@tiflash_fine_grained_shuffle_stream_count = -1") diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index 157c72bb93701..39a267328d7d0 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -56,6 +56,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util/tablesampler" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/table" @@ -4095,7 +4096,7 @@ func getStatsTable(ctx base.PlanContext, tblInfo *model.TableInfo, pid int64) *s // In OptObjectiveDeterminate mode, we need to ignore the real-time stats. // To achieve this, we copy the statsTbl and reset the real-time stats fields (set ModifyCount to 0 and set // RealtimeCount to the row count from the ANALYZE, which is fetched from loaded stats in GetAnalyzeRowCount()). - if ctx.GetSessionVars().GetOptObjective() == variable.OptObjectiveDeterminate { + if ctx.GetSessionVars().GetOptObjective() == vardef.OptObjectiveDeterminate { analyzeCount := max(int64(statsTbl.GetAnalyzeRowCount()), 0) // If the two fields are already the values we want, we don't need to modify it, and also we don't need to copy. if statsTbl.RealtimeCount != analyzeCount || statsTbl.ModifyCount != 0 { @@ -4160,7 +4161,7 @@ func getLatestVersionFromStatsTable(ctx sessionctx.Context, tblInfo *model.Table // 2. Table row count from statistics is zero. Pseudo stats table. realtimeRowCount := statsTbl.RealtimeCount - if ctx.GetSessionVars().GetOptObjective() == variable.OptObjectiveDeterminate { + if ctx.GetSessionVars().GetOptObjective() == vardef.OptObjectiveDeterminate { realtimeRowCount = max(int64(statsTbl.GetAnalyzeRowCount()), 0) } if realtimeRowCount == 0 { diff --git a/pkg/planner/core/logical_plans_test.go b/pkg/planner/core/logical_plans_test.go index 486d04d420f78..b013166424500 100644 --- a/pkg/planner/core/logical_plans_test.go +++ b/pkg/planner/core/logical_plans_test.go @@ -40,7 +40,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" "github.com/pingcap/tidb/pkg/util/hint" @@ -507,7 +507,7 @@ func TestDupRandJoinCondsPushDown(t *testing.T) { } func TestTablePartition(t *testing.T) { - variable.EnableMDL.Store(false) + vardef.EnableMDL.Store(false) definitions := []model.PartitionDefinition{ { ID: 41, @@ -1127,7 +1127,7 @@ func TestAggPrune(t *testing.T) { } func TestVisitInfo(t *testing.T) { - variable.EnableMDL.Store(false) + vardef.EnableMDL.Store(false) tests := []struct { sql string ans []visitInfo @@ -1827,8 +1827,8 @@ func TestWindowFunction(t *testing.T) { defer s.plannerSuite.Close() s.optimizeVars = map[string]string{ - variable.TiDBWindowConcurrency: "1", - variable.TiDBCostModelVersion: "1", + vardef.TiDBWindowConcurrency: "1", + vardef.TiDBCostModelVersion: "1", } defer func() { s.optimizeVars = nil @@ -1849,8 +1849,8 @@ func TestWindowParallelFunction(t *testing.T) { s.plannerSuite = createPlannerSuite() defer s.plannerSuite.Close() s.optimizeVars = map[string]string{ - variable.TiDBWindowConcurrency: "4", - variable.TiDBCostModelVersion: "1", + vardef.TiDBWindowConcurrency: "4", + vardef.TiDBCostModelVersion: "1", } defer func() { s.optimizeVars = nil diff --git a/pkg/planner/core/mock.go b/pkg/planner/core/mock.go index 72fef5e690e6b..20cb4de5b463a 100644 --- a/pkg/planner/core/mock.go +++ b/pkg/planner/core/mock.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/mock" ) @@ -424,7 +424,7 @@ func MockContext() *mock.Context { Client: &mock.Client{}, } ctx.GetSessionVars().CurrentDB = "test" - ctx.GetSessionVars().DivPrecisionIncrement = variable.DefDivPrecisionIncrement + ctx.GetSessionVars().DivPrecisionIncrement = vardef.DefDivPrecisionIncrement do := domain.NewMockDomain() if err := do.CreateStatsHandle(ctx, initStatsCtx); err != nil { panic(fmt.Sprintf("create mock context panic: %+v", err)) diff --git a/pkg/planner/core/optimizer.go b/pkg/planner/core/optimizer.go index 5a82e05fdf9e5..693b1807e25ac 100644 --- a/pkg/planner/core/optimizer.go +++ b/pkg/planner/core/optimizer.go @@ -48,6 +48,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -836,7 +837,7 @@ func inferFineGrainedShuffleStreamCountForWindow(ctx context.Context, sctx base. func setDefaultStreamCount(streamCountInfo *tiflashClusterInfo) { (*streamCountInfo).itemStatus = initialized - (*streamCountInfo).itemValue = variable.DefStreamCountWhenMaxThreadsNotSet + (*streamCountInfo).itemValue = vardef.DefStreamCountWhenMaxThreadsNotSet } func setupFineGrainedShuffleInternal(ctx context.Context, sctx base.PlanContext, plan base.PhysicalPlan, helper *fineGrainedShuffleHelper, streamCountInfo *tiflashClusterInfo, tiflashServerCountInfo *tiflashClusterInfo) { diff --git a/pkg/planner/core/physical_plans.go b/pkg/planner/core/physical_plans.go index 39b67b1930f90..5e9bd2e66435f 100644 --- a/pkg/planner/core/physical_plans.go +++ b/pkg/planner/core/physical_plans.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util/tablesampler" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -1849,7 +1850,7 @@ type PhysicalExchangeSender struct { HashCols []*property.MPPPartitionColumn // Tasks is the mpp task for current PhysicalExchangeSender. Tasks []*kv.MPPTask - CompressionMode kv.ExchangeCompressionMode + CompressionMode vardef.ExchangeCompressionMode } // Clone implements op.PhysicalPlan interface. diff --git a/pkg/planner/core/plan_cache.go b/pkg/planner/core/plan_cache.go index 9fc4fde550872..3ead33c698aa8 100644 --- a/pkg/planner/core/plan_cache.go +++ b/pkg/planner/core/plan_cache.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util/debugtrace" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn/staleread" "github.com/pingcap/tidb/pkg/types" driver "github.com/pingcap/tidb/pkg/types/parser_driver" @@ -277,7 +277,7 @@ func instancePlanCacheEnabled(ctx context.Context) bool { if intest.InTest && ctx.Value(PlanCacheKeyEnableInstancePlanCache{}) != nil { return true } - enableInstancePlanCache := variable.EnableInstancePlanCache.Load() + enableInstancePlanCache := vardef.EnableInstancePlanCache.Load() return enableInstancePlanCache } diff --git a/pkg/planner/core/plan_cache_utils.go b/pkg/planner/core/plan_cache_utils.go index 47a465aaa726c..280db886a2e04 100644 --- a/pkg/planner/core/plan_cache_utils.go +++ b/pkg/planner/core/plan_cache_utils.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" @@ -337,8 +338,8 @@ func NewPlanCacheKey(sctx sessionctx.Context, stmt *PlanCacheStmt) (key, binding hash = append(hash, hack.Slice(connCharset)...) hash = append(hash, hack.Slice(connCollation)...) hash = append(hash, hack.Slice(strconv.FormatBool(vars.InRestrictedSQL))...) - hash = append(hash, hack.Slice(strconv.FormatBool(variable.RestrictedReadOnly.Load()))...) - hash = append(hash, hack.Slice(strconv.FormatBool(variable.VarTiDBSuperReadOnly.Load()))...) + hash = append(hash, hack.Slice(strconv.FormatBool(vardef.RestrictedReadOnly.Load()))...) + hash = append(hash, hack.Slice(strconv.FormatBool(vardef.VarTiDBSuperReadOnly.Load()))...) // expr-pushdown-blacklist can affect query optimization, so we need to consider it in plan cache. hash = codec.EncodeInt(hash, expression.ExprPushDownBlackListReloadTimeStamp.Load()) diff --git a/pkg/planner/core/plan_cost_ver1.go b/pkg/planner/core/plan_cost_ver1.go index c94c406a5bef9..f638fe553a408 100644 --- a/pkg/planner/core/plan_cost_ver1.go +++ b/pkg/planner/core/plan_cost_ver1.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util/costusage" "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/util/paging" ) @@ -836,7 +836,7 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, _ bool, costFlag uint64, build = p.Children()[1] } sessVars := p.SCtx().GetSessionVars() - oomUseTmpStorage := variable.EnableTmpStorageOnOOM.Load() + oomUseTmpStorage := vardef.EnableTmpStorageOnOOM.Load() memQuota := sessVars.MemTracker.GetBytesLimit() // sessVars.MemQuotaQuery && hint rowSize := getAvgRowSize(build.StatsInfo(), build.Schema().Columns) spill := oomUseTmpStorage && memQuota > 0 && rowSize*buildCnt > float64(memQuota) && p.storeTp != kv.TiFlash @@ -1025,7 +1025,7 @@ func (p *PhysicalSort) GetCost(count float64, schema *expression.Schema) float64 cpuCost := count * math.Log2(count) * sessVars.GetCPUFactor() memoryCost := count * sessVars.GetMemoryFactor() - oomUseTmpStorage := variable.EnableTmpStorageOnOOM.Load() + oomUseTmpStorage := vardef.EnableTmpStorageOnOOM.Load() memQuota := sessVars.MemTracker.GetBytesLimit() // sessVars.MemQuotaQuery && hint rowSize := getAvgRowSize(p.StatsInfo(), schema.Columns) spill := oomUseTmpStorage && memQuota > 0 && rowSize*count > float64(memQuota) diff --git a/pkg/planner/core/plan_cost_ver2.go b/pkg/planner/core/plan_cost_ver2.go index f57e319d0c811..12cc9f4f58472 100644 --- a/pkg/planner/core/plan_cost_ver2.go +++ b/pkg/planner/core/plan_cost_ver2.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/costusage" "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util/paging" "github.com/pingcap/tidb/pkg/util/ranger" "github.com/pingcap/tipb/go-tipb" @@ -387,7 +387,7 @@ func (p *PhysicalSort) GetPlanCostVer2(taskType property.TaskType, option *optim cpuFactor := getTaskCPUFactorVer2(p, taskType) memFactor := getTaskMemFactorVer2(p, taskType) diskFactor := defaultVer2Factors.TiDBDisk - oomUseTmpStorage := variable.EnableTmpStorageOnOOM.Load() + oomUseTmpStorage := vardef.EnableTmpStorageOnOOM.Load() memQuota := p.SCtx().GetSessionVars().MemTracker.GetBytesLimit() spill := taskType == property.RootTaskType && // only TiDB can spill oomUseTmpStorage && // spill is enabled diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index 6f0b90a4c9d2d..fd5c39588d143 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -50,6 +50,7 @@ import ( "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn/staleread" "github.com/pingcap/tidb/pkg/statistics" @@ -1260,7 +1261,7 @@ func getPossibleAccessPaths(ctx base.PlanContext, tableHints *hint.PlanHints, in if !isolationReadEnginesHasTiKV { if hint.IndexNames != nil { - engineVals, _ := ctx.GetSessionVars().GetSystemVar(variable.TiDBIsolationReadEngines) + engineVals, _ := ctx.GetSessionVars().GetSystemVar(vardef.TiDBIsolationReadEngines) err := fmt.Errorf("TiDB doesn't support index in the isolation read engines(value: '%v')", engineVals) if i < indexHintsLen { return nil, err @@ -1364,7 +1365,7 @@ func filterPathByIsolationRead(ctx base.PlanContext, paths []*util.AccessPath, t } } var err error - engineVals, _ := ctx.GetSessionVars().GetSystemVar(variable.TiDBIsolationReadEngines) + engineVals, _ := ctx.GetSessionVars().GetSystemVar(vardef.TiDBIsolationReadEngines) if len(paths) == 0 { helpMsg := "" if engineVals == "tiflash" { @@ -1374,7 +1375,7 @@ func filterPathByIsolationRead(ctx base.PlanContext, paths []*util.AccessPath, t } } err = plannererrors.ErrInternal.GenWithStackByArgs(fmt.Sprintf("No access path for table '%s' is found with '%v' = '%v', valid values can be '%s'%s.", tblName.String(), - variable.TiDBIsolationReadEngines, engineVals, availableEngineStr, helpMsg)) + vardef.TiDBIsolationReadEngines, engineVals, availableEngineStr, helpMsg)) } if _, ok := isolationReadEngines[kv.TiFlash]; !ok { if ctx.GetSessionVars().StmtCtx.TiFlashEngineRemovedDueToStrictSQLMode { @@ -1382,7 +1383,7 @@ func filterPathByIsolationRead(ctx base.PlanContext, paths []*util.AccessPath, t "MPP mode may be blocked because the query is not readonly and sql mode is strict.") } else { ctx.GetSessionVars().RaiseWarningWhenMPPEnforced( - fmt.Sprintf("MPP mode may be blocked because '%v'(value: '%v') not match, need 'tiflash'.", variable.TiDBIsolationReadEngines, engineVals)) + fmt.Sprintf("MPP mode may be blocked because '%v'(value: '%v') not match, need 'tiflash'.", vardef.TiDBIsolationReadEngines, engineVals)) } } return paths, err @@ -2150,7 +2151,7 @@ func (b *PlanBuilder) getFullAnalyzeColumnsInfo( switch columnChoice { case ast.DefaultChoice: - columnOptions := variable.AnalyzeColumnOptions.Load() + columnOptions := vardef.AnalyzeColumnOptions.Load() switch columnOptions { case ast.AllColumns.String(): return tbl.TableInfo.Columns, nil, nil @@ -2346,7 +2347,7 @@ func (b *PlanBuilder) filterSkipColumnTypes(origin []*model.ColumnInfo, tbl *res skipTypes := b.ctx.GetSessionVars().AnalyzeSkipColumnTypes if b.ctx.GetSessionVars().InRestrictedSQL { // For auto analyze, we need to use @@global.tidb_analyze_skip_column_types. - val, err1 := b.ctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBAnalyzeSkipColumnTypes) + val, err1 := b.ctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBAnalyzeSkipColumnTypes) if err1 != nil { logutil.BgLogger().Error("loading tidb_analyze_skip_column_types failed", zap.Error(err1)) result = origin @@ -2733,7 +2734,7 @@ func pickColumnList(astColChoice ast.ColumnChoice, astColList []*model.ColumnInf func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64, version int) (base.Plan, error) { p := &Analyze{Opts: opts} p.OptionsMap = make(map[int64]V2AnalyzeOptions) - usePersistedOptions := variable.PersistAnalyzeOptions.Load() + usePersistedOptions := vardef.PersistAnalyzeOptions.Load() // Construct tasks for each table. for _, tbl := range as.TableNames { @@ -3686,7 +3687,7 @@ func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (base. p.StaleTxnStartTS = startTS } case *ast.SetResourceGroupStmt: - if variable.EnableResourceControlStrictMode.Load() { + if vardef.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) } @@ -3818,7 +3819,7 @@ func collectVisitInfoFromGrantStmt(sctx base.PlanContext, vi []visitInfo, stmt * } func genAuthErrForGrantStmt(sctx base.PlanContext, dbName string) error { - if !strings.EqualFold(dbName, variable.PerformanceSchema) { + if !strings.EqualFold(dbName, vardef.PerformanceSchema) { return nil } user := sctx.GetSessionVars().User @@ -5977,9 +5978,9 @@ func checkAlterDDLJobOptValue(opt *AlterDDLJobOpt) error { if err != nil { return err } - if thread < 1 || thread > variable.MaxConfigurableConcurrency { + if thread < 1 || thread > vardef.MaxConfigurableConcurrency { return fmt.Errorf("the value %v for %s is out of range [1, %v]", - thread, opt.Name, variable.MaxConfigurableConcurrency) + thread, opt.Name, vardef.MaxConfigurableConcurrency) } case AlterDDLJobBatchSize: batchSize, err := GetThreadOrBatchSizeFromExpression(opt) @@ -5987,9 +5988,9 @@ func checkAlterDDLJobOptValue(opt *AlterDDLJobOpt) error { return err } bs := int32(batchSize) - if bs < variable.MinDDLReorgBatchSize || bs > variable.MaxDDLReorgBatchSize { + if bs < vardef.MinDDLReorgBatchSize || bs > vardef.MaxDDLReorgBatchSize { return fmt.Errorf("the value %v for %s is out of range [%v, %v]", - bs, opt.Name, variable.MinDDLReorgBatchSize, variable.MaxDDLReorgBatchSize) + bs, opt.Name, vardef.MinDDLReorgBatchSize, vardef.MaxDDLReorgBatchSize) } case AlterDDLJobMaxWriteSpeed: speed, err := GetMaxWriteSpeedFromExpression(opt) diff --git a/pkg/planner/core/rule_collect_plan_stats.go b/pkg/planner/core/rule_collect_plan_stats.go index 742db96bf352c..ae305dc88f9db 100644 --- a/pkg/planner/core/rule_collect_plan_stats.go +++ b/pkg/planner/core/rule_collect_plan_stats.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/planctx" "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/asyncload" "github.com/pingcap/tidb/pkg/util/intset" @@ -258,7 +258,7 @@ func RequestLoadStats(ctx base.PlanContext, neededHistItems []model.StatsLoadIte err := domain.GetDomain(ctx).StatsHandle().SendLoadRequests(stmtCtx, neededHistItems, timeout) if err != nil { stmtCtx.IsSyncStatsFailed = true - if variable.StatsLoadPseudoTimeout.Load() { + if vardef.StatsLoadPseudoTimeout.Load() { logutil.BgLogger().Warn("RequestLoadStats failed", zap.Error(err)) stmtCtx.AppendWarning(err) return nil @@ -278,7 +278,7 @@ func SyncWaitStatsLoad(plan base.LogicalPlan) error { err := domain.GetDomain(plan.SCtx()).StatsHandle().SyncWaitStatsLoad(stmtCtx) if err != nil { stmtCtx.IsSyncStatsFailed = true - if variable.StatsLoadPseudoTimeout.Load() { + if vardef.StatsLoadPseudoTimeout.Load() { logutil.BgLogger().Warn("SyncWaitStatsLoad failed", zap.Error(err)) stmtCtx.AppendWarning(err) return nil diff --git a/pkg/planner/memo/BUILD.bazel b/pkg/planner/memo/BUILD.bazel index 5ca76fe078a01..04d96e8f1cb3d 100644 --- a/pkg/planner/memo/BUILD.bazel +++ b/pkg/planner/memo/BUILD.bazel @@ -44,7 +44,7 @@ go_test( "//pkg/planner/core/operator/logicalop", "//pkg/planner/core/resolve", "//pkg/planner/property", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit/testsetup", "@com_github_stretchr_testify//require", "@io_opencensus_go//stats/view", diff --git a/pkg/planner/memo/group_test.go b/pkg/planner/memo/group_test.go index f3a637327f91e..6af441d6c280a 100644 --- a/pkg/planner/memo/group_test.go +++ b/pkg/planner/memo/group_test.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/planner/property" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/stretchr/testify/require" ) @@ -97,7 +97,7 @@ func TestGroupExists(t *testing.T) { } func TestGroupFingerPrint(t *testing.T) { - variable.EnableMDL.Store(false) + vardef.EnableMDL.Store(false) p := parser.New() stmt1, err := p.ParseOneStmt("select * from t where a > 1 and a < 100", "", "") require.NoError(t, err) @@ -229,7 +229,7 @@ func TestFirstElemAfterDelete(t *testing.T) { } func TestBuildKeyInfo(t *testing.T) { - variable.EnableMDL.Store(false) + vardef.EnableMDL.Store(false) p := parser.New() ctx := plannercore.MockContext() defer func() { diff --git a/pkg/planner/optimize.go b/pkg/planner/optimize.go index 526e23c14905e..d2b9a4b6b16bf 100644 --- a/pkg/planner/optimize.go +++ b/pkg/planner/optimize.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util/optimizetrace" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" @@ -138,7 +139,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node *resolve.NodeW, defer debugtrace.LeaveContextCommon(pctx) } - if !sessVars.InRestrictedSQL && (variable.RestrictedReadOnly.Load() || variable.VarTiDBSuperReadOnly.Load()) { + if !sessVars.InRestrictedSQL && (vardef.RestrictedReadOnly.Load() || vardef.VarTiDBSuperReadOnly.Load()) { allowed, err := allowInReadOnlyMode(pctx, node.Node) if err != nil { return nil, nil, err @@ -180,10 +181,10 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node *resolve.NodeW, defer func() { // Override the resource group if the hint is set. if retErr == nil && sessVars.StmtCtx.StmtHints.HasResourceGroup { - if variable.EnableResourceControl.Load() { + if vardef.EnableResourceControl.Load() { hasPriv := true // only check dynamic privilege when strict-mode is enabled. - if variable.EnableResourceControlStrictMode.Load() { + if vardef.EnableResourceControlStrictMode.Load() { checker := privilege.GetPrivilegeManager(sctx) if checker != nil { hasRgAdminPriv := checker.RequestDynamicVerification(sctx.GetSessionVars().ActiveRoles, "RESOURCE_GROUP_ADMIN", false) diff --git a/pkg/plugin/conn_ip_example/BUILD.bazel b/pkg/plugin/conn_ip_example/BUILD.bazel index f720963731967..3efb4b83bb1d6 100644 --- a/pkg/plugin/conn_ip_example/BUILD.bazel +++ b/pkg/plugin/conn_ip_example/BUILD.bazel @@ -7,6 +7,7 @@ go_library( visibility = ["//visibility:private"], deps = [ "//pkg/plugin", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", ], ) diff --git a/pkg/plugin/conn_ip_example/conn_ip_example.go b/pkg/plugin/conn_ip_example/conn_ip_example.go index b2dfc54602d4d..7fa03e598a4f6 100644 --- a/pkg/plugin/conn_ip_example/conn_ip_example.go +++ b/pkg/plugin/conn_ip_example/conn_ip_example.go @@ -21,6 +21,7 @@ import ( "sync/atomic" "github.com/pingcap/tidb/pkg/plugin" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" ) @@ -47,13 +48,13 @@ func OnInit(ctx context.Context, manifest *plugin.Manifest) error { // With the server. sv := &variable.SysVar{ Name: "conn_ip_example_key", - Scope: variable.ScopeGlobal | variable.ScopeSession, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Value: "v1", - Type: variable.TypeStr, // default. + Type: vardef.TypeStr, // default. // (Optional) specifying a validation function helps to normalize the value before setting it. // The "normalizedValue" applies if the value has a Type associated, where some formatting may have already // been applied. i.e. TypeBool: ON/oN/1/on -> ON - Validation: func(vars *variable.SessionVars, normalizedValue string, originalValue string, scope variable.ScopeFlag) (string, error) { + Validation: func(vars *variable.SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { fmt.Println("The validation function was called") return strings.ToLower(normalizedValue), nil }, diff --git a/pkg/privilege/privileges/BUILD.bazel b/pkg/privilege/privileges/BUILD.bazel index b08fc60a8adac..da9778d93e599 100644 --- a/pkg/privilege/privileges/BUILD.bazel +++ b/pkg/privilege/privileges/BUILD.bazel @@ -24,6 +24,7 @@ go_library( "//pkg/privilege/privileges/ldap", "//pkg/sessionctx", "//pkg/sessionctx/sessionstates", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/types", "//pkg/util", @@ -67,6 +68,7 @@ go_test( "//pkg/session", "//pkg/sessionctx", "//pkg/sessionctx/sessionstates", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/testkit", "//pkg/testkit/testsetup", diff --git a/pkg/privilege/privileges/cache.go b/pkg/privilege/privileges/cache.go index 30db8dff136d3..d3b835cd3facd 100644 --- a/pkg/privilege/privileges/cache.go +++ b/pkg/privilege/privileges/cache.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -920,7 +921,7 @@ func (p *immutable) decodeUserTableRow(row chunk.Row, fs []*resolve.ResultField) var value UserRecord defaultAuthPlugin := "" if p.globalVars != nil { - val, err := p.globalVars.GetGlobalSysVar(variable.DefaultAuthPlugin) + val, err := p.globalVars.GetGlobalSysVar(vardef.DefaultAuthPlugin) if err == nil { defaultAuthPlugin = val } @@ -1256,7 +1257,7 @@ func (p *MySQLPrivilege) matchIdentity(sctx sqlexec.RestrictedSQLExecutor, user, // If skip-name resolve is not enabled, and the host is not localhost // we can fallback and try to resolve with all addrs that match. // TODO: this is imported from previous code in session.Auth(), and can be improved in future. - if !skipNameResolve && host != variable.DefHostname { + if !skipNameResolve && host != vardef.DefHostname { addrs, err := net.LookupAddr(host) if err != nil { logutil.BgLogger().Warn( diff --git a/pkg/privilege/privileges/cache_test.go b/pkg/privilege/privileges/cache_test.go index 2f734fc5e14fc..ccf4e6f70f600 100644 --- a/pkg/privilege/privileges/cache_test.go +++ b/pkg/privilege/privileges/cache_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/privilege/privileges" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" @@ -73,7 +73,7 @@ func TestLoadUserTable(t *testing.T) { for _, plugin := range []string{mysql.AuthNativePassword, mysql.AuthCachingSha2Password, mysql.AuthTiDBSM3Password} { p = privileges.MySQLPrivilege{} p.SetGlobalVarsAccessor(se.GetSessionVars().GlobalVarsAccessor) - require.NoError(t, se.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.DefaultAuthPlugin, plugin)) + require.NoError(t, se.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.DefaultAuthPlugin, plugin)) require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) require.Equal(t, plugin, p.User()[0].AuthPlugin) } diff --git a/pkg/privilege/privileges/privileges.go b/pkg/privilege/privileges/privileges.go index a08477973182a..41f9b845e5d5f 100644 --- a/pkg/privilege/privileges/privileges.go +++ b/pkg/privilege/privileges/privileges.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/privilege/privileges/ldap" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -451,7 +452,7 @@ func checkAuthTokenClaims(claims map[string]any, record *UserRecord, tokenLife t // CheckPasswordExpired checks whether the password has been expired. func (*UserPrivileges) CheckPasswordExpired(sessionVars *variable.SessionVars, record *UserRecord) (bool, error) { - isSandBoxModeEnabled := variable.IsSandBoxModeEnabled.Load() + isSandBoxModeEnabled := vardef.IsSandBoxModeEnabled.Load() if record.PasswordExpired { if isSandBoxModeEnabled { return true, nil @@ -461,7 +462,7 @@ func (*UserPrivileges) CheckPasswordExpired(sessionVars *variable.SessionVars, r if record.PasswordLifeTime != 0 { lifeTime := record.PasswordLifeTime if lifeTime == -1 { - pwdLifeTimeStr, err := sessionVars.GlobalVarsAccessor.GetGlobalSysVar(variable.DefaultPasswordLifetime) + pwdLifeTimeStr, err := sessionVars.GlobalVarsAccessor.GetGlobalSysVar(vardef.DefaultPasswordLifetime) if err != nil { return false, err } diff --git a/pkg/privilege/privileges/privileges_test.go b/pkg/privilege/privileges/privileges_test.go index 8ec8e217124d2..6f26de43a3af5 100644 --- a/pkg/privilege/privileges/privileges_test.go +++ b/pkg/privilege/privileges/privileges_test.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testutil" @@ -1325,7 +1326,7 @@ func TestSecurityEnhancedModeSysVars(t *testing.T) { tk.MustQuery(`SELECT @@global.tidb_force_priority`).Check(testkit.Rows("NO_PRIORITY")) tk.MustQuery(`SELECT @@global.tidb_enable_telemetry`).Check(testkit.Rows("0")) - tk.MustQuery(`SELECT @@hostname`).Check(testkit.Rows(variable.DefHostname)) + tk.MustQuery(`SELECT @@hostname`).Check(testkit.Rows(vardef.DefHostname)) sem.Disable() if hostname, err := os.Hostname(); err == nil { tk.MustQuery(`SELECT @@hostname`).Check(testkit.Rows(hostname)) @@ -1911,7 +1912,7 @@ func TestCheckPasswordExpired(t *testing.T) { require.ErrorContains(t, err, "Your password has expired. To log in you must change it using a client that supports expired passwords") record.PasswordExpired = false - err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.DefaultPasswordLifetime, "2") + err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.DefaultPasswordLifetime, "2") require.NoError(t, err) // use default_password_lifetime record.PasswordLifeTime = -1 @@ -1982,7 +1983,7 @@ func TestPasswordExpireWithSandBoxMode(t *testing.T) { store := createStoreAndPrepareDB(t) rootTk := testkit.NewTestKit(t, store) rootTk.MustExec(`CREATE USER 'testuser'@'localhost' PASSWORD EXPIRE`) - variable.IsSandBoxModeEnabled.Store(true) + vardef.IsSandBoxModeEnabled.Store(true) // PASSWORD EXPIRE user := &auth.UserIdentity{Username: "testuser", Hostname: "localhost"} @@ -2054,13 +2055,13 @@ func TestVerificationInfoWithSessionTokenPlugin(t *testing.T) { require.False(t, tk.Session().InSandBoxMode()) // Test password expiration with sandbox. - variable.IsSandBoxModeEnabled.Store(true) + vardef.IsSandBoxModeEnabled.Store(true) err = tk.Session().Auth(user, tokenBytes, nil, nil) require.NoError(t, err) require.False(t, tk.Session().InSandBoxMode()) // Enable resource group. - variable.EnableResourceControl.Store(true) + vardef.EnableResourceControl.Store(true) err = tk.Session().Auth(user, tokenBytes, nil, nil) require.NoError(t, err) require.Equal(t, "default", tk.Session().GetSessionVars().ResourceGroupName) diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 4d54f8db8eafd..1947e1e8729c4 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -31,7 +31,6 @@ go_library( "//pkg/extension", "//pkg/infoschema", "//pkg/kv", - "//pkg/meta/model", "//pkg/metrics", "//pkg/param", "//pkg/parser", @@ -69,6 +68,7 @@ go_library( "//pkg/sessionctx", "//pkg/sessionctx/sessionstates", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/statistics/handle", @@ -178,7 +178,7 @@ go_test( "//pkg/server/internal/testutil", "//pkg/server/internal/util", "//pkg/session", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/sessiontxn", "//pkg/store/mockstore", "//pkg/store/mockstore/unistore", diff --git a/pkg/server/conn.go b/pkg/server/conn.go index f7a45129d4cce..94e488dc57bc3 100644 --- a/pkg/server/conn.go +++ b/pkg/server/conn.go @@ -65,7 +65,6 @@ import ( "github.com/pingcap/tidb/pkg/extension" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" @@ -92,6 +91,7 @@ import ( server_metrics "github.com/pingcap/tidb/pkg/server/metrics" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" storeerr "github.com/pingcap/tidb/pkg/store/driver/error" @@ -460,7 +460,7 @@ func (cc *clientConn) writeInitialHandshake(ctx context.Context) error { return err } } - defAuthPlugin, err := cc.ctx.GetSessionVars().GetGlobalSystemVar(context.Background(), variable.DefaultAuthPlugin) + defAuthPlugin, err := cc.ctx.GetSessionVars().GetGlobalSystemVar(context.Background(), vardef.DefaultAuthPlugin) if err != nil { return err } @@ -507,15 +507,15 @@ func (cc *clientConn) getWaitTimeout(ctx context.Context) uint64 { // getSessionVarsWaitTimeout get session variable wait_timeout func (cc *clientConn) getSessionVarsWaitTimeout(ctx context.Context) uint64 { - valStr, exists := cc.ctx.GetSessionVars().GetSystemVar(variable.WaitTimeout) + valStr, exists := cc.ctx.GetSessionVars().GetSystemVar(vardef.WaitTimeout) if !exists { - return variable.DefWaitTimeout + return vardef.DefWaitTimeout } waitTimeout, err := strconv.ParseUint(valStr, 10, 64) if err != nil { logutil.Logger(ctx).Warn("get sysval wait_timeout failed, use default value", zap.Error(err)) // if get waitTimeout error, use default value - return variable.DefWaitTimeout + return vardef.DefWaitTimeout } return waitTimeout } @@ -934,7 +934,7 @@ func (cc *clientConn) PeerHost(hasPassword string, update bool) (host, port stri return cc.peerHost, cc.peerPort, nil } } - host = variable.DefHostname + host = vardef.DefHostname if cc.isUnixSocket { cc.peerHost = host cc.serverHost = host @@ -973,7 +973,7 @@ func (cc *clientConn) skipInitConnect() bool { // initResultEncoder initialize the result encoder for current connection. func (cc *clientConn) initResultEncoder(ctx context.Context) { - chs, err := cc.ctx.GetSessionVars().GetSessionOrGlobalSystemVar(context.Background(), variable.CharacterSetResults) + chs, err := cc.ctx.GetSessionVars().GetSessionOrGlobalSystemVar(context.Background(), vardef.CharacterSetResults) if err != nil { chs = "" logutil.Logger(ctx).Warn("get character_set_results system variable failed", zap.Error(err)) @@ -982,7 +982,7 @@ func (cc *clientConn) initResultEncoder(ctx context.Context) { } func (cc *clientConn) initInputEncoder(ctx context.Context) { - chs, err := cc.ctx.GetSessionVars().GetSessionOrGlobalSystemVar(context.Background(), variable.CharacterSetClient) + chs, err := cc.ctx.GetSessionVars().GetSessionOrGlobalSystemVar(context.Background(), vardef.CharacterSetClient) if err != nil { chs = "" logutil.Logger(ctx).Warn("get character_set_client system variable failed", zap.Error(err)) @@ -993,7 +993,7 @@ func (cc *clientConn) initInputEncoder(ctx context.Context) { // initConnect runs the initConnect SQL statement if it has been specified. // The semantics are MySQL compatible. func (cc *clientConn) initConnect(ctx context.Context) error { - val, err := cc.ctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.InitConnect) + val, err := cc.ctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.InitConnect) if err != nil { return err } @@ -1054,7 +1054,7 @@ func (cc *clientConn) Run(ctx context.Context) { }() parentCtx := ctx - var traceInfo *model.TraceInfo + var traceInfo *tracing.TraceInfo // Usually, client connection status changes between [dispatching] <=> [reading]. // When some event happens, server may notify this client connection by setting // the status to special values, for example: kill or graceful shutdown. @@ -1064,7 +1064,7 @@ func (cc *clientConn) Run(ctx context.Context) { sessVars := cc.ctx.GetSessionVars() if alias := sessVars.SessionAlias; traceInfo == nil || traceInfo.SessionAlias != alias { // We should reset the context trace info when traceInfo not inited or session alias changed. - traceInfo = &model.TraceInfo{ + traceInfo = &tracing.TraceInfo{ ConnectionID: cc.connectionID, SessionAlias: alias, } @@ -1302,7 +1302,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { sqlID := cc.ctx.GetSessionVars().SQLCPUUsages.AllocNewSQLID() ctx = topsql.AttachAndRegisterProcessInfo(ctx, cc.connectionID, sqlID) } - if variable.EnablePProfSQLCPU.Load() { + if vardef.EnablePProfSQLCPU.Load() { label := getLastStmtInConn{cc}.PProfLabel() if len(label) > 0 { defer pprof.SetGoroutineLabels(ctx) @@ -2705,8 +2705,8 @@ func (*compressionStats) Stats(vars *variable.SessionVars) (map[string]any, erro } // GetScope gets the status variables scope. -func (*compressionStats) GetScope(_ string) variable.ScopeFlag { - return variable.ScopeSession +func (*compressionStats) GetScope(_ string) vardef.ScopeFlag { + return vardef.ScopeSession } func init() { diff --git a/pkg/server/conn_stmt.go b/pkg/server/conn_stmt.go index 19ac430500944..d0188c2dddfd8 100644 --- a/pkg/server/conn_stmt.go +++ b/pkg/server/conn_stmt.go @@ -54,7 +54,7 @@ import ( "github.com/pingcap/tidb/pkg/server/internal/dump" "github.com/pingcap/tidb/pkg/server/internal/parse" "github.com/pingcap/tidb/pkg/server/internal/resultset" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" storeerr "github.com/pingcap/tidb/pkg/store/driver/error" "github.com/pingcap/tidb/pkg/util/chunk" @@ -375,7 +375,7 @@ func (cc *clientConn) executeWithCursor(ctx context.Context, stmt PreparedStatem rowContainer.GetMemTracker().SetLabel(memory.LabelForCursorFetch) rowContainer.GetDiskTracker().AttachTo(vars.DiskTracker) rowContainer.GetDiskTracker().SetLabel(memory.LabelForCursorFetch) - if variable.EnableTmpStorageOnOOM.Load() { + if vardef.EnableTmpStorageOnOOM.Load() { failpoint.Inject("testCursorFetchSpill", func(val failpoint.Value) { if val, ok := val.(bool); val && ok { actionSpill := rowContainer.ActionSpillForTest() diff --git a/pkg/server/conn_test.go b/pkg/server/conn_test.go index 03b81fb0fa043..395b6acbb98e5 100644 --- a/pkg/server/conn_test.go +++ b/pkg/server/conn_test.go @@ -45,7 +45,7 @@ import ( "github.com/pingcap/tidb/pkg/server/internal/testutil" serverutil "github.com/pingcap/tidb/pkg/server/internal/util" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/store/mockstore/unistore" "github.com/pingcap/tidb/pkg/testkit" @@ -663,7 +663,7 @@ func TestGetSessionVarsWaitTimeout(t *testing.T) { }, } cc.SetCtx(tc) - require.Equal(t, uint64(variable.DefWaitTimeout), cc.getSessionVarsWaitTimeout(context.Background())) + require.Equal(t, uint64(vardef.DefWaitTimeout), cc.getSessionVarsWaitTimeout(context.Background())) } func mapIdentical(m1, m2 map[string]string) bool { @@ -872,7 +872,7 @@ func TestPrefetchPointKeys4Update(t *testing.T) { tk := testkit.NewTestKit(t, store) cc.SetCtx(&TiDBContext{Session: tk.Session()}) ctx := context.Background() - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly tk.MustExec("use test") tk.MustExec("create table prefetch (a int, b int, c int, primary key (a, b))") tk.MustExec("insert prefetch values (1, 1, 1), (2, 2, 2), (3, 3, 3)") @@ -921,7 +921,7 @@ func TestPrefetchPointKeys4Delete(t *testing.T) { tk := testkit.NewTestKit(t, store) cc.SetCtx(&TiDBContext{Session: tk.Session()}) ctx := context.Background() - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly tk.MustExec("use test") tk.MustExec("create table prefetch (a int, b int, c int, primary key (a, b))") tk.MustExec("insert prefetch values (1, 1, 1), (2, 2, 2), (3, 3, 3)") diff --git a/pkg/server/handler/optimizor/BUILD.bazel b/pkg/server/handler/optimizor/BUILD.bazel index 1c30409cf01ee..a0c13d59500a5 100644 --- a/pkg/server/handler/optimizor/BUILD.bazel +++ b/pkg/server/handler/optimizor/BUILD.bazel @@ -17,7 +17,7 @@ go_library( "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/server/handler", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/statistics/handle", "//pkg/statistics/util", "//pkg/table", diff --git a/pkg/server/handler/optimizor/statistics_handler.go b/pkg/server/handler/optimizor/statistics_handler.go index 56fce1180c241..87ecf51c87251 100644 --- a/pkg/server/handler/optimizor/statistics_handler.go +++ b/pkg/server/handler/optimizor/statistics_handler.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/server/handler" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/logutil" @@ -100,7 +100,7 @@ func (sh StatsHistoryHandler) ServeHTTP(w http.ResponseWriter, req *http.Request return } if !enabeld { - handler.WriteError(w, fmt.Errorf("%v should be enabled", variable.TiDBEnableHistoricalStats)) + handler.WriteError(w, fmt.Errorf("%v should be enabled", vardef.TiDBEnableHistoricalStats)) return } diff --git a/pkg/server/handler/tests/BUILD.bazel b/pkg/server/handler/tests/BUILD.bazel index 015f338779a74..acd94daa68a05 100644 --- a/pkg/server/handler/tests/BUILD.bazel +++ b/pkg/server/handler/tests/BUILD.bazel @@ -35,7 +35,7 @@ go_test( "//pkg/session", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/statistics/handle/ddl/testutil", "//pkg/store/helper", "//pkg/store/mockstore", diff --git a/pkg/server/handler/tests/http_handler_serial_test.go b/pkg/server/handler/tests/http_handler_serial_test.go index 0944533fc4ba9..3337cde6a6630 100644 --- a/pkg/server/handler/tests/http_handler_serial_test.go +++ b/pkg/server/handler/tests/http_handler_serial_test.go @@ -41,7 +41,7 @@ import ( "github.com/pingcap/tidb/pkg/server/handler" "github.com/pingcap/tidb/pkg/server/handler/tikvhandler" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/deadlockhistory" @@ -73,13 +73,13 @@ func TestPostSettings(t *testing.T) { require.NoError(t, resp.Body.Close()) require.Equal(t, zap.ErrorLevel, log.GetLevel()) require.Equal(t, "error", config.GetGlobalConfig().Log.Level) - require.True(t, variable.ProcessGeneralLog.Load()) - val, err := se.GetSessionVars().GetGlobalSystemVar(context.Background(), variable.TiDBEnableAsyncCommit) + require.True(t, vardef.ProcessGeneralLog.Load()) + val, err := se.GetSessionVars().GetGlobalSystemVar(context.Background(), vardef.TiDBEnableAsyncCommit) require.NoError(t, err) - require.Equal(t, variable.On, val) - val, err = se.GetSessionVars().GetGlobalSystemVar(context.Background(), variable.TiDBEnable1PC) + require.Equal(t, vardef.On, val) + val, err = se.GetSessionVars().GetGlobalSystemVar(context.Background(), vardef.TiDBEnable1PC) require.NoError(t, err) - require.Equal(t, variable.On, val) + require.Equal(t, vardef.On, val) form = make(url.Values) form.Set("log_level", "fatal") @@ -90,15 +90,15 @@ func TestPostSettings(t *testing.T) { require.NoError(t, err) require.Equal(t, http.StatusOK, resp.StatusCode) require.NoError(t, resp.Body.Close()) - require.False(t, variable.ProcessGeneralLog.Load()) + require.False(t, vardef.ProcessGeneralLog.Load()) require.Equal(t, zap.FatalLevel, log.GetLevel()) require.Equal(t, "fatal", config.GetGlobalConfig().Log.Level) - val, err = se.GetSessionVars().GetGlobalSystemVar(context.Background(), variable.TiDBEnableAsyncCommit) + val, err = se.GetSessionVars().GetGlobalSystemVar(context.Background(), vardef.TiDBEnableAsyncCommit) require.NoError(t, err) - require.Equal(t, variable.Off, val) - val, err = se.GetSessionVars().GetGlobalSystemVar(context.Background(), variable.TiDBEnable1PC) + require.Equal(t, vardef.Off, val) + val, err = se.GetSessionVars().GetGlobalSystemVar(context.Background(), vardef.TiDBEnable1PC) require.NoError(t, err) - require.Equal(t, variable.Off, val) + require.Equal(t, vardef.Off, val) form.Set("log_level", os.Getenv("log_level")) // test ddl_slow_threshold @@ -108,7 +108,7 @@ func TestPostSettings(t *testing.T) { require.NoError(t, err) require.Equal(t, http.StatusOK, resp.StatusCode) require.NoError(t, resp.Body.Close()) - require.Equal(t, uint32(200), atomic.LoadUint32(&variable.DDLSlowOprThreshold)) + require.Equal(t, uint32(200), atomic.LoadUint32(&vardef.DDLSlowOprThreshold)) // test check_mb4_value_in_utf8 db, err := sql.Open("mysql", ts.GetDSN()) diff --git a/pkg/server/handler/tikvhandler/BUILD.bazel b/pkg/server/handler/tikvhandler/BUILD.bazel index 08dc460b24a10..748310da161df 100644 --- a/pkg/server/handler/tikvhandler/BUILD.bazel +++ b/pkg/server/handler/tikvhandler/BUILD.bazel @@ -23,6 +23,7 @@ go_library( "//pkg/session/txninfo", "//pkg/session/types", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/store/gcworker", "//pkg/store/helper", diff --git a/pkg/server/handler/tikvhandler/tikv_handler.go b/pkg/server/handler/tikvhandler/tikv_handler.go index 7707d9f562df7..841dd74554e88 100644 --- a/pkg/server/handler/tikvhandler/tikv_handler.go +++ b/pkg/server/handler/tikvhandler/tikv_handler.go @@ -51,6 +51,7 @@ import ( "github.com/pingcap/tidb/pkg/session/txninfo" sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/store/gcworker" "github.com/pingcap/tidb/pkg/store/helper" @@ -428,9 +429,9 @@ func (h SettingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { if generalLog := req.Form.Get("tidb_general_log"); generalLog != "" { switch generalLog { case "0": - variable.ProcessGeneralLog.Store(false) + vardef.ProcessGeneralLog.Store(false) case "1": - variable.ProcessGeneralLog.Store(true) + vardef.ProcessGeneralLog.Store(true) default: handler.WriteError(w, errors.New("illegal argument")) return @@ -446,9 +447,9 @@ func (h SettingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { switch asyncCommit { case "0": - err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.TiDBEnableAsyncCommit, variable.Off) + err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.TiDBEnableAsyncCommit, vardef.Off) case "1": - err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.TiDBEnableAsyncCommit, variable.On) + err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.TiDBEnableAsyncCommit, vardef.On) default: handler.WriteError(w, errors.New("illegal argument")) return @@ -468,9 +469,9 @@ func (h SettingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { switch onePC { case "0": - err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.TiDBEnable1PC, variable.Off) + err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.TiDBEnable1PC, vardef.Off) case "1": - err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.TiDBEnable1PC, variable.On) + err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.TiDBEnable1PC, vardef.On) default: handler.WriteError(w, errors.New("illegal argument")) return @@ -487,7 +488,7 @@ func (h SettingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { return } if threshold > 0 { - atomic.StoreUint32(&variable.DDLSlowOprThreshold, uint32(threshold)) + atomic.StoreUint32(&vardef.DDLSlowOprThreshold, uint32(threshold)) } } if checkMb4ValueInUtf8 := req.Form.Get("check_mb4_value_in_utf8"); checkMb4ValueInUtf8 != "" { @@ -535,9 +536,9 @@ func (h SettingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { switch mutationChecker { case "0": - err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.TiDBEnableMutationChecker, variable.Off) + err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.TiDBEnableMutationChecker, vardef.Off) case "1": - err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.TiDBEnableMutationChecker, variable.On) + err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.TiDBEnableMutationChecker, vardef.On) default: handler.WriteError(w, errors.New("illegal argument")) return diff --git a/pkg/server/internal/BUILD.bazel b/pkg/server/internal/BUILD.bazel index ade7fae050388..621b70753e495 100644 --- a/pkg/server/internal/BUILD.bazel +++ b/pkg/server/internal/BUILD.bazel @@ -11,7 +11,7 @@ go_library( "//pkg/server/err", "//pkg/server/internal/util", "//pkg/server/metrics", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "@com_github_klauspost_compress//zstd", "@com_github_pingcap_errors//:errors", ], diff --git a/pkg/server/internal/packetio.go b/pkg/server/internal/packetio.go index ba3eb4165a194..fa824578f2d5a 100644 --- a/pkg/server/internal/packetio.go +++ b/pkg/server/internal/packetio.go @@ -49,7 +49,7 @@ import ( server_err "github.com/pingcap/tidb/pkg/server/err" "github.com/pingcap/tidb/pkg/server/internal/util" server_metrics "github.com/pingcap/tidb/pkg/server/metrics" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" ) const defaultWriterSize = 16 * 1024 @@ -76,7 +76,7 @@ type PacketIO struct { func NewPacketIO(bufReadConn *util.BufferedReadConn) *PacketIO { p := &PacketIO{sequence: 0, compressionAlgorithm: mysql.CompressionNone, compressedSequence: 0, zstdLevel: 3} p.SetBufferedReadConn(bufReadConn) - p.SetMaxAllowedPacket(variable.DefMaxAllowedPacket) + p.SetMaxAllowedPacket(vardef.DefMaxAllowedPacket) return p } diff --git a/pkg/server/rpc_server.go b/pkg/server/rpc_server.go index f91e317fc7ec7..3d7b926aa7745 100644 --- a/pkg/server/rpc_server.go +++ b/pkg/server/rpc_server.go @@ -35,7 +35,7 @@ import ( "github.com/pingcap/tidb/pkg/privilege/privileges" "github.com/pingcap/tidb/pkg/session" sessiontypes "github.com/pingcap/tidb/pkg/session/types" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/memory" @@ -237,11 +237,11 @@ func (s *rpcServer) createSession() (sessiontypes.Session, error) { vars.SetHashAggFinalConcurrency(1) vars.StmtCtx.InitMemTracker(memory.LabelForSQLText, -1) vars.StmtCtx.MemTracker.AttachTo(vars.MemTracker) - if variable.OOMAction.Load() == variable.OOMActionCancel { + if vardef.OOMAction.Load() == vardef.OOMActionCancel { action := &memory.PanicOnExceed{Killer: &vars.SQLKiller} vars.MemTracker.SetActionOnExceed(action) } - if err = vars.SetSystemVar(variable.MaxAllowedPacket, strconv.FormatUint(variable.DefMaxAllowedPacket, 10)); err != nil { + if err = vars.SetSystemVar(vardef.MaxAllowedPacket, strconv.FormatUint(vardef.DefMaxAllowedPacket, 10)); err != nil { return nil, err } se.SetExtensions(extensions.NewSessionExtensions()) diff --git a/pkg/server/server.go b/pkg/server/server.go index 59e16a4f2a5cc..edf4f425a46e1 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -68,6 +68,7 @@ import ( servererr "github.com/pingcap/tidb/pkg/server/err" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/session/txninfo" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/util" @@ -411,8 +412,8 @@ func setSSLVariable(ca, key, cert string) { } func setTxnScope() { - variable.SetSysVar(variable.TiDBTxnScope, func() string { - if !variable.EnableLocalTxn.Load() { + variable.SetSysVar(vardef.TiDBTxnScope, func() string { + if !vardef.EnableLocalTxn.Load() { return kv.GlobalTxnScope } if txnScope := config.GetTxnScopeFromConfig(); txnScope == kv.GlobalTxnScope { diff --git a/pkg/server/stat.go b/pkg/server/stat.go index 1ce9e09535daf..f36d8dc5c0585 100644 --- a/pkg/server/stat.go +++ b/pkg/server/stat.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/server/handler/tikvhandler" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util/logutil" "go.uber.org/zap" @@ -32,13 +33,13 @@ var ( ) var defaultStatus = map[string]*variable.StatusVal{ - serverNotAfter: {Scope: variable.ScopeGlobal | variable.ScopeSession, Value: ""}, - serverNotBefore: {Scope: variable.ScopeGlobal | variable.ScopeSession, Value: ""}, - upTime: {Scope: variable.ScopeGlobal, Value: 0}, + serverNotAfter: {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Value: ""}, + serverNotBefore: {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Value: ""}, + upTime: {Scope: vardef.ScopeGlobal, Value: 0}, } // GetScope gets the Status variables scope. -func (*Server) GetScope(_ string) variable.ScopeFlag { +func (*Server) GetScope(_ string) vardef.ScopeFlag { return variable.DefaultStatusVarScopeFlag } diff --git a/pkg/session/BUILD.bazel b/pkg/session/BUILD.bazel index f8844b4717e78..b684099d16ef0 100644 --- a/pkg/session/BUILD.bazel +++ b/pkg/session/BUILD.bazel @@ -74,6 +74,7 @@ go_library( "//pkg/sessionctx", "//pkg/sessionctx/sessionstates", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/sessiontxn/isolation", @@ -169,6 +170,7 @@ go_test( "//pkg/parser/auth", "//pkg/session/types", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/store/mockstore", diff --git a/pkg/session/bootstrap.go b/pkg/session/bootstrap.go index 68aaea4b2a3bc..f05500af7acb0 100644 --- a/pkg/session/bootstrap.go +++ b/pkg/session/bootstrap.go @@ -48,6 +48,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" sessiontypes "github.com/pingcap/tidb/pkg/session/types" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" storepkg "github.com/pingcap/tidb/pkg/store" "github.com/pingcap/tidb/pkg/table/tables" @@ -1519,7 +1520,7 @@ func checkDistTask(s sessiontypes.Session, ver int64) { return } ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap) - rs, err := s.ExecuteInternal(ctx, "SELECT HIGH_PRIORITY variable_value from mysql.global_variables where variable_name = %?;", variable.TiDBEnableDistTask) + rs, err := s.ExecuteInternal(ctx, "SELECT HIGH_PRIORITY variable_value from mysql.global_variables where variable_name = %?;", vardef.TiDBEnableDistTask) if err != nil { logutil.BgLogger().Fatal("check dist task failed, getting tidb_enable_dist_task failed", zap.Error(err)) } @@ -1532,7 +1533,7 @@ func checkDistTask(s sessiontypes.Session, ver int64) { if req.NumRows() == 0 { // Not set yet. return - } else if req.GetRow(0).GetString(0) == variable.On { + } else if req.GetRow(0).GetString(0) == vardef.On { logutil.BgLogger().Fatal("cannot upgrade when tidb_enable_dist_task is enabled, "+ "please set tidb_enable_dist_task to off before upgrade", zap.Error(err)) } @@ -1657,7 +1658,7 @@ func upgradeToVer2(s sessiontypes.Session, ver int64) { } // Version 2 add two system variable for DistSQL concurrency controlling. // Insert distsql related system variable. - distSQLVars := []string{variable.TiDBDistSQLScanConcurrency} + distSQLVars := []string{vardef.TiDBDistSQLScanConcurrency} values := make([]string, 0, len(distSQLVars)) for _, v := range distSQLVars { value := fmt.Sprintf(`("%s", "%s")`, v, variable.GetSysVar(v).Value) @@ -1946,7 +1947,7 @@ func upgradeToVer25(s sessiontypes.Session, ver int64) { return } sql := fmt.Sprintf("UPDATE HIGH_PRIORITY %[1]s.%[2]s SET VARIABLE_VALUE = '%[4]d' WHERE VARIABLE_NAME = '%[3]s' AND VARIABLE_VALUE < %[4]d", - mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBMaxChunkSize, variable.DefInitChunkSize) + mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBMaxChunkSize, vardef.DefInitChunkSize) mustExecute(s, sql) } @@ -2029,7 +2030,7 @@ func upgradeToVer35(s sessiontypes.Session, ver int64) { return } sql := fmt.Sprintf("UPDATE HIGH_PRIORITY %s.%s SET VARIABLE_NAME = '%s' WHERE VARIABLE_NAME = 'tidb_back_off_weight'", - mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBBackOffWeight) + mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBBackOffWeight) mustExecute(s, sql) } @@ -2049,7 +2050,7 @@ func upgradeToVer37(s sessiontypes.Session, ver int64) { } // when upgrade from old tidb and no 'tidb_enable_window_function' in GLOBAL_VARIABLES, init it with 0. sql := fmt.Sprintf("INSERT IGNORE INTO %s.%s (`VARIABLE_NAME`, `VARIABLE_VALUE`) VALUES ('%s', '%d')", - mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableWindowFunction, 0) + mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBEnableWindowFunction, 0) mustExecute(s, sql) } @@ -2105,12 +2106,12 @@ func upgradeToVer42(s sessiontypes.Session, ver int64) { // Convert statement summary global variables to non-empty values. func writeStmtSummaryVars(s sessiontypes.Session) { sql := "UPDATE %n.%n SET variable_value= %? WHERE variable_name= %? AND variable_value=''" - mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, variable.BoolToOnOff(variable.DefTiDBEnableStmtSummary), variable.TiDBEnableStmtSummary) - mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, variable.BoolToOnOff(variable.DefTiDBStmtSummaryInternalQuery), variable.TiDBStmtSummaryInternalQuery) - mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, strconv.Itoa(variable.DefTiDBStmtSummaryRefreshInterval), variable.TiDBStmtSummaryRefreshInterval) - mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, strconv.Itoa(variable.DefTiDBStmtSummaryHistorySize), variable.TiDBStmtSummaryHistorySize) - mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, strconv.FormatUint(uint64(variable.DefTiDBStmtSummaryMaxStmtCount), 10), variable.TiDBStmtSummaryMaxStmtCount) - mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, strconv.FormatUint(uint64(variable.DefTiDBStmtSummaryMaxSQLLength), 10), variable.TiDBStmtSummaryMaxSQLLength) + mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, variable.BoolToOnOff(vardef.DefTiDBEnableStmtSummary), vardef.TiDBEnableStmtSummary) + mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, variable.BoolToOnOff(vardef.DefTiDBStmtSummaryInternalQuery), vardef.TiDBStmtSummaryInternalQuery) + mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, strconv.Itoa(vardef.DefTiDBStmtSummaryRefreshInterval), vardef.TiDBStmtSummaryRefreshInterval) + mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, strconv.Itoa(vardef.DefTiDBStmtSummaryHistorySize), vardef.TiDBStmtSummaryHistorySize) + mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, strconv.FormatUint(uint64(vardef.DefTiDBStmtSummaryMaxStmtCount), 10), vardef.TiDBStmtSummaryMaxStmtCount) + mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, strconv.FormatUint(uint64(vardef.DefTiDBStmtSummaryMaxSQLLength), 10), vardef.TiDBStmtSummaryMaxSQLLength) } func upgradeToVer43(s sessiontypes.Session, ver int64) { @@ -2174,7 +2175,7 @@ func upgradeToVer53(s sessiontypes.Session, ver int64) { } // when upgrade from old tidb and no `tidb_enable_strict_double_type_check` in GLOBAL_VARIABLES, init it with 1` sql := fmt.Sprintf("INSERT IGNORE INTO %s.%s (`VARIABLE_NAME`, `VARIABLE_VALUE`) VALUES ('%s', '%d')", - mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableStrictDoubleTypeCheck, 0) + mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBEnableStrictDoubleTypeCheck, 0) mustExecute(s, sql) } @@ -2208,13 +2209,13 @@ func upgradeToVer55(s sessiontypes.Session, ver int64) { return } defValues := map[string]string{ - variable.TiDBIndexLookupConcurrency: "4", - variable.TiDBIndexLookupJoinConcurrency: "4", - variable.TiDBHashAggFinalConcurrency: "4", - variable.TiDBHashAggPartialConcurrency: "4", - variable.TiDBWindowConcurrency: "4", - variable.TiDBProjectionConcurrency: "4", - variable.TiDBHashJoinConcurrency: "5", + vardef.TiDBIndexLookupConcurrency: "4", + vardef.TiDBIndexLookupJoinConcurrency: "4", + vardef.TiDBHashAggFinalConcurrency: "4", + vardef.TiDBHashAggPartialConcurrency: "4", + vardef.TiDBWindowConcurrency: "4", + vardef.TiDBProjectionConcurrency: "4", + vardef.TiDBHashJoinConcurrency: "5", } names := make([]string, 0, len(defValues)) for n := range defValues { @@ -2242,7 +2243,7 @@ func upgradeToVer55(s sessiontypes.Session, ver int64) { terror.MustNil(err) mustExecute(s, "BEGIN") - v := strconv.Itoa(variable.ConcurrencyUnset) + v := strconv.Itoa(vardef.ConcurrencyUnset) sql := fmt.Sprintf("UPDATE %s.%s SET variable_value='%%s' WHERE variable_name='%%s'", mysql.SystemDB, mysql.GlobalVariablesTable) for _, name := range names { mustExecute(s, fmt.Sprintf(sql, v, name)) @@ -2499,7 +2500,7 @@ func upgradeToVer74(s sessiontypes.Session, ver int64) { return } // The old default value of `tidb_stmt_summary_max_stmt_count` is 200, we want to enlarge this to the new default value when TiDB upgrade. - mustExecute(s, fmt.Sprintf("UPDATE mysql.global_variables SET VARIABLE_VALUE='%[1]v' WHERE VARIABLE_NAME = 'tidb_stmt_summary_max_stmt_count' AND CAST(VARIABLE_VALUE AS SIGNED) = 200", variable.DefTiDBStmtSummaryMaxStmtCount)) + mustExecute(s, fmt.Sprintf("UPDATE mysql.global_variables SET VARIABLE_VALUE='%[1]v' WHERE VARIABLE_NAME = 'tidb_stmt_summary_max_stmt_count' AND CAST(VARIABLE_VALUE AS SIGNED) = 200", vardef.DefTiDBStmtSummaryMaxStmtCount)) } func upgradeToVer75(s sessiontypes.Session, ver int64) { @@ -2549,7 +2550,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. - initGlobalVariableIfNotExists(s, variable.TiDBAnalyzeVersion, 1) + initGlobalVariableIfNotExists(s, vardef.TiDBAnalyzeVersion, 1) } // For users that upgrade TiDB from a pre-4.0 version, we want to disable index merge by default. @@ -2560,7 +2561,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". - initGlobalVariableIfNotExists(s, variable.TiDBEnableIndexMerge, variable.Off) + initGlobalVariableIfNotExists(s, vardef.TiDBEnableIndexMerge, vardef.Off) } func upgradeToVer82(s sessiontypes.Session, ver int64) { @@ -2640,17 +2641,17 @@ func upgradeToVer90(s sessiontypes.Session, ver int64) { return } valStr := variable.BoolToOnOff(config.GetGlobalConfig().EnableBatchDML) - importConfigOption(s, "enable-batch-dml", variable.TiDBEnableBatchDML, valStr) + importConfigOption(s, "enable-batch-dml", vardef.TiDBEnableBatchDML, valStr) valStr = fmt.Sprint(config.GetGlobalConfig().MemQuotaQuery) - importConfigOption(s, "mem-quota-query", variable.TiDBMemQuotaQuery, valStr) + importConfigOption(s, "mem-quota-query", vardef.TiDBMemQuotaQuery, valStr) valStr = fmt.Sprint(config.GetGlobalConfig().Log.QueryLogMaxLen) - importConfigOption(s, "query-log-max-len", variable.TiDBQueryLogMaxLen, valStr) + importConfigOption(s, "query-log-max-len", vardef.TiDBQueryLogMaxLen, valStr) valStr = fmt.Sprint(config.GetGlobalConfig().Performance.CommitterConcurrency) - importConfigOption(s, "committer-concurrency", variable.TiDBCommitterConcurrency, valStr) + importConfigOption(s, "committer-concurrency", vardef.TiDBCommitterConcurrency, valStr) valStr = variable.BoolToOnOff(config.GetGlobalConfig().Performance.RunAutoAnalyze) - importConfigOption(s, "run-auto-analyze", variable.TiDBEnableAutoAnalyze, valStr) + importConfigOption(s, "run-auto-analyze", vardef.TiDBEnableAutoAnalyze, valStr) valStr = config.GetGlobalConfig().OOMAction - importConfigOption(s, "oom-action", variable.TiDBMemOOMAction, valStr) + importConfigOption(s, "oom-action", vardef.TiDBMemOOMAction, valStr) } func upgradeToVer91(s sessiontypes.Session, ver int64) { @@ -2658,13 +2659,13 @@ func upgradeToVer91(s sessiontypes.Session, ver int64) { return } valStr := variable.BoolToOnOff(config.GetGlobalConfig().PreparedPlanCache.Enabled) - importConfigOption(s, "prepared-plan-cache.enable", variable.TiDBEnablePrepPlanCache, valStr) + importConfigOption(s, "prepared-plan-cache.enable", vardef.TiDBEnablePrepPlanCache, valStr) valStr = strconv.Itoa(int(config.GetGlobalConfig().PreparedPlanCache.Capacity)) - importConfigOption(s, "prepared-plan-cache.capacity", variable.TiDBPrepPlanCacheSize, valStr) + importConfigOption(s, "prepared-plan-cache.capacity", vardef.TiDBPrepPlanCacheSize, valStr) valStr = strconv.FormatFloat(config.GetGlobalConfig().PreparedPlanCache.MemoryGuardRatio, 'f', -1, 64) - importConfigOption(s, "prepared-plan-cache.memory-guard-ratio", variable.TiDBPrepPlanCacheMemoryGuardRatio, valStr) + importConfigOption(s, "prepared-plan-cache.memory-guard-ratio", vardef.TiDBPrepPlanCacheMemoryGuardRatio, valStr) } func upgradeToVer93(s sessiontypes.Session, ver int64) { @@ -2672,7 +2673,7 @@ func upgradeToVer93(s sessiontypes.Session, ver int64) { return } valStr := variable.BoolToOnOff(config.GetGlobalConfig().OOMUseTmpStorage) - importConfigOption(s, "oom-use-tmp-storage", variable.TiDBEnableTmpStorageOnOOM, valStr) + importConfigOption(s, "oom-use-tmp-storage", vardef.TiDBEnableTmpStorageOnOOM, valStr) } func upgradeToVer94(s sessiontypes.Session, ver int64) { @@ -2695,7 +2696,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. - initGlobalVariableIfNotExists(s, variable.TiDBOptRangeMaxSize, 0) + initGlobalVariableIfNotExists(s, vardef.TiDBOptRangeMaxSize, 0) } func upgradeToVer98(s sessiontypes.Session, ver int64) { @@ -2707,12 +2708,12 @@ func upgradeToVer98(s sessiontypes.Session, ver int64) { func upgradeToVer99Before(s sessiontypes.Session) { mustExecute(s, "INSERT HIGH_PRIORITY IGNORE INTO %n.%n VALUES (%?, %?);", - mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableMDL, 0) + mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBEnableMDL, 0) } func upgradeToVer99After(s sessiontypes.Session) { sql := fmt.Sprintf("UPDATE HIGH_PRIORITY %[1]s.%[2]s SET VARIABLE_VALUE = %[4]d WHERE VARIABLE_NAME = '%[3]s'", - mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableMDL, 1) + mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBEnableMDL, 1) mustExecute(s, sql) err := kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), s.GetStore(), true, func(_ context.Context, txn kv.Transaction) error { t := meta.NewMutator(txn) @@ -2726,7 +2727,7 @@ func upgradeToVer100(s sessiontypes.Session, ver int64) { return } valStr := strconv.Itoa(int(config.GetGlobalConfig().Performance.ServerMemoryQuota)) - importConfigOption(s, "performance.server-memory-quota", variable.TiDBServerMemoryLimit, valStr) + importConfigOption(s, "performance.server-memory-quota", vardef.TiDBServerMemoryLimit, valStr) } func upgradeToVer101(s sessiontypes.Session, ver int64) { @@ -2764,7 +2765,7 @@ func upgradeToVer105(s sessiontypes.Session, ver int64) { if ver >= version105 { return } - initGlobalVariableIfNotExists(s, variable.TiDBCostModelVersion, "1") + initGlobalVariableIfNotExists(s, vardef.TiDBCostModelVersion, "1") } func upgradeToVer106(s sessiontypes.Session, ver int64) { @@ -2798,7 +2799,7 @@ func upgradeToVer109(s sessiontypes.Session, ver int64) { return } mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", - mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableGCAwareMemoryTrack, 0) + mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBEnableGCAwareMemoryTrack, 0) } // For users that upgrade TiDB from a 5.4-6.4 version, we want to enable tidb tidb_stats_load_pseudo_timeout by default. @@ -2807,7 +2808,7 @@ func upgradeToVer110(s sessiontypes.Session, ver int64) { return } mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", - mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBStatsLoadPseudoTimeout, 1) + mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBStatsLoadPseudoTimeout, 1) } func upgradeToVer130(s sessiontypes.Session, ver int64) { @@ -2837,18 +2838,18 @@ func upgradeToVer133(s sessiontypes.Session, ver int64) { return } mustExecute(s, "UPDATE HIGH_PRIORITY %n.%n set VARIABLE_VALUE = %? where VARIABLE_NAME = %? and VARIABLE_VALUE = %?;", - mysql.SystemDB, mysql.GlobalVariablesTable, variable.DefTiDBServerMemoryLimit, variable.TiDBServerMemoryLimit, "0") + mysql.SystemDB, mysql.GlobalVariablesTable, vardef.DefTiDBServerMemoryLimit, vardef.TiDBServerMemoryLimit, "0") } func upgradeToVer134(s sessiontypes.Session, ver int64) { if ver >= version134 { return } - mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, variable.ForeignKeyChecks, variable.On) - mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableForeignKey, variable.On) - mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableHistoricalStats, variable.On) - mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnablePlanReplayerCapture, variable.On) - mustExecute(s, "UPDATE HIGH_PRIORITY %n.%n SET VARIABLE_VALUE = %? WHERE VARIABLE_NAME = %? AND VARIABLE_VALUE = %?;", mysql.SystemDB, mysql.GlobalVariablesTable, "4", variable.TiDBStoreBatchSize, "0") + mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, vardef.ForeignKeyChecks, vardef.On) + mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBEnableForeignKey, vardef.On) + mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBEnableHistoricalStats, vardef.On) + mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBEnablePlanReplayerCapture, vardef.On) + mustExecute(s, "UPDATE HIGH_PRIORITY %n.%n SET VARIABLE_VALUE = %? WHERE VARIABLE_NAME = %? AND VARIABLE_VALUE = %?;", mysql.SystemDB, mysql.GlobalVariablesTable, "4", vardef.TiDBStoreBatchSize, "0") } // For users that upgrade TiDB from a pre-7.0 version, we want to set tidb_opt_advanced_join_hint to off by default to keep plans unchanged. @@ -2856,7 +2857,7 @@ func upgradeToVer135(s sessiontypes.Session, ver int64) { if ver >= version135 { return } - initGlobalVariableIfNotExists(s, variable.TiDBOptAdvancedJoinHint, false) + initGlobalVariableIfNotExists(s, vardef.TiDBOptAdvancedJoinHint, false) } func upgradeToVer136(s sessiontypes.Session, ver int64) { @@ -2877,7 +2878,7 @@ func upgradeToVer138(s sessiontypes.Session, ver int64) { if ver >= version138 { return } - mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBOptimizerEnableNAAJ, variable.On) + mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBOptimizerEnableNAAJ, vardef.On) } func upgradeToVer139(sessiontypes.Session, int64) {} @@ -2898,7 +2899,7 @@ func upgradeToVer141(s sessiontypes.Session, ver int64) { } 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.TiDBPrepPlanCacheSize) + mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBPrepPlanCacheSize) terror.MustNil(err) req := rs.NewChunk(nil) err = rs.Next(ctx, req) @@ -2912,15 +2913,15 @@ func upgradeToVer141(s sessiontypes.Session, ver int64) { val := row.GetString(0) mustExecute(s, "INSERT HIGH_PRIORITY IGNORE INTO %n.%n VALUES (%?, %?);", - mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBSessionPlanCacheSize, val) - mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBLoadBasedReplicaReadThreshold, variable.DefTiDBLoadBasedReplicaReadThreshold.String()) + mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBSessionPlanCacheSize, val) + mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, vardef.TiDBLoadBasedReplicaReadThreshold, vardef.DefTiDBLoadBasedReplicaReadThreshold.String()) } func upgradeToVer142(s sessiontypes.Session, ver int64) { if ver >= version142 { return } - initGlobalVariableIfNotExists(s, variable.TiDBEnableNonPreparedPlanCache, variable.Off) + initGlobalVariableIfNotExists(s, vardef.TiDBEnableNonPreparedPlanCache, vardef.Off) } func upgradeToVer143(s sessiontypes.Session, ver int64) { @@ -2936,7 +2937,7 @@ func upgradeToVer144(s sessiontypes.Session, ver int64) { return } - initGlobalVariableIfNotExists(s, variable.TiDBPlanCacheInvalidationOnFreshStats, variable.Off) + initGlobalVariableIfNotExists(s, vardef.TiDBPlanCacheInvalidationOnFreshStats, vardef.Off) } func upgradeToVer146(s sessiontypes.Session, ver int64) { @@ -3078,7 +3079,7 @@ func upgradeToVer177(s sessiontypes.Session, ver int64) { } // ignore error when upgrading from v7.4 to higher version. doReentrantDDL(s, CreateDistFrameworkMeta, infoschema.ErrTableExists) - err := s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.TiDBEnableAsyncMergeGlobalStats, variable.Off) + err := s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.TiDBEnableAsyncMergeGlobalStats, vardef.Off) if err != nil { logutil.BgLogger().Fatal("upgradeToVer177 error", zap.Error(err)) } @@ -3155,7 +3156,7 @@ func upgradeToVer191(s sessiontypes.Session, ver int64) { } sql := fmt.Sprintf("INSERT HIGH_PRIORITY IGNORE INTO %s.%s VALUES('%s', '%s')", mysql.SystemDB, mysql.GlobalVariablesTable, - variable.TiDBTxnMode, variable.OptimisticTxnMode) + vardef.TiDBTxnMode, vardef.OptimisticTxnMode) mustExecute(s, sql) } @@ -3218,7 +3219,7 @@ func upgradeToVer209(s sessiontypes.Session, ver int64) { return } - initGlobalVariableIfNotExists(s, variable.TiDBResourceControlStrictMode, variable.Off) + initGlobalVariableIfNotExists(s, vardef.TiDBResourceControlStrictMode, vardef.Off) } func upgradeToVer210(s sessiontypes.Session, ver int64) { @@ -3228,11 +3229,11 @@ func upgradeToVer210(s sessiontypes.Session, ver int64) { // Check if tidb_analyze_column_options exists in mysql.GLOBAL_VARIABLES. // If not, set tidb_analyze_column_options to ALL since this is the old behavior before we introduce this variable. - initGlobalVariableIfNotExists(s, variable.TiDBAnalyzeColumnOptions, ast.AllColumns.String()) + initGlobalVariableIfNotExists(s, vardef.TiDBAnalyzeColumnOptions, ast.AllColumns.String()) // Check if tidb_opt_projection_push_down exists in mysql.GLOBAL_VARIABLES. // If not, set tidb_opt_projection_push_down to Off since this is the old behavior before we introduce this variable. - initGlobalVariableIfNotExists(s, variable.TiDBOptProjectionPushDown, variable.Off) + initGlobalVariableIfNotExists(s, vardef.TiDBOptProjectionPushDown, vardef.Off) } func upgradeToVer211(s sessiontypes.Session, ver int64) { @@ -3298,7 +3299,7 @@ func upgradeToVer215(s sessiontypes.Session, ver int64) { return } - initGlobalVariableIfNotExists(s, variable.TiDBEnableINLJoinInnerMultiPattern, variable.Off) + initGlobalVariableIfNotExists(s, vardef.TiDBEnableINLJoinInnerMultiPattern, vardef.Off) } func upgradeToVer216(s sessiontypes.Session, ver int64) { @@ -3385,7 +3386,7 @@ func initGlobalVariableIfNotExists(s sessiontypes.Session, name string, val any) 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= %?`, - mysql.SystemDB, mysql.TiDBTable, tidbDefOOMAction, variable.OOMActionLog, comment, variable.OOMActionLog, + mysql.SystemDB, mysql.TiDBTable, tidbDefOOMAction, vardef.OOMActionLog, comment, vardef.OOMActionLog, ) } diff --git a/pkg/session/bootstrap_test.go b/pkg/session/bootstrap_test.go index 7bf76aebf7b31..391b77abd56a4 100644 --- a/pkg/session/bootstrap_test.go +++ b/pkg/session/bootstrap_test.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/store/mockstore" @@ -346,7 +347,7 @@ func TestUpgrade(t *testing.T) { require.NoError(t, err) MustExec(t, se1, `delete from mysql.TiDB where VARIABLE_NAME="tidb_server_version"`) MustExec(t, se1, "update mysql.global_variables set variable_value='off' where variable_name='tidb_enable_dist_task'") - MustExec(t, se1, fmt.Sprintf(`delete from mysql.global_variables where VARIABLE_NAME="%s"`, variable.TiDBDistSQLScanConcurrency)) + MustExec(t, se1, fmt.Sprintf(`delete from mysql.global_variables where VARIABLE_NAME="%s"`, vardef.TiDBDistSQLScanConcurrency)) MustExec(t, se1, `commit`) store.SetOption(StoreBootstrappedKey, nil) revertVersionAndVariables(t, se1, 0) @@ -436,7 +437,7 @@ func TestIssue17979_1(t *testing.T) { r := MustExecToRecodeSet(t, seV4, "select variable_value from mysql.tidb where variable_name='default_oom_action'") req := r.NewChunk(nil) require.NoError(t, r.Next(ctx, req)) - require.Equal(t, variable.OOMActionLog, req.GetRow(0).GetString(0)) + require.Equal(t, vardef.OOMActionLog, req.GetRow(0).GetString(0)) domV4.Close() } @@ -737,7 +738,7 @@ func TestAnalyzeVersionUpgradeFrom300To500(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV3, ver300) - MustExec(t, seV3, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBAnalyzeVersion)) + MustExec(t, seV3, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBAnalyzeVersion)) MustExec(t, seV3, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV3) @@ -745,7 +746,7 @@ func TestAnalyzeVersionUpgradeFrom300To500(t *testing.T) { require.Equal(t, int64(ver300), ver) // We are now in 3.0.0, check tidb_analyze_version should not exist. - res := MustExecToRecodeSet(t, seV3, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBAnalyzeVersion)) + res := MustExecToRecodeSet(t, seV3, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBAnalyzeVersion)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -812,7 +813,7 @@ func TestIndexMergeUpgradeFrom300To540(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV3, ver300) - MustExec(t, seV3, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableIndexMerge)) + MustExec(t, seV3, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBEnableIndexMerge)) MustExec(t, seV3, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV3) @@ -820,7 +821,7 @@ func TestIndexMergeUpgradeFrom300To540(t *testing.T) { require.Equal(t, int64(ver300), ver) // We are now in 3.0.0, check tidb_enable_index_merge should not exist. - res := MustExecToRecodeSet(t, seV3, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableIndexMerge)) + res := MustExecToRecodeSet(t, seV3, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBEnableIndexMerge)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -871,7 +872,7 @@ func testIndexMergeUpgradeFrom400To540(t *testing.T, enable bool) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV4, ver400) - MustExec(t, seV4, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", variable.Off, variable.TiDBEnableIndexMerge)) + MustExec(t, seV4, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", vardef.Off, vardef.TiDBEnableIndexMerge)) MustExec(t, seV4, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV4) @@ -879,14 +880,14 @@ func testIndexMergeUpgradeFrom400To540(t *testing.T, enable bool) { require.Equal(t, int64(ver400), ver) // We are now in 4.0.0, tidb_enable_index_merge is off. - res := MustExecToRecodeSet(t, seV4, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableIndexMerge)) + res := MustExecToRecodeSet(t, seV4, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBEnableIndexMerge)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) require.Equal(t, 1, chk.NumRows()) row := chk.GetRow(0) require.Equal(t, 2, row.Len()) - require.Equal(t, variable.Off, row.GetString(1)) + require.Equal(t, vardef.Off, row.GetString(1)) if enable { // For the first time, We set tidb_enable_index_merge as on. @@ -965,7 +966,7 @@ func TestTiDBEnablePagingVariable(t *testing.T) { require.NotEqual(t, 0, req.NumRows()) rows := statistics.RowToDatums(req.GetRow(0), r.Fields()) - if variable.DefTiDBEnablePaging { + if vardef.DefTiDBEnablePaging { match(t, rows, "1") } else { match(t, rows, "0") @@ -990,7 +991,7 @@ func TestTiDBOptRangeMaxSizeWhenUpgrading(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV630, ver94) - MustExec(t, seV630, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBOptRangeMaxSize)) + MustExec(t, seV630, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBOptRangeMaxSize)) MustExec(t, seV630, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV630) @@ -998,7 +999,7 @@ func TestTiDBOptRangeMaxSizeWhenUpgrading(t *testing.T) { require.Equal(t, int64(ver94), ver) // We are now in 6.3.0, check tidb_opt_range_max_size should not exist. - res := MustExecToRecodeSet(t, seV630, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBOptRangeMaxSize)) + res := MustExecToRecodeSet(t, seV630, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBOptRangeMaxSize)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1048,7 +1049,7 @@ func TestTiDBOptAdvancedJoinHintWhenUpgrading(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV660, ver134) - MustExec(t, seV660, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBOptAdvancedJoinHint)) + MustExec(t, seV660, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBOptAdvancedJoinHint)) MustExec(t, seV660, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV660) @@ -1056,7 +1057,7 @@ func TestTiDBOptAdvancedJoinHintWhenUpgrading(t *testing.T) { require.Equal(t, int64(ver134), ver) // We are now in 6.6.0, check tidb_opt_advanced_join_hint should not exist. - res := MustExecToRecodeSet(t, seV660, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBOptAdvancedJoinHint)) + res := MustExecToRecodeSet(t, seV660, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBOptAdvancedJoinHint)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1158,7 +1159,7 @@ func TestTiDBCostModelUpgradeFrom300To650(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV3, ver300) - MustExec(t, seV3, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBCostModelVersion)) + MustExec(t, seV3, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBCostModelVersion)) MustExec(t, seV3, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV3) @@ -1166,7 +1167,7 @@ func TestTiDBCostModelUpgradeFrom300To650(t *testing.T) { require.Equal(t, int64(ver300), ver) // We are now in 3.0.0, check TiDBCostModelVersion should not exist. - res := MustExecToRecodeSet(t, seV3, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBCostModelVersion)) + res := MustExecToRecodeSet(t, seV3, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBCostModelVersion)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1210,7 +1211,7 @@ func TestTiDBCostModelUpgradeFrom610To650(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV61, ver61) - MustExec(t, seV61, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "1", variable.TiDBCostModelVersion)) + MustExec(t, seV61, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "1", vardef.TiDBCostModelVersion)) MustExec(t, seV61, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV61) @@ -1218,7 +1219,7 @@ func TestTiDBCostModelUpgradeFrom610To650(t *testing.T) { require.Equal(t, int64(ver61), ver) // We are now in 6.1, tidb_cost_model_version is 1. - res := MustExecToRecodeSet(t, seV61, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBCostModelVersion)) + res := MustExecToRecodeSet(t, seV61, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBCostModelVersion)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1278,7 +1279,7 @@ func TestTiDBGCAwareUpgradeFrom630To650(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV63, ver63) - MustExec(t, seV63, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "1", variable.TiDBEnableGCAwareMemoryTrack)) + MustExec(t, seV63, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "1", vardef.TiDBEnableGCAwareMemoryTrack)) MustExec(t, seV63, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV63) @@ -1286,7 +1287,7 @@ func TestTiDBGCAwareUpgradeFrom630To650(t *testing.T) { require.Equal(t, int64(ver63), ver) // We are now in 6.3, tidb_enable_gc_aware_memory_track is ON. - res := MustExecToRecodeSet(t, seV63, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableGCAwareMemoryTrack)) + res := MustExecToRecodeSet(t, seV63, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBEnableGCAwareMemoryTrack)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1306,7 +1307,7 @@ func TestTiDBGCAwareUpgradeFrom630To650(t *testing.T) { require.Equal(t, currentBootstrapVersion, ver) // We are now in 6.5. - res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableGCAwareMemoryTrack)) + res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBEnableGCAwareMemoryTrack)) chk = res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1332,7 +1333,7 @@ func TestTiDBServerMemoryLimitUpgradeTo651_1(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV132, ver132) - MustExec(t, seV132, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "0", variable.TiDBServerMemoryLimit)) + MustExec(t, seV132, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "0", vardef.TiDBServerMemoryLimit)) MustExec(t, seV132, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV132) @@ -1340,7 +1341,7 @@ func TestTiDBServerMemoryLimitUpgradeTo651_1(t *testing.T) { require.Equal(t, int64(ver132), ver) // We are now in 6.5.0, tidb_server_memory_limit is 0. - res := MustExecToRecodeSet(t, seV132, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBServerMemoryLimit)) + res := MustExecToRecodeSet(t, seV132, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBServerMemoryLimit)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1360,14 +1361,14 @@ func TestTiDBServerMemoryLimitUpgradeTo651_1(t *testing.T) { require.Equal(t, currentBootstrapVersion, ver) // We are now in 6.5.1+. - res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBServerMemoryLimit)) + res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBServerMemoryLimit)) chk = res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) require.Equal(t, 1, chk.NumRows()) row = chk.GetRow(0) require.Equal(t, 2, row.Len()) - require.Equal(t, variable.DefTiDBServerMemoryLimit, row.GetString(1)) + require.Equal(t, vardef.DefTiDBServerMemoryLimit, row.GetString(1)) } func TestTiDBServerMemoryLimitUpgradeTo651_2(t *testing.T) { @@ -1386,7 +1387,7 @@ func TestTiDBServerMemoryLimitUpgradeTo651_2(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV132, ver132) - MustExec(t, seV132, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "70%", variable.TiDBServerMemoryLimit)) + MustExec(t, seV132, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "70%", vardef.TiDBServerMemoryLimit)) MustExec(t, seV132, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV132) @@ -1394,7 +1395,7 @@ func TestTiDBServerMemoryLimitUpgradeTo651_2(t *testing.T) { require.Equal(t, int64(ver132), ver) // We are now in 6.5.0, tidb_server_memory_limit is "70%". - res := MustExecToRecodeSet(t, seV132, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBServerMemoryLimit)) + res := MustExecToRecodeSet(t, seV132, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBServerMemoryLimit)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1414,7 +1415,7 @@ func TestTiDBServerMemoryLimitUpgradeTo651_2(t *testing.T) { require.Equal(t, currentBootstrapVersion, ver) // We are now in 6.5.1+. - res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBServerMemoryLimit)) + res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBServerMemoryLimit)) chk = res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1440,10 +1441,10 @@ func TestTiDBGlobalVariablesDefaultValueUpgradeFrom630To660(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV630, ver630) - MustExec(t, seV630, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "OFF", variable.TiDBEnableForeignKey)) - MustExec(t, seV630, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "OFF", variable.ForeignKeyChecks)) - MustExec(t, seV630, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "OFF", variable.TiDBEnableHistoricalStats)) - MustExec(t, seV630, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "OFF", variable.TiDBEnablePlanReplayerCapture)) + MustExec(t, seV630, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "OFF", vardef.TiDBEnableForeignKey)) + MustExec(t, seV630, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "OFF", vardef.ForeignKeyChecks)) + MustExec(t, seV630, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "OFF", vardef.TiDBEnableHistoricalStats)) + MustExec(t, seV630, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "OFF", vardef.TiDBEnablePlanReplayerCapture)) MustExec(t, seV630, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV630) @@ -1451,7 +1452,7 @@ func TestTiDBGlobalVariablesDefaultValueUpgradeFrom630To660(t *testing.T) { require.Equal(t, int64(ver630), ver) // We are now in 6.3.0. - upgradeVars := []string{variable.TiDBEnableForeignKey, variable.ForeignKeyChecks, variable.TiDBEnableHistoricalStats, variable.TiDBEnablePlanReplayerCapture} + upgradeVars := []string{vardef.TiDBEnableForeignKey, vardef.ForeignKeyChecks, vardef.TiDBEnableHistoricalStats, vardef.TiDBEnablePlanReplayerCapture} varsValueList := []string{"OFF", "OFF", "OFF", "OFF"} for i := range upgradeVars { res := MustExecToRecodeSet(t, seV630, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", upgradeVars[i])) @@ -1506,7 +1507,7 @@ func TestTiDBStoreBatchSizeUpgradeFrom650To660(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV65, ver65) - MustExec(t, seV65, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "0", variable.TiDBStoreBatchSize)) + MustExec(t, seV65, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "0", vardef.TiDBStoreBatchSize)) MustExec(t, seV65, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV65) @@ -1514,7 +1515,7 @@ func TestTiDBStoreBatchSizeUpgradeFrom650To660(t *testing.T) { require.Equal(t, int64(ver65), ver) // We are now in 6.5, tidb_store_batch_size is 0. - res := MustExecToRecodeSet(t, seV65, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBStoreBatchSize)) + res := MustExecToRecodeSet(t, seV65, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBStoreBatchSize)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1659,7 +1660,7 @@ func TestTiDBNonPrepPlanCacheUpgradeFrom540To700(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV54, ver54) - MustExec(t, seV54, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableNonPreparedPlanCache)) + MustExec(t, seV54, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBEnableNonPreparedPlanCache)) MustExec(t, seV54, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV54) @@ -1667,7 +1668,7 @@ func TestTiDBNonPrepPlanCacheUpgradeFrom540To700(t *testing.T) { require.Equal(t, int64(ver54), ver) // We are now in 5.4, check TiDBCostModelVersion should not exist. - res := MustExecToRecodeSet(t, seV54, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableNonPreparedPlanCache)) + res := MustExecToRecodeSet(t, seV54, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBEnableNonPreparedPlanCache)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1684,7 +1685,7 @@ func TestTiDBNonPrepPlanCacheUpgradeFrom540To700(t *testing.T) { require.Equal(t, currentBootstrapVersion, ver) // We are now in 7.0 - res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableNonPreparedPlanCache)) + res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBEnableNonPreparedPlanCache)) chk = res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1693,7 +1694,7 @@ func TestTiDBNonPrepPlanCacheUpgradeFrom540To700(t *testing.T) { require.Equal(t, 2, row.Len()) require.Equal(t, "OFF", row.GetString(1)) // tidb_enable_non_prepared_plan_cache = off - res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBNonPreparedPlanCacheSize)) + res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBNonPreparedPlanCacheSize)) chk = res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1719,7 +1720,7 @@ func TestTiDBStatsLoadPseudoTimeoutUpgradeFrom610To650(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV61, ver61) - MustExec(t, seV61, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "0", variable.TiDBStatsLoadPseudoTimeout)) + MustExec(t, seV61, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "0", vardef.TiDBStatsLoadPseudoTimeout)) MustExec(t, seV61, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV61) @@ -1727,7 +1728,7 @@ func TestTiDBStatsLoadPseudoTimeoutUpgradeFrom610To650(t *testing.T) { require.Equal(t, int64(ver61), ver) // We are now in 6.1, tidb_stats_load_pseudo_timeout is OFF. - res := MustExecToRecodeSet(t, seV61, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBStatsLoadPseudoTimeout)) + res := MustExecToRecodeSet(t, seV61, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBStatsLoadPseudoTimeout)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1747,7 +1748,7 @@ func TestTiDBStatsLoadPseudoTimeoutUpgradeFrom610To650(t *testing.T) { require.Equal(t, currentBootstrapVersion, ver) // We are now in 6.5. - res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBStatsLoadPseudoTimeout)) + res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBStatsLoadPseudoTimeout)) chk = res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1855,7 +1856,7 @@ func TestTiDBLoadBasedReplicaReadThresholdUpgradingToVer141(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV70, ver70) - MustExec(t, seV70, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "0", variable.TiDBLoadBasedReplicaReadThreshold)) + MustExec(t, seV70, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "0", vardef.TiDBLoadBasedReplicaReadThreshold)) MustExec(t, seV70, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV70) @@ -1863,7 +1864,7 @@ func TestTiDBLoadBasedReplicaReadThresholdUpgradingToVer141(t *testing.T) { require.Equal(t, int64(ver70), ver) // We are now in 7.0, tidb_load_based_replica_read_threshold is 0. - res := MustExecToRecodeSet(t, seV70, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBLoadBasedReplicaReadThreshold)) + res := MustExecToRecodeSet(t, seV70, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBLoadBasedReplicaReadThreshold)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -1883,7 +1884,7 @@ func TestTiDBLoadBasedReplicaReadThresholdUpgradingToVer141(t *testing.T) { require.Equal(t, currentBootstrapVersion, ver) // We are now in 7.1. - res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBLoadBasedReplicaReadThreshold)) + res = MustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBLoadBasedReplicaReadThreshold)) chk = res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) @@ -2350,7 +2351,7 @@ func TestTiDBUpgradeToVer209(t *testing.T) { 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()) + require.Equal(t, false, vardef.EnableResourceControlStrictMode.Load()) } func TestTiDBUpgradeWithDistTaskEnable(t *testing.T) { @@ -2497,7 +2498,7 @@ func TestIndexJoinMultiPatternByUpgrade650To840(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) revertVersionAndVariables(t, seV7, ver650) - MustExec(t, seV7, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableINLJoinInnerMultiPattern)) + MustExec(t, seV7, fmt.Sprintf("delete from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBEnableINLJoinInnerMultiPattern)) MustExec(t, seV7, "commit") store.SetOption(StoreBootstrappedKey, nil) ver, err := getBootstrapVersion(seV7) @@ -2505,7 +2506,7 @@ func TestIndexJoinMultiPatternByUpgrade650To840(t *testing.T) { require.Equal(t, int64(ver650), ver) // We are now in 6.5.0, check tidb_enable_inl_join_inner_multi_pattern should not exist. - res := MustExecToRecodeSet(t, seV7, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableINLJoinInnerMultiPattern)) + res := MustExecToRecodeSet(t, seV7, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", vardef.TiDBEnableINLJoinInnerMultiPattern)) chk := res.NewChunk(nil) err = res.Next(ctx, chk) require.NoError(t, err) diff --git a/pkg/session/clusteredindextest/BUILD.bazel b/pkg/session/clusteredindextest/BUILD.bazel index a1d61c12e84be..a1d1e24f98235 100644 --- a/pkg/session/clusteredindextest/BUILD.bazel +++ b/pkg/session/clusteredindextest/BUILD.bazel @@ -15,7 +15,7 @@ go_test( "//pkg/config", "//pkg/kv", "//pkg/session", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/testkit/testmain", "//pkg/testkit/testsetup", diff --git a/pkg/session/clusteredindextest/clustered_index_test.go b/pkg/session/clusteredindextest/clustered_index_test.go index 8eb3e9dc2d4b5..134f2689c6382 100644 --- a/pkg/session/clusteredindextest/clustered_index_test.go +++ b/pkg/session/clusteredindextest/clustered_index_test.go @@ -21,7 +21,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -29,7 +29,7 @@ import ( func createTestKit(t *testing.T, store kv.Storage) *testkit.TestKit { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn return tk } diff --git a/pkg/session/nontransactional.go b/pkg/session/nontransactional.go index 8272d9d4e4806..b05b9ccce798e 100644 --- a/pkg/session/nontransactional.go +++ b/pkg/session/nontransactional.go @@ -34,7 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/resolve" session_metrics "github.com/pingcap/tidb/pkg/session/metrics" sessiontypes "github.com/pingcap/tidb/pkg/session/types" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" driver "github.com/pingcap/tidb/pkg/types/parser_driver" @@ -186,7 +186,7 @@ func checkConstraint(stmt *ast.NonTransactionalDMLStmt, se sessiontypes.Session) return errors.Errorf("non-transactional DML can only run in auto-commit mode. auto-commit:%v, inTxn:%v", se.GetSessionVars().IsAutocommit(), se.GetSessionVars().InTxn()) } - if variable.EnableBatchDML.Load() && sessVars.DMLBatchSize > 0 && (sessVars.BatchDelete || sessVars.BatchInsert) { + if vardef.EnableBatchDML.Load() && sessVars.DMLBatchSize > 0 && (sessVars.BatchDelete || sessVars.BatchInsert) { return errors.Errorf("can't run non-transactional DML with batch-dml") } diff --git a/pkg/session/schematest/BUILD.bazel b/pkg/session/schematest/BUILD.bazel index 8a6b85c2ac110..e478f8f69aa64 100644 --- a/pkg/session/schematest/BUILD.bazel +++ b/pkg/session/schematest/BUILD.bazel @@ -16,6 +16,7 @@ go_test( "//pkg/parser/ast", "//pkg/server", "//pkg/session", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/store/mockstore", "//pkg/tablecodec", diff --git a/pkg/session/schematest/schema_test.go b/pkg/session/schematest/schema_test.go index 383edf8c7cc7f..774f45d874b94 100644 --- a/pkg/session/schematest/schema_test.go +++ b/pkg/session/schematest/schema_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/server" "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/tablecodec" @@ -122,7 +123,7 @@ func TestTableReaderChunk(t *testing.T) { tk.Session().GetSessionVars().SetDistSQLScanConcurrency(1) tk.MustExec("set tidb_init_chunk_size = 2") defer func() { - tk.MustExec(fmt.Sprintf("set tidb_init_chunk_size = %d", variable.DefInitChunkSize)) + tk.MustExec(fmt.Sprintf("set tidb_init_chunk_size = %d", vardef.DefInitChunkSize)) }() rs, err := tk.Exec("select * from chk") require.NoError(t, err) @@ -386,7 +387,7 @@ func TestValidationRecursion(t *testing.T) { // This tests for a regression where GetGlobalSysVar() can not safely call the validation // function because it might cause infinite recursion. // See: https://github.com/pingcap/tidb/issues/30255 - sv := variable.SysVar{Scope: variable.ScopeGlobal, Name: "mynewsysvar", Value: "test", Validation: func(vars *variable.SessionVars, normalizedValue string, originalValue string, scope variable.ScopeFlag) (string, error) { + sv := variable.SysVar{Scope: vardef.ScopeGlobal, Name: "mynewsysvar", Value: "test", Validation: func(vars *variable.SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { return vars.GlobalVarsAccessor.GetGlobalSysVar("mynewsysvar") }} variable.RegisterSysVar(&sv) @@ -396,7 +397,7 @@ func TestValidationRecursion(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - val, err := sv.Validate(tk.Session().GetSessionVars(), "test2", variable.ScopeGlobal) + val, err := sv.Validate(tk.Session().GetSessionVars(), "test2", vardef.ScopeGlobal) require.NoError(t, err) require.Equal(t, "test", val) } diff --git a/pkg/session/session.go b/pkg/session/session.go index 1cdf1bb96a8d8..e3770a7772d97 100644 --- a/pkg/session/session.go +++ b/pkg/session/session.go @@ -90,6 +90,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/statistics/handle/syncload" @@ -391,10 +392,10 @@ func (s *session) SetCollation(coID int) error { // If new collations are enabled, switch to the default // collation if this one is not supported. co = collate.SubstituteMissingCollationToDefault(co) - for _, v := range variable.SetNamesVariables { + for _, v := range vardef.SetNamesVariables { terror.Log(s.sessionVars.SetSystemVarWithoutValidation(v, cs)) } - return s.sessionVars.SetSystemVarWithoutValidation(variable.CollationConnection, co) + return s.sessionVars.SetSystemVarWithoutValidation(vardef.CollationConnection, co) } func (s *session) GetSessionPlanCache() sessionctx.SessionPlanCache { @@ -404,7 +405,7 @@ func (s *session) GetSessionPlanCache() sessionctx.SessionPlanCache { } if s.sessionPlanCache == nil { // lazy construction s.sessionPlanCache = plannercore.NewLRUPlanCache(uint(s.GetSessionVars().SessionPlanCacheSize), - variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load(), s, false) + vardef.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load(), s, false) } return s.sessionPlanCache } @@ -507,7 +508,7 @@ func (s *session) doCommit(ctx context.Context) error { return nil } // check if the cluster is read-only - if !s.sessionVars.InRestrictedSQL && (variable.RestrictedReadOnly.Load() || variable.VarTiDBSuperReadOnly.Load()) { + if !s.sessionVars.InRestrictedSQL && (vardef.RestrictedReadOnly.Load() || vardef.VarTiDBSuperReadOnly.Load()) { // It is not internal SQL, and the cluster has one of RestrictedReadOnly or SuperReadOnly // We need to privilege check again: a privilege check occurred during planning, but we need // to prevent the case that a long running auto-commit statement is now trying to commit. @@ -1177,19 +1178,19 @@ func createSessionFunc(store kv.Storage) pools.Factory { if err != nil { return nil, err } - err = se.sessionVars.SetSystemVar(variable.AutoCommit, "1") + err = se.sessionVars.SetSystemVar(vardef.AutoCommit, "1") if err != nil { return nil, err } - err = se.sessionVars.SetSystemVar(variable.MaxExecutionTime, "0") + err = se.sessionVars.SetSystemVar(vardef.MaxExecutionTime, "0") if err != nil { return nil, errors.Trace(err) } - err = se.sessionVars.SetSystemVar(variable.MaxAllowedPacket, strconv.FormatUint(variable.DefMaxAllowedPacket, 10)) + err = se.sessionVars.SetSystemVar(vardef.MaxAllowedPacket, strconv.FormatUint(vardef.DefMaxAllowedPacket, 10)) if err != nil { return nil, errors.Trace(err) } - err = se.sessionVars.SetSystemVar(variable.TiDBConstraintCheckInPlacePessimistic, variable.On) + err = se.sessionVars.SetSystemVar(vardef.TiDBConstraintCheckInPlacePessimistic, vardef.On) if err != nil { return nil, errors.Trace(err) } @@ -1207,19 +1208,19 @@ func createSessionWithDomainFunc(store kv.Storage) func(*domain.Domain) (pools.R if err != nil { return nil, err } - err = se.sessionVars.SetSystemVar(variable.AutoCommit, "1") + err = se.sessionVars.SetSystemVar(vardef.AutoCommit, "1") if err != nil { return nil, err } - err = se.sessionVars.SetSystemVar(variable.MaxExecutionTime, "0") + err = se.sessionVars.SetSystemVar(vardef.MaxExecutionTime, "0") if err != nil { return nil, errors.Trace(err) } - err = se.sessionVars.SetSystemVar(variable.MaxAllowedPacket, strconv.FormatUint(variable.DefMaxAllowedPacket, 10)) + err = se.sessionVars.SetSystemVar(vardef.MaxAllowedPacket, strconv.FormatUint(vardef.DefMaxAllowedPacket, 10)) if err != nil { return nil, errors.Trace(err) } - err = se.sessionVars.SetSystemVar(variable.TiDBConstraintCheckInPlacePessimistic, variable.On) + err = se.sessionVars.SetSystemVar(vardef.TiDBConstraintCheckInPlacePessimistic, vardef.On) if err != nil { return nil, errors.Trace(err) } @@ -1313,7 +1314,7 @@ func (s *session) GetGlobalSysVar(name string) (string, error) { // See https://github.com/pingcap/tidb/issues/30255 for why we don't do full validation. // If validation fails, we should return the default value: // See: https://github.com/pingcap/tidb/pull/31566 - sysVar, err = sv.ValidateFromType(s.GetSessionVars(), sysVar, variable.ScopeGlobal) + sysVar, err = sv.ValidateFromType(s.GetSessionVars(), sysVar, vardef.ScopeGlobal) if err != nil { return sv.Value, nil } @@ -1327,7 +1328,7 @@ func (s *session) SetGlobalSysVar(ctx context.Context, name string, value string if sv == nil { return variable.ErrUnknownSystemVar.GenWithStackByArgs(name) } - if value, err = sv.Validate(s.sessionVars, value, variable.ScopeGlobal); err != nil { + if value, err = sv.Validate(s.sessionVars, value, vardef.ScopeGlobal); err != nil { return err } if err = sv.SetGlobalFromHook(ctx, s.sessionVars, value, false); err != nil { @@ -1826,7 +1827,7 @@ func (s *session) useCurrentSession(execOption sqlexec.ExecOption) (*session, fu var err error orgSnapshotInfoSchema, orgSnapshotTS := s.sessionVars.SnapshotInfoschema, s.sessionVars.SnapshotTS if execOption.SnapshotTS != 0 { - if err = s.sessionVars.SetSystemVar(variable.TiDBSnapshot, strconv.FormatUint(execOption.SnapshotTS, 10)); err != nil { + if err = s.sessionVars.SetSystemVar(vardef.TiDBSnapshot, strconv.FormatUint(execOption.SnapshotTS, 10)); err != nil { return nil, nil, err } s.sessionVars.SnapshotInfoschema, err = getSnapshotInfoSchema(s, execOption.SnapshotTS) @@ -1852,7 +1853,7 @@ func (s *session) useCurrentSession(execOption sqlexec.ExecOption) (*session, fu return s, func() { s.sessionVars.AnalyzeVersion = prevStatsVer s.sessionVars.EnableAnalyzeSnapshot = prevAnalyzeSnapshot - if err := s.sessionVars.SetSystemVar(variable.TiDBSnapshot, ""); err != nil { + if err := s.sessionVars.SetSystemVar(vardef.TiDBSnapshot, ""); err != nil { logutil.BgLogger().Error("set tidbSnapshot error", zap.Error(err)) } s.sessionVars.SnapshotInfoschema = orgSnapshotInfoSchema @@ -1883,7 +1884,7 @@ func (s *session) getInternalSession(execOption sqlexec.ExecOption) (*session, f se.sessionVars.SkipMissingPartitionStats = s.sessionVars.SkipMissingPartitionStats if execOption.SnapshotTS != 0 { - if err := se.sessionVars.SetSystemVar(variable.TiDBSnapshot, strconv.FormatUint(execOption.SnapshotTS, 10)); err != nil { + if err := se.sessionVars.SetSystemVar(vardef.TiDBSnapshot, strconv.FormatUint(execOption.SnapshotTS, 10)); err != nil { return nil, nil, err } se.sessionVars.SnapshotInfoschema, err = getSnapshotInfoSchema(s, execOption.SnapshotTS) @@ -1910,7 +1911,7 @@ func (s *session) getInternalSession(execOption sqlexec.ExecOption) (*session, f return se, func() { se.sessionVars.AnalyzeVersion = prevStatsVer se.sessionVars.EnableAnalyzeSnapshot = prevAnalyzeSnapshot - if err := se.sessionVars.SetSystemVar(variable.TiDBSnapshot, ""); err != nil { + if err := se.sessionVars.SetSystemVar(vardef.TiDBSnapshot, ""); err != nil { logutil.BgLogger().Error("set tidbSnapshot error", zap.Error(err)) } se.sessionVars.SnapshotInfoschema = nil @@ -2796,7 +2797,7 @@ func (s *session) Auth(user *auth.UserIdentity, authentication, salt []byte, aut return err } - if variable.EnableResourceControl.Load() && info.ResourceGroupName != "" { + if vardef.EnableResourceControl.Load() && info.ResourceGroupName != "" { s.sessionVars.SetResourceGroupName(info.ResourceGroupName) } @@ -3050,7 +3051,7 @@ func (s *session) MatchIdentity(ctx context.Context, username, remoteHost string var success bool var skipNameResolve bool var user = &auth.UserIdentity{} - varVal, err := s.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.SkipNameResolve) + varVal, err := s.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.SkipNameResolve) if err == nil && variable.TiDBOptOn(varVal) { skipNameResolve = true } @@ -3118,10 +3119,10 @@ func CreateSession4TestWithOpt(store kv.Storage, opt *Opt) (types.Session, error // initialize session variables for test. s.GetSessionVars().InitChunkSize = 2 s.GetSessionVars().MaxChunkSize = 32 - s.GetSessionVars().MinPagingSize = variable.DefMinPagingSize - s.GetSessionVars().EnablePaging = variable.DefTiDBEnablePaging + s.GetSessionVars().MinPagingSize = vardef.DefMinPagingSize + s.GetSessionVars().EnablePaging = vardef.DefTiDBEnablePaging s.GetSessionVars().StmtCtx.SetTimeZone(s.GetSessionVars().Location()) - err = s.GetSessionVars().SetSystemVarWithoutValidation(variable.CharacterSetConnection, "utf8mb4") + err = s.GetSessionVars().SetSystemVarWithoutValidation(vardef.CharacterSetConnection, "utf8mb4") } return s, err } @@ -3216,12 +3217,12 @@ var ( func splitAndScatterTable(store kv.Storage, tableIDs []int64) { if s, ok := store.(kv.SplittableStore); ok && atomic.LoadUint32(&ddl.EnableSplitTableRegion) == 1 { - ctxWithTimeout, cancel := context.WithTimeout(context.Background(), variable.DefWaitSplitRegionTimeout*time.Second) + ctxWithTimeout, cancel := context.WithTimeout(context.Background(), vardef.DefWaitSplitRegionTimeout*time.Second) var regionIDs []uint64 for _, id := range tableIDs { - regionIDs = append(regionIDs, ddl.SplitRecordRegion(ctxWithTimeout, s, id, id, variable.DefTiDBScatterRegion)) + regionIDs = append(regionIDs, ddl.SplitRecordRegion(ctxWithTimeout, s, id, id, vardef.DefTiDBScatterRegion)) } - if variable.DefTiDBScatterRegion != variable.ScatterOff { + if vardef.DefTiDBScatterRegion != vardef.ScatterOff { ddl.WaitScatterRegionFinish(ctxWithTimeout, s, regionIDs...) } cancel() @@ -3325,7 +3326,7 @@ func InitMDLVariableForBootstrap(store kv.Storage) error { if err != nil { return err } - variable.EnableMDL.Store(true) + vardef.EnableMDL.Store(true) return nil } @@ -3343,7 +3344,7 @@ func InitTiDBSchemaCacheSize(store kv.Storage) error { return errors.Trace(err) } if isNull { - size = variable.DefTiDBSchemaCacheSize + size = vardef.DefTiDBSchemaCacheSize return t.SetSchemaCacheSize(size) } return nil @@ -3351,7 +3352,7 @@ func InitTiDBSchemaCacheSize(store kv.Storage) error { if err != nil { return errors.Trace(err) } - variable.SchemaCacheSize.Store(size) + vardef.SchemaCacheSize.Store(size) return nil } @@ -3369,9 +3370,9 @@ func InitMDLVariableForUpgrade(store kv.Storage) (bool, error) { return nil }) if isNull || !enable { - variable.EnableMDL.Store(false) + vardef.EnableMDL.Store(false) } else { - variable.EnableMDL.Store(true) + vardef.EnableMDL.Store(true) } return isNull, err } @@ -3398,7 +3399,7 @@ func InitMDLVariable(store kv.Storage) error { } return nil }) - variable.EnableMDL.Store(enable) + vardef.EnableMDL.Store(enable) return err } @@ -3727,7 +3728,7 @@ func runInBootstrapSession(store kv.Storage, ver int64) { // For the bootstrap SQLs, the following variables should be compatible with old TiDB versions. // TODO we should have a createBootstrapSession to init those special variables. - s.sessionVars.EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + s.sessionVars.EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly s.SetValue(sessionctx.Initing, true) if startMode == ddl.Bootstrap { @@ -3936,7 +3937,7 @@ func (s *session) loadCommonGlobalVariablesIfNeeded() error { if s.Value(sessionctx.Initing) != nil { // When running bootstrap or upgrade, we should not access global storage. // But we need to init max_allowed_packet to use concat function during bootstrap or upgrade. - err := vars.SetSystemVar(variable.MaxAllowedPacket, strconv.FormatUint(variable.DefMaxAllowedPacket, 10)) + err := vars.SetSystemVar(vardef.MaxAllowedPacket, strconv.FormatUint(vardef.DefMaxAllowedPacket, 10)) if err != nil { logutil.BgLogger().Error("set system variable max_allowed_packet error", zap.Error(err)) } @@ -3963,8 +3964,8 @@ func (s *session) loadCommonGlobalVariablesIfNeeded() error { } // when client set Capability Flags CLIENT_INTERACTIVE, init wait_timeout with interactive_timeout if vars.ClientCapability&mysql.ClientInteractive > 0 { - if varVal, ok := vars.GetSystemVar(variable.InteractiveTimeout); ok { - if err := vars.SetSystemVar(variable.WaitTimeout, varVal); err != nil { + if varVal, ok := vars.GetSystemVar(vardef.InteractiveTimeout); ok { + if err := vars.SetSystemVar(vardef.WaitTimeout, varVal); err != nil { return err } } @@ -4126,7 +4127,7 @@ func logStmt(execStmt *executor.ExecStmt, s *session) { func logGeneralQuery(execStmt *executor.ExecStmt, s *session, isPrepared bool) { vars := s.GetSessionVars() - if variable.ProcessGeneralLog.Load() && !vars.InRestrictedSQL { + if vardef.ProcessGeneralLog.Load() && !vars.InRestrictedSQL { var query string if isPrepared { query = execStmt.OriginText() @@ -4525,7 +4526,7 @@ func (s *session) usePipelinedDmlOrWarn(ctx context.Context) bool { ) return false } - if (vars.BatchCommit || vars.BatchInsert || vars.BatchDelete) && vars.DMLBatchSize > 0 && variable.EnableBatchDML.Load() { + if (vars.BatchCommit || vars.BatchInsert || vars.BatchDelete) && vars.DMLBatchSize > 0 && vardef.EnableBatchDML.Load() { stmtCtx.AppendWarning(errors.New("Pipelined DML can not be used with the deprecated Batch DML. Fallback to standard mode")) return false } diff --git a/pkg/session/test/BUILD.bazel b/pkg/session/test/BUILD.bazel index cd560cb769d00..6dbe06bf455fd 100644 --- a/pkg/session/test/BUILD.bazel +++ b/pkg/session/test/BUILD.bazel @@ -21,6 +21,7 @@ go_test( "//pkg/parser/terror", "//pkg/planner/core", "//pkg/session", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/store/mockstore", "//pkg/tablecodec", diff --git a/pkg/session/test/session_test.go b/pkg/session/test/session_test.go index 7ee9aa2391f3a..d8e52ce58a821 100644 --- a/pkg/session/test/session_test.go +++ b/pkg/session/test/session_test.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" @@ -917,10 +918,10 @@ func TestBootstrapSQLWithExtension(t *testing.T) { extension.WithCustomAuthPlugins(authChecks), extension.WithCustomSysVariables([]*variable.SysVar{ { - Scope: variable.ScopeGlobal, + Scope: vardef.ScopeGlobal, Name: "extension_authentication_plugin", Value: mysql.AuthNativePassword, - Type: variable.TypeEnum, + Type: vardef.TypeEnum, PossibleValues: []string{authChecks[0].Name}, }, }), diff --git a/pkg/session/test/variable/BUILD.bazel b/pkg/session/test/variable/BUILD.bazel index 4f19705c2dc7a..70ed74553b56e 100644 --- a/pkg/session/test/variable/BUILD.bazel +++ b/pkg/session/test/variable/BUILD.bazel @@ -13,6 +13,7 @@ go_test( "//pkg/config", "//pkg/kv", "//pkg/session", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/store/copr", "//pkg/testkit", diff --git a/pkg/session/test/variable/variable_test.go b/pkg/session/test/variable/variable_test.go index 3d200251e2479..f07f73f886431 100644 --- a/pkg/session/test/variable/variable_test.go +++ b/pkg/session/test/variable/variable_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/store/copr" "github.com/pingcap/tidb/pkg/testkit" @@ -196,10 +197,10 @@ func TestCorrectScopeError(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeNone, Name: "sv_none", Value: "acdc"}) - variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeGlobal, Name: "sv_global", Value: "acdc"}) - variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeSession, Name: "sv_session", Value: "acdc"}) - variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeGlobal | variable.ScopeSession, Name: "sv_both", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: vardef.ScopeNone, Name: "sv_none", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: vardef.ScopeGlobal, Name: "sv_global", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: vardef.ScopeSession, Name: "sv_session", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "sv_both", Value: "acdc"}) // check set behavior @@ -417,8 +418,8 @@ func TestGeneralLogNonzeroTxnStartTS(t *testing.T) { defer func() { logutil.GeneralLogger = oldGL }() // enable general log - oldVar := variable.ProcessGeneralLog.Swap(true) - defer variable.ProcessGeneralLog.Store(oldVar) + oldVar := vardef.ProcessGeneralLog.Swap(true) + defer vardef.ProcessGeneralLog.Store(oldVar) store := testkit.CreateMockStore(t) diff --git a/pkg/session/test/vars/BUILD.bazel b/pkg/session/test/vars/BUILD.bazel index 6bacd56b34aa9..5aa77f30e7a38 100644 --- a/pkg/session/test/vars/BUILD.bazel +++ b/pkg/session/test/vars/BUILD.bazel @@ -14,6 +14,7 @@ go_test( "//pkg/domain", "//pkg/kv", "//pkg/parser/terror", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/testkit", "//pkg/testkit/testmain", diff --git a/pkg/session/test/vars/vars_test.go b/pkg/session/test/vars/vars_test.go index 8968ce7a65dcd..6f44b66faae56 100644 --- a/pkg/session/test/vars/vars_test.go +++ b/pkg/session/test/vars/vars_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" tikv "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/hint" @@ -69,7 +70,7 @@ func TestRemovedSysVars(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeGlobal | variable.ScopeSession, Name: "bogus_var", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "bogus_var", Value: "acdc"}) result := tk.MustQuery("SHOW GLOBAL VARIABLES LIKE 'bogus_var'") result.Check(testkit.Rows("bogus_var acdc")) result = tk.MustQuery("SELECT @@GLOBAL.bogus_var") @@ -249,7 +250,7 @@ func TestTimeZone(t *testing.T) { func TestGlobalVarAccessor(t *testing.T) { varName := "max_allowed_packet" - varValue := strconv.FormatUint(variable.DefMaxAllowedPacket, 10) // This is the default value for max_allowed_packet + varValue := strconv.FormatUint(vardef.DefMaxAllowedPacket, 10) // This is the default value for max_allowed_packet // The value of max_allowed_packet should be a multiple of 1024, // so the setting of varValue1 and varValue2 would be truncated to varValue0 diff --git a/pkg/session/tidb_test.go b/pkg/session/tidb_test.go index 6d306c76064f0..b521a8a2a8bfe 100644 --- a/pkg/session/tidb_test.go +++ b/pkg/session/tidb_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" @@ -87,7 +87,7 @@ func TestSchemaCacheSizeVar(t *testing.T) { m = meta.NewMutator(txn) size, isNull, err = m.GetSchemaCacheSize() require.NoError(t, err) - require.Equal(t, size, uint64(variable.DefTiDBSchemaCacheSize)) + require.Equal(t, size, uint64(vardef.DefTiDBSchemaCacheSize)) require.Equal(t, isNull, false) require.NoError(t, txn.Rollback()) } diff --git a/pkg/sessionctx/sessionstates/BUILD.bazel b/pkg/sessionctx/sessionstates/BUILD.bazel index 5451a8de47875..bfc585d809b93 100644 --- a/pkg/sessionctx/sessionstates/BUILD.bazel +++ b/pkg/sessionctx/sessionstates/BUILD.bazel @@ -40,7 +40,7 @@ go_test( "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/server", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/util", "//pkg/util/sem", diff --git a/pkg/sessionctx/sessionstates/session_states_test.go b/pkg/sessionctx/sessionstates/session_states_test.go index f1337448f3561..85309efc6b7fb 100644 --- a/pkg/sessionctx/sessionstates/session_states_test.go +++ b/pkg/sessionctx/sessionstates/session_states_test.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/server" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/sem" "github.com/stretchr/testify/require" @@ -105,37 +105,37 @@ func TestSystemVars(t *testing.T) { { // normal variable inSessionStates: true, - varName: variable.TiDBMaxTiFlashThreads, - expectedValue: strconv.Itoa(variable.DefTiFlashMaxThreads), + varName: vardef.TiDBMaxTiFlashThreads, + expectedValue: strconv.Itoa(vardef.DefTiFlashMaxThreads), }, { // hidden variable inSessionStates: true, - varName: variable.TiDBTxnReadTS, + varName: vardef.TiDBTxnReadTS, expectedValue: "", }, { // none-scoped variable inSessionStates: false, - varName: variable.DataDir, + varName: vardef.DataDir, expectedValue: "/usr/local/mysql/data/", }, { // instance-scoped variable inSessionStates: false, - varName: variable.TiDBGeneralLog, + varName: vardef.TiDBGeneralLog, expectedValue: "0", }, { // global-scoped variable inSessionStates: false, - varName: variable.TiDBAutoAnalyzeStartTime, - expectedValue: variable.DefAutoAnalyzeStartTime, + varName: vardef.TiDBAutoAnalyzeStartTime, + expectedValue: vardef.DefAutoAnalyzeStartTime, }, { // sem invisible variable inSessionStates: false, - varName: variable.TiDBConfig, + varName: vardef.TiDBConfig, }, { // noop variables @@ -152,25 +152,25 @@ func TestSystemVars(t *testing.T) { }, { inSessionStates: false, - varName: variable.Timestamp, + varName: vardef.Timestamp, }, { stmts: []string{"set timestamp=100"}, inSessionStates: true, - varName: variable.Timestamp, + varName: vardef.Timestamp, expectedValue: "100", }, { stmts: []string{"set rand_seed1=10000000, rand_seed2=1000000"}, inSessionStates: true, - varName: variable.RandSeed1, + varName: vardef.RandSeed1, checkStmt: "select rand()", expectedValue: "0.028870999839968048", }, { stmts: []string{"set rand_seed1=10000000, rand_seed2=1000000", "select rand()"}, inSessionStates: true, - varName: variable.RandSeed1, + varName: vardef.RandSeed1, checkStmt: "select rand()", expectedValue: "0.11641535266900002", }, @@ -182,7 +182,7 @@ func TestSystemVars(t *testing.T) { "set @@tidb_enforce_mpp=1", }, inSessionStates: true, - varName: variable.TiDBEnforceMPPExecution, + varName: vardef.TiDBEnforceMPPExecution, expectedValue: "1", }, { @@ -193,7 +193,7 @@ func TestSystemVars(t *testing.T) { "set @@tx_read_only=1", }, inSessionStates: true, - varName: variable.TxReadOnly, + varName: vardef.TxReadOnly, expectedValue: "1", }, } @@ -268,21 +268,21 @@ func TestInvisibleVars(t *testing.T) { // The value is changed and the user has the privilege. hasPriv: true, stmt: "set tidb_opt_write_row_id=true", - varName: variable.TiDBOptWriteRowID, + varName: vardef.TiDBOptWriteRowID, expectedValue: "1", }, { // The value has a global scope. hasPriv: true, stmt: "set tidb_row_format_version=1", - varName: variable.TiDBRowFormatVersion, + varName: vardef.TiDBRowFormatVersion, expectedValue: "1", }, { // The global value is changed, so the session value is still different with global. hasPriv: true, stmt: "set global tidb_row_format_version=1", - varName: variable.TiDBRowFormatVersion, + varName: vardef.TiDBRowFormatVersion, cleanStmt: "set global tidb_row_format_version=2", expectedValue: "2", }, diff --git a/pkg/sessionctx/vardef/BUILD.bazel b/pkg/sessionctx/vardef/BUILD.bazel new file mode 100644 index 0000000000000..7b10c9d7648e9 --- /dev/null +++ b/pkg/sessionctx/vardef/BUILD.bazel @@ -0,0 +1,21 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "vardef", + srcs = [ + "sysvar.go", + "tidb_vars.go", + ], + importpath = "github.com/pingcap/tidb/pkg/sessionctx/vardef", + visibility = ["//visibility:public"], + deps = [ + "//pkg/config", + "//pkg/executor/join/joinversion", + "//pkg/parser/mysql", + "//pkg/util/memory", + "//pkg/util/paging", + "//pkg/util/size", + "@com_github_pingcap_tipb//go-tipb", + "@org_uber_go_atomic//:atomic", + ], +) diff --git a/pkg/sessionctx/vardef/OWNERS b/pkg/sessionctx/vardef/OWNERS new file mode 100644 index 0000000000000..dc9645b9c8b79 --- /dev/null +++ b/pkg/sessionctx/vardef/OWNERS @@ -0,0 +1,8 @@ +# See the OWNERS docs at https://go.k8s.io/owners +options: + no_parent_owners: true +filters: + "(OWNERS|(sysvar|tidb_vars)\\.go)$": + approvers: + - sig-critical-approvers-tidb-server + diff --git a/pkg/sessionctx/vardef/sysvar.go b/pkg/sessionctx/vardef/sysvar.go new file mode 100644 index 0000000000000..756f563eda0db --- /dev/null +++ b/pkg/sessionctx/vardef/sysvar.go @@ -0,0 +1,342 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package vardef + +// SetNamesVariables is the system variable names related to set names statements. +var SetNamesVariables = []string{ + CharacterSetClient, + CharacterSetConnection, + CharacterSetResults, +} + +// SetCharsetVariables is the system variable names related to set charset statements. +var SetCharsetVariables = []string{ + CharacterSetClient, + CharacterSetResults, +} + +const ( + // MaskPwd is the mask of password for LDAP variables. + MaskPwd = "******" + + // PessimisticTxnMode is the name for tidb_txn_mode system variable. + PessimisticTxnMode = "pessimistic" + // OptimisticTxnMode is the name for tidb_txn_mode system variable. + OptimisticTxnMode = "optimistic" +) + +const ( + // CharacterSetConnection is the name for character_set_connection system variable. + CharacterSetConnection = "character_set_connection" + // CollationConnection is the name for collation_connection system variable. + CollationConnection = "collation_connection" + // CharsetDatabase is the name for character_set_database system variable. + CharsetDatabase = "character_set_database" + // CollationDatabase is the name for collation_database system variable. + CollationDatabase = "collation_database" + // CharacterSetFilesystem is the name for character_set_filesystem system variable. + CharacterSetFilesystem = "character_set_filesystem" + // CharacterSetClient is the name for character_set_client system variable. + CharacterSetClient = "character_set_client" + // CharacterSetSystem is the name for character_set_system system variable. + CharacterSetSystem = "character_set_system" + // GeneralLog is the name for 'general_log' system variable. + GeneralLog = "general_log" + // AvoidTemporalUpgrade is the name for 'avoid_temporal_upgrade' system variable. + AvoidTemporalUpgrade = "avoid_temporal_upgrade" + // MaxPreparedStmtCount is the name for 'max_prepared_stmt_count' system variable. + MaxPreparedStmtCount = "max_prepared_stmt_count" + // BigTables is the name for 'big_tables' system variable. + BigTables = "big_tables" + // CheckProxyUsers is the name for 'check_proxy_users' system variable. + CheckProxyUsers = "check_proxy_users" + // CoreFile is the name for 'core_file' system variable. + CoreFile = "core_file" + // DefaultWeekFormat is the name for 'default_week_format' system variable. + DefaultWeekFormat = "default_week_format" + // GroupConcatMaxLen is the name for 'group_concat_max_len' system variable. + GroupConcatMaxLen = "group_concat_max_len" + // DelayKeyWrite is the name for 'delay_key_write' system variable. + DelayKeyWrite = "delay_key_write" + // EndMarkersInJSON is the name for 'end_markers_in_json' system variable. + EndMarkersInJSON = "end_markers_in_json" + // Hostname is the name for 'hostname' system variable. + Hostname = "hostname" + // InnodbCommitConcurrency is the name for 'innodb_commit_concurrency' system variable. + InnodbCommitConcurrency = "innodb_commit_concurrency" + // InnodbFastShutdown is the name for 'innodb_fast_shutdown' system variable. + InnodbFastShutdown = "innodb_fast_shutdown" + // InnodbLockWaitTimeout is the name for 'innodb_lock_wait_timeout' system variable. + InnodbLockWaitTimeout = "innodb_lock_wait_timeout" + // MaxSortLength is the name for 'max_sort_length' system variable. + MaxSortLength = "max_sort_length" + // MaxSpRecursionDepth is the name for 'max_sp_recursion_depth' system variable. + MaxSpRecursionDepth = "max_sp_recursion_depth" + // MaxUserConnections is the name for 'max_user_connections' system variable. + MaxUserConnections = "max_user_connections" + // OfflineMode is the name for 'offline_mode' system variable. + OfflineMode = "offline_mode" + // InteractiveTimeout is the name for 'interactive_timeout' system variable. + InteractiveTimeout = "interactive_timeout" + // FlushTime is the name for 'flush_time' system variable. + FlushTime = "flush_time" + // PseudoSlaveMode is the name for 'pseudo_slave_mode' system variable. + PseudoSlaveMode = "pseudo_slave_mode" + // LowPriorityUpdates is the name for 'low_priority_updates' system variable. + LowPriorityUpdates = "low_priority_updates" + // LowerCaseTableNames is the name for 'lower_case_table_names' system variable. + LowerCaseTableNames = "lower_case_table_names" + // SessionTrackGtids is the name for 'session_track_gtids' system variable. + SessionTrackGtids = "session_track_gtids" + // OldPasswords is the name for 'old_passwords' system variable. + OldPasswords = "old_passwords" + // MaxConnections is the name for 'max_connections' system variable. + MaxConnections = "max_connections" + // SkipNameResolve is the name for 'skip_name_resolve' system variable. + SkipNameResolve = "skip_name_resolve" + // ForeignKeyChecks is the name for 'foreign_key_checks' system variable. + ForeignKeyChecks = "foreign_key_checks" + // SQLSafeUpdates is the name for 'sql_safe_updates' system variable. + SQLSafeUpdates = "sql_safe_updates" + // WarningCount is the name for 'warning_count' system variable. + WarningCount = "warning_count" + // ErrorCount is the name for 'error_count' system variable. + ErrorCount = "error_count" + // DefaultPasswordLifetime is the name for 'default_password_lifetime' system variable. + DefaultPasswordLifetime = "default_password_lifetime" + // DisconnectOnExpiredPassword is the name for 'disconnect_on_expired_password' system variable. + DisconnectOnExpiredPassword = "disconnect_on_expired_password" + // SQLSelectLimit is the name for 'sql_select_limit' system variable. + SQLSelectLimit = "sql_select_limit" + // MaxConnectErrors is the name for 'max_connect_errors' system variable. + MaxConnectErrors = "max_connect_errors" + // TableDefinitionCache is the name for 'table_definition_cache' system variable. + TableDefinitionCache = "table_definition_cache" + // Timestamp is the name for 'timestamp' system variable. + Timestamp = "timestamp" + // ConnectTimeout is the name for 'connect_timeout' system variable. + ConnectTimeout = "connect_timeout" + // SyncBinlog is the name for 'sync_binlog' system variable. + SyncBinlog = "sync_binlog" + // BlockEncryptionMode is the name for 'block_encryption_mode' system variable. + BlockEncryptionMode = "block_encryption_mode" + // WaitTimeout is the name for 'wait_timeout' system variable. + WaitTimeout = "wait_timeout" + // Version is the name of 'version' system variable. + Version = "version" + // VersionComment is the name of 'version_comment' system variable. + VersionComment = "version_comment" + // PluginDir is the name of 'plugin_dir' system variable. + PluginDir = "plugin_dir" + // PluginLoad is the name of 'plugin_load' system variable. + PluginLoad = "plugin_load" + // TiDBEnableDDL indicates whether the tidb-server campaigns the DDL owner, + TiDBEnableDDL = "tidb_enable_ddl" + // TiDBEnableStatsOwner indicates whether the tidb-server campaigns the Stats owner, + TiDBEnableStatsOwner = "tidb_enable_stats_owner" + // Port is the name for 'port' system variable. + Port = "port" + // DataDir is the name for 'datadir' system variable. + DataDir = "datadir" + // Profiling is the name for 'Profiling' system variable. + Profiling = "profiling" + // Socket is the name for 'socket' system variable. + Socket = "socket" + // BinlogOrderCommits is the name for 'binlog_order_commits' system variable. + BinlogOrderCommits = "binlog_order_commits" + // MasterVerifyChecksum is the name for 'master_verify_checksum' system variable. + MasterVerifyChecksum = "master_verify_checksum" + // SuperReadOnly is the name for 'super_read_only' system variable. + SuperReadOnly = "super_read_only" + // SQLNotes is the name for 'sql_notes' system variable. + SQLNotes = "sql_notes" + // SlaveCompressedProtocol is the name for 'slave_compressed_protocol' system variable. + SlaveCompressedProtocol = "slave_compressed_protocol" + // BinlogRowQueryLogEvents is the name for 'binlog_rows_query_log_events' system variable. + BinlogRowQueryLogEvents = "binlog_rows_query_log_events" + // LogSlowSlaveStatements is the name for 'log_slow_slave_statements' system variable. + LogSlowSlaveStatements = "log_slow_slave_statements" + // LogSlowAdminStatements is the name for 'log_slow_admin_statements' system variable. + LogSlowAdminStatements = "log_slow_admin_statements" + // LogQueriesNotUsingIndexes is the name for 'log_queries_not_using_indexes' system variable. + LogQueriesNotUsingIndexes = "log_queries_not_using_indexes" + // SQLAutoIsNull is the name for 'sql_auto_is_null' system variable. + SQLAutoIsNull = "sql_auto_is_null" + // RelayLogPurge is the name for 'relay_log_purge' system variable. + RelayLogPurge = "relay_log_purge" + // AutomaticSpPrivileges is the name for 'automatic_sp_privileges' system variable. + AutomaticSpPrivileges = "automatic_sp_privileges" + // SQLQuoteShowCreate is the name for 'sql_quote_show_create' system variable. + SQLQuoteShowCreate = "sql_quote_show_create" + // SlowQueryLog is the name for 'slow_query_log' system variable. + SlowQueryLog = "slow_query_log" + // BinlogDirectNonTransactionalUpdates is the name for 'binlog_direct_non_transactional_updates' system variable. + BinlogDirectNonTransactionalUpdates = "binlog_direct_non_transactional_updates" + // SQLBigSelects is the name for 'sql_big_selects' system variable. + SQLBigSelects = "sql_big_selects" + // LogBinTrustFunctionCreators is the name for 'log_bin_trust_function_creators' system variable. + LogBinTrustFunctionCreators = "log_bin_trust_function_creators" + // OldAlterTable is the name for 'old_alter_table' system variable. + OldAlterTable = "old_alter_table" + // EnforceGtidConsistency is the name for 'enforce_gtid_consistency' system variable. + EnforceGtidConsistency = "enforce_gtid_consistency" + // SecureAuth is the name for 'secure_auth' system variable. + SecureAuth = "secure_auth" + // UniqueChecks is the name for 'unique_checks' system variable. + UniqueChecks = "unique_checks" + // SQLWarnings is the name for 'sql_warnings' system variable. + SQLWarnings = "sql_warnings" + // AutoCommit is the name for 'autocommit' system variable. + AutoCommit = "autocommit" + // KeepFilesOnCreate is the name for 'keep_files_on_create' system variable. + KeepFilesOnCreate = "keep_files_on_create" + // ShowOldTemporals is the name for 'show_old_temporals' system variable. + ShowOldTemporals = "show_old_temporals" + // LocalInFile is the name for 'local_infile' system variable. + LocalInFile = "local_infile" + // PerformanceSchema is the name for 'performance_schema' system variable. + PerformanceSchema = "performance_schema" + // Flush is the name for 'flush' system variable. + Flush = "flush" + // SlaveAllowBatching is the name for 'slave_allow_batching' system variable. + SlaveAllowBatching = "slave_allow_batching" + // MyISAMUseMmap is the name for 'myisam_use_mmap' system variable. + MyISAMUseMmap = "myisam_use_mmap" + // InnodbFilePerTable is the name for 'innodb_file_per_table' system variable. + InnodbFilePerTable = "innodb_file_per_table" + // InnodbLogCompressedPages is the name for 'innodb_log_compressed_pages' system variable. + InnodbLogCompressedPages = "innodb_log_compressed_pages" + // InnodbPrintAllDeadlocks is the name for 'innodb_print_all_deadlocks' system variable. + InnodbPrintAllDeadlocks = "innodb_print_all_deadlocks" + // InnodbStrictMode is the name for 'innodb_strict_mode' system variable. + InnodbStrictMode = "innodb_strict_mode" + // InnodbCmpPerIndexEnabled is the name for 'innodb_cmp_per_index_enabled' system variable. + InnodbCmpPerIndexEnabled = "innodb_cmp_per_index_enabled" + // InnodbBufferPoolDumpAtShutdown is the name for 'innodb_buffer_pool_dump_at_shutdown' system variable. + InnodbBufferPoolDumpAtShutdown = "innodb_buffer_pool_dump_at_shutdown" + // InnodbAdaptiveHashIndex is the name for 'innodb_adaptive_hash_index' system variable. + InnodbAdaptiveHashIndex = "innodb_adaptive_hash_index" + // InnodbFtEnableStopword is the name for 'innodb_ft_enable_stopword' system variable. + InnodbFtEnableStopword = "innodb_ft_enable_stopword" // #nosec G101 + // InnodbOptimizeFullTextOnly is the name for 'innodb_optimize_fulltext_only' system variable. + InnodbOptimizeFullTextOnly = "innodb_optimize_fulltext_only" + // InnodbStatusOutputLocks is the name for 'innodb_status_output_locks' system variable. + InnodbStatusOutputLocks = "innodb_status_output_locks" + // InnodbBufferPoolDumpNow is the name for 'innodb_buffer_pool_dump_now' system variable. + InnodbBufferPoolDumpNow = "innodb_buffer_pool_dump_now" + // InnodbBufferPoolLoadNow is the name for 'innodb_buffer_pool_load_now' system variable. + InnodbBufferPoolLoadNow = "innodb_buffer_pool_load_now" + // InnodbStatsOnMetadata is the name for 'innodb_stats_on_metadata' system variable. + InnodbStatsOnMetadata = "innodb_stats_on_metadata" + // InnodbDisableSortFileCache is the name for 'innodb_disable_sort_file_cache' system variable. + InnodbDisableSortFileCache = "innodb_disable_sort_file_cache" + // InnodbStatsAutoRecalc is the name for 'innodb_stats_auto_recalc' system variable. + InnodbStatsAutoRecalc = "innodb_stats_auto_recalc" + // InnodbBufferPoolLoadAbort is the name for 'innodb_buffer_pool_load_abort' system variable. + InnodbBufferPoolLoadAbort = "innodb_buffer_pool_load_abort" + // InnodbStatsPersistent is the name for 'innodb_stats_persistent' system variable. + InnodbStatsPersistent = "innodb_stats_persistent" + // InnodbRandomReadAhead is the name for 'innodb_random_read_ahead' system variable. + InnodbRandomReadAhead = "innodb_random_read_ahead" + // InnodbAdaptiveFlushing is the name for 'innodb_adaptive_flushing' system variable. + InnodbAdaptiveFlushing = "innodb_adaptive_flushing" + // InnodbTableLocks is the name for 'innodb_table_locks' system variable. + InnodbTableLocks = "innodb_table_locks" + // InnodbStatusOutput is the name for 'innodb_status_output' system variable. + InnodbStatusOutput = "innodb_status_output" + // NetBufferLength is the name for 'net_buffer_length' system variable. + NetBufferLength = "net_buffer_length" + // TxReadOnly is the name of 'tx_read_only' system variable. + TxReadOnly = "tx_read_only" + // TransactionReadOnly is the name of 'transaction_read_only' system variable. + TransactionReadOnly = "transaction_read_only" + // CharacterSetServer is the name of 'character_set_server' system variable. + CharacterSetServer = "character_set_server" + // AutoIncrementIncrement is the name of 'auto_increment_increment' system variable. + AutoIncrementIncrement = "auto_increment_increment" + // AutoIncrementOffset is the name of 'auto_increment_offset' system variable. + AutoIncrementOffset = "auto_increment_offset" + // InitConnect is the name of 'init_connect' system variable. + InitConnect = "init_connect" + // CollationServer is the name of 'collation_server' variable. + CollationServer = "collation_server" + // DefaultCollationForUTF8MB4 is the name of 'default_collation_for_utf8mb4' variable. + DefaultCollationForUTF8MB4 = "default_collation_for_utf8mb4" + // NetWriteTimeout is the name of 'net_write_timeout' variable. + NetWriteTimeout = "net_write_timeout" + // ThreadPoolSize is the name of 'thread_pool_size' variable. + ThreadPoolSize = "thread_pool_size" + // WindowingUseHighPrecision is the name of 'windowing_use_high_precision' system variable. + WindowingUseHighPrecision = "windowing_use_high_precision" + // OptimizerSwitch is the name of 'optimizer_switch' system variable. + OptimizerSwitch = "optimizer_switch" + // SystemTimeZone is the name of 'system_time_zone' system variable. + SystemTimeZone = "system_time_zone" + // CTEMaxRecursionDepth is the name of 'cte_max_recursion_depth' system variable. + CTEMaxRecursionDepth = "cte_max_recursion_depth" + // SQLModeVar is the name of the 'sql_mode' system variable. + SQLModeVar = "sql_mode" + // CharacterSetResults is the name of the 'character_set_results' system variable. + CharacterSetResults = "character_set_results" + // MaxAllowedPacket is the name of the 'max_allowed_packet' system variable. + MaxAllowedPacket = "max_allowed_packet" + // TimeZone is the name of the 'time_zone' system variable. + TimeZone = "time_zone" + // TxnIsolation is the name of the 'tx_isolation' system variable. + TxnIsolation = "tx_isolation" + // TransactionIsolation is the name of the 'transaction_isolation' system variable. + TransactionIsolation = "transaction_isolation" + // TxnIsolationOneShot is the name of the 'tx_isolation_one_shot' system variable. + TxnIsolationOneShot = "tx_isolation_one_shot" + // MaxExecutionTime is the name of the 'max_execution_time' system variable. + MaxExecutionTime = "max_execution_time" + // TiKVClientReadTimeout is the name of the 'tikv_client_read_timeout' system variable. + TiKVClientReadTimeout = "tikv_client_read_timeout" + // TiDBLoadBindingTimeout is the name of the 'tidb_load_binding_timeout' system variable. + TiDBLoadBindingTimeout = "tidb_load_binding_timeout" + // ReadOnly is the name of the 'read_only' system variable. + ReadOnly = "read_only" + // DefaultAuthPlugin is the name of 'default_authentication_plugin' system variable. + DefaultAuthPlugin = "default_authentication_plugin" + // LastInsertID is the name of 'last_insert_id' system variable. + LastInsertID = "last_insert_id" + // Identity is the name of 'identity' system variable. + Identity = "identity" + // TiDBAllowFunctionForExpressionIndex is the name of `TiDBAllowFunctionForExpressionIndex` system variable. + TiDBAllowFunctionForExpressionIndex = "tidb_allow_function_for_expression_index" + // RandSeed1 is the name of 'rand_seed1' system variable. + RandSeed1 = "rand_seed1" + // RandSeed2 is the name of 'rand_seed2' system variable. + RandSeed2 = "rand_seed2" + // SQLRequirePrimaryKey is the name of `sql_require_primary_key` system variable. + SQLRequirePrimaryKey = "sql_require_primary_key" + // ValidatePasswordEnable turns on/off the validation of password. + ValidatePasswordEnable = "validate_password.enable" + // ValidatePasswordPolicy specifies the password policy enforced by validate_password. + ValidatePasswordPolicy = "validate_password.policy" + // ValidatePasswordCheckUserName controls whether validate_password compares passwords to the user name part of + // the effective user account for the current session + ValidatePasswordCheckUserName = "validate_password.check_user_name" + // ValidatePasswordLength specified the minimum number of characters that validate_password requires passwords to have + ValidatePasswordLength = "validate_password.length" + // ValidatePasswordMixedCaseCount specified the minimum number of lowercase and uppercase characters that validate_password requires + ValidatePasswordMixedCaseCount = "validate_password.mixed_case_count" + // ValidatePasswordNumberCount specified the minimum number of numeric (digit) characters that validate_password requires + ValidatePasswordNumberCount = "validate_password.number_count" + // ValidatePasswordSpecialCharCount specified the minimum number of nonalphanumeric characters that validate_password requires + ValidatePasswordSpecialCharCount = "validate_password.special_char_count" + // ValidatePasswordDictionary specified the dictionary that validate_password uses for checking passwords. Each word is separated by semicolon (;). + ValidatePasswordDictionary = "validate_password.dictionary" +) diff --git a/pkg/sessionctx/vardef/tidb_vars.go b/pkg/sessionctx/vardef/tidb_vars.go new file mode 100644 index 0000000000000..f96d78f582f4a --- /dev/null +++ b/pkg/sessionctx/vardef/tidb_vars.go @@ -0,0 +1,2014 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package vardef + +import ( + "fmt" + "math" + "strconv" + "strings" + goatomic "sync/atomic" + "time" + + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/executor/join/joinversion" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/util/memory" + "github.com/pingcap/tidb/pkg/util/paging" + "github.com/pingcap/tidb/pkg/util/size" + "github.com/pingcap/tipb/go-tipb" + "go.uber.org/atomic" +) + +/* + Steps to add a new TiDB specific system variable: + + 1. Add a new variable name with comment in this file. + 2. Add the default value of the new variable in this file. + 3. Add SysVar instance in 'defaultSysVars' slice. +*/ + +// TiDB system variable names that only in session scope. +const ( + TiDBDDLSlowOprThreshold = "ddl_slow_threshold" + + // TiDBSnapshot is used for reading history data, the default value is empty string. + // The value can be a datetime string like '2017-11-11 20:20:20' or a tso string. When this variable is set, the session reads history data of that time. + TiDBSnapshot = "tidb_snapshot" + + // TiDBOptAggPushDown is used to enable/disable the optimizer rule of aggregation push down. + TiDBOptAggPushDown = "tidb_opt_agg_push_down" + + // TiDBOptDeriveTopN is used to enable/disable the optimizer rule of deriving topN. + TiDBOptDeriveTopN = "tidb_opt_derive_topn" + + // TiDBOptCartesianBCJ is used to disable/enable broadcast cartesian join in MPP mode + TiDBOptCartesianBCJ = "tidb_opt_broadcast_cartesian_join" + + TiDBOptMPPOuterJoinFixedBuildSide = "tidb_opt_mpp_outer_join_fixed_build_side" + + // TiDBOptDistinctAggPushDown is used to decide whether agg with distinct should be pushed to tikv/tiflash. + TiDBOptDistinctAggPushDown = "tidb_opt_distinct_agg_push_down" + + // TiDBOptSkewDistinctAgg is used to indicate the distinct agg has data skew + TiDBOptSkewDistinctAgg = "tidb_opt_skew_distinct_agg" + + // TiDBOpt3StageDistinctAgg is used to indicate whether to plan and execute the distinct agg in 3 stages + TiDBOpt3StageDistinctAgg = "tidb_opt_three_stage_distinct_agg" + + // TiDBOptEnable3StageMultiDistinctAgg is used to indicate whether to plan and execute the multi distinct agg in 3 stages + TiDBOptEnable3StageMultiDistinctAgg = "tidb_opt_enable_three_stage_multi_distinct_agg" + + TiDBOptExplainNoEvaledSubQuery = "tidb_opt_enable_non_eval_scalar_subquery" + + // TiDBBCJThresholdSize is used to limit the size of small table for mpp broadcast join. + // Its unit is bytes, if the size of small table is larger than it, we will not use bcj. + TiDBBCJThresholdSize = "tidb_broadcast_join_threshold_size" + + // TiDBBCJThresholdCount is used to limit the count of small table for mpp broadcast join. + // If we can't estimate the size of one side of join child, we will check if its row number exceeds this limitation. + TiDBBCJThresholdCount = "tidb_broadcast_join_threshold_count" + + // TiDBPreferBCJByExchangeDataSize indicates the method used to choose mpp broadcast join + TiDBPreferBCJByExchangeDataSize = "tidb_prefer_broadcast_join_by_exchange_data_size" + + // TiDBOptWriteRowID is used to enable/disable the operations of insert、replace and update to _tidb_rowid. + TiDBOptWriteRowID = "tidb_opt_write_row_id" + + // TiDBAutoAnalyzeRatio will run if (table modify count)/(table row count) is greater than this value. + TiDBAutoAnalyzeRatio = "tidb_auto_analyze_ratio" + + // TiDBAutoAnalyzeStartTime will run if current time is within start time and end time. + TiDBAutoAnalyzeStartTime = "tidb_auto_analyze_start_time" + TiDBAutoAnalyzeEndTime = "tidb_auto_analyze_end_time" + + // TiDBChecksumTableConcurrency is used to speed up the ADMIN CHECKSUM TABLE + // statement, when a table has multiple indices, those indices can be + // scanned concurrently, with the cost of higher system performance impact. + TiDBChecksumTableConcurrency = "tidb_checksum_table_concurrency" + + // TiDBCurrentTS is used to get the current transaction timestamp. + // It is read-only. + TiDBCurrentTS = "tidb_current_ts" + + // TiDBLastTxnInfo is used to get the last transaction info within the current session. + TiDBLastTxnInfo = "tidb_last_txn_info" + + // TiDBLastQueryInfo is used to get the last query info within the current session. + TiDBLastQueryInfo = "tidb_last_query_info" + + // TiDBLastDDLInfo is used to get the last ddl info within the current session. + TiDBLastDDLInfo = "tidb_last_ddl_info" + + // TiDBLastPlanReplayerToken is used to get the last plan replayer token within the current session + TiDBLastPlanReplayerToken = "tidb_last_plan_replayer_token" + + // TiDBConfig is a read-only variable that shows the config of the current server. + TiDBConfig = "tidb_config" + + // TiDBBatchInsert is used to enable/disable auto-split insert data. If set this option on, insert executor will automatically + // insert data into multiple batches and use a single txn for each batch. This will be helpful when inserting large data. + TiDBBatchInsert = "tidb_batch_insert" + + // TiDBBatchDelete is used to enable/disable auto-split delete data. If set this option on, delete executor will automatically + // split data into multiple batches and use a single txn for each batch. This will be helpful when deleting large data. + TiDBBatchDelete = "tidb_batch_delete" + + // TiDBBatchCommit is used to enable/disable auto-split the transaction. + // If set this option on, the transaction will be committed when it reaches stmt-count-limit and starts a new transaction. + TiDBBatchCommit = "tidb_batch_commit" + + // TiDBDMLBatchSize is used to split the insert/delete data into small batches. + // It only takes effort when tidb_batch_insert/tidb_batch_delete is on. + // Its default value is 20000. When the row size is large, 20k rows could be larger than 100MB. + // User could change it to a smaller one to avoid breaking the transaction size limitation. + TiDBDMLBatchSize = "tidb_dml_batch_size" + + // The following session variables controls the memory quota during query execution. + + // TiDBMemQuotaQuery controls the memory quota of a query. + TiDBMemQuotaQuery = "tidb_mem_quota_query" // Bytes. + // TiDBMemQuotaApplyCache controls the memory quota of a query. + TiDBMemQuotaApplyCache = "tidb_mem_quota_apply_cache" + + // TiDBGeneralLog is used to log every query in the server in info level. + TiDBGeneralLog = "tidb_general_log" + + // TiDBLogFileMaxDays is used to log every query in the server in info level. + TiDBLogFileMaxDays = "tidb_log_file_max_days" + + // TiDBPProfSQLCPU is used to add label sql label to pprof result. + TiDBPProfSQLCPU = "tidb_pprof_sql_cpu" + + // TiDBRetryLimit is the maximum number of retries when committing a transaction. + TiDBRetryLimit = "tidb_retry_limit" + + // TiDBDisableTxnAutoRetry disables transaction auto retry. + // Deprecated: This variable is deprecated, please do not use this variable. + TiDBDisableTxnAutoRetry = "tidb_disable_txn_auto_retry" + + // TiDBEnableChunkRPC enables TiDB to use Chunk format for coprocessor requests. + TiDBEnableChunkRPC = "tidb_enable_chunk_rpc" + + // TiDBOptimizerSelectivityLevel is used to control the selectivity estimation level. + TiDBOptimizerSelectivityLevel = "tidb_optimizer_selectivity_level" + + // TiDBOptimizerEnableNewOnlyFullGroupByCheck is used to open the newly only_full_group_by check by maintaining functional dependency. + TiDBOptimizerEnableNewOnlyFullGroupByCheck = "tidb_enable_new_only_full_group_by_check" + + TiDBOptimizerEnableOuterJoinReorder = "tidb_enable_outer_join_reorder" + + // TiDBOptimizerEnableNAAJ is used to open the newly null-aware anti join + TiDBOptimizerEnableNAAJ = "tidb_enable_null_aware_anti_join" + + // TiDBTxnMode is used to control the transaction behavior. + TiDBTxnMode = "tidb_txn_mode" + + // TiDBRowFormatVersion is used to control tidb row format version current. + TiDBRowFormatVersion = "tidb_row_format_version" + + // TiDBEnableRowLevelChecksum is used to control whether to append checksum to row values. + TiDBEnableRowLevelChecksum = "tidb_enable_row_level_checksum" + + // TiDBEnableTablePartition is used to control table partition feature. + // The valid value include auto/on/off: + // on or auto: enable table partition if the partition type is implemented. + // off: always disable table partition. + TiDBEnableTablePartition = "tidb_enable_table_partition" + + // TiDBEnableListTablePartition is used to control list table partition feature. + // Deprecated: This variable is deprecated, please do not use this variable. + TiDBEnableListTablePartition = "tidb_enable_list_partition" + + // TiDBSkipIsolationLevelCheck is used to control whether to return error when set unsupported transaction + // isolation level. + TiDBSkipIsolationLevelCheck = "tidb_skip_isolation_level_check" + + // TiDBLowResolutionTSO is used for reading data with low resolution TSO which is updated once every two seconds + TiDBLowResolutionTSO = "tidb_low_resolution_tso" + + // TiDBReplicaRead is used for reading data from replicas, followers for example. + TiDBReplicaRead = "tidb_replica_read" + + // TiDBAdaptiveClosestReadThreshold is for reading data from closest replicas(with same 'zone' label). + // TiKV client should send read request to the closest replica(leader/follower) if the estimated response + // size exceeds this threshold; otherwise, this request should be sent to leader. + // This variable only take effect when `tidb_replica_read` is 'closest-adaptive'. + TiDBAdaptiveClosestReadThreshold = "tidb_adaptive_closest_read_threshold" + + // TiDBAllowRemoveAutoInc indicates whether a user can drop the auto_increment column attribute or not. + TiDBAllowRemoveAutoInc = "tidb_allow_remove_auto_inc" + + // TiDBMultiStatementMode enables multi statement at the risk of SQL injection + // provides backwards compatibility + TiDBMultiStatementMode = "tidb_multi_statement_mode" + + // TiDBEvolvePlanTaskMaxTime controls the max time of a single evolution task. + TiDBEvolvePlanTaskMaxTime = "tidb_evolve_plan_task_max_time" + + // TiDBEvolvePlanTaskStartTime is the start time of evolution task. + TiDBEvolvePlanTaskStartTime = "tidb_evolve_plan_task_start_time" + // TiDBEvolvePlanTaskEndTime is the end time of evolution task. + TiDBEvolvePlanTaskEndTime = "tidb_evolve_plan_task_end_time" + + // TiDBSlowLogThreshold is used to set the slow log threshold in the server. + TiDBSlowLogThreshold = "tidb_slow_log_threshold" + + // TiDBSlowTxnLogThreshold is used to set the slow transaction log threshold in the server. + TiDBSlowTxnLogThreshold = "tidb_slow_txn_log_threshold" + + // TiDBRecordPlanInSlowLog is used to log the plan of the slow query. + TiDBRecordPlanInSlowLog = "tidb_record_plan_in_slow_log" + + // TiDBEnableSlowLog enables TiDB to log slow queries. + TiDBEnableSlowLog = "tidb_enable_slow_log" + + // TiDBCheckMb4ValueInUTF8 is used to control whether to enable the check wrong utf8 value. + TiDBCheckMb4ValueInUTF8 = "tidb_check_mb4_value_in_utf8" + + // TiDBFoundInPlanCache indicates whether the last statement was found in plan cache + TiDBFoundInPlanCache = "last_plan_from_cache" + + // TiDBFoundInBinding indicates whether the last statement was matched with the hints in the binding. + TiDBFoundInBinding = "last_plan_from_binding" + + // TiDBAllowAutoRandExplicitInsert indicates whether explicit insertion on auto_random column is allowed. + TiDBAllowAutoRandExplicitInsert = "allow_auto_random_explicit_insert" + + // TiDBTxnScope indicates whether using global transactions or local transactions. + TiDBTxnScope = "txn_scope" + + // TiDBTxnReadTS indicates the next transaction should be staleness transaction and provide the startTS + TiDBTxnReadTS = "tx_read_ts" + + // TiDBReadStaleness indicates the staleness duration for following statement + TiDBReadStaleness = "tidb_read_staleness" + + // TiDBEnablePaging indicates whether paging is enabled in coprocessor requests. + TiDBEnablePaging = "tidb_enable_paging" + + // TiDBReadConsistency indicates whether the autocommit read statement goes through TiKV RC. + TiDBReadConsistency = "tidb_read_consistency" + + // TiDBSysdateIsNow is the name of the `tidb_sysdate_is_now` system variable + TiDBSysdateIsNow = "tidb_sysdate_is_now" + + // RequireSecureTransport indicates the secure mode for data transport + RequireSecureTransport = "require_secure_transport" + + // TiFlashFastScan indicates whether use fast scan in tiflash. + TiFlashFastScan = "tiflash_fastscan" + + // TiDBEnableUnsafeSubstitute indicates whether to enable generate column takes unsafe substitute. + TiDBEnableUnsafeSubstitute = "tidb_enable_unsafe_substitute" + + // TiDBEnableTiFlashReadForWriteStmt indicates whether to enable TiFlash to read for write statements. + TiDBEnableTiFlashReadForWriteStmt = "tidb_enable_tiflash_read_for_write_stmt" + + // TiDBUseAlloc indicates whether the last statement used chunk alloc + TiDBUseAlloc = "last_sql_use_alloc" + + // TiDBExplicitRequestSourceType indicates the source of the request, it's a complement of RequestSourceType. + // The value maybe "lightning", "br", "dumpling" etc. + TiDBExplicitRequestSourceType = "tidb_request_source_type" +) + +// TiDB system variable names that both in session and global scope. +const ( + // TiDBBuildStatsConcurrency specifies the number of concurrent workers used for analyzing tables or partitions. + // When multiple tables or partitions are specified in the analyze statement, TiDB will process them concurrently. + // Additionally, this setting controls the concurrency for building NDV (Number of Distinct Values) for special indexes, + // such as generated columns composed indexes. + TiDBBuildStatsConcurrency = "tidb_build_stats_concurrency" + + // TiDBBuildSamplingStatsConcurrency is used to control the concurrency of building stats using sampling. + // 1. The number of concurrent workers to merge FMSketches and Sample Data from different regions. + // 2. The number of concurrent workers to build TopN and Histogram concurrently. + TiDBBuildSamplingStatsConcurrency = "tidb_build_sampling_stats_concurrency" + + // TiDBDistSQLScanConcurrency is used to set the concurrency of a distsql scan task. + // A distsql scan task can be a table scan or a index scan, which may be distributed to many TiKV nodes. + // Higher concurrency may reduce latency, but with the cost of higher memory usage and system performance impact. + // If the query has a LIMIT clause, high concurrency makes the system do much more work than needed. + TiDBDistSQLScanConcurrency = "tidb_distsql_scan_concurrency" + + // TiDBAnalyzeDistSQLScanConcurrency is the number of concurrent workers to scan regions to collect statistics (FMSketch, Samples). + // For auto analyze, the value is controlled by tidb_sysproc_scan_concurrency variable. + TiDBAnalyzeDistSQLScanConcurrency = "tidb_analyze_distsql_scan_concurrency" + + // TiDBOptInSubqToJoinAndAgg is used to enable/disable the optimizer rule of rewriting IN subquery. + TiDBOptInSubqToJoinAndAgg = "tidb_opt_insubq_to_join_and_agg" + + // TiDBOptPreferRangeScan is used to enable/disable the optimizer to always prefer range scan over table scan, ignoring their costs. + TiDBOptPreferRangeScan = "tidb_opt_prefer_range_scan" + + // TiDBOptEnableCorrelationAdjustment is used to indicates if enable correlation adjustment. + TiDBOptEnableCorrelationAdjustment = "tidb_opt_enable_correlation_adjustment" + + // TiDBOptLimitPushDownThreshold determines if push Limit or TopN down to TiKV forcibly. + TiDBOptLimitPushDownThreshold = "tidb_opt_limit_push_down_threshold" + + // TiDBOptCorrelationThreshold is a guard to enable row count estimation using column order correlation. + TiDBOptCorrelationThreshold = "tidb_opt_correlation_threshold" + + // TiDBOptCorrelationExpFactor is an exponential factor to control heuristic approach when tidb_opt_correlation_threshold is not satisfied. + TiDBOptCorrelationExpFactor = "tidb_opt_correlation_exp_factor" + + // TiDBOptCPUFactor is the CPU cost of processing one expression for one row. + TiDBOptCPUFactor = "tidb_opt_cpu_factor" + // TiDBOptCopCPUFactor is the CPU cost of processing one expression for one row in coprocessor. + TiDBOptCopCPUFactor = "tidb_opt_copcpu_factor" + // TiDBOptTiFlashConcurrencyFactor is concurrency number of tiflash computation. + TiDBOptTiFlashConcurrencyFactor = "tidb_opt_tiflash_concurrency_factor" + // TiDBOptNetworkFactor is the network cost of transferring 1 byte data. + TiDBOptNetworkFactor = "tidb_opt_network_factor" + // TiDBOptScanFactor is the IO cost of scanning 1 byte data on TiKV. + TiDBOptScanFactor = "tidb_opt_scan_factor" + // TiDBOptDescScanFactor is the IO cost of scanning 1 byte data on TiKV in desc order. + TiDBOptDescScanFactor = "tidb_opt_desc_factor" + // TiDBOptSeekFactor is the IO cost of seeking the start value in a range on TiKV or TiFlash. + TiDBOptSeekFactor = "tidb_opt_seek_factor" + // TiDBOptMemoryFactor is the memory cost of storing one tuple. + TiDBOptMemoryFactor = "tidb_opt_memory_factor" + // TiDBOptDiskFactor is the IO cost of reading/writing one byte to temporary disk. + TiDBOptDiskFactor = "tidb_opt_disk_factor" + // TiDBOptConcurrencyFactor is the CPU cost of additional one goroutine. + TiDBOptConcurrencyFactor = "tidb_opt_concurrency_factor" + // TiDBOptForceInlineCTE is used to enable/disable inline CTE + TiDBOptForceInlineCTE = "tidb_opt_force_inline_cte" + + // TiDBIndexJoinBatchSize is used to set the batch size of an index lookup join. + // The index lookup join fetches batches of data from outer executor and constructs ranges for inner executor. + // This value controls how much of data in a batch to do the index join. + // Large value may reduce the latency but consumes more system resource. + TiDBIndexJoinBatchSize = "tidb_index_join_batch_size" + + // TiDBIndexLookupSize is used for index lookup executor. + // The index lookup executor first scan a batch of handles from a index, then use those handles to lookup the table + // rows, this value controls how much of handles in a batch to do a lookup task. + // Small value sends more RPCs to TiKV, consume more system resource. + // Large value may do more work than needed if the query has a limit. + TiDBIndexLookupSize = "tidb_index_lookup_size" + + // TiDBIndexLookupConcurrency is used for index lookup executor. + // A lookup task may have 'tidb_index_lookup_size' of handles at maximum, the handles may be distributed + // in many TiKV nodes, we execute multiple concurrent index lookup tasks concurrently to reduce the time + // waiting for a task to finish. + // Set this value higher may reduce the latency but consumes more system resource. + // tidb_index_lookup_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBIndexLookupConcurrency = "tidb_index_lookup_concurrency" + + // TiDBIndexLookupJoinConcurrency is used for index lookup join executor. + // IndexLookUpJoin starts "tidb_index_lookup_join_concurrency" inner workers + // to fetch inner rows and join the matched (outer, inner) row pairs. + // tidb_index_lookup_join_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBIndexLookupJoinConcurrency = "tidb_index_lookup_join_concurrency" + + // TiDBIndexSerialScanConcurrency is used for controlling the concurrency of index scan operation + // when we need to keep the data output order the same as the order of index data. + TiDBIndexSerialScanConcurrency = "tidb_index_serial_scan_concurrency" + + // TiDBMaxChunkSize is used to control the max chunk size during query execution. + TiDBMaxChunkSize = "tidb_max_chunk_size" + + // TiDBAllowBatchCop means if we should send batch coprocessor to TiFlash. It can be set to 0, 1 and 2. + // 0 means never use batch cop, 1 means use batch cop in case of aggregation and join, 2, means to force sending batch cop for any query. + // The default value is 0 + TiDBAllowBatchCop = "tidb_allow_batch_cop" + + // TiDBShardRowIDBits means all the tables created in the current session will be sharded. + // The default value is 0 + TiDBShardRowIDBits = "tidb_shard_row_id_bits" + + // TiDBPreSplitRegions means all the tables created in the current session will be pre-splited. + // The default value is 0 + TiDBPreSplitRegions = "tidb_pre_split_regions" + + // TiDBAllowMPPExecution means if we should use mpp way to execute query or not. + // Default value is `true`, means to be determined by the optimizer. + // Value set to `false` means never use mpp. + TiDBAllowMPPExecution = "tidb_allow_mpp" + + // TiDBAllowTiFlashCop means we only use MPP mode to query data. + // Default value is `true`, means to be determined by the optimizer. + // Value set to `false` means we may fall back to TiFlash cop plan if possible. + TiDBAllowTiFlashCop = "tidb_allow_tiflash_cop" + + // TiDBHashExchangeWithNewCollation means if hash exchange is supported when new collation is on. + // Default value is `true`, means support hash exchange when new collation is on. + // Value set to `false` means not support hash exchange when new collation is on. + TiDBHashExchangeWithNewCollation = "tidb_hash_exchange_with_new_collation" + + // TiDBEnforceMPPExecution means if we should enforce mpp way to execute query or not. + // Default value is `false`, means to be determined by variable `tidb_allow_mpp`. + // Value set to `true` means enforce use mpp. + // Note if you want to set `tidb_enforce_mpp` to `true`, you must set `tidb_allow_mpp` to `true` first. + TiDBEnforceMPPExecution = "tidb_enforce_mpp" + + // TiDBMaxTiFlashThreads is the maximum number of threads to execute the request which is pushed down to tiflash. + // Default value is -1, means it will not be pushed down to tiflash. + // If the value is bigger than -1, it will be pushed down to tiflash and used to create db context in tiflash. + TiDBMaxTiFlashThreads = "tidb_max_tiflash_threads" + + // TiDBMaxBytesBeforeTiFlashExternalJoin is the maximum bytes used by a TiFlash join before spill to disk + TiDBMaxBytesBeforeTiFlashExternalJoin = "tidb_max_bytes_before_tiflash_external_join" + + // TiDBMaxBytesBeforeTiFlashExternalGroupBy is the maximum bytes used by a TiFlash hash aggregation before spill to disk + TiDBMaxBytesBeforeTiFlashExternalGroupBy = "tidb_max_bytes_before_tiflash_external_group_by" + + // TiDBMaxBytesBeforeTiFlashExternalSort is the maximum bytes used by a TiFlash sort/TopN before spill to disk + TiDBMaxBytesBeforeTiFlashExternalSort = "tidb_max_bytes_before_tiflash_external_sort" + + // TiFlashMemQuotaQueryPerNode is the maximum bytes used by a TiFlash Query on each TiFlash node + TiFlashMemQuotaQueryPerNode = "tiflash_mem_quota_query_per_node" + + // TiFlashQuerySpillRatio is the threshold that TiFlash will trigger auto spill when the memory usage is above this percentage + TiFlashQuerySpillRatio = "tiflash_query_spill_ratio" + + // TiDBMPPStoreFailTTL is the unavailable time when a store is detected failed. During that time, tidb will not send any task to + // TiFlash even though the failed TiFlash node has been recovered. + TiDBMPPStoreFailTTL = "tidb_mpp_store_fail_ttl" + + // TiDBInitChunkSize is used to control the init chunk size during query execution. + TiDBInitChunkSize = "tidb_init_chunk_size" + + // TiDBMinPagingSize is used to control the min paging size in the coprocessor paging protocol. + TiDBMinPagingSize = "tidb_min_paging_size" + + // TiDBMaxPagingSize is used to control the max paging size in the coprocessor paging protocol. + TiDBMaxPagingSize = "tidb_max_paging_size" + + // TiDBEnableCascadesPlanner is used to control whether to enable the cascades planner. + TiDBEnableCascadesPlanner = "tidb_enable_cascades_planner" + + // TiDBSkipUTF8Check skips the UTF8 validate process, validate UTF8 has performance cost, if we can make sure + // the input string values are valid, we can skip the check. + TiDBSkipUTF8Check = "tidb_skip_utf8_check" + + // TiDBSkipASCIICheck skips the ASCII validate process + // old tidb may already have fields with invalid ASCII bytes + // disable ASCII validate can guarantee a safe replication + TiDBSkipASCIICheck = "tidb_skip_ascii_check" + + // TiDBHashJoinConcurrency is used for hash join executor. + // The hash join outer executor starts multiple concurrent join workers to probe the hash table. + // tidb_hash_join_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBHashJoinConcurrency = "tidb_hash_join_concurrency" + + // TiDBProjectionConcurrency is used for projection operator. + // This variable controls the worker number of projection operator. + // tidb_projection_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBProjectionConcurrency = "tidb_projection_concurrency" + + // TiDBHashAggPartialConcurrency is used for hash agg executor. + // The hash agg executor starts multiple concurrent partial workers to do partial aggregate works. + // tidb_hashagg_partial_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBHashAggPartialConcurrency = "tidb_hashagg_partial_concurrency" + + // TiDBHashAggFinalConcurrency is used for hash agg executor. + // The hash agg executor starts multiple concurrent final workers to do final aggregate works. + // tidb_hashagg_final_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBHashAggFinalConcurrency = "tidb_hashagg_final_concurrency" + + // TiDBWindowConcurrency is used for window parallel executor. + // tidb_window_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBWindowConcurrency = "tidb_window_concurrency" + + // TiDBMergeJoinConcurrency is used for merge join parallel executor + TiDBMergeJoinConcurrency = "tidb_merge_join_concurrency" + + // TiDBStreamAggConcurrency is used for stream aggregation parallel executor. + // tidb_stream_agg_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBStreamAggConcurrency = "tidb_streamagg_concurrency" + + // TiDBIndexMergeIntersectionConcurrency is used for parallel worker of index merge intersection. + TiDBIndexMergeIntersectionConcurrency = "tidb_index_merge_intersection_concurrency" + + // TiDBEnableParallelApply is used for parallel apply. + TiDBEnableParallelApply = "tidb_enable_parallel_apply" + + // TiDBBackoffLockFast is used for tikv backoff base time in milliseconds. + TiDBBackoffLockFast = "tidb_backoff_lock_fast" + + // TiDBBackOffWeight is used to control the max back off time in TiDB. + // The default maximum back off time is a small value. + // BackOffWeight could multiply it to let the user adjust the maximum time for retrying. + // Only positive integers can be accepted, which means that the maximum back off time can only grow. + TiDBBackOffWeight = "tidb_backoff_weight" + + // TiDBDDLReorgWorkerCount defines the count of ddl reorg workers. + TiDBDDLReorgWorkerCount = "tidb_ddl_reorg_worker_cnt" + + // TiDBDDLFlashbackConcurrency defines the count of ddl flashback workers. + TiDBDDLFlashbackConcurrency = "tidb_ddl_flashback_concurrency" + + // TiDBDDLReorgBatchSize defines the transaction batch size of ddl reorg workers. + TiDBDDLReorgBatchSize = "tidb_ddl_reorg_batch_size" + + // TiDBDDLErrorCountLimit defines the count of ddl error limit. + TiDBDDLErrorCountLimit = "tidb_ddl_error_count_limit" + + // TiDBDDLReorgPriority defines the operations' priority of adding indices. + // It can be: PRIORITY_LOW, PRIORITY_NORMAL, PRIORITY_HIGH + TiDBDDLReorgPriority = "tidb_ddl_reorg_priority" + + // TiDBDDLReorgMaxWriteSpeed defines the max write limitation for the lightning local backend + TiDBDDLReorgMaxWriteSpeed = "tidb_ddl_reorg_max_write_speed" + + // TiDBEnableAutoIncrementInGenerated disables the mysql compatibility check on using auto-incremented columns in + // expression indexes and generated columns described here https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html for details. + TiDBEnableAutoIncrementInGenerated = "tidb_enable_auto_increment_in_generated" + + // TiDBPlacementMode is used to control the mode for placement + TiDBPlacementMode = "tidb_placement_mode" + + // TiDBMaxDeltaSchemaCount defines the max length of deltaSchemaInfos. + // deltaSchemaInfos is a queue that maintains the history of schema changes. + TiDBMaxDeltaSchemaCount = "tidb_max_delta_schema_count" + + // TiDBScatterRegion will scatter the regions for DDLs when it is "table" or "global", "" indicates not trigger scatter. + TiDBScatterRegion = "tidb_scatter_region" + + // TiDBWaitSplitRegionFinish defines the split region behaviour is sync or async. + TiDBWaitSplitRegionFinish = "tidb_wait_split_region_finish" + + // TiDBWaitSplitRegionTimeout uses to set the split and scatter region back off time. + TiDBWaitSplitRegionTimeout = "tidb_wait_split_region_timeout" + + // TiDBForcePriority defines the operations' priority of all statements. + // It can be "NO_PRIORITY", "LOW_PRIORITY", "HIGH_PRIORITY", "DELAYED" + TiDBForcePriority = "tidb_force_priority" + + // TiDBConstraintCheckInPlace indicates to check the constraint when the SQL executing. + // It could hurt the performance of bulking insert when it is ON. + TiDBConstraintCheckInPlace = "tidb_constraint_check_in_place" + + // TiDBEnableWindowFunction is used to control whether to enable the window function. + TiDBEnableWindowFunction = "tidb_enable_window_function" + + // TiDBEnablePipelinedWindowFunction is used to control whether to use pipelined window function, it only works when tidb_enable_window_function = true. + TiDBEnablePipelinedWindowFunction = "tidb_enable_pipelined_window_function" + + // TiDBEnableStrictDoubleTypeCheck is used to control table field double type syntax check. + TiDBEnableStrictDoubleTypeCheck = "tidb_enable_strict_double_type_check" + + // TiDBOptProjectionPushDown is used to control whether to pushdown projection to coprocessor. + TiDBOptProjectionPushDown = "tidb_opt_projection_push_down" + + // TiDBEnableVectorizedExpression is used to control whether to enable the vectorized expression evaluation. + TiDBEnableVectorizedExpression = "tidb_enable_vectorized_expression" + + // TiDBOptJoinReorderThreshold defines the threshold less than which + // we'll choose a rather time-consuming algorithm to calculate the join order. + TiDBOptJoinReorderThreshold = "tidb_opt_join_reorder_threshold" + + // TiDBSlowQueryFile indicates which slow query log file for SLOW_QUERY table to parse. + TiDBSlowQueryFile = "tidb_slow_query_file" + + // TiDBEnableFastAnalyze indicates to use fast analyze. + // Deprecated: This variable is deprecated, please do not use this variable. + TiDBEnableFastAnalyze = "tidb_enable_fast_analyze" + + // TiDBExpensiveQueryTimeThreshold indicates the time threshold of expensive query. + TiDBExpensiveQueryTimeThreshold = "tidb_expensive_query_time_threshold" + + // TiDBExpensiveTxnTimeThreshold indicates the time threshold of expensive transaction. + TiDBExpensiveTxnTimeThreshold = "tidb_expensive_txn_time_threshold" + + // TiDBEnableIndexMerge indicates to generate IndexMergePath. + TiDBEnableIndexMerge = "tidb_enable_index_merge" + + // TiDBEnableNoopFuncs set true will enable using fake funcs(like get_lock release_lock) + TiDBEnableNoopFuncs = "tidb_enable_noop_functions" + + // TiDBEnableStmtSummary indicates whether the statement summary is enabled. + TiDBEnableStmtSummary = "tidb_enable_stmt_summary" + + // TiDBStmtSummaryInternalQuery indicates whether the statement summary contain internal query. + TiDBStmtSummaryInternalQuery = "tidb_stmt_summary_internal_query" + + // TiDBStmtSummaryRefreshInterval indicates the refresh interval in seconds for each statement summary. + TiDBStmtSummaryRefreshInterval = "tidb_stmt_summary_refresh_interval" + + // TiDBStmtSummaryHistorySize indicates the history size of each statement summary. + TiDBStmtSummaryHistorySize = "tidb_stmt_summary_history_size" + + // TiDBStmtSummaryMaxStmtCount indicates the max number of statements kept in memory. + TiDBStmtSummaryMaxStmtCount = "tidb_stmt_summary_max_stmt_count" + + // TiDBStmtSummaryMaxSQLLength indicates the max length of displayed normalized sql and sample sql. + TiDBStmtSummaryMaxSQLLength = "tidb_stmt_summary_max_sql_length" + + // TiDBIgnoreInlistPlanDigest enables TiDB to generate the same plan digest with SQL using different in-list arguments. + TiDBIgnoreInlistPlanDigest = "tidb_ignore_inlist_plan_digest" + + // TiDBCapturePlanBaseline indicates whether the capture of plan baselines is enabled. + TiDBCapturePlanBaseline = "tidb_capture_plan_baselines" + + // TiDBUsePlanBaselines indicates whether the use of plan baselines is enabled. + TiDBUsePlanBaselines = "tidb_use_plan_baselines" + + // TiDBEvolvePlanBaselines indicates whether the evolution of plan baselines is enabled. + TiDBEvolvePlanBaselines = "tidb_evolve_plan_baselines" + + // TiDBOptEnableFuzzyBinding indicates whether to enable the universal binding. + TiDBOptEnableFuzzyBinding = "tidb_opt_enable_fuzzy_binding" + + // TiDBEnableExtendedStats indicates whether the extended statistics feature is enabled. + TiDBEnableExtendedStats = "tidb_enable_extended_stats" + + // TiDBIsolationReadEngines indicates the tidb only read from the stores whose engine type is involved in IsolationReadEngines. + // Now, only support TiKV and TiFlash. + TiDBIsolationReadEngines = "tidb_isolation_read_engines" + + // TiDBStoreLimit indicates the limit of sending request to a store, 0 means without limit. + TiDBStoreLimit = "tidb_store_limit" + + // TiDBMetricSchemaStep indicates the step when query metric schema. + TiDBMetricSchemaStep = "tidb_metric_query_step" + + // TiDBCDCWriteSource indicates the following data is written by TiCDC if it is not 0. + TiDBCDCWriteSource = "tidb_cdc_write_source" + + // TiDBMetricSchemaRangeDuration indicates the range duration when query metric schema. + TiDBMetricSchemaRangeDuration = "tidb_metric_query_range_duration" + + // TiDBEnableCollectExecutionInfo indicates that whether execution info is collected. + TiDBEnableCollectExecutionInfo = "tidb_enable_collect_execution_info" + + // TiDBExecutorConcurrency is used for controlling the concurrency of all types of executors. + TiDBExecutorConcurrency = "tidb_executor_concurrency" + + // TiDBEnableClusteredIndex indicates if clustered index feature is enabled. + TiDBEnableClusteredIndex = "tidb_enable_clustered_index" + + // TiDBEnableGlobalIndex means if we could create an global index on a partition table or not. + // Deprecated, will always be ON + TiDBEnableGlobalIndex = "tidb_enable_global_index" + + // TiDBPartitionPruneMode indicates the partition prune mode used. + TiDBPartitionPruneMode = "tidb_partition_prune_mode" + + // TiDBRedactLog indicates that whether redact log. + TiDBRedactLog = "tidb_redact_log" + + // TiDBRestrictedReadOnly is meant for the cloud admin to toggle the cluster read only + TiDBRestrictedReadOnly = "tidb_restricted_read_only" + + // TiDBSuperReadOnly is tidb's variant of mysql's super_read_only, which has some differences from mysql's super_read_only. + TiDBSuperReadOnly = "tidb_super_read_only" + + // TiDBShardAllocateStep indicates the max size of continuous rowid shard in one transaction. + TiDBShardAllocateStep = "tidb_shard_allocate_step" + // TiDBEnableTelemetry indicates that whether usage data report to PingCAP is enabled. + // Deprecated: it is 'off' always since Telemetry has been removed from TiDB. + TiDBEnableTelemetry = "tidb_enable_telemetry" + + // TiDBMemoryUsageAlarmRatio indicates the alarm threshold when memory usage of the tidb-server exceeds. + TiDBMemoryUsageAlarmRatio = "tidb_memory_usage_alarm_ratio" + + // TiDBMemoryUsageAlarmKeepRecordNum indicates the number of saved alarm files. + TiDBMemoryUsageAlarmKeepRecordNum = "tidb_memory_usage_alarm_keep_record_num" + + // TiDBEnableRateLimitAction indicates whether enabled ratelimit action + TiDBEnableRateLimitAction = "tidb_enable_rate_limit_action" + + // TiDBEnableAsyncCommit indicates whether to enable the async commit feature. + TiDBEnableAsyncCommit = "tidb_enable_async_commit" + + // TiDBEnable1PC indicates whether to enable the one-phase commit feature. + TiDBEnable1PC = "tidb_enable_1pc" + + // TiDBGuaranteeLinearizability indicates whether to guarantee linearizability. + TiDBGuaranteeLinearizability = "tidb_guarantee_linearizability" + + // TiDBAnalyzeVersion indicates how tidb collects the analyzed statistics and how use to it. + TiDBAnalyzeVersion = "tidb_analyze_version" + + // TiDBAutoAnalyzePartitionBatchSize indicates the batch size for partition tables for auto analyze in dynamic mode + // Deprecated: This variable is deprecated, please do not use this variable. + TiDBAutoAnalyzePartitionBatchSize = "tidb_auto_analyze_partition_batch_size" + + // TiDBEnableIndexMergeJoin indicates whether to enable index merge join. + TiDBEnableIndexMergeJoin = "tidb_enable_index_merge_join" + + // TiDBTrackAggregateMemoryUsage indicates whether track the memory usage of aggregate function. + TiDBTrackAggregateMemoryUsage = "tidb_track_aggregate_memory_usage" + + // TiDBEnableExchangePartition indicates whether to enable exchange partition. + TiDBEnableExchangePartition = "tidb_enable_exchange_partition" + + // TiDBAllowFallbackToTiKV indicates the engine types whose unavailability triggers fallback to TiKV. + // Now we only support TiFlash. + TiDBAllowFallbackToTiKV = "tidb_allow_fallback_to_tikv" + + // TiDBEnableTopSQL indicates whether the top SQL is enabled. + TiDBEnableTopSQL = "tidb_enable_top_sql" + + // TiDBSourceID indicates the source ID of the TiDB server. + TiDBSourceID = "tidb_source_id" + + // TiDBTopSQLMaxTimeSeriesCount indicates the max number of statements been collected in each time series. + TiDBTopSQLMaxTimeSeriesCount = "tidb_top_sql_max_time_series_count" + + // TiDBTopSQLMaxMetaCount indicates the max capacity of the collect meta per second. + TiDBTopSQLMaxMetaCount = "tidb_top_sql_max_meta_count" + + // TiDBEnableLocalTxn indicates whether to enable Local Txn. + TiDBEnableLocalTxn = "tidb_enable_local_txn" + + // TiDBEnableMDL indicates whether to enable MDL. + TiDBEnableMDL = "tidb_enable_metadata_lock" + + // TiDBTSOClientBatchMaxWaitTime indicates the max value of the TSO Batch Wait interval time of PD client. + TiDBTSOClientBatchMaxWaitTime = "tidb_tso_client_batch_max_wait_time" + + // TiDBTxnCommitBatchSize is used to control the batch size of transaction commit related requests sent by TiDB to TiKV. + // If a single transaction has a large amount of writes, you can increase the batch size to improve the batch effect, + // setting too large will exceed TiKV's raft-entry-max-size limit and cause commit failure. + TiDBTxnCommitBatchSize = "tidb_txn_commit_batch_size" + + // TiDBEnableTSOFollowerProxy indicates whether to enable the TSO Follower Proxy feature of PD client. + TiDBEnableTSOFollowerProxy = "tidb_enable_tso_follower_proxy" + + // PDEnableFollowerHandleRegion indicates whether to enable the PD Follower handle region API. + PDEnableFollowerHandleRegion = "pd_enable_follower_handle_region" + + // TiDBEnableOrderedResultMode indicates if stabilize query results. + TiDBEnableOrderedResultMode = "tidb_enable_ordered_result_mode" + + // TiDBRemoveOrderbyInSubquery indicates whether to remove ORDER BY in subquery. + TiDBRemoveOrderbyInSubquery = "tidb_remove_orderby_in_subquery" + + // TiDBEnablePseudoForOutdatedStats indicates whether use pseudo for outdated stats + TiDBEnablePseudoForOutdatedStats = "tidb_enable_pseudo_for_outdated_stats" + + // TiDBRegardNULLAsPoint indicates whether regard NULL as point when optimizing + TiDBRegardNULLAsPoint = "tidb_regard_null_as_point" + + // TiDBTmpTableMaxSize indicates the max memory size of temporary tables. + TiDBTmpTableMaxSize = "tidb_tmp_table_max_size" + + // TiDBEnableLegacyInstanceScope indicates if instance scope can be set with SET SESSION. + TiDBEnableLegacyInstanceScope = "tidb_enable_legacy_instance_scope" + + // TiDBTableCacheLease indicates the read lock lease of a cached table. + TiDBTableCacheLease = "tidb_table_cache_lease" + + // TiDBStatsLoadSyncWait indicates the time sql execution will sync-wait for stats load. + TiDBStatsLoadSyncWait = "tidb_stats_load_sync_wait" + + // TiDBEnableMutationChecker indicates whether to check data consistency for mutations + TiDBEnableMutationChecker = "tidb_enable_mutation_checker" + // TiDBTxnAssertionLevel indicates how strict the assertion will be, which helps to detect and preventing data & + // index inconsistency problems. + TiDBTxnAssertionLevel = "tidb_txn_assertion_level" + + // TiDBIgnorePreparedCacheCloseStmt indicates whether to ignore close-stmt commands for prepared statements. + TiDBIgnorePreparedCacheCloseStmt = "tidb_ignore_prepared_cache_close_stmt" + + // TiDBEnableNewCostInterface is a internal switch to indicates whether to use the new cost calculation interface. + TiDBEnableNewCostInterface = "tidb_enable_new_cost_interface" + + // TiDBCostModelVersion is a internal switch to indicates the cost model version. + TiDBCostModelVersion = "tidb_cost_model_version" + + // TiDBIndexJoinDoubleReadPenaltyCostRate indicates whether to add some penalty cost to IndexJoin and how much of it. + // IndexJoin can cause plenty of extra double read tasks, which consume lots of resources and take a long time. + // Since the number of double read tasks is hard to estimated accurately, we leave this variable to let us can adjust this + // part of cost manually. + TiDBIndexJoinDoubleReadPenaltyCostRate = "tidb_index_join_double_read_penalty_cost_rate" + + // TiDBBatchPendingTiFlashCount indicates the maximum count of non-available TiFlash tables. + TiDBBatchPendingTiFlashCount = "tidb_batch_pending_tiflash_count" + + // TiDBQueryLogMaxLen is used to set the max length of the query in the log. + TiDBQueryLogMaxLen = "tidb_query_log_max_len" + + // TiDBEnableNoopVariables is used to indicate if noops appear in SHOW [GLOBAL] VARIABLES + TiDBEnableNoopVariables = "tidb_enable_noop_variables" + + // TiDBNonTransactionalIgnoreError is used to ignore error in non-transactional DMLs. + // When set to false, a non-transactional DML returns when it meets the first error. + // When set to true, a non-transactional DML finishes all batches even if errors are met in some batches. + TiDBNonTransactionalIgnoreError = "tidb_nontransactional_ignore_error" + + // Fine grained shuffle is disabled when TiFlashFineGrainedShuffleStreamCount is zero. + TiFlashFineGrainedShuffleStreamCount = "tiflash_fine_grained_shuffle_stream_count" + TiFlashFineGrainedShuffleBatchSize = "tiflash_fine_grained_shuffle_batch_size" + + // TiDBSimplifiedMetrics controls whether to unregister some unused metrics. + TiDBSimplifiedMetrics = "tidb_simplified_metrics" + + // TiDBMemoryDebugModeMinHeapInUse is used to set tidb memory debug mode trigger threshold. + // When set to 0, the function is disabled. + // When set to a negative integer, use memory debug mode to detect the issue of frequent allocation and release of memory. + // We do not actively trigger gc, and check whether the `tracker memory * (1+bias ratio) > heap in use` each 5s. + // When set to a positive integer, use memory debug mode to detect the issue of memory tracking inaccurate. + // We trigger runtime.GC() each 5s, and check whether the `tracker memory * (1+bias ratio) > heap in use`. + TiDBMemoryDebugModeMinHeapInUse = "tidb_memory_debug_mode_min_heap_inuse" + // TiDBMemoryDebugModeAlarmRatio is used set tidb memory debug mode bias ratio. Treat memory bias less than this ratio as noise. + TiDBMemoryDebugModeAlarmRatio = "tidb_memory_debug_mode_alarm_ratio" + + // TiDBEnableAnalyzeSnapshot indicates whether to read data on snapshot when collecting statistics. + // When set to false, ANALYZE reads the latest data. + // When set to true, ANALYZE reads data on the snapshot at the beginning of ANALYZE. + TiDBEnableAnalyzeSnapshot = "tidb_enable_analyze_snapshot" + + // TiDBDefaultStrMatchSelectivity controls some special cardinality estimation strategy for string match functions (like and regexp). + // When set to 0, Selectivity() will try to evaluate those functions with TopN and NULL in the stats to estimate, + // and the default selectivity and the selectivity for the histogram part will be 0.1. + // When set to (0, 1], Selectivity() will use the value of this variable as the default selectivity of those + // functions instead of the selectionFactor (0.8). + TiDBDefaultStrMatchSelectivity = "tidb_default_string_match_selectivity" + + // TiDBEnablePrepPlanCache indicates whether to enable prepared plan cache + TiDBEnablePrepPlanCache = "tidb_enable_prepared_plan_cache" + // TiDBPrepPlanCacheSize indicates the number of cached statements. + // This variable is deprecated, use tidb_session_plan_cache_size instead. + TiDBPrepPlanCacheSize = "tidb_prepared_plan_cache_size" + // TiDBEnablePrepPlanCacheMemoryMonitor indicates whether to enable prepared plan cache monitor + TiDBEnablePrepPlanCacheMemoryMonitor = "tidb_enable_prepared_plan_cache_memory_monitor" + + // TiDBEnableNonPreparedPlanCache indicates whether to enable non-prepared plan cache. + TiDBEnableNonPreparedPlanCache = "tidb_enable_non_prepared_plan_cache" + // TiDBEnableNonPreparedPlanCacheForDML indicates whether to enable non-prepared plan cache for DML statements. + TiDBEnableNonPreparedPlanCacheForDML = "tidb_enable_non_prepared_plan_cache_for_dml" + // TiDBNonPreparedPlanCacheSize controls the size of non-prepared plan cache. + // This variable is deprecated, use tidb_session_plan_cache_size instead. + TiDBNonPreparedPlanCacheSize = "tidb_non_prepared_plan_cache_size" + // TiDBPlanCacheMaxPlanSize controls the maximum size of a plan that can be cached. + TiDBPlanCacheMaxPlanSize = "tidb_plan_cache_max_plan_size" + // TiDBPlanCacheInvalidationOnFreshStats controls if plan cache will be invalidated automatically when + // related stats are analyzed after the plan cache is generated. + TiDBPlanCacheInvalidationOnFreshStats = "tidb_plan_cache_invalidation_on_fresh_stats" + // TiDBSessionPlanCacheSize controls the size of session plan cache. + TiDBSessionPlanCacheSize = "tidb_session_plan_cache_size" + + // TiDBEnableInstancePlanCache indicates whether to enable instance plan cache. + // If this variable is false, session-level plan cache will be used. + TiDBEnableInstancePlanCache = "tidb_enable_instance_plan_cache" + // TiDBInstancePlanCacheReservedPercentage indicates the percentage memory to evict. + TiDBInstancePlanCacheReservedPercentage = "tidb_instance_plan_cache_reserved_percentage" + // TiDBInstancePlanCacheMaxMemSize indicates the maximum memory size of instance plan cache. + TiDBInstancePlanCacheMaxMemSize = "tidb_instance_plan_cache_max_size" + + // TiDBConstraintCheckInPlacePessimistic controls whether to skip certain kinds of pessimistic locks. + TiDBConstraintCheckInPlacePessimistic = "tidb_constraint_check_in_place_pessimistic" + + // TiDBEnableForeignKey indicates whether to enable foreign key feature. + // TODO(crazycs520): remove this after foreign key GA. + TiDBEnableForeignKey = "tidb_enable_foreign_key" + + // TiDBOptRangeMaxSize is the max memory limit for ranges. When the optimizer estimates that the memory usage of complete + // ranges would exceed the limit, it chooses less accurate ranges such as full range. 0 indicates that there is no memory + // limit for ranges. + TiDBOptRangeMaxSize = "tidb_opt_range_max_size" + + // TiDBOptAdvancedJoinHint indicates whether the join method hint is compatible with join order hint. + TiDBOptAdvancedJoinHint = "tidb_opt_advanced_join_hint" + // TiDBOptUseInvisibleIndexes indicates whether to use invisible indexes. + TiDBOptUseInvisibleIndexes = "tidb_opt_use_invisible_indexes" + // TiDBAnalyzePartitionConcurrency is the number of concurrent workers to save statistics to the system tables. + TiDBAnalyzePartitionConcurrency = "tidb_analyze_partition_concurrency" + // TiDBMergePartitionStatsConcurrency indicates the concurrency when merge partition stats into global stats + TiDBMergePartitionStatsConcurrency = "tidb_merge_partition_stats_concurrency" + // TiDBEnableAsyncMergeGlobalStats indicates whether to enable async merge global stats + TiDBEnableAsyncMergeGlobalStats = "tidb_enable_async_merge_global_stats" + // TiDBOptPrefixIndexSingleScan indicates whether to do some optimizations to avoid double scan for prefix index. + // When set to true, `col is (not) null`(`col` is index prefix column) is regarded as index filter rather than table filter. + TiDBOptPrefixIndexSingleScan = "tidb_opt_prefix_index_single_scan" + + // TiDBEnableExternalTSRead indicates whether to enable read through an external ts + TiDBEnableExternalTSRead = "tidb_enable_external_ts_read" + + // TiDBEnablePlanReplayerCapture indicates whether to enable plan replayer capture + TiDBEnablePlanReplayerCapture = "tidb_enable_plan_replayer_capture" + + // TiDBEnablePlanReplayerContinuousCapture indicates whether to enable continuous capture + TiDBEnablePlanReplayerContinuousCapture = "tidb_enable_plan_replayer_continuous_capture" + // TiDBEnableReusechunk indicates whether to enable chunk alloc + TiDBEnableReusechunk = "tidb_enable_reuse_chunk" + + // TiDBStoreBatchSize indicates the batch size of coprocessor in the same store. + TiDBStoreBatchSize = "tidb_store_batch_size" + + // MppExchangeCompressionMode indicates the data compression method in mpp exchange operator + MppExchangeCompressionMode = "mpp_exchange_compression_mode" + + // MppVersion indicates the mpp-version used to build mpp plan + MppVersion = "mpp_version" + + // TiDBPessimisticTransactionFairLocking controls whether fair locking for pessimistic transaction + // is enabled. + TiDBPessimisticTransactionFairLocking = "tidb_pessimistic_txn_fair_locking" + + // TiDBEnablePlanCacheForParamLimit controls whether prepare statement with parameterized limit can be cached + TiDBEnablePlanCacheForParamLimit = "tidb_enable_plan_cache_for_param_limit" + + // TiDBEnableINLJoinInnerMultiPattern indicates whether enable multi pattern for inner side of inl join + TiDBEnableINLJoinInnerMultiPattern = "tidb_enable_inl_join_inner_multi_pattern" + + // TiFlashComputeDispatchPolicy indicates how to dispatch task to tiflash_compute nodes. + TiFlashComputeDispatchPolicy = "tiflash_compute_dispatch_policy" + + // TiDBEnablePlanCacheForSubquery controls whether prepare statement with subquery can be cached + TiDBEnablePlanCacheForSubquery = "tidb_enable_plan_cache_for_subquery" + + // TiDBOptEnableLateMaterialization indicates whether to enable late materialization + TiDBOptEnableLateMaterialization = "tidb_opt_enable_late_materialization" + // TiDBLoadBasedReplicaReadThreshold is the wait duration threshold to enable replica read automatically. + TiDBLoadBasedReplicaReadThreshold = "tidb_load_based_replica_read_threshold" + + // TiDBOptOrderingIdxSelThresh is the threshold for optimizer to consider the ordering index. + TiDBOptOrderingIdxSelThresh = "tidb_opt_ordering_index_selectivity_threshold" + + // TiDBOptOrderingIdxSelRatio is the ratio the optimizer will assume applies when non indexed filtering rows are found + // via the ordering index. + TiDBOptOrderingIdxSelRatio = "tidb_opt_ordering_index_selectivity_ratio" + + // TiDBOptEnableMPPSharedCTEExecution indicates whether the optimizer try to build shared CTE scan during MPP execution. + TiDBOptEnableMPPSharedCTEExecution = "tidb_opt_enable_mpp_shared_cte_execution" + // TiDBOptFixControl makes the user able to control some details of the optimizer behavior. + TiDBOptFixControl = "tidb_opt_fix_control" + + // TiFlashReplicaRead is used to set the policy of TiFlash replica read when the query needs the TiFlash engine. + TiFlashReplicaRead = "tiflash_replica_read" + + // TiDBLockUnchangedKeys indicates whether to lock duplicate keys in INSERT IGNORE and REPLACE statements, + // or unchanged unique keys in UPDATE statements, see PR #42210 and #42713 + TiDBLockUnchangedKeys = "tidb_lock_unchanged_keys" + + // TiDBFastCheckTable enables fast check table. + TiDBFastCheckTable = "tidb_enable_fast_table_check" + + // TiDBAnalyzeSkipColumnTypes indicates the column types whose statistics would not be collected when executing the ANALYZE command. + TiDBAnalyzeSkipColumnTypes = "tidb_analyze_skip_column_types" + + // TiDBEnableCheckConstraint indicates whether to enable check constraint feature. + TiDBEnableCheckConstraint = "tidb_enable_check_constraint" + + // TiDBOptEnableHashJoin indicates whether to enable hash join. + TiDBOptEnableHashJoin = "tidb_opt_enable_hash_join" + + // TiDBHashJoinVersion indicates whether to use hash join implementation v2. + TiDBHashJoinVersion = "tidb_hash_join_version" + + // TiDBOptObjective indicates whether the optimizer should be more stable, predictable or more aggressive. + // Please see comments of SessionVars.OptObjective for details. + TiDBOptObjective = "tidb_opt_objective" + + // TiDBEnableParallelHashaggSpill is the name of the `tidb_enable_parallel_hashagg_spill` system variable + TiDBEnableParallelHashaggSpill = "tidb_enable_parallel_hashagg_spill" + + // TiDBTxnEntrySizeLimit indicates the max size of a entry in membuf. + TiDBTxnEntrySizeLimit = "tidb_txn_entry_size_limit" + + // TiDBSchemaCacheSize indicates the size of infoschema meta data which are cached in V2 implementation. + TiDBSchemaCacheSize = "tidb_schema_cache_size" + + // DivPrecisionIncrement indicates the number of digits by which to increase the scale of the result of + // division operations performed with the / operator. + DivPrecisionIncrement = "div_precision_increment" + + // TiDBEnableSharedLockPromotion indicates whether the `select for share` statement would be executed + // as `select for update` statements which do acquire pessimistic locks. + TiDBEnableSharedLockPromotion = "tidb_enable_shared_lock_promotion" +) + +// TiDB vars that have only global scope +const ( + // TiDBGCEnable turns garbage collection on or OFF + TiDBGCEnable = "tidb_gc_enable" + // TiDBGCRunInterval sets the interval that GC runs + TiDBGCRunInterval = "tidb_gc_run_interval" + // TiDBGCLifetime sets the retention window of older versions + TiDBGCLifetime = "tidb_gc_life_time" + // TiDBGCConcurrency sets the concurrency of garbage collection. -1 = AUTO value + TiDBGCConcurrency = "tidb_gc_concurrency" + // TiDBGCScanLockMode enables the green GC feature (deprecated) + TiDBGCScanLockMode = "tidb_gc_scan_lock_mode" + // TiDBGCMaxWaitTime sets max time for gc advances the safepoint delayed by active transactions + TiDBGCMaxWaitTime = "tidb_gc_max_wait_time" + // TiDBEnableEnhancedSecurity restricts SUPER users from certain operations. + TiDBEnableEnhancedSecurity = "tidb_enable_enhanced_security" + // TiDBEnableHistoricalStats enables the historical statistics feature (default off) + TiDBEnableHistoricalStats = "tidb_enable_historical_stats" + // TiDBPersistAnalyzeOptions persists analyze options for later analyze and auto-analyze + TiDBPersistAnalyzeOptions = "tidb_persist_analyze_options" + // TiDBEnableColumnTracking enables collecting predicate columns. + // DEPRECATED: This variable is deprecated, please do not use this variable. + TiDBEnableColumnTracking = "tidb_enable_column_tracking" + // TiDBAnalyzeColumnOptions specifies the default column selection strategy for both manual and automatic analyze operations. + // It accepts two values: + // `PREDICATE`: Analyze only the columns that are used in the predicates of the query. + // `ALL`: Analyze all columns in the table. + TiDBAnalyzeColumnOptions = "tidb_analyze_column_options" + // TiDBDisableColumnTrackingTime records the last time TiDBEnableColumnTracking is set off. + // It is used to invalidate the collected predicate columns after turning off TiDBEnableColumnTracking, which avoids physical deletion. + // It doesn't have cache in memory, and we directly get/set the variable value from/to mysql.tidb. + // DEPRECATED: This variable is deprecated, please do not use this variable. + TiDBDisableColumnTrackingTime = "tidb_disable_column_tracking_time" + // TiDBStatsLoadPseudoTimeout indicates whether to fallback to pseudo stats after load timeout. + TiDBStatsLoadPseudoTimeout = "tidb_stats_load_pseudo_timeout" + // TiDBMemQuotaBindingCache indicates the memory quota for the bind cache. + TiDBMemQuotaBindingCache = "tidb_mem_quota_binding_cache" + // TiDBRCReadCheckTS indicates the tso optimization for read-consistency read is enabled. + TiDBRCReadCheckTS = "tidb_rc_read_check_ts" + // TiDBRCWriteCheckTs indicates whether some special write statements don't get latest tso from PD at RC + TiDBRCWriteCheckTs = "tidb_rc_write_check_ts" + // TiDBCommitterConcurrency controls the number of running concurrent requests in the commit phase. + TiDBCommitterConcurrency = "tidb_committer_concurrency" + // TiDBEnableBatchDML enables batch dml. + TiDBEnableBatchDML = "tidb_enable_batch_dml" + // TiDBStatsCacheMemQuota records stats cache quota + TiDBStatsCacheMemQuota = "tidb_stats_cache_mem_quota" + // TiDBMemQuotaAnalyze indicates the memory quota for all analyze jobs. + TiDBMemQuotaAnalyze = "tidb_mem_quota_analyze" + // TiDBEnableAutoAnalyze determines whether TiDB executes automatic analysis. + // In test, we disable it by default. See GlobalSystemVariableInitialValue for details. + TiDBEnableAutoAnalyze = "tidb_enable_auto_analyze" + // TiDBEnableAutoAnalyzePriorityQueue determines whether TiDB executes automatic analysis with priority queue. + TiDBEnableAutoAnalyzePriorityQueue = "tidb_enable_auto_analyze_priority_queue" + // TiDBMemOOMAction indicates what operation TiDB perform when a single SQL statement exceeds + // the memory quota specified by tidb_mem_quota_query and cannot be spilled to disk. + TiDBMemOOMAction = "tidb_mem_oom_action" + // TiDBPrepPlanCacheMemoryGuardRatio is used to prevent [performance.max-memory] from being exceeded + TiDBPrepPlanCacheMemoryGuardRatio = "tidb_prepared_plan_cache_memory_guard_ratio" + // TiDBMaxAutoAnalyzeTime is the max time that auto analyze can run. If auto analyze runs longer than the value, it + // will be killed. 0 indicates that there is no time limit. + TiDBMaxAutoAnalyzeTime = "tidb_max_auto_analyze_time" + // TiDBAutoAnalyzeConcurrency is the concurrency of the auto analyze + TiDBAutoAnalyzeConcurrency = "tidb_auto_analyze_concurrency" + // TiDBEnableDistTask indicates whether to enable the distributed execute background tasks(For example DDL, Import etc). + TiDBEnableDistTask = "tidb_enable_dist_task" + // TiDBEnableFastCreateTable indicates whether to enable the fast create table feature. + TiDBEnableFastCreateTable = "tidb_enable_fast_create_table" + // TiDBGenerateBinaryPlan indicates whether binary plan should be generated in slow log and statements summary. + TiDBGenerateBinaryPlan = "tidb_generate_binary_plan" + // TiDBEnableGCAwareMemoryTrack indicates whether to turn-on GC-aware memory track. + TiDBEnableGCAwareMemoryTrack = "tidb_enable_gc_aware_memory_track" + // TiDBEnableTmpStorageOnOOM controls whether to enable the temporary storage for some operators + // when a single SQL statement exceeds the memory quota specified by the memory quota. + TiDBEnableTmpStorageOnOOM = "tidb_enable_tmp_storage_on_oom" + // TiDBDDLEnableFastReorg indicates whether to use lighting backfill process for adding index. + TiDBDDLEnableFastReorg = "tidb_ddl_enable_fast_reorg" + // TiDBDDLDiskQuota used to set disk quota for lightning add index. + TiDBDDLDiskQuota = "tidb_ddl_disk_quota" + // TiDBCloudStorageURI used to set a cloud storage uri for ddl add index and import into. + TiDBCloudStorageURI = "tidb_cloud_storage_uri" + // TiDBAutoBuildStatsConcurrency is the number of concurrent workers to automatically analyze tables or partitions. + // It is very similar to the `tidb_build_stats_concurrency` variable, but it is used for the auto analyze feature. + TiDBAutoBuildStatsConcurrency = "tidb_auto_build_stats_concurrency" + // TiDBSysProcScanConcurrency is used to set the scan concurrency of for backend system processes, like auto-analyze. + // For now, it controls the number of concurrent workers to scan regions to collect statistics (FMSketch, Samples). + TiDBSysProcScanConcurrency = "tidb_sysproc_scan_concurrency" + // TiDBServerMemoryLimit indicates the memory limit of the tidb-server instance. + TiDBServerMemoryLimit = "tidb_server_memory_limit" + // TiDBServerMemoryLimitSessMinSize indicates the minimal memory used of a session, that becomes a candidate for session kill. + TiDBServerMemoryLimitSessMinSize = "tidb_server_memory_limit_sess_min_size" + // TiDBServerMemoryLimitGCTrigger indicates the gc percentage of the TiDBServerMemoryLimit. + TiDBServerMemoryLimitGCTrigger = "tidb_server_memory_limit_gc_trigger" + // TiDBEnableGOGCTuner is to enable GOGC tuner. it can tuner GOGC + TiDBEnableGOGCTuner = "tidb_enable_gogc_tuner" + // TiDBGOGCTunerThreshold is to control the threshold of GOGC tuner. + TiDBGOGCTunerThreshold = "tidb_gogc_tuner_threshold" + // TiDBGOGCTunerMaxValue is the max value of GOGC that GOGC tuner can change to. + TiDBGOGCTunerMaxValue = "tidb_gogc_tuner_max_value" + // TiDBGOGCTunerMinValue is the min value of GOGC that GOGC tuner can change to. + TiDBGOGCTunerMinValue = "tidb_gogc_tuner_min_value" + // TiDBExternalTS is the ts to read through when the `TiDBEnableExternalTsRead` is on + TiDBExternalTS = "tidb_external_ts" + // TiDBTTLJobEnable is used to enable/disable scheduling ttl job + TiDBTTLJobEnable = "tidb_ttl_job_enable" + // TiDBTTLScanBatchSize is used to control the batch size in the SELECT statement for TTL jobs + TiDBTTLScanBatchSize = "tidb_ttl_scan_batch_size" + // TiDBTTLDeleteBatchSize is used to control the batch size in the DELETE statement for TTL jobs + TiDBTTLDeleteBatchSize = "tidb_ttl_delete_batch_size" + // TiDBTTLDeleteRateLimit is used to control the delete rate limit for TTL jobs in each node + TiDBTTLDeleteRateLimit = "tidb_ttl_delete_rate_limit" + // TiDBTTLJobScheduleWindowStartTime is used to restrict the start time of the time window of scheduling the ttl jobs. + TiDBTTLJobScheduleWindowStartTime = "tidb_ttl_job_schedule_window_start_time" + // TiDBTTLJobScheduleWindowEndTime is used to restrict the end time of the time window of scheduling the ttl jobs. + TiDBTTLJobScheduleWindowEndTime = "tidb_ttl_job_schedule_window_end_time" + // TiDBTTLScanWorkerCount indicates the count of the scan workers in each TiDB node + TiDBTTLScanWorkerCount = "tidb_ttl_scan_worker_count" + // TiDBTTLDeleteWorkerCount indicates the count of the delete workers in each TiDB node + TiDBTTLDeleteWorkerCount = "tidb_ttl_delete_worker_count" + // PasswordReuseHistory limit a few passwords to reuse. + PasswordReuseHistory = "password_history" + // PasswordReuseTime limit how long passwords can be reused. + PasswordReuseTime = "password_reuse_interval" + // TiDBHistoricalStatsDuration indicates the duration to remain tidb historical stats + TiDBHistoricalStatsDuration = "tidb_historical_stats_duration" + // TiDBEnableHistoricalStatsForCapture indicates whether use historical stats in plan replayer capture + 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. + TiDBStmtSummaryFilename = "tidb_stmt_summary_filename" + // TiDBStmtSummaryFileMaxDays indicates how many days the files written by stmtsummary will be kept. + TiDBStmtSummaryFileMaxDays = "tidb_stmt_summary_file_max_days" + // TiDBStmtSummaryFileMaxSize indicates the maximum size (in mb) of a single file written by stmtsummary. + TiDBStmtSummaryFileMaxSize = "tidb_stmt_summary_file_max_size" + // TiDBStmtSummaryFileMaxBackups indicates the maximum number of files written by stmtsummary. + TiDBStmtSummaryFileMaxBackups = "tidb_stmt_summary_file_max_backups" + // TiDBTTLRunningTasks limits the count of running ttl tasks. Default to 0, means 3 times the count of TiKV (or no + // limitation, if the storage is not TiKV). + TiDBTTLRunningTasks = "tidb_ttl_running_tasks" + // AuthenticationLDAPSASLAuthMethodName defines the authentication method used by LDAP SASL authentication plugin + AuthenticationLDAPSASLAuthMethodName = "authentication_ldap_sasl_auth_method_name" + // AuthenticationLDAPSASLCAPath defines the ca certificate to verify LDAP connection in LDAP SASL authentication plugin + AuthenticationLDAPSASLCAPath = "authentication_ldap_sasl_ca_path" + // AuthenticationLDAPSASLTLS defines whether to use TLS connection in LDAP SASL authentication plugin + AuthenticationLDAPSASLTLS = "authentication_ldap_sasl_tls" + // AuthenticationLDAPSASLServerHost defines the server host of LDAP server for LDAP SASL authentication plugin + AuthenticationLDAPSASLServerHost = "authentication_ldap_sasl_server_host" + // AuthenticationLDAPSASLServerPort defines the port of LDAP server for LDAP SASL authentication plugin + AuthenticationLDAPSASLServerPort = "authentication_ldap_sasl_server_port" + // AuthenticationLDAPSASLReferral defines whether to enable LDAP referral for LDAP SASL authentication plugin + AuthenticationLDAPSASLReferral = "authentication_ldap_sasl_referral" + // AuthenticationLDAPSASLUserSearchAttr defines the attribute of username in LDAP server + AuthenticationLDAPSASLUserSearchAttr = "authentication_ldap_sasl_user_search_attr" + // AuthenticationLDAPSASLBindBaseDN defines the `dn` to search the users in. It's used to limit the search scope of TiDB. + AuthenticationLDAPSASLBindBaseDN = "authentication_ldap_sasl_bind_base_dn" + // AuthenticationLDAPSASLBindRootDN defines the `dn` of the user to login the LDAP server and perform search. + AuthenticationLDAPSASLBindRootDN = "authentication_ldap_sasl_bind_root_dn" + // AuthenticationLDAPSASLBindRootPWD defines the password of the user to login the LDAP server and perform search. + AuthenticationLDAPSASLBindRootPWD = "authentication_ldap_sasl_bind_root_pwd" + // AuthenticationLDAPSASLInitPoolSize defines the init size of connection pool to LDAP server for SASL plugin. + AuthenticationLDAPSASLInitPoolSize = "authentication_ldap_sasl_init_pool_size" + // AuthenticationLDAPSASLMaxPoolSize defines the max size of connection pool to LDAP server for SASL plugin. + AuthenticationLDAPSASLMaxPoolSize = "authentication_ldap_sasl_max_pool_size" + // AuthenticationLDAPSimpleAuthMethodName defines the authentication method used by LDAP Simple authentication plugin + AuthenticationLDAPSimpleAuthMethodName = "authentication_ldap_simple_auth_method_name" + // AuthenticationLDAPSimpleCAPath defines the ca certificate to verify LDAP connection in LDAP Simple authentication plugin + AuthenticationLDAPSimpleCAPath = "authentication_ldap_simple_ca_path" + // AuthenticationLDAPSimpleTLS defines whether to use TLS connection in LDAP Simple authentication plugin + AuthenticationLDAPSimpleTLS = "authentication_ldap_simple_tls" + // AuthenticationLDAPSimpleServerHost defines the server host of LDAP server for LDAP Simple authentication plugin + AuthenticationLDAPSimpleServerHost = "authentication_ldap_simple_server_host" + // AuthenticationLDAPSimpleServerPort defines the port of LDAP server for LDAP Simple authentication plugin + AuthenticationLDAPSimpleServerPort = "authentication_ldap_simple_server_port" + // AuthenticationLDAPSimpleReferral defines whether to enable LDAP referral for LDAP Simple authentication plugin + AuthenticationLDAPSimpleReferral = "authentication_ldap_simple_referral" + // AuthenticationLDAPSimpleUserSearchAttr defines the attribute of username in LDAP server + AuthenticationLDAPSimpleUserSearchAttr = "authentication_ldap_simple_user_search_attr" + // AuthenticationLDAPSimpleBindBaseDN defines the `dn` to search the users in. It's used to limit the search scope of TiDB. + AuthenticationLDAPSimpleBindBaseDN = "authentication_ldap_simple_bind_base_dn" + // AuthenticationLDAPSimpleBindRootDN defines the `dn` of the user to login the LDAP server and perform search. + AuthenticationLDAPSimpleBindRootDN = "authentication_ldap_simple_bind_root_dn" + // AuthenticationLDAPSimpleBindRootPWD defines the password of the user to login the LDAP server and perform search. + AuthenticationLDAPSimpleBindRootPWD = "authentication_ldap_simple_bind_root_pwd" + // AuthenticationLDAPSimpleInitPoolSize defines the init size of connection pool to LDAP server for SASL plugin. + AuthenticationLDAPSimpleInitPoolSize = "authentication_ldap_simple_init_pool_size" + // AuthenticationLDAPSimpleMaxPoolSize defines the max size of connection pool to LDAP server for SASL plugin. + AuthenticationLDAPSimpleMaxPoolSize = "authentication_ldap_simple_max_pool_size" + // TiDBRuntimeFilterTypeName the value of is string, a runtime filter type list split by ",", such as: "IN,MIN_MAX" + TiDBRuntimeFilterTypeName = "tidb_runtime_filter_type" + // TiDBRuntimeFilterModeName the mode of runtime filter, such as "OFF", "LOCAL" + TiDBRuntimeFilterModeName = "tidb_runtime_filter_mode" + // TiDBSkipMissingPartitionStats controls how to handle missing partition stats when merging partition stats to global stats. + // When set to true, skip missing partition stats and continue to merge other partition stats to global stats. + // When set to false, give up merging partition stats to global stats. + TiDBSkipMissingPartitionStats = "tidb_skip_missing_partition_stats" + // TiDBSessionAlias indicates the alias of a session which is used for tracing. + TiDBSessionAlias = "tidb_session_alias" + // TiDBServiceScope indicates the role for tidb for distributed task framework. + TiDBServiceScope = "tidb_service_scope" + // TiDBSchemaVersionCacheLimit defines the capacity size of domain infoSchema cache. + TiDBSchemaVersionCacheLimit = "tidb_schema_version_cache_limit" + // TiDBEnableTiFlashPipelineMode means if we should use pipeline model to execute query or not in tiflash. + // It's deprecated and setting it will not have any effect. + TiDBEnableTiFlashPipelineMode = "tidb_enable_tiflash_pipeline_model" + // TiDBIdleTransactionTimeout indicates the maximum time duration a transaction could be idle, unit is second. + // Any idle transaction will be killed after being idle for `tidb_idle_transaction_timeout` seconds. + // This is similar to https://docs.percona.com/percona-server/5.7/management/innodb_kill_idle_trx.html and https://mariadb.com/kb/en/transaction-timeouts/ + TiDBIdleTransactionTimeout = "tidb_idle_transaction_timeout" + // TiDBLowResolutionTSOUpdateInterval defines how often to refresh low resolution timestamps. + TiDBLowResolutionTSOUpdateInterval = "tidb_low_resolution_tso_update_interval" + // TiDBDMLType indicates the execution type of DML in TiDB. + // The value can be STANDARD, BULK. + // Currently, the BULK mode only affects auto-committed DML. + TiDBDMLType = "tidb_dml_type" + // TiFlashHashAggPreAggMode indicates the policy of 1st hashagg. + TiFlashHashAggPreAggMode = "tiflash_hashagg_preaggregation_mode" + // TiDBEnableLazyCursorFetch defines whether to enable the lazy cursor fetch. If it's `OFF`, all results of + // of a cursor will be stored in the tidb node in `EXECUTE` command. + TiDBEnableLazyCursorFetch = "tidb_enable_lazy_cursor_fetch" + // TiDBTSOClientRPCMode controls how the TSO client performs the TSO RPC requests. It internally controls the + // concurrency of the RPC. This variable provides an approach to tune the latency of getting timestamps from PD. + TiDBTSOClientRPCMode = "tidb_tso_client_rpc_mode" +) + +// TiDB intentional limits, can be raised in the future. +const ( + // MaxConfigurableConcurrency is the maximum number of "threads" (goroutines) that can be specified + // for any type of configuration item that has concurrent workers. + MaxConfigurableConcurrency = 256 + + // MaxShardRowIDBits is the maximum number of bits that can be used for row-id sharding. + MaxShardRowIDBits = 15 + + // MaxPreSplitRegions is the maximum number of regions that can be pre-split. + MaxPreSplitRegions = 15 +) + +// Default TiDB system variable values. +const ( + DefHostname = "localhost" + DefIndexLookupConcurrency = ConcurrencyUnset + DefIndexLookupJoinConcurrency = ConcurrencyUnset + DefIndexSerialScanConcurrency = 1 + DefIndexJoinBatchSize = 25000 + DefIndexLookupSize = 20000 + DefDistSQLScanConcurrency = 15 + DefAnalyzeDistSQLScanConcurrency = 4 + DefBuildStatsConcurrency = 2 + DefBuildSamplingStatsConcurrency = 2 + DefAutoAnalyzeRatio = 0.5 + DefAutoAnalyzeStartTime = "00:00 +0000" + DefAutoAnalyzeEndTime = "23:59 +0000" + DefAutoIncrementIncrement = 1 + DefAutoIncrementOffset = 1 + DefChecksumTableConcurrency = 4 + DefSkipUTF8Check = false + DefSkipASCIICheck = false + DefOptAggPushDown = false + DefOptDeriveTopN = false + DefOptCartesianBCJ = 1 + DefOptMPPOuterJoinFixedBuildSide = false + DefOptWriteRowID = false + DefOptEnableCorrelationAdjustment = true + DefOptLimitPushDownThreshold = 100 + DefOptCorrelationThreshold = 0.9 + DefOptCorrelationExpFactor = 1 + DefOptCPUFactor = 3.0 + DefOptCopCPUFactor = 3.0 + DefOptTiFlashConcurrencyFactor = 24.0 + DefOptNetworkFactor = 1.0 + DefOptScanFactor = 1.5 + DefOptDescScanFactor = 3.0 + DefOptSeekFactor = 20.0 + DefOptMemoryFactor = 0.001 + DefOptDiskFactor = 1.5 + DefOptConcurrencyFactor = 3.0 + DefOptForceInlineCTE = false + DefOptInSubqToJoinAndAgg = true + DefOptPreferRangeScan = true + DefBatchInsert = false + DefBatchDelete = false + DefBatchCommit = false + DefCurretTS = 0 + DefInitChunkSize = 32 + DefMinPagingSize = int(paging.MinPagingSize) + DefMaxPagingSize = int(paging.MaxPagingSize) + DefMaxChunkSize = 1024 + DefDMLBatchSize = 0 + DefMaxPreparedStmtCount = -1 + DefWaitTimeout = 28800 + DefTiDBMemQuotaApplyCache = 32 << 20 // 32MB. + DefTiDBMemQuotaBindingCache = 64 << 20 // 64MB. + DefTiDBGeneralLog = false + DefTiDBPProfSQLCPU = 0 + DefTiDBRetryLimit = 10 + DefTiDBDisableTxnAutoRetry = true + DefTiDBConstraintCheckInPlace = false + DefTiDBHashJoinConcurrency = ConcurrencyUnset + DefTiDBProjectionConcurrency = ConcurrencyUnset + DefBroadcastJoinThresholdSize = 100 * 1024 * 1024 + DefBroadcastJoinThresholdCount = 10 * 1024 + DefPreferBCJByExchangeDataSize = false + DefTiDBOptimizerSelectivityLevel = 0 + DefTiDBOptimizerEnableNewOFGB = false + DefTiDBEnableOuterJoinReorder = true + DefTiDBEnableNAAJ = true + DefTiDBAllowBatchCop = 1 + DefShardRowIDBits = 0 + DefPreSplitRegions = 0 + DefBlockEncryptionMode = "aes-128-ecb" + DefTiDBAllowMPPExecution = true + DefTiDBAllowTiFlashCop = false + DefTiDBHashExchangeWithNewCollation = true + DefTiDBEnforceMPPExecution = false + DefTiFlashMaxThreads = -1 + DefTiFlashMaxBytesBeforeExternalJoin = -1 + DefTiFlashMaxBytesBeforeExternalGroupBy = -1 + DefTiFlashMaxBytesBeforeExternalSort = -1 + DefTiFlashMemQuotaQueryPerNode = 0 + DefTiFlashQuerySpillRatio = 0.7 + DefTiDBEnableTiFlashPipelineMode = true + DefTiDBMPPStoreFailTTL = "60s" + DefTiDBTxnMode = PessimisticTxnMode + DefTiDBRowFormatV1 = 1 + DefTiDBRowFormatV2 = 2 + DefTiDBDDLReorgWorkerCount = 4 + DefTiDBDDLReorgBatchSize = 256 + DefTiDBDDLFlashbackConcurrency = 64 + DefTiDBDDLErrorCountLimit = 512 + DefTiDBDDLReorgMaxWriteSpeed = 0 + DefTiDBMaxDeltaSchemaCount = 1024 + DefTiDBPlacementMode = PlacementModeStrict + DefTiDBEnableAutoIncrementInGenerated = false + DefTiDBHashAggPartialConcurrency = ConcurrencyUnset + DefTiDBHashAggFinalConcurrency = ConcurrencyUnset + DefTiDBWindowConcurrency = ConcurrencyUnset + DefTiDBMergeJoinConcurrency = 1 // disable optimization by default + DefTiDBStreamAggConcurrency = 1 + DefTiDBForcePriority = mysql.NoPriority + DefEnableWindowFunction = true + DefEnablePipelinedWindowFunction = true + DefEnableStrictDoubleTypeCheck = true + DefEnableVectorizedExpression = true + DefTiDBOptJoinReorderThreshold = 0 + DefTiDBDDLSlowOprThreshold = 300 + DefTiDBUseFastAnalyze = false + DefTiDBSkipIsolationLevelCheck = false + DefTiDBExpensiveQueryTimeThreshold = 60 // 60s + DefTiDBExpensiveTxnTimeThreshold = 60 * 10 // 10 minutes + DefTiDBScatterRegion = ScatterOff + DefTiDBWaitSplitRegionFinish = true + DefWaitSplitRegionTimeout = 300 // 300s + DefTiDBEnableNoopFuncs = Off + DefTiDBEnableNoopVariables = true + DefTiDBAllowRemoveAutoInc = false + DefTiDBUsePlanBaselines = true + DefTiDBEvolvePlanBaselines = false + DefTiDBEvolvePlanTaskMaxTime = 600 // 600s + DefTiDBEvolvePlanTaskStartTime = "00:00 +0000" + DefTiDBEvolvePlanTaskEndTime = "23:59 +0000" + DefInnodbLockWaitTimeout = 50 // 50s + DefTiDBStoreLimit = 0 + DefTiDBMetricSchemaStep = 60 // 60s + DefTiDBMetricSchemaRangeDuration = 60 // 60s + DefTiDBFoundInPlanCache = false + DefTiDBFoundInBinding = false + DefTiDBEnableCollectExecutionInfo = true + DefTiDBAllowAutoRandExplicitInsert = false + DefTiDBEnableClusteredIndex = ClusteredIndexDefModeOn + DefTiDBRedactLog = Off + DefTiDBRestrictedReadOnly = false + DefTiDBSuperReadOnly = false + DefTiDBShardAllocateStep = math.MaxInt64 + DefTiDBEnableTelemetry = false + DefTiDBEnableParallelApply = false + DefTiDBPartitionPruneMode = "dynamic" + DefTiDBEnableRateLimitAction = false + DefTiDBEnableAsyncCommit = false + DefTiDBEnable1PC = false + DefTiDBGuaranteeLinearizability = true + DefTiDBAnalyzeVersion = 2 + // Deprecated: This variable is deprecated, please do not use this variable. + DefTiDBAutoAnalyzePartitionBatchSize = mysql.PartitionCountLimit + DefTiDBEnableIndexMergeJoin = false + DefTiDBTrackAggregateMemoryUsage = true + DefCTEMaxRecursionDepth = 1000 + DefTiDBTmpTableMaxSize = 64 << 20 // 64MB. + DefTiDBEnableLocalTxn = false + DefTiDBTSOClientBatchMaxWaitTime = 0.0 // 0ms + DefTiDBEnableTSOFollowerProxy = false + DefPDEnableFollowerHandleRegion = true + DefTiDBEnableOrderedResultMode = false + DefTiDBEnablePseudoForOutdatedStats = false + DefTiDBRegardNULLAsPoint = true + DefEnablePlacementCheck = true + DefTimestamp = "0" + DefTimestampFloat = 0.0 + DefTiDBEnableStmtSummary = true + DefTiDBStmtSummaryInternalQuery = false + DefTiDBStmtSummaryRefreshInterval = 1800 + DefTiDBStmtSummaryHistorySize = 24 + DefTiDBStmtSummaryMaxStmtCount = 3000 + DefTiDBStmtSummaryMaxSQLLength = 4096 + DefTiDBCapturePlanBaseline = Off + DefTiDBIgnoreInlistPlanDigest = false + DefTiDBEnableIndexMerge = true + DefEnableLegacyInstanceScope = true + DefTiDBTableCacheLease = 3 // 3s + DefTiDBPersistAnalyzeOptions = true + DefTiDBStatsLoadSyncWait = 100 + DefTiDBStatsLoadPseudoTimeout = true + DefSysdateIsNow = false + DefTiDBEnableParallelHashaggSpill = true + DefTiDBEnableMutationChecker = false + DefTiDBTxnAssertionLevel = AssertionOffStr + DefTiDBIgnorePreparedCacheCloseStmt = false + DefTiDBBatchPendingTiFlashCount = 4000 + DefRCReadCheckTS = false + DefTiDBRemoveOrderbyInSubquery = true + DefTiDBSkewDistinctAgg = false + DefTiDB3StageDistinctAgg = true + DefTiDB3StageMultiDistinctAgg = false + DefTiDBOptExplainEvaledSubquery = false + DefTiDBReadStaleness = 0 + DefTiDBGCMaxWaitTime = 24 * 60 * 60 + DefMaxAllowedPacket uint64 = 67108864 + DefTiDBEnableBatchDML = false + DefTiDBMemQuotaQuery = memory.DefMemQuotaQuery // 1GB + DefTiDBStatsCacheMemQuota = 0 + MaxTiDBStatsCacheMemQuota = 1024 * 1024 * 1024 * 1024 // 1TB + DefTiDBQueryLogMaxLen = 4096 + DefRequireSecureTransport = false + DefTiDBCommitterConcurrency = 128 + DefTiDBBatchDMLIgnoreError = false + DefTiDBMemQuotaAnalyze = -1 + DefTiDBEnableAutoAnalyze = true + DefTiDBEnableAutoAnalyzePriorityQueue = true + DefTiDBAnalyzeColumnOptions = "PREDICATE" + DefTiDBMemOOMAction = "CANCEL" + DefTiDBMaxAutoAnalyzeTime = 12 * 60 * 60 + DefTiDBAutoAnalyzeConcurrency = 1 + DefTiDBEnablePrepPlanCache = true + DefTiDBPrepPlanCacheSize = 100 + DefTiDBSessionPlanCacheSize = 100 + DefTiDBEnablePrepPlanCacheMemoryMonitor = true + DefTiDBPrepPlanCacheMemoryGuardRatio = 0.1 + DefTiDBEnableWorkloadBasedLearning = false + DefTiDBWorkloadBasedLearningInterval = 24 * time.Hour + DefTiDBEnableDistTask = true + DefTiDBEnableFastCreateTable = true + DefTiDBSimplifiedMetrics = false + DefTiDBEnablePaging = true + DefTiFlashFineGrainedShuffleStreamCount = 0 + DefStreamCountWhenMaxThreadsNotSet = 8 + DefTiFlashFineGrainedShuffleBatchSize = 8192 + DefAdaptiveClosestReadThreshold = 4096 + DefTiDBEnableAnalyzeSnapshot = false + DefTiDBGenerateBinaryPlan = true + DefEnableTiDBGCAwareMemoryTrack = false + DefTiDBDefaultStrMatchSelectivity = 0.8 + DefTiDBEnableTmpStorageOnOOM = true + DefTiDBEnableMDL = true + DefTiFlashFastScan = false + DefMemoryUsageAlarmRatio = 0.7 + DefMemoryUsageAlarmKeepRecordNum = 5 + DefTiDBEnableFastReorg = true + DefTiDBDDLDiskQuota = 100 * 1024 * 1024 * 1024 // 100GB + DefExecutorConcurrency = 5 + DefTiDBEnableNonPreparedPlanCache = false + DefTiDBEnableNonPreparedPlanCacheForDML = false + DefTiDBNonPreparedPlanCacheSize = 100 + DefTiDBPlanCacheMaxPlanSize = 2 * size.MB + DefTiDBInstancePlanCacheMaxMemSize = 100 * size.MB + MinTiDBInstancePlanCacheMemSize = 100 * size.MB + DefTiDBInstancePlanCacheReservedPercentage = 0.1 + // MaxDDLReorgBatchSize is exported for testing. + MaxDDLReorgBatchSize int32 = 10240 + MinDDLReorgBatchSize int32 = 32 + MinExpensiveQueryTimeThreshold uint64 = 10 // 10s + MinExpensiveTxnTimeThreshold uint64 = 60 // 60s + DefTiDBAutoBuildStatsConcurrency = 1 + DefTiDBSysProcScanConcurrency = 1 + DefTiDBRcWriteCheckTs = false + DefTiDBForeignKeyChecks = true + DefTiDBOptAdvancedJoinHint = true + DefTiDBAnalyzePartitionConcurrency = 2 + DefTiDBOptRangeMaxSize = 64 * int64(size.MB) // 64 MB + DefTiDBCostModelVer = 2 + DefTiDBServerMemoryLimitSessMinSize = 128 << 20 + DefTiDBMergePartitionStatsConcurrency = 1 + DefTiDBServerMemoryLimitGCTrigger = 0.7 + DefTiDBEnableGOGCTuner = true + // DefTiDBGOGCTunerThreshold is to limit TiDBGOGCTunerThreshold. + DefTiDBGOGCTunerThreshold float64 = 0.6 + DefTiDBGOGCMaxValue = 500 + DefTiDBGOGCMinValue = 100 + DefTiDBOptPrefixIndexSingleScan = true + DefTiDBEnableAsyncMergeGlobalStats = true + DefTiDBExternalTS = 0 + DefTiDBEnableExternalTSRead = false + DefTiDBEnableReusechunk = true + DefTiDBUseAlloc = false + DefTiDBEnablePlanReplayerCapture = true + DefTiDBIndexMergeIntersectionConcurrency = ConcurrencyUnset + DefTiDBTTLJobEnable = true + DefTiDBTTLScanBatchSize = 500 + DefTiDBTTLScanBatchMaxSize = 10240 + DefTiDBTTLScanBatchMinSize = 1 + DefTiDBTTLDeleteBatchSize = 100 + DefTiDBTTLDeleteBatchMaxSize = 10240 + DefTiDBTTLDeleteBatchMinSize = 1 + DefTiDBTTLDeleteRateLimit = 0 + DefTiDBTTLRunningTasks = -1 + DefPasswordReuseHistory = 0 + DefPasswordReuseTime = 0 + DefTiDBStoreBatchSize = 4 + DefTiDBHistoricalStatsDuration = 7 * 24 * time.Hour + DefTiDBEnableHistoricalStatsForCapture = false + DefTiDBTTLJobScheduleWindowStartTime = "00:00 +0000" + DefTiDBTTLJobScheduleWindowEndTime = "23:59 +0000" + DefTiDBTTLScanWorkerCount = 4 + DefTiDBTTLDeleteWorkerCount = 4 + DefaultExchangeCompressionMode = ExchangeCompressionModeUnspecified + DefTiDBEnableResourceControl = true + DefTiDBResourceControlStrictMode = true + DefTiDBPessimisticTransactionFairLocking = false + DefTiDBEnablePlanCacheForParamLimit = true + DefTiDBEnableINLJoinMultiPattern = true + DefTiFlashComputeDispatchPolicy = DispatchPolicyConsistentHashStr + DefTiDBEnablePlanCacheForSubquery = true + DefTiDBLoadBasedReplicaReadThreshold = time.Second + DefTiDBOptEnableLateMaterialization = true + DefTiDBOptOrderingIdxSelThresh = 0.0 + DefTiDBOptOrderingIdxSelRatio = -1 + DefTiDBOptEnableMPPSharedCTEExecution = false + DefTiDBPlanCacheInvalidationOnFreshStats = true + DefTiDBEnableRowLevelChecksum = false + DefAuthenticationLDAPSASLAuthMethodName = "SCRAM-SHA-1" + DefAuthenticationLDAPSASLServerPort = 389 + DefAuthenticationLDAPSASLTLS = false + DefAuthenticationLDAPSASLUserSearchAttr = "uid" + DefAuthenticationLDAPSASLInitPoolSize = 10 + DefAuthenticationLDAPSASLMaxPoolSize = 1000 + DefAuthenticationLDAPSimpleAuthMethodName = "SIMPLE" + DefAuthenticationLDAPSimpleServerPort = 389 + DefAuthenticationLDAPSimpleTLS = false + DefAuthenticationLDAPSimpleUserSearchAttr = "uid" + DefAuthenticationLDAPSimpleInitPoolSize = 10 + DefAuthenticationLDAPSimpleMaxPoolSize = 1000 + DefTiFlashReplicaRead = AllReplicaStr + DefTiDBEnableFastCheckTable = true + DefRuntimeFilterType = "IN" + DefRuntimeFilterMode = "OFF" + DefTiDBLockUnchangedKeys = true + DefTiDBEnableCheckConstraint = false + DefTiDBSkipMissingPartitionStats = true + DefTiDBOptEnableHashJoin = true + DefTiDBHashJoinVersion = joinversion.HashJoinVersionOptimized + DefTiDBOptObjective = OptObjectiveModerate + DefTiDBSchemaVersionCacheLimit = 16 + DefTiDBIdleTransactionTimeout = 0 + DefTiDBTxnEntrySizeLimit = 0 + DefTiDBSchemaCacheSize = 512 * 1024 * 1024 + DefTiDBLowResolutionTSOUpdateInterval = 2000 + DefDivPrecisionIncrement = 4 + DefTiDBDMLType = "STANDARD" + DefGroupConcatMaxLen = uint64(1024) + DefDefaultWeekFormat = "0" + DefTiFlashPreAggMode = ForcePreAggStr + DefTiDBEnableLazyCursorFetch = false + DefOptEnableProjectionPushDown = true + DefTiDBEnableSharedLockPromotion = false + DefTiDBTSOClientRPCMode = TSOClientRPCModeDefault +) + +// Process global variables. +var ( + ProcessGeneralLog = atomic.NewBool(false) + RunAutoAnalyze = atomic.NewBool(DefTiDBEnableAutoAnalyze) + EnableAutoAnalyzePriorityQueue = atomic.NewBool(DefTiDBEnableAutoAnalyzePriorityQueue) + // AnalyzeColumnOptions is a global variable that indicates the default column choice for ANALYZE. + // The value of this variable is a string that can be one of the following values: + // "PREDICATE", "ALL". + // The behavior of the analyze operation depends on the value of `tidb_persist_analyze_options`: + // 1. If `tidb_persist_analyze_options` is enabled and the column choice from the analyze options record is set to `default`, + // the value of `tidb_analyze_column_options` determines the behavior of the analyze operation. + // 2. If `tidb_persist_analyze_options` is disabled, `tidb_analyze_column_options` is used directly to decide + // whether to analyze all columns or just the predicate columns. + AnalyzeColumnOptions = atomic.NewString(DefTiDBAnalyzeColumnOptions) + GlobalLogMaxDays = atomic.NewInt32(int32(config.GetGlobalConfig().Log.File.MaxDays)) + QueryLogMaxLen = atomic.NewInt32(DefTiDBQueryLogMaxLen) + EnablePProfSQLCPU = atomic.NewBool(false) + EnableBatchDML = atomic.NewBool(false) + EnableTmpStorageOnOOM = atomic.NewBool(DefTiDBEnableTmpStorageOnOOM) + DDLReorgWorkerCounter int32 = DefTiDBDDLReorgWorkerCount + DDLReorgBatchSize int32 = DefTiDBDDLReorgBatchSize + DDLFlashbackConcurrency int32 = DefTiDBDDLFlashbackConcurrency + DDLErrorCountLimit int64 = DefTiDBDDLErrorCountLimit + DDLReorgRowFormat int64 = DefTiDBRowFormatV2 + DDLReorgMaxWriteSpeed = atomic.NewInt64(DefTiDBDDLReorgMaxWriteSpeed) + MaxDeltaSchemaCount int64 = DefTiDBMaxDeltaSchemaCount + // DDLSlowOprThreshold is the threshold for ddl slow operations, uint is millisecond. + DDLSlowOprThreshold = config.GetGlobalConfig().Instance.DDLSlowOprThreshold + ForcePriority = int32(DefTiDBForcePriority) + MaxOfMaxAllowedPacket uint64 = 1073741824 + ExpensiveQueryTimeThreshold uint64 = DefTiDBExpensiveQueryTimeThreshold + ExpensiveTxnTimeThreshold uint64 = DefTiDBExpensiveTxnTimeThreshold + MemoryUsageAlarmRatio = atomic.NewFloat64(DefMemoryUsageAlarmRatio) + MemoryUsageAlarmKeepRecordNum = atomic.NewInt64(DefMemoryUsageAlarmKeepRecordNum) + EnableLocalTxn = atomic.NewBool(DefTiDBEnableLocalTxn) + MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) + EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) + EnablePDFollowerHandleRegion = atomic.NewBool(DefPDEnableFollowerHandleRegion) + RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) + VarTiDBSuperReadOnly = atomic.NewBool(DefTiDBSuperReadOnly) + PersistAnalyzeOptions = atomic.NewBool(DefTiDBPersistAnalyzeOptions) + TableCacheLease = atomic.NewInt64(DefTiDBTableCacheLease) + StatsLoadSyncWait = atomic.NewInt64(DefTiDBStatsLoadSyncWait) + StatsLoadPseudoTimeout = atomic.NewBool(DefTiDBStatsLoadPseudoTimeout) + MemQuotaBindingCache = atomic.NewInt64(DefTiDBMemQuotaBindingCache) + GCMaxWaitTime = atomic.NewInt64(DefTiDBGCMaxWaitTime) + StatsCacheMemQuota = atomic.NewInt64(DefTiDBStatsCacheMemQuota) + OOMAction = atomic.NewString(DefTiDBMemOOMAction) + MaxAutoAnalyzeTime = atomic.NewInt64(DefTiDBMaxAutoAnalyzeTime) + // variables for plan cache + PreparedPlanCacheMemoryGuardRatio = atomic.NewFloat64(DefTiDBPrepPlanCacheMemoryGuardRatio) + EnableInstancePlanCache = atomic.NewBool(false) + InstancePlanCacheReservedPercentage = atomic.NewFloat64(0.1) + InstancePlanCacheMaxMemSize = atomic.NewInt64(int64(DefTiDBInstancePlanCacheMaxMemSize)) + EnableDistTask = atomic.NewBool(DefTiDBEnableDistTask) + EnableFastCreateTable = atomic.NewBool(DefTiDBEnableFastCreateTable) + EnableNoopVariables = atomic.NewBool(DefTiDBEnableNoopVariables) + EnableMDL = atomic.NewBool(false) + AutoAnalyzePartitionBatchSize = atomic.NewInt64(DefTiDBAutoAnalyzePartitionBatchSize) + AutoAnalyzeConcurrency = atomic.NewInt32(DefTiDBAutoAnalyzeConcurrency) + // TODO: set value by session variable + EnableWorkloadBasedLearning = atomic.NewBool(DefTiDBEnableWorkloadBasedLearning) + WorkloadBasedLearningInterval = atomic.NewDuration(DefTiDBWorkloadBasedLearningInterval) + // EnableFastReorg indicates whether to use lightning to enhance DDL reorg performance. + EnableFastReorg = atomic.NewBool(DefTiDBEnableFastReorg) + // DDLDiskQuota is the temporary variable for set disk quota for lightning + DDLDiskQuota = atomic.NewUint64(DefTiDBDDLDiskQuota) + // EnableForeignKey indicates whether to enable foreign key feature. + EnableForeignKey = atomic.NewBool(true) + EnableRCReadCheckTS = atomic.NewBool(false) + // EnableRowLevelChecksum indicates whether to append checksum to row values. + EnableRowLevelChecksum = atomic.NewBool(DefTiDBEnableRowLevelChecksum) + LowResolutionTSOUpdateInterval = atomic.NewUint32(DefTiDBLowResolutionTSOUpdateInterval) + + // DefTiDBServerMemoryLimit indicates the default value of TiDBServerMemoryLimit(TotalMem * 80%). + // It should be a const and shouldn't be modified after tidb is started. + DefTiDBServerMemoryLimit = serverMemoryLimitDefaultValue() + GOGCTunerThreshold = atomic.NewFloat64(DefTiDBGOGCTunerThreshold) + PasswordValidationLength = atomic.NewInt32(8) + PasswordValidationMixedCaseCount = atomic.NewInt32(1) + PasswordValidtaionNumberCount = atomic.NewInt32(1) + PasswordValidationSpecialCharCount = atomic.NewInt32(1) + EnableTTLJob = atomic.NewBool(DefTiDBTTLJobEnable) + TTLScanBatchSize = atomic.NewInt64(DefTiDBTTLScanBatchSize) + TTLDeleteBatchSize = atomic.NewInt64(DefTiDBTTLDeleteBatchSize) + TTLDeleteRateLimit = atomic.NewInt64(DefTiDBTTLDeleteRateLimit) + TTLJobScheduleWindowStartTime = atomic.NewTime( + mustParseTime( + FullDayTimeFormat, + DefTiDBTTLJobScheduleWindowStartTime, + ), + ) + TTLJobScheduleWindowEndTime = atomic.NewTime( + mustParseTime( + FullDayTimeFormat, + DefTiDBTTLJobScheduleWindowEndTime, + ), + ) + TTLScanWorkerCount = atomic.NewInt32(DefTiDBTTLScanWorkerCount) + TTLDeleteWorkerCount = atomic.NewInt32(DefTiDBTTLDeleteWorkerCount) + PasswordHistory = atomic.NewInt64(DefPasswordReuseHistory) + PasswordReuseInterval = atomic.NewInt64(DefPasswordReuseTime) + IsSandBoxModeEnabled = atomic.NewBool(false) + MaxPreparedStmtCountValue = atomic.NewInt64(DefMaxPreparedStmtCount) + HistoricalStatsDuration = atomic.NewDuration(DefTiDBHistoricalStatsDuration) + EnableHistoricalStatsForCapture = atomic.NewBool(DefTiDBEnableHistoricalStatsForCapture) + 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) + 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.NewUint64(DefTiDBSchemaCacheSize) + SchemaCacheSizeOriginText = atomic.NewString(strconv.Itoa(DefTiDBSchemaCacheSize)) +) + +func serverMemoryLimitDefaultValue() string { + total, err := memory.MemTotal() + if err == nil && total != 0 { + return "80%" + } + return "0" +} + +func mustParseDuration(str string) time.Duration { + duration, err := time.ParseDuration(str) + if err != nil { + panic(fmt.Sprintf("%s is not a duration", str)) + } + + return duration +} + +func mustParseTime(layout string, str string) time.Time { + time, err := time.ParseInLocation(layout, str, time.UTC) + if err != nil { + panic(fmt.Sprintf("%s is not in %s duration format", str, layout)) + } + + return time +} + +const ( + // OptObjectiveModerate is a possible value and the default value for TiDBOptObjective. + // Please see comments of SessionVars.OptObjective for details. + OptObjectiveModerate string = "moderate" + // OptObjectiveDeterminate is a possible value for TiDBOptObjective. + OptObjectiveDeterminate = "determinate" +) + +// ForcePreAggStr means 1st hashagg will be pre aggregated. +// AutoStr means TiFlash will decide which policy for 1st hashagg. +// ForceStreamingStr means 1st hashagg will for pass through all blocks. +const ( + ForcePreAggStr = "force_preagg" + AutoStr = "auto" + ForceStreamingStr = "force_streaming" +) + +const ( + // AllReplicaStr is the string value of AllReplicas. + AllReplicaStr = "all_replicas" + // ClosestAdaptiveStr is the string value of ClosestAdaptive. + ClosestAdaptiveStr = "closest_adaptive" + // ClosestReplicasStr is the string value of ClosestReplicas. + ClosestReplicasStr = "closest_replicas" +) + +const ( + // DispatchPolicyRRStr is string value for DispatchPolicyRR. + DispatchPolicyRRStr = "round_robin" + // DispatchPolicyConsistentHashStr is string value for DispatchPolicyConsistentHash. + DispatchPolicyConsistentHashStr = "consistent_hash" + // DispatchPolicyInvalidStr is string value for DispatchPolicyInvalid. + DispatchPolicyInvalidStr = "invalid" +) + +// ConcurrencyUnset means the value the of the concurrency related variable is unset. +const ConcurrencyUnset = -1 + +// ExchangeCompressionMode means the compress method used in exchange operator +type ExchangeCompressionMode int + +const ( + // ExchangeCompressionModeNONE indicates no compression + ExchangeCompressionModeNONE ExchangeCompressionMode = iota + // ExchangeCompressionModeFast indicates fast compression/decompression speed, compression ratio is lower than HC mode + ExchangeCompressionModeFast + // ExchangeCompressionModeHC indicates high compression (HC) ratio mode + ExchangeCompressionModeHC + // ExchangeCompressionModeUnspecified indicates unspecified compress method, let TiDB choose one + ExchangeCompressionModeUnspecified + + // RecommendedExchangeCompressionMode indicates recommended compression mode + RecommendedExchangeCompressionMode ExchangeCompressionMode = ExchangeCompressionModeFast + + exchangeCompressionModeUnspecifiedName string = "UNSPECIFIED" +) + +// Name returns the name of ExchangeCompressionMode +func (t ExchangeCompressionMode) Name() string { + if t == ExchangeCompressionModeUnspecified { + return exchangeCompressionModeUnspecifiedName + } + return t.ToTipbCompressionMode().String() +} + +// ToExchangeCompressionMode returns the ExchangeCompressionMode from name +func ToExchangeCompressionMode(name string) (ExchangeCompressionMode, bool) { + name = strings.ToUpper(name) + if name == exchangeCompressionModeUnspecifiedName { + return ExchangeCompressionModeUnspecified, true + } + value, ok := tipb.CompressionMode_value[name] + if ok { + return ExchangeCompressionMode(value), true + } + return ExchangeCompressionModeNONE, false +} + +// ToTipbCompressionMode returns tipb.CompressionMode from kv.ExchangeCompressionMode +func (t ExchangeCompressionMode) ToTipbCompressionMode() tipb.CompressionMode { + switch t { + case ExchangeCompressionModeNONE: + return tipb.CompressionMode_NONE + case ExchangeCompressionModeFast: + return tipb.CompressionMode_FAST + case ExchangeCompressionModeHC: + return tipb.CompressionMode_HIGH_COMPRESSION + } + return tipb.CompressionMode_NONE +} + +// ScopeFlag is for system variable whether can be changed in global/session dynamically or not. +type ScopeFlag uint8 + +// TypeFlag is the SysVar type, which doesn't exactly match MySQL types. +type TypeFlag byte + +const ( + // ScopeNone means the system variable can not be changed dynamically. + ScopeNone ScopeFlag = 0 + // ScopeGlobal means the system variable can be changed globally. + ScopeGlobal ScopeFlag = 1 << 0 + // ScopeSession means the system variable can only be changed in current session. + ScopeSession ScopeFlag = 1 << 1 + // ScopeInstance means it is similar to global but doesn't propagate to other TiDB servers. + ScopeInstance ScopeFlag = 1 << 2 + + // TypeStr is the default + TypeStr TypeFlag = iota + // TypeBool for boolean + TypeBool + // TypeInt for integer + TypeInt + // TypeEnum for Enum + TypeEnum + // TypeFloat for Double + TypeFloat + // TypeUnsigned for Unsigned integer + TypeUnsigned + // TypeTime for time of day (a TiDB extension) + TypeTime + // TypeDuration for a golang duration (a TiDB extension) + TypeDuration + + // On is the canonical string for ON + On = "ON" + // Off is the canonical string for OFF + Off = "OFF" + // Warn means return warnings + Warn = "WARN" + // IntOnly means enable for int type + IntOnly = "INT_ONLY" + // Marker is a special log redact behavior + Marker = "MARKER" + + // AssertionStrictStr is a choice of variable TiDBTxnAssertionLevel that means full assertions should be performed, + // even if the performance might be slowed down. + AssertionStrictStr = "STRICT" + // AssertionFastStr is a choice of variable TiDBTxnAssertionLevel that means assertions that doesn't affect + // performance should be performed. + AssertionFastStr = "FAST" + // AssertionOffStr is a choice of variable TiDBTxnAssertionLevel that means no assertion should be performed. + AssertionOffStr = "OFF" + // OOMActionCancel constants represents the valid action configurations for OOMAction "CANCEL". + OOMActionCancel = "CANCEL" + // OOMActionLog constants represents the valid action configurations for OOMAction "LOG". + OOMActionLog = "LOG" + + // TSOClientRPCModeDefault is a choice of variable TiDBTSOClientRPCMode. In this mode, the TSO client sends batched + // TSO requests serially. + TSOClientRPCModeDefault = "DEFAULT" + // TSOClientRPCModeParallel is a choice of variable TiDBTSOClientRPCMode. In this mode, the TSO client tries to + // keep approximately 2 batched TSO requests running in parallel. This option tries to reduce the batch-waiting time + // by half, at the expense of about twice the amount of TSO RPC calls. + TSOClientRPCModeParallel = "PARALLEL" + // TSOClientRPCModeParallelFast is a choice of variable TiDBTSOClientRPCMode. In this mode, the TSO client tries to + // keep approximately 4 batched TSO requests running in parallel. This option tries to reduce the batch-waiting time + // by 3/4, at the expense of about 4 times the amount of TSO RPC calls. + TSOClientRPCModeParallelFast = "PARALLEL-FAST" +) + +// Global config name list. +const ( + GlobalConfigEnableTopSQL = "enable_resource_metering" + GlobalConfigSourceID = "source_id" +) + +func (s ScopeFlag) String() string { + var scopes []string + if s == ScopeNone { + return "NONE" + } + if s&ScopeSession != 0 { + scopes = append(scopes, "SESSION") + } + if s&ScopeGlobal != 0 { + scopes = append(scopes, "GLOBAL") + } + if s&ScopeInstance != 0 { + scopes = append(scopes, "INSTANCE") + } + return strings.Join(scopes, ",") +} + +// ClusteredIndexDefMode controls the default clustered property for primary key. +type ClusteredIndexDefMode int + +const ( + // ClusteredIndexDefModeIntOnly indicates only single int primary key will default be clustered. + ClusteredIndexDefModeIntOnly ClusteredIndexDefMode = 0 + // ClusteredIndexDefModeOn indicates primary key will default be clustered. + ClusteredIndexDefModeOn ClusteredIndexDefMode = 1 + // ClusteredIndexDefModeOff indicates primary key will default be non-clustered. + ClusteredIndexDefModeOff ClusteredIndexDefMode = 2 +) + +// TiDBOptEnableClustered converts enable clustered options to ClusteredIndexDefMode. +func TiDBOptEnableClustered(opt string) ClusteredIndexDefMode { + switch opt { + case On: + return ClusteredIndexDefModeOn + case Off: + return ClusteredIndexDefModeOff + default: + return ClusteredIndexDefModeIntOnly + } +} + +const ( + // ScatterOff means default, will not scatter region + ScatterOff string = "" + // ScatterTable means scatter region at table level + ScatterTable string = "table" + // ScatterGlobal means scatter region at global level + ScatterGlobal string = "global" +) + +const ( + // PlacementModeStrict indicates all placement operations should be checked strictly in ddl + PlacementModeStrict string = "STRICT" + // PlacementModeIgnore indicates ignore all placement operations in ddl + PlacementModeIgnore string = "IGNORE" +) + +const ( + // LocalDayTimeFormat is the local format of analyze start time and end time. + LocalDayTimeFormat = "15:04" + // FullDayTimeFormat is the full format of analyze start time and end time. + FullDayTimeFormat = "15:04 -0700" +) + +// SetDDLReorgWorkerCounter sets DDLReorgWorkerCounter count. +// Sysvar validation enforces the range to already be correct. +func SetDDLReorgWorkerCounter(cnt int32) { + goatomic.StoreInt32(&DDLReorgWorkerCounter, cnt) +} + +// GetDDLReorgWorkerCounter gets DDLReorgWorkerCounter. +func GetDDLReorgWorkerCounter() int32 { + return goatomic.LoadInt32(&DDLReorgWorkerCounter) +} + +// SetDDLFlashbackConcurrency sets DDLFlashbackConcurrency count. +// Sysvar validation enforces the range to already be correct. +func SetDDLFlashbackConcurrency(cnt int32) { + goatomic.StoreInt32(&DDLFlashbackConcurrency, cnt) +} + +// GetDDLFlashbackConcurrency gets DDLFlashbackConcurrency count. +func GetDDLFlashbackConcurrency() int32 { + return goatomic.LoadInt32(&DDLFlashbackConcurrency) +} + +// SetDDLReorgBatchSize sets DDLReorgBatchSize size. +// Sysvar validation enforces the range to already be correct. +func SetDDLReorgBatchSize(cnt int32) { + goatomic.StoreInt32(&DDLReorgBatchSize, cnt) +} + +// GetDDLReorgBatchSize gets DDLReorgBatchSize. +func GetDDLReorgBatchSize() int32 { + return goatomic.LoadInt32(&DDLReorgBatchSize) +} + +// SetDDLErrorCountLimit sets ddlErrorCountlimit size. +func SetDDLErrorCountLimit(cnt int64) { + goatomic.StoreInt64(&DDLErrorCountLimit, cnt) +} + +// GetDDLErrorCountLimit gets ddlErrorCountlimit size. +func GetDDLErrorCountLimit() int64 { + return goatomic.LoadInt64(&DDLErrorCountLimit) +} + +// SetDDLReorgRowFormat sets DDLReorgRowFormat version. +func SetDDLReorgRowFormat(format int64) { + goatomic.StoreInt64(&DDLReorgRowFormat, format) +} + +// GetDDLReorgRowFormat gets DDLReorgRowFormat version. +func GetDDLReorgRowFormat() int64 { + return goatomic.LoadInt64(&DDLReorgRowFormat) +} + +// SetMaxDeltaSchemaCount sets MaxDeltaSchemaCount size. +func SetMaxDeltaSchemaCount(cnt int64) { + goatomic.StoreInt64(&MaxDeltaSchemaCount, cnt) +} + +// GetMaxDeltaSchemaCount gets MaxDeltaSchemaCount size. +func GetMaxDeltaSchemaCount() int64 { + return goatomic.LoadInt64(&MaxDeltaSchemaCount) +} diff --git a/pkg/sessionctx/variable/BUILD.bazel b/pkg/sessionctx/variable/BUILD.bazel index de5ab75eaa0cf..29d5e6050db4d 100644 --- a/pkg/sessionctx/variable/BUILD.bazel +++ b/pkg/sessionctx/variable/BUILD.bazel @@ -37,6 +37,7 @@ go_library( "//pkg/resourcegroup", "//pkg/sessionctx/sessionstates", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/types", "//pkg/types/parser_driver", "//pkg/util", @@ -52,13 +53,11 @@ go_library( "//pkg/util/logutil", "//pkg/util/mathutil", "//pkg/util/memory", - "//pkg/util/paging", "//pkg/util/ppcpuusage", "//pkg/util/redact", "//pkg/util/replayer", "//pkg/util/rowcodec", "//pkg/util/servicescope", - "//pkg/util/size", "//pkg/util/sqlkiller", "//pkg/util/stmtsummary", "//pkg/util/stmtsummary/v2:stmtsummary", @@ -113,6 +112,7 @@ go_test( "//pkg/planner/core", "//pkg/sessionctx/sessionstates", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/testkit/testsetup", "//pkg/types", diff --git a/pkg/sessionctx/variable/mock_globalaccessor.go b/pkg/sessionctx/variable/mock_globalaccessor.go index 5477c054257e4..0a86e10234739 100644 --- a/pkg/sessionctx/variable/mock_globalaccessor.go +++ b/pkg/sessionctx/variable/mock_globalaccessor.go @@ -14,7 +14,11 @@ package variable -import "context" +import ( + "context" + + "github.com/pingcap/tidb/pkg/sessionctx/vardef" +) // MockGlobalAccessor implements GlobalVarAccessor interface. it's used in tests type MockGlobalAccessor struct { @@ -76,7 +80,7 @@ func (m *MockGlobalAccessor) SetGlobalSysVar(ctx context.Context, name string, v if sv == nil { return ErrUnknownSystemVar.GenWithStackByArgs(name) } - if value, err = sv.Validate(m.SessionVars, value, ScopeGlobal); err != nil { + if value, err = sv.Validate(m.SessionVars, value, vardef.ScopeGlobal); err != nil { return err } if err = sv.SetGlobalFromHook(ctx, m.SessionVars, value, false); err != nil { @@ -100,7 +104,7 @@ func (m *MockGlobalAccessor) SetGlobalSysVarOnly(ctx context.Context, name strin func (m *MockGlobalAccessor) GetTiDBTableValue(name string) (string, error) { // add for test tidb_gc_max_wait_time validation if name == "tikv_gc_life_time" { - sv := GetSysVar(TiDBGCLifetime) + sv := GetSysVar(vardef.TiDBGCLifetime) if sv == nil { panic("Get SysVar Failed") } diff --git a/pkg/sessionctx/variable/mock_globalaccessor_test.go b/pkg/sessionctx/variable/mock_globalaccessor_test.go index 76f4b1d39b608..d0f4970f52272 100644 --- a/pkg/sessionctx/variable/mock_globalaccessor_test.go +++ b/pkg/sessionctx/variable/mock_globalaccessor_test.go @@ -18,6 +18,7 @@ import ( "context" "testing" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/stretchr/testify/require" "go.opencensus.io/stats/view" ) @@ -40,13 +41,13 @@ func TestMockAPI(t *testing.T) { require.Error(t, err) // valid option, invalid value - err = mock.SetGlobalSysVar(context.Background(), DefaultAuthPlugin, "invalidvalue") + err = mock.SetGlobalSysVar(context.Background(), vardef.DefaultAuthPlugin, "invalidvalue") require.Error(t, err) // valid option, valid value - err = mock.SetGlobalSysVar(context.Background(), DefaultAuthPlugin, "mysql_native_password") + err = mock.SetGlobalSysVar(context.Background(), vardef.DefaultAuthPlugin, "mysql_native_password") require.NoError(t, err) - err = mock.SetGlobalSysVarOnly(context.Background(), DefaultAuthPlugin, "mysql_native_password", true) + err = mock.SetGlobalSysVarOnly(context.Background(), vardef.DefaultAuthPlugin, "mysql_native_password", true) require.NoError(t, err) // Test GetTiDBTableValue diff --git a/pkg/sessionctx/variable/noop.go b/pkg/sessionctx/variable/noop.go index bbdaa249ee9ce..13f919de45436 100644 --- a/pkg/sessionctx/variable/noop.go +++ b/pkg/sessionctx/variable/noop.go @@ -18,6 +18,7 @@ import ( "math" "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" ) // The following sysVars are noops. @@ -29,622 +30,622 @@ var noopSysVars = []*SysVar{ // It is unsafe to pretend that any variation of "read only" is enabled when the server // does not support it. It is possible that these features will be supported in future, // but until then... - {Scope: ScopeGlobal | ScopeSession, Name: TxReadOnly, Value: Off, Type: TypeBool, Aliases: []string{TransactionReadOnly}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TxReadOnly, Value: vardef.Off, Type: vardef.TypeBool, Aliases: []string{vardef.TransactionReadOnly}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, - {Scope: ScopeGlobal | ScopeSession, Name: TransactionReadOnly, Value: Off, Type: TypeBool, Aliases: []string{TxReadOnly}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TransactionReadOnly, Value: vardef.Off, Type: vardef.TypeBool, Aliases: []string{vardef.TxReadOnly}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, - {Scope: ScopeGlobal, Name: OfflineMode, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal, Name: vardef.OfflineMode, Value: vardef.Off, Type: vardef.TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, true) }}, - {Scope: ScopeGlobal, Name: SuperReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal, Name: vardef.SuperReadOnly, Value: vardef.Off, Type: vardef.TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, - {Scope: ScopeGlobal, Name: ReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal, Name: vardef.ReadOnly, Value: vardef.Off, Type: vardef.TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, - {Scope: ScopeGlobal, Name: ConnectTimeout, Value: "10", Type: TypeUnsigned, MinValue: 2, MaxValue: secondsPerYear}, + {Scope: vardef.ScopeGlobal, Name: vardef.ConnectTimeout, Value: "10", Type: vardef.TypeUnsigned, MinValue: 2, MaxValue: secondsPerYear}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "sql_buffer_result", - Value: Off, + Value: vardef.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}, - {Scope: ScopeNone, Name: "performance_schema_max_mutex_classes", Value: "200"}, - {Scope: ScopeGlobal | ScopeSession, Name: LowPriorityUpdates, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: SessionTrackGtids, Value: Off, Type: TypeEnum, PossibleValues: []string{Off, "OWN_GTID", "ALL_GTIDS"}}, - {Scope: ScopeGlobal | ScopeSession, Name: "ndbinfo_max_rows", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: "ndb_index_stat_option", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: OldPasswords, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 2}, - {Scope: ScopeNone, Name: "innodb_version", Value: "5.6.25"}, - {Scope: ScopeGlobal | ScopeSession, Name: BigTables, Value: Off, Type: TypeBool}, - {Scope: ScopeNone, Name: "skip_external_locking", Value: "1"}, - {Scope: ScopeNone, Name: "innodb_sync_array_size", Value: "1"}, - {Scope: ScopeSession, Name: "gtid_next", Value: ""}, - {Scope: ScopeGlobal, Name: "ndb_show_foreign_key_mock_tables", Value: ""}, - {Scope: ScopeNone, Name: "multi_range_count", Value: "256"}, - {Scope: ScopeGlobal | ScopeSession, Name: "binlog_error_action", Value: "IGNORE_ERROR"}, - {Scope: ScopeGlobal | ScopeSession, Name: "default_storage_engine", Value: "InnoDB"}, - {Scope: ScopeNone, Name: "ft_query_expansion_limit", Value: "20"}, - {Scope: ScopeGlobal, Name: MaxConnectErrors, Value: "100", Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, - {Scope: ScopeGlobal, Name: SyncBinlog, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 4294967295}, - {Scope: ScopeNone, Name: "max_digest_length", Value: "1024"}, - {Scope: ScopeNone, Name: "innodb_force_load_corrupted", Value: "0"}, - {Scope: ScopeNone, Name: "performance_schema_max_table_handles", Value: "4000"}, - {Scope: ScopeGlobal, Name: InnodbFastShutdown, Value: "1", Type: TypeUnsigned, MinValue: 0, MaxValue: 2}, - {Scope: ScopeNone, Name: "ft_max_word_len", Value: "84"}, - {Scope: ScopeGlobal, Name: "log_backward_compatible_user_definitions", Value: ""}, - {Scope: ScopeNone, Name: "lc_messages_dir", Value: "/usr/local/mysql-5.6.25-osx10.8-x86_64/share/"}, - {Scope: ScopeGlobal, Name: "ft_boolean_syntax", Value: `+ -><()~*:""&|`}, - {Scope: ScopeGlobal, Name: TableDefinitionCache, Value: "2000", Type: TypeUnsigned, MinValue: 400, MaxValue: 524288}, - {Scope: ScopeNone, Name: "performance_schema_max_file_handles", Value: "32768"}, - {Scope: ScopeSession, Name: "transaction_allow_batching", Value: ""}, - {Scope: ScopeNone, Name: "performance_schema_max_statement_classes", Value: "168"}, - {Scope: ScopeGlobal, Name: "server_id", Value: "0"}, - {Scope: ScopeGlobal, Name: "innodb_flushing_avg_loops", Value: "30"}, - {Scope: ScopeGlobal, Name: "innodb_max_purge_lag", Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: "preload_buffer_size", Value: "32768"}, - {Scope: ScopeGlobal, Name: CheckProxyUsers, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "innodb_flush_log_at_timeout", Value: "1"}, - {Scope: ScopeGlobal, Name: "innodb_max_undo_log_size", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: vardef.MyISAMUseMmap, Value: vardef.Off, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeGlobal, Name: "gtid_mode", Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: vardef.FlushTime, Value: "0", Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: secondsPerYear}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_mutex_classes", Value: "200"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.LowPriorityUpdates, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.SessionTrackGtids, Value: vardef.Off, Type: vardef.TypeEnum, PossibleValues: []string{vardef.Off, "OWN_GTID", "ALL_GTIDS"}}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "ndbinfo_max_rows", Value: ""}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "ndb_index_stat_option", Value: ""}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.OldPasswords, Value: "0", Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: 2}, + {Scope: vardef.ScopeNone, Name: "innodb_version", Value: "5.6.25"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.BigTables, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeNone, Name: "skip_external_locking", Value: "1"}, + {Scope: vardef.ScopeNone, Name: "innodb_sync_array_size", Value: "1"}, + {Scope: vardef.ScopeSession, Name: "gtid_next", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "ndb_show_foreign_key_mock_tables", Value: ""}, + {Scope: vardef.ScopeNone, Name: "multi_range_count", Value: "256"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "binlog_error_action", Value: "IGNORE_ERROR"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "default_storage_engine", Value: "InnoDB"}, + {Scope: vardef.ScopeNone, Name: "ft_query_expansion_limit", Value: "20"}, + {Scope: vardef.ScopeGlobal, Name: vardef.MaxConnectErrors, Value: "100", Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64}, + {Scope: vardef.ScopeGlobal, Name: vardef.SyncBinlog, Value: "0", Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: 4294967295}, + {Scope: vardef.ScopeNone, Name: "max_digest_length", Value: "1024"}, + {Scope: vardef.ScopeNone, Name: "innodb_force_load_corrupted", Value: "0"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_table_handles", Value: "4000"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbFastShutdown, Value: "1", Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: 2}, + {Scope: vardef.ScopeNone, Name: "ft_max_word_len", Value: "84"}, + {Scope: vardef.ScopeGlobal, Name: "log_backward_compatible_user_definitions", Value: ""}, + {Scope: vardef.ScopeNone, Name: "lc_messages_dir", Value: "/usr/local/mysql-5.6.25-osx10.8-x86_64/share/"}, + {Scope: vardef.ScopeGlobal, Name: "ft_boolean_syntax", Value: `+ -><()~*:""&|`}, + {Scope: vardef.ScopeGlobal, Name: vardef.TableDefinitionCache, Value: "2000", Type: vardef.TypeUnsigned, MinValue: 400, MaxValue: 524288}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_file_handles", Value: "32768"}, + {Scope: vardef.ScopeSession, Name: "transaction_allow_batching", Value: ""}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_statement_classes", Value: "168"}, + {Scope: vardef.ScopeGlobal, Name: "server_id", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_flushing_avg_loops", Value: "30"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_max_purge_lag", Value: "0"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "preload_buffer_size", Value: "32768"}, + {Scope: vardef.ScopeGlobal, Name: vardef.CheckProxyUsers, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "innodb_flush_log_at_timeout", Value: "1"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_max_undo_log_size", Value: ""}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.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_default_row_format", Value: "dynamic", Type: TypeEnum, PossibleValues: []string{"redundant", "compact", "dynamic"}}, - {Scope: ScopeGlobal, Name: "innodb_compression_failure_threshold_pct", Value: "5"}, - {Scope: ScopeNone, Name: "performance_schema_events_waits_history_long_size", Value: "10000"}, - {Scope: ScopeGlobal, Name: "innodb_checksum_algorithm", Value: "innodb"}, - {Scope: ScopeNone, Name: "innodb_ft_sort_pll_degree", Value: "2"}, - {Scope: ScopeNone, Name: "thread_stack", Value: "262144"}, - {Scope: ScopeGlobal, Name: "relay_log_info_repository", Value: "FILE"}, - {Scope: ScopeGlobal | ScopeSession, Name: "max_delayed_threads", Value: "20"}, - {Scope: ScopeNone, Name: "protocol_version", Value: "10"}, - {Scope: ScopeGlobal | ScopeSession, Name: "new", Value: Off}, - {Scope: ScopeGlobal | ScopeSession, Name: "myisam_sort_buffer_size", Value: "8388608"}, - {Scope: ScopeGlobal | ScopeSession, Name: "optimizer_trace_offset", Value: "-1"}, - {Scope: ScopeGlobal, Name: InnodbBufferPoolDumpAtShutdown, Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLNotes, Value: "1"}, - {Scope: ScopeGlobal, Name: InnodbCmpPerIndexEnabled, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeGlobal, Name: "innodb_ft_server_stopword_table", Value: ""}, - {Scope: ScopeNone, Name: "performance_schema_max_file_instances", Value: "7693"}, - {Scope: ScopeNone, Name: "log_output", Value: "FILE"}, - {Scope: ScopeGlobal, Name: "binlog_group_commit_sync_delay", Value: ""}, - {Scope: ScopeGlobal, Name: "binlog_group_commit_sync_no_delay_count", Value: ""}, - {Scope: ScopeNone, Name: "have_crypt", Value: "YES"}, - {Scope: ScopeGlobal, Name: "innodb_log_write_ahead_size", Value: ""}, - {Scope: ScopeNone, Name: "innodb_log_group_home_dir", Value: "./"}, - {Scope: ScopeNone, Name: "performance_schema_events_statements_history_size", Value: "10"}, - {Scope: ScopeGlobal, Name: GeneralLog, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: BinlogOrderCommits, Value: On, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "key_cache_division_limit", Value: "100"}, - {Scope: ScopeGlobal | ScopeSession, Name: "max_insert_delayed_threads", Value: "20"}, - {Scope: ScopeNone, Name: "performance_schema_session_connect_attrs_size", Value: "512"}, - {Scope: ScopeGlobal, Name: "innodb_max_dirty_pages_pct", Value: "75"}, - {Scope: ScopeGlobal, Name: InnodbFilePerTable, Value: On, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeGlobal, Name: InnodbLogCompressedPages, Value: "1"}, - {Scope: ScopeNone, Name: "skip_networking", Value: "0"}, - {Scope: ScopeGlobal, Name: "innodb_monitor_reset", Value: ""}, - {Scope: ScopeNone, Name: "ssl_cipher", Value: ""}, - {Scope: ScopeNone, Name: "tls_version", Value: "TLSv1,TLSv1.1,TLSv1.2"}, - {Scope: ScopeGlobal, Name: InnodbPrintAllDeadlocks, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeNone, Name: "innodb_autoinc_lock_mode", Value: "1"}, - {Scope: ScopeGlobal, Name: "key_buffer_size", Value: "8388608"}, - {Scope: ScopeGlobal, Name: "host_cache_size", Value: "279"}, - {Scope: ScopeGlobal, Name: DelayKeyWrite, Value: On, Type: TypeEnum, PossibleValues: []string{Off, On, "ALL"}}, - {Scope: ScopeNone, Name: "metadata_locks_cache_size", Value: "1024"}, - {Scope: ScopeNone, Name: "innodb_force_recovery", Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: "debug", Value: ""}, - {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: vardef.ScopeNone, Name: "have_rtree_keys", Value: "YES"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_old_blocks_pct", Value: "37"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_default_row_format", Value: "dynamic", Type: vardef.TypeEnum, PossibleValues: []string{"redundant", "compact", "dynamic"}}, + {Scope: vardef.ScopeGlobal, Name: "innodb_compression_failure_threshold_pct", Value: "5"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_events_waits_history_long_size", Value: "10000"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_checksum_algorithm", Value: "innodb"}, + {Scope: vardef.ScopeNone, Name: "innodb_ft_sort_pll_degree", Value: "2"}, + {Scope: vardef.ScopeNone, Name: "thread_stack", Value: "262144"}, + {Scope: vardef.ScopeGlobal, Name: "relay_log_info_repository", Value: "FILE"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "max_delayed_threads", Value: "20"}, + {Scope: vardef.ScopeNone, Name: "protocol_version", Value: "10"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "new", Value: vardef.Off}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "myisam_sort_buffer_size", Value: "8388608"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "optimizer_trace_offset", Value: "-1"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbBufferPoolDumpAtShutdown, Value: "0"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.SQLNotes, Value: "1"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbCmpPerIndexEnabled, Value: vardef.Off, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeGlobal, Name: "innodb_ft_server_stopword_table", Value: ""}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_file_instances", Value: "7693"}, + {Scope: vardef.ScopeNone, Name: "log_output", Value: "FILE"}, + {Scope: vardef.ScopeGlobal, Name: "binlog_group_commit_sync_delay", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "binlog_group_commit_sync_no_delay_count", Value: ""}, + {Scope: vardef.ScopeNone, Name: "have_crypt", Value: "YES"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_log_write_ahead_size", Value: ""}, + {Scope: vardef.ScopeNone, Name: "innodb_log_group_home_dir", Value: "./"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_events_statements_history_size", Value: "10"}, + {Scope: vardef.ScopeGlobal, Name: vardef.GeneralLog, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: vardef.BinlogOrderCommits, Value: vardef.On, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "key_cache_division_limit", Value: "100"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "max_insert_delayed_threads", Value: "20"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_session_connect_attrs_size", Value: "512"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_max_dirty_pages_pct", Value: "75"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbFilePerTable, Value: vardef.On, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbLogCompressedPages, Value: "1"}, + {Scope: vardef.ScopeNone, Name: "skip_networking", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_monitor_reset", Value: ""}, + {Scope: vardef.ScopeNone, Name: "ssl_cipher", Value: ""}, + {Scope: vardef.ScopeNone, Name: "tls_version", Value: "TLSv1,TLSv1.1,TLSv1.2"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbPrintAllDeadlocks, Value: vardef.Off, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeNone, Name: "innodb_autoinc_lock_mode", Value: "1"}, + {Scope: vardef.ScopeGlobal, Name: "key_buffer_size", Value: "8388608"}, + {Scope: vardef.ScopeGlobal, Name: "host_cache_size", Value: "279"}, + {Scope: vardef.ScopeGlobal, Name: vardef.DelayKeyWrite, Value: vardef.On, Type: vardef.TypeEnum, PossibleValues: []string{vardef.Off, vardef.On, "ALL"}}, + {Scope: vardef.ScopeNone, Name: "metadata_locks_cache_size", Value: "1024"}, + {Scope: vardef.ScopeNone, Name: "innodb_force_recovery", Value: "0"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "debug", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "log_warnings", Value: "1"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.InnodbStrictMode, Value: vardef.On, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeGlobal, Name: "innodb_rollback_segments", Value: "128"}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "join_buffer_size", Value: "262144", IsHintUpdatableVerified: true, }, - {Scope: ScopeGlobal, Name: "max_binlog_size", Value: "1073741824"}, - {Scope: ScopeGlobal, Name: "concurrent_insert", Value: "AUTO"}, - {Scope: ScopeGlobal, Name: InnodbAdaptiveHashIndex, Value: On, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeGlobal, Name: InnodbFtEnableStopword, Value: On, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeGlobal, Name: "general_log_file", Value: "/usr/local/mysql/data/localhost.log"}, - {Scope: ScopeGlobal, Name: "innodb_compression_level", Value: "6"}, - {Scope: ScopeNone, Name: "myisam_mmap_size", Value: "18446744073709551615"}, - {Scope: ScopeNone, Name: "innodb_buffer_pool_instances", Value: "8"}, + {Scope: vardef.ScopeGlobal, Name: "max_binlog_size", Value: "1073741824"}, + {Scope: vardef.ScopeGlobal, Name: "concurrent_insert", Value: "AUTO"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbAdaptiveHashIndex, Value: vardef.On, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbFtEnableStopword, Value: vardef.On, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeGlobal, Name: "general_log_file", Value: "/usr/local/mysql/data/localhost.log"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_compression_level", Value: "6"}, + {Scope: vardef.ScopeNone, Name: "myisam_mmap_size", Value: "18446744073709551615"}, + {Scope: vardef.ScopeNone, Name: "innodb_buffer_pool_instances", Value: "8"}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.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) + {Scope: vardef.ScopeNone, Name: vardef.CharacterSetSystem, Value: "utf8"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.CharacterSetFilesystem, Value: "binary", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + return checkCharacterSet(normalizedValue, vardef.CharacterSetFilesystem) }}, - {Scope: ScopeGlobal, Name: InnodbOptimizeFullTextOnly, Value: "0"}, - {Scope: ScopeNone, Name: "character_sets_dir", Value: "/usr/local/mysql-5.6.25-osx10.8-x86_64/share/charsets/"}, - {Scope: ScopeNone, Name: "innodb_rollback_on_timeout", Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: "query_alloc_block_size", Value: "8192"}, - {Scope: ScopeNone, Name: "have_compress", Value: "YES"}, - {Scope: ScopeNone, Name: "thread_concurrency", Value: "10"}, - {Scope: ScopeGlobal | ScopeSession, Name: "query_prealloc_size", Value: "8192"}, - {Scope: ScopeNone, Name: "relay_log_space_limit", Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: MaxUserConnections, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 4294967295}, - {Scope: ScopeNone, Name: "performance_schema_max_thread_classes", Value: "50"}, - {Scope: ScopeGlobal, Name: "innodb_api_trx_level", Value: "0"}, - {Scope: ScopeNone, Name: "performance_schema_max_file_classes", Value: "50"}, - {Scope: ScopeGlobal, Name: "expire_logs_days", Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: BinlogRowQueryLogEvents, Value: Off, Type: TypeBool}, - {Scope: ScopeNone, Name: "pid_file", Value: "/usr/local/mysql/data/localhost.pid"}, - {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: vardef.ScopeGlobal, Name: vardef.InnodbOptimizeFullTextOnly, Value: "0"}, + {Scope: vardef.ScopeNone, Name: "character_sets_dir", Value: "/usr/local/mysql-5.6.25-osx10.8-x86_64/share/charsets/"}, + {Scope: vardef.ScopeNone, Name: "innodb_rollback_on_timeout", Value: "0"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "query_alloc_block_size", Value: "8192"}, + {Scope: vardef.ScopeNone, Name: "have_compress", Value: "YES"}, + {Scope: vardef.ScopeNone, Name: "thread_concurrency", Value: "10"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "query_prealloc_size", Value: "8192"}, + {Scope: vardef.ScopeNone, Name: "relay_log_space_limit", Value: "0"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.MaxUserConnections, Value: "0", Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: 4294967295}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_thread_classes", Value: "50"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_api_trx_level", Value: "0"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_file_classes", Value: "50"}, + {Scope: vardef.ScopeGlobal, Name: "expire_logs_days", Value: "0"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.BinlogRowQueryLogEvents, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeNone, Name: "pid_file", Value: "/usr/local/mysql/data/localhost.pid"}, + {Scope: vardef.ScopeNone, Name: "innodb_undo_tablespaces", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbStatusOutputLocks, Value: vardef.Off, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeNone, Name: "performance_schema_accounts_size", Value: "100"}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.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"}, - {Scope: ScopeGlobal, Name: "innodb_stats_persistent_sample_pages", Value: "20"}, - {Scope: ScopeGlobal, Name: "show_compatibility_56", Value: ""}, - {Scope: ScopeNone, Name: "innodb_open_files", Value: "2000"}, - {Scope: ScopeGlobal, Name: "innodb_spin_wait_delay", Value: "6"}, - {Scope: ScopeGlobal, Name: "thread_cache_size", Value: "9"}, - {Scope: ScopeGlobal, Name: LogSlowAdminStatements, Value: Off, Type: TypeBool}, - {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: vardef.ScopeGlobal, Name: "max_write_lock_count", Value: "18446744073709551615"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_socket_instances", Value: "322"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_table_instances", Value: "12500"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_stats_persistent_sample_pages", Value: "20"}, + {Scope: vardef.ScopeGlobal, Name: "show_compatibility_56", Value: ""}, + {Scope: vardef.ScopeNone, Name: "innodb_open_files", Value: "2000"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_spin_wait_delay", Value: "6"}, + {Scope: vardef.ScopeGlobal, Name: "thread_cache_size", Value: "9"}, + {Scope: vardef.ScopeGlobal, Name: vardef.LogSlowAdminStatements, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeNone, Name: "ft_stopword_file", Value: "(built-in)"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_max_dirty_pages_pct_lwm", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: vardef.LogQueriesNotUsingIndexes, Value: vardef.Off, Type: vardef.TypeBool}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "max_heap_table_size", Value: "16777216", IsHintUpdatableVerified: true, }, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "tmp_table_size", Value: "16777216", - Type: TypeUnsigned, + Type: vardef.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: vardef.ScopeGlobal, Name: "innodb_lru_scan_depth", Value: "1024"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_purge_rseg_truncate_frequency", Value: ""}, { - Scope: ScopeGlobal | ScopeSession, - Name: SQLAutoIsNull, - Value: Off, - Type: TypeBool, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.SQLAutoIsNull, + Value: vardef.Off, + Type: vardef.TypeBool, IsHintUpdatableVerified: true, Validation: func( - vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag, + vars *SessionVars, normalizedValue string, originalValue string, scope vardef.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 scope == vardef.ScopeSession && vars.NoopFuncsMode != OnInt { if vars.NoopFuncsMode == OffInt { - return Off, errors.Trace(errMsg) + return vardef.Off, errors.Trace(errMsg) } vars.StmtCtx.AppendWarning(errMsg) } - if scope == ScopeGlobal { - val, err := vars.GlobalVarsAccessor.GetGlobalSysVar(TiDBEnableNoopFuncs) + if scope == vardef.ScopeGlobal { + val, err := vars.GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBEnableNoopFuncs) if err != nil { - return originalValue, errUnknownSystemVariable.GenWithStackByArgs(TiDBEnableNoopFuncs) + return originalValue, errUnknownSystemVariable.GenWithStackByArgs(vardef.TiDBEnableNoopFuncs) } - if val == Off { - return Off, errors.Trace(errMsg) + if val == vardef.Off { + return vardef.Off, errors.Trace(errMsg) } - if val == Warn { + if val == vardef.Warn { vars.StmtCtx.AppendWarning(errMsg) } } } 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: ScopeNone, Name: "innodb_buffer_pool_load_at_startup", Value: "1"}, + {Scope: vardef.ScopeNone, Name: "innodb_api_enable_binlog", Value: "0"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "innodb_ft_user_stopword_table", Value: ""}, + {Scope: vardef.ScopeNone, Name: "server_id_bits", Value: "32"}, + {Scope: vardef.ScopeNone, Name: "innodb_buffer_pool_load_at_startup", Value: "1"}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "sort_buffer_size", Value: "262144", IsHintUpdatableVerified: true, }, - {Scope: ScopeGlobal, Name: "innodb_flush_neighbors", Value: "1"}, - {Scope: ScopeNone, Name: "performance_schema_max_socket_classes", Value: "10"}, - {Scope: ScopeNone, Name: "performance_schema_max_stage_classes", Value: "150"}, - {Scope: ScopeGlobal, Name: "innodb_purge_batch_size", Value: "300"}, - {Scope: ScopeNone, Name: "have_profiling", Value: "NO"}, - {Scope: ScopeGlobal, Name: InnodbBufferPoolDumpNow, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeGlobal, Name: RelayLogPurge, Value: On, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "ndb_distribution", Value: ""}, - {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: vardef.ScopeGlobal, Name: "innodb_flush_neighbors", Value: "1"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_socket_classes", Value: "10"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_stage_classes", Value: "150"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_purge_batch_size", Value: "300"}, + {Scope: vardef.ScopeNone, Name: "have_profiling", Value: "NO"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbBufferPoolDumpNow, Value: vardef.Off, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeGlobal, Name: vardef.RelayLogPurge, Value: vardef.On, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "ndb_distribution", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "myisam_data_pointer_size", Value: "6"}, + {Scope: vardef.ScopeGlobal, Name: "ndb_optimization_delay", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "innodb_ft_num_word_optimize", Value: "2000"}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "max_join_size", Value: "18446744073709551615", IsHintUpdatableVerified: true, }, - {Scope: ScopeNone, Name: CoreFile, Value: Off, Type: TypeBool}, + {Scope: vardef.ScopeNone, Name: vardef.CoreFile, Value: vardef.Off, Type: vardef.TypeBool}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.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: vardef.ScopeNone, Name: "innodb_log_buffer_size", Value: "8388608"}, + {Scope: vardef.ScopeGlobal, Name: "delayed_insert_timeout", Value: "300"}, + {Scope: vardef.ScopeGlobal, Name: "max_relay_log_size", Value: "0"}, { - Scope: ScopeGlobal | ScopeSession, - Name: MaxSortLength, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.MaxSortLength, Value: "1024", - Type: TypeUnsigned, + Type: vardef.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"}, - {Scope: ScopeGlobal, Name: "binlog_max_flush_queue_time", Value: "0"}, - {Scope: ScopeGlobal, Name: "innodb_fill_factor", Value: ""}, - {Scope: ScopeGlobal, Name: "log_syslog_facility", Value: ""}, - {Scope: ScopeNone, Name: "innodb_ft_min_token_size", Value: "3"}, - {Scope: ScopeGlobal | ScopeSession, Name: "transaction_write_set_extraction", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: "ndb_blob_write_batch_bytes", Value: ""}, - {Scope: ScopeGlobal, Name: "automatic_sp_privileges", Value: "1"}, - {Scope: ScopeGlobal, Name: "innodb_flush_sync", Value: ""}, - {Scope: ScopeNone, Name: "performance_schema_events_statements_history_long_size", Value: "10000"}, - {Scope: ScopeGlobal, Name: "innodb_monitor_disable", Value: ""}, - {Scope: ScopeNone, Name: "innodb_doublewrite", Value: "1"}, - {Scope: ScopeNone, Name: "log_bin_use_v1_row_events", Value: "0"}, - {Scope: ScopeSession, Name: "innodb_optimize_point_storage", Value: ""}, - {Scope: ScopeNone, Name: "innodb_api_disable_rowlock", Value: "0"}, - {Scope: ScopeGlobal, Name: "innodb_adaptive_flushing_lwm", Value: "10"}, - {Scope: ScopeNone, Name: "innodb_log_files_in_group", Value: "2"}, - {Scope: ScopeGlobal, Name: InnodbBufferPoolLoadNow, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeNone, Name: "performance_schema_max_rwlock_classes", Value: "40"}, - {Scope: ScopeNone, Name: "binlog_gtid_simple_recovery", Value: "1"}, - {Scope: ScopeNone, Name: "performance_schema_digests_size", Value: "10000"}, - {Scope: ScopeGlobal | ScopeSession, Name: Profiling, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "sha256_password_proxy_users", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLQuoteShowCreate, Value: On, Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: "binlogging_impossible_mode", Value: "IGNORE_ERROR"}, - {Scope: ScopeGlobal, Name: "innodb_stats_transient_sample_pages", Value: "8"}, - {Scope: ScopeGlobal, Name: InnodbStatsOnMetadata, Value: "0"}, - {Scope: ScopeNone, Name: "server_uuid", Value: "00000000-0000-0000-0000-000000000000"}, - {Scope: ScopeNone, Name: "open_files_limit", Value: "5000"}, - {Scope: ScopeGlobal | ScopeSession, Name: "ndb_force_send", Value: ""}, - {Scope: ScopeNone, Name: "skip_show_database", Value: "0"}, - {Scope: ScopeGlobal, Name: "log_timestamps", Value: ""}, - {Scope: ScopeGlobal, Name: "event_scheduler", Value: Off}, - {Scope: ScopeGlobal | ScopeSession, Name: "ndb_deferred_constraints", Value: ""}, - {Scope: ScopeGlobal, Name: "log_syslog_include_pid", Value: ""}, - {Scope: ScopeNone, Name: "innodb_ft_cache_size", Value: "8000000"}, - {Scope: ScopeGlobal, Name: InnodbDisableSortFileCache, Value: "0"}, - {Scope: ScopeGlobal, Name: "log_error_verbosity", Value: ""}, - {Scope: ScopeNone, Name: "performance_schema_hosts_size", Value: "100"}, - {Scope: ScopeGlobal, Name: "innodb_replication_delay", Value: "0"}, - {Scope: ScopeGlobal, Name: SlowQueryLog, Value: "0"}, - {Scope: ScopeSession, Name: "debug_sync", Value: ""}, - {Scope: ScopeGlobal, Name: InnodbStatsAutoRecalc, Value: "1"}, + {Scope: vardef.ScopeNone, Name: "metadata_locks_hash_instances", Value: "8"}, + {Scope: vardef.ScopeGlobal, Name: "ndb_eventbuffer_free_percent", Value: ""}, + {Scope: vardef.ScopeNone, Name: "large_files_support", Value: "1"}, + {Scope: vardef.ScopeGlobal, Name: "binlog_max_flush_queue_time", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_fill_factor", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "log_syslog_facility", Value: ""}, + {Scope: vardef.ScopeNone, Name: "innodb_ft_min_token_size", Value: "3"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "transaction_write_set_extraction", Value: ""}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "ndb_blob_write_batch_bytes", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "automatic_sp_privileges", Value: "1"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_flush_sync", Value: ""}, + {Scope: vardef.ScopeNone, Name: "performance_schema_events_statements_history_long_size", Value: "10000"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_monitor_disable", Value: ""}, + {Scope: vardef.ScopeNone, Name: "innodb_doublewrite", Value: "1"}, + {Scope: vardef.ScopeNone, Name: "log_bin_use_v1_row_events", Value: "0"}, + {Scope: vardef.ScopeSession, Name: "innodb_optimize_point_storage", Value: ""}, + {Scope: vardef.ScopeNone, Name: "innodb_api_disable_rowlock", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_adaptive_flushing_lwm", Value: "10"}, + {Scope: vardef.ScopeNone, Name: "innodb_log_files_in_group", Value: "2"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbBufferPoolLoadNow, Value: vardef.Off, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_rwlock_classes", Value: "40"}, + {Scope: vardef.ScopeNone, Name: "binlog_gtid_simple_recovery", Value: "1"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_digests_size", Value: "10000"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.Profiling, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "sha256_password_proxy_users", Value: ""}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.SQLQuoteShowCreate, Value: vardef.On, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "binlogging_impossible_mode", Value: "IGNORE_ERROR"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_stats_transient_sample_pages", Value: "8"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbStatsOnMetadata, Value: "0"}, + {Scope: vardef.ScopeNone, Name: "server_uuid", Value: "00000000-0000-0000-0000-000000000000"}, + {Scope: vardef.ScopeNone, Name: "open_files_limit", Value: "5000"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "ndb_force_send", Value: ""}, + {Scope: vardef.ScopeNone, Name: "skip_show_database", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "log_timestamps", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "event_scheduler", Value: vardef.Off}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "ndb_deferred_constraints", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "log_syslog_include_pid", Value: ""}, + {Scope: vardef.ScopeNone, Name: "innodb_ft_cache_size", Value: "8000000"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbDisableSortFileCache, Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "log_error_verbosity", Value: ""}, + {Scope: vardef.ScopeNone, Name: "performance_schema_hosts_size", Value: "100"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_replication_delay", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: vardef.SlowQueryLog, Value: "0"}, + {Scope: vardef.ScopeSession, Name: "debug_sync", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: vardef.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: vardef.ScopeGlobal | vardef.ScopeSession, Name: "lc_messages", Value: "en_US"}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.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: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.BinlogDirectNonTransactionalUpdates, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "innodb_change_buffering", Value: "all"}, { - Scope: ScopeGlobal | ScopeSession, - Name: SQLBigSelects, - Value: On, - Type: TypeBool, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.SQLBigSelects, + Value: vardef.On, + Type: vardef.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: vardef.ScopeGlobal, Name: "innodb_max_purge_lag_delay", Value: "0"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "session_track_schema", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "innodb_io_capacity_max", Value: "2000"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_autoextend_increment", Value: "64"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "binlog_format", Value: "STATEMENT"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "optimizer_trace", Value: "enabled=off,one_line=off"}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.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"}, - {Scope: ScopeNone, Name: "performance_schema_setup_objects_size", Value: "100"}, - {Scope: ScopeGlobal, Name: "sync_relay_log", Value: "10000"}, - {Scope: ScopeGlobal, Name: "innodb_ft_result_cache_limit", Value: "2000000000"}, - {Scope: ScopeNone, Name: "innodb_sort_buffer_size", Value: "1048576"}, - {Scope: ScopeGlobal, Name: "innodb_ft_enable_diag_print", Type: TypeBool, Value: Off}, - {Scope: ScopeNone, Name: "thread_handling", Value: "one-thread-per-connection"}, - {Scope: ScopeGlobal, Name: "stored_program_cache", Value: "256"}, - {Scope: ScopeNone, Name: "performance_schema_max_mutex_instances", Value: "15906"}, - {Scope: ScopeGlobal, Name: "innodb_adaptive_max_sleep_delay", Value: "150000"}, - {Scope: ScopeNone, Name: "large_pages", Value: Off}, - {Scope: ScopeGlobal | ScopeSession, Name: "session_track_system_variables", Value: ""}, - {Scope: ScopeGlobal, Name: "innodb_change_buffer_max_size", Value: "25"}, - {Scope: ScopeGlobal, Name: LogBinTrustFunctionCreators, Value: Off, Type: TypeBool}, - {Scope: ScopeNone, Name: "innodb_write_io_threads", Value: "4"}, - {Scope: ScopeGlobal, Name: "mysql_native_password_proxy_users", Value: ""}, - {Scope: ScopeNone, Name: "large_page_size", Value: "0"}, - {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: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.NetWriteTimeout, Value: "60"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbBufferPoolLoadAbort, Value: vardef.Off, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "transaction_prealloc_size", Value: "4096"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_setup_objects_size", Value: "100"}, + {Scope: vardef.ScopeGlobal, Name: "sync_relay_log", Value: "10000"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_ft_result_cache_limit", Value: "2000000000"}, + {Scope: vardef.ScopeNone, Name: "innodb_sort_buffer_size", Value: "1048576"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_ft_enable_diag_print", Type: vardef.TypeBool, Value: vardef.Off}, + {Scope: vardef.ScopeNone, Name: "thread_handling", Value: "one-thread-per-connection"}, + {Scope: vardef.ScopeGlobal, Name: "stored_program_cache", Value: "256"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_mutex_instances", Value: "15906"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_adaptive_max_sleep_delay", Value: "150000"}, + {Scope: vardef.ScopeNone, Name: "large_pages", Value: vardef.Off}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "session_track_system_variables", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "innodb_change_buffer_max_size", Value: "25"}, + {Scope: vardef.ScopeGlobal, Name: vardef.LogBinTrustFunctionCreators, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeNone, Name: "innodb_write_io_threads", Value: "4"}, + {Scope: vardef.ScopeGlobal, Name: "mysql_native_password_proxy_users", Value: ""}, + {Scope: vardef.ScopeNone, Name: "large_page_size", Value: "0"}, + {Scope: vardef.ScopeNone, Name: "table_open_cache_instances", Value: "1"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbStatsPersistent, Value: vardef.On, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "session_track_state_change", Value: ""}, { - Scope: ScopeNone, - Name: OptimizerSwitch, + Scope: vardef.ScopeNone, + Name: vardef.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: ScopeGlobal, Name: "log_syslog", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: "transaction_alloc_block_size", Value: "8192"}, - {Scope: ScopeNone, Name: "performance_schema_max_cond_classes", Value: "80"}, - {Scope: ScopeGlobal, Name: "innodb_io_capacity", Value: "200"}, - {Scope: ScopeGlobal, Name: "max_binlog_cache_size", Value: "18446744073709547520"}, - {Scope: ScopeGlobal | ScopeSession, Name: "ndb_index_stat_enable", Value: ""}, - {Scope: ScopeGlobal, Name: "executed_gtids_compression_period", Value: ""}, - {Scope: ScopeNone, Name: "time_format", Value: "%H:%i:%s"}, - {Scope: ScopeGlobal | ScopeSession, Name: OldAlterTable, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: "long_query_time", Value: "10.000000"}, - {Scope: ScopeNone, Name: "innodb_use_native_aio", Value: "0"}, - {Scope: ScopeGlobal, Name: "log_throttle_queries_not_using_indexes", Value: "0"}, - {Scope: ScopeNone, Name: "locked_in_memory", Value: "0"}, - {Scope: ScopeNone, Name: "innodb_api_enable_mdl", Value: "0"}, - {Scope: ScopeGlobal, Name: "binlog_cache_size", Value: "32768"}, - {Scope: ScopeGlobal, Name: "innodb_compression_pad_pct_max", Value: "50"}, - {Scope: ScopeGlobal, Name: InnodbCommitConcurrency, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: 1000}, - {Scope: ScopeNone, Name: "ft_min_word_len", Value: "4"}, - {Scope: ScopeGlobal, Name: EnforceGtidConsistency, Value: Off, Type: TypeEnum, PossibleValues: []string{Off, On, "WARN"}}, - {Scope: ScopeGlobal, Name: SecureAuth, Value: On, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal, Name: "delayed_queue_size", Value: "1000"}, + {Scope: vardef.ScopeNone, Name: "innodb_read_only", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "log_syslog", Value: ""}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "transaction_alloc_block_size", Value: "8192"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_cond_classes", Value: "80"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_io_capacity", Value: "200"}, + {Scope: vardef.ScopeGlobal, Name: "max_binlog_cache_size", Value: "18446744073709547520"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "ndb_index_stat_enable", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "executed_gtids_compression_period", Value: ""}, + {Scope: vardef.ScopeNone, Name: "time_format", Value: "%H:%i:%s"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.OldAlterTable, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "long_query_time", Value: "10.000000"}, + {Scope: vardef.ScopeNone, Name: "innodb_use_native_aio", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "log_throttle_queries_not_using_indexes", Value: "0"}, + {Scope: vardef.ScopeNone, Name: "locked_in_memory", Value: "0"}, + {Scope: vardef.ScopeNone, Name: "innodb_api_enable_mdl", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "binlog_cache_size", Value: "32768"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_compression_pad_pct_max", Value: "50"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbCommitConcurrency, Value: "0", Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: 1000}, + {Scope: vardef.ScopeNone, Name: "ft_min_word_len", Value: "4"}, + {Scope: vardef.ScopeGlobal, Name: vardef.EnforceGtidConsistency, Value: vardef.Off, Type: vardef.TypeEnum, PossibleValues: []string{vardef.Off, vardef.On, "WARN"}}, + {Scope: vardef.ScopeGlobal, Name: vardef.SecureAuth, Value: vardef.On, Type: vardef.TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { if TiDBOptOn(normalizedValue) { - return On, nil + return vardef.On, nil } - return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(SecureAuth, originalValue) + return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(vardef.SecureAuth, originalValue) }}, - {Scope: ScopeGlobal, Name: InnodbRandomReadAhead, Value: Off, Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbRandomReadAhead, Value: vardef.Off, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, { - Scope: ScopeGlobal | ScopeSession, - Name: UniqueChecks, - Value: On, - Type: TypeBool, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.UniqueChecks, + Value: vardef.On, + Type: vardef.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: ""}, - {Scope: ScopeGlobal, Name: "innodb_read_ahead_threshold", Value: "56"}, - {Scope: ScopeGlobal, Name: "key_cache_block_size", Value: "1024"}, - {Scope: ScopeNone, Name: "ndb_recv_thread_cpu_mask", Value: ""}, - {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: vardef.ScopeGlobal, Name: "internal_tmp_disk_storage_engine", Value: ""}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "myisam_repair_threads", Value: "1"}, + {Scope: vardef.ScopeGlobal, Name: "ndb_eventbuffer_max_alloc", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "innodb_read_ahead_threshold", Value: "56"}, + {Scope: vardef.ScopeGlobal, Name: "key_cache_block_size", Value: "1024"}, + {Scope: vardef.ScopeNone, Name: "ndb_recv_thread_cpu_mask", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "gtid_purged", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "max_binlog_stmt_cache_size", Value: "18446744073709547520"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "lock_wait_timeout", Value: "31536000"}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.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"}, - {Scope: ScopeGlobal, Name: "slow_query_log_file", Value: "/usr/local/mysql/data/localhost-slow.log"}, - {Scope: ScopeGlobal, Name: "innodb_thread_sleep_delay", Value: "10000"}, - {Scope: ScopeGlobal, Name: "innodb_ft_aux_table", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLWarnings, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: KeepFilesOnCreate, Value: Off, Type: TypeBool}, - {Scope: ScopeNone, Name: "innodb_data_file_path", Value: "ibdata1:12M:autoextend"}, - {Scope: ScopeNone, Name: "performance_schema_setup_actors_size", Value: "100"}, - {Scope: ScopeNone, Name: "log_error", Value: "/usr/local/mysql/data/localhost.err"}, - {Scope: ScopeGlobal, Name: "binlog_stmt_cache_size", Value: "32768"}, - {Scope: ScopeNone, Name: "relay_log_info_file", Value: "relay-log.info"}, - {Scope: ScopeNone, Name: "innodb_ft_total_cache_size", Value: "640000000"}, - {Scope: ScopeNone, Name: "performance_schema_max_rwlock_instances", Value: "9102"}, - {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: vardef.ScopeNone, Name: "innodb_read_io_threads", Value: "4"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.MaxSpRecursionDepth, Value: "0", Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: 255}, + {Scope: vardef.ScopeNone, Name: "ignore_builtin_innodb", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "slow_query_log_file", Value: "/usr/local/mysql/data/localhost-slow.log"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_thread_sleep_delay", Value: "10000"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_ft_aux_table", Value: ""}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.SQLWarnings, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.KeepFilesOnCreate, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeNone, Name: "innodb_data_file_path", Value: "ibdata1:12M:autoextend"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_setup_actors_size", Value: "100"}, + {Scope: vardef.ScopeNone, Name: "log_error", Value: "/usr/local/mysql/data/localhost.err"}, + {Scope: vardef.ScopeGlobal, Name: "binlog_stmt_cache_size", Value: "32768"}, + {Scope: vardef.ScopeNone, Name: "relay_log_info_file", Value: "relay-log.info"}, + {Scope: vardef.ScopeNone, Name: "innodb_ft_total_cache_size", Value: "640000000"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_rwlock_instances", Value: "9102"}, + {Scope: vardef.ScopeGlobal, Name: "table_open_cache", Value: "2000"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_events_stages_history_long_size", Value: "10000"}, + {Scope: vardef.ScopeSession, Name: "insert_id", Value: ""}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "default_tmp_storage_engine", Value: "InnoDB", IsHintUpdatableVerified: true, }, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "optimizer_search_depth", Value: "62", IsHintUpdatableVerified: true, }, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "max_points_in_geometry", Value: "65536", IsHintUpdatableVerified: true, }, - {Scope: ScopeGlobal | ScopeSession, Name: "profiling_history_size", Value: "15"}, - {Scope: ScopeNone, Name: "have_symlink", Value: "YES"}, - {Scope: ScopeGlobal | ScopeSession, Name: "storage_engine", Value: "InnoDB"}, - {Scope: ScopeGlobal | ScopeSession, Name: "sql_log_off", Value: "0"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "profiling_history_size", Value: "15"}, + {Scope: vardef.ScopeNone, Name: "have_symlink", Value: "YES"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "storage_engine", Value: "InnoDB"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "sql_log_off", Value: "0"}, // In MySQL, the default value of `explicit_defaults_for_timestamp` is `0`. // But In TiDB, it's set to `1` to be consistent with TiDB timestamp behavior. // See: https://github.com/pingcap/tidb/pull/6068 for details - {Scope: ScopeNone, Name: "explicit_defaults_for_timestamp", Value: On, Type: TypeBool}, - {Scope: ScopeNone, Name: "performance_schema_events_waits_history_size", Value: "10"}, - {Scope: ScopeGlobal, Name: "log_syslog_tag", Value: ""}, - {Scope: ScopeGlobal, Name: "innodb_undo_log_truncate", Value: ""}, - {Scope: ScopeSession, Name: "innodb_create_intrinsic", Value: ""}, - {Scope: ScopeGlobal, Name: "gtid_executed_compression_period", Value: ""}, - {Scope: ScopeGlobal, Name: "ndb_log_empty_epochs", Value: ""}, - {Scope: ScopeNone, Name: "have_geometry", Value: "YES"}, - {Scope: ScopeGlobal | ScopeSession, Name: "optimizer_trace_max_mem_size", Value: "16384"}, - {Scope: ScopeGlobal | ScopeSession, Name: "net_retry_count", Value: "10"}, - {Scope: ScopeSession, Name: "ndb_table_no_logging", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: "optimizer_trace_features", Value: "greedy_search=on,range_optimizer=on,dynamic_range=on,repeated_subselect=on"}, - {Scope: ScopeGlobal, Name: "innodb_flush_log_at_trx_commit", Value: "1"}, - {Scope: ScopeGlobal, Name: "rewriter_enabled", Value: ""}, + {Scope: vardef.ScopeNone, Name: "explicit_defaults_for_timestamp", Value: vardef.On, Type: vardef.TypeBool}, + {Scope: vardef.ScopeNone, Name: "performance_schema_events_waits_history_size", Value: "10"}, + {Scope: vardef.ScopeGlobal, Name: "log_syslog_tag", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "innodb_undo_log_truncate", Value: ""}, + {Scope: vardef.ScopeSession, Name: "innodb_create_intrinsic", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "gtid_executed_compression_period", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "ndb_log_empty_epochs", Value: ""}, + {Scope: vardef.ScopeNone, Name: "have_geometry", Value: "YES"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "optimizer_trace_max_mem_size", Value: "16384"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "net_retry_count", Value: "10"}, + {Scope: vardef.ScopeSession, Name: "ndb_table_no_logging", Value: ""}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "optimizer_trace_features", Value: "greedy_search=on,range_optimizer=on,dynamic_range=on,repeated_subselect=on"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_flush_log_at_trx_commit", Value: "1"}, + {Scope: vardef.ScopeGlobal, Name: "rewriter_enabled", Value: ""}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "updatable_views_with_limit", Value: "YES", IsHintUpdatableVerified: true, }, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.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"}, - {Scope: ScopeGlobal | ScopeSession, Name: ShowOldTemporals, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "innodb_buffer_pool_size", Value: "4294967296"}, - {Scope: ScopeGlobal, Name: InnodbAdaptiveFlushing, Value: On, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeGlobal, Name: "innodb_monitor_enable", Value: ""}, - {Scope: ScopeGlobal, Name: "innodb_buffer_pool_filename", Value: "ib_buffer_pool"}, - {Scope: ScopeGlobal, Name: "slow_launch_time", Value: "2"}, - {Scope: ScopeGlobal | ScopeSession, Name: "ndb_use_transactions", Value: ""}, - {Scope: ScopeNone, Name: "innodb_purge_threads", Value: "1"}, - {Scope: ScopeGlobal, Name: "innodb_concurrency_tickets", Value: "5000"}, - {Scope: ScopeGlobal, Name: "innodb_monitor_reset_all", Value: ""}, - {Scope: ScopeNone, Name: "performance_schema_users_size", Value: "100"}, - {Scope: ScopeGlobal, Name: "ndb_log_updated_only", Value: ""}, - {Scope: ScopeNone, Name: "basedir", Value: "/usr/local/mysql"}, - {Scope: ScopeGlobal, Name: "innodb_old_blocks_time", Value: "1000"}, - {Scope: ScopeGlobal, Name: "innodb_stats_method", Value: "nulls_equal"}, - {Scope: ScopeGlobal, Name: LocalInFile, Value: On, Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: "myisam_stats_method", Value: "nulls_unequal"}, - {Scope: ScopeNone, Name: "relay_log_recovery", Value: "0"}, - {Scope: ScopeNone, Name: "old", Value: "0"}, - {Scope: ScopeGlobal | ScopeSession, Name: InnodbTableLocks, Value: On, Type: TypeBool, AutoConvertNegativeBool: true}, - {Scope: ScopeNone, Name: PerformanceSchema, Value: Off, Type: TypeBool}, - {Scope: ScopeNone, Name: "myisam_recover_options", Value: Off}, - {Scope: ScopeGlobal | ScopeSession, Name: NetBufferLength, Value: "16384", Type: TypeUnsigned, MinValue: 1024, MaxValue: 1048576}, - {Scope: ScopeGlobal | ScopeSession, Name: "binlog_row_image", Value: "FULL"}, - {Scope: ScopeSession, Name: "rbr_exec_mode", Value: ""}, - {Scope: ScopeGlobal, Name: "myisam_max_sort_file_size", Value: "9223372036853727232"}, - {Scope: ScopeNone, Name: "back_log", Value: "80"}, - {Scope: ScopeSession, Name: "pseudo_thread_id", Value: ""}, - {Scope: ScopeNone, Name: "have_dynamic_loading", Value: "YES"}, - {Scope: ScopeGlobal, Name: "rewriter_verbose", Value: ""}, - {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: vardef.ScopeGlobal | vardef.ScopeSession, Name: "completion_type", Value: "NO_CHAIN"}, + {Scope: vardef.ScopeGlobal, Name: "binlog_checksum", Value: "CRC32"}, + {Scope: vardef.ScopeNone, Name: "report_port", Value: "3306"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.ShowOldTemporals, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "innodb_buffer_pool_size", Value: "4294967296"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbAdaptiveFlushing, Value: vardef.On, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeGlobal, Name: "innodb_monitor_enable", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "innodb_buffer_pool_filename", Value: "ib_buffer_pool"}, + {Scope: vardef.ScopeGlobal, Name: "slow_launch_time", Value: "2"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "ndb_use_transactions", Value: ""}, + {Scope: vardef.ScopeNone, Name: "innodb_purge_threads", Value: "1"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_concurrency_tickets", Value: "5000"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_monitor_reset_all", Value: ""}, + {Scope: vardef.ScopeNone, Name: "performance_schema_users_size", Value: "100"}, + {Scope: vardef.ScopeGlobal, Name: "ndb_log_updated_only", Value: ""}, + {Scope: vardef.ScopeNone, Name: "basedir", Value: "/usr/local/mysql"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_old_blocks_time", Value: "1000"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_stats_method", Value: "nulls_equal"}, + {Scope: vardef.ScopeGlobal, Name: vardef.LocalInFile, Value: vardef.On, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "myisam_stats_method", Value: "nulls_unequal"}, + {Scope: vardef.ScopeNone, Name: "relay_log_recovery", Value: "0"}, + {Scope: vardef.ScopeNone, Name: "old", Value: "0"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.InnodbTableLocks, Value: vardef.On, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeNone, Name: vardef.PerformanceSchema, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeNone, Name: "myisam_recover_options", Value: vardef.Off}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.NetBufferLength, Value: "16384", Type: vardef.TypeUnsigned, MinValue: 1024, MaxValue: 1048576}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "binlog_row_image", Value: "FULL"}, + {Scope: vardef.ScopeSession, Name: "rbr_exec_mode", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "myisam_max_sort_file_size", Value: "9223372036853727232"}, + {Scope: vardef.ScopeNone, Name: "back_log", Value: "80"}, + {Scope: vardef.ScopeSession, Name: "pseudo_thread_id", Value: ""}, + {Scope: vardef.ScopeNone, Name: "have_dynamic_loading", Value: "YES"}, + {Scope: vardef.ScopeGlobal, Name: "rewriter_verbose", Value: ""}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_cond_instances", Value: "3504"}, + {Scope: vardef.ScopeGlobal, Name: "delayed_insert_limit", Value: "100"}, + {Scope: vardef.ScopeGlobal, Name: vardef.Flush, Value: vardef.Off, Type: vardef.TypeBool}, { - Scope: ScopeGlobal | ScopeSession, + Scope: vardef.ScopeGlobal | vardef.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"}, - {Scope: ScopeGlobal | ScopeSession, Name: "ndbinfo_show_hidden", Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: "net_read_timeout", Value: "30"}, - {Scope: ScopeNone, Name: "innodb_page_size", Value: "16384"}, - {Scope: ScopeNone, Name: "innodb_log_file_size", Value: "50331648"}, - {Scope: ScopeGlobal, Name: "sync_relay_log_info", Value: "10000"}, - {Scope: ScopeGlobal | ScopeSession, Name: "optimizer_trace_limit", Value: "1"}, - {Scope: ScopeNone, Name: "innodb_ft_max_token_size", Value: "84"}, - {Scope: ScopeGlobal, Name: "ndb_log_binlog_index", Value: ""}, - {Scope: ScopeGlobal, Name: "innodb_api_bk_commit_interval", Value: "5"}, - {Scope: ScopeNone, Name: "innodb_undo_directory", Value: "."}, - {Scope: ScopeNone, Name: "bind_address", Value: "*"}, - {Scope: ScopeGlobal, Name: "innodb_sync_spin_loops", Value: "30"}, + {Scope: vardef.ScopeNone, Name: "performance_schema_events_stages_history_size", Value: "10"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "ndb_join_pushdown", Value: ""}, + {Scope: vardef.ScopeNone, Name: "performance_schema_max_thread_instances", Value: "402"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "ndbinfo_show_hidden", Value: ""}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "net_read_timeout", Value: "30"}, + {Scope: vardef.ScopeNone, Name: "innodb_page_size", Value: "16384"}, + {Scope: vardef.ScopeNone, Name: "innodb_log_file_size", Value: "50331648"}, + {Scope: vardef.ScopeGlobal, Name: "sync_relay_log_info", Value: "10000"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "optimizer_trace_limit", Value: "1"}, + {Scope: vardef.ScopeNone, Name: "innodb_ft_max_token_size", Value: "84"}, + {Scope: vardef.ScopeGlobal, Name: "ndb_log_binlog_index", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "innodb_api_bk_commit_interval", Value: "5"}, + {Scope: vardef.ScopeNone, Name: "innodb_undo_directory", Value: "."}, + {Scope: vardef.ScopeNone, Name: "bind_address", Value: "*"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_sync_spin_loops", Value: "30"}, { - Scope: ScopeGlobal | ScopeSession, - Name: SQLSafeUpdates, - Value: Off, - Type: TypeBool, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.SQLSafeUpdates, + Value: vardef.Off, + Type: vardef.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: vardef.ScopeNone, Name: "tmpdir", Value: "/var/tmp/"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_thread_concurrency", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "innodb_buffer_pool_dump_pct", Value: ""}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "lc_time_names", Value: "en_US", ReadOnly: true}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "max_statement_time", Value: ""}, { - Scope: ScopeGlobal | ScopeSession, - Name: EndMarkersInJSON, - Value: Off, - Type: TypeBool, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.EndMarkersInJSON, + Value: vardef.Off, + Type: vardef.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}, - {Scope: ScopeGlobal | ScopeSession, Name: "min_examined_row_limit", Value: "0"}, - {Scope: ScopeGlobal, Name: "sync_frm", Type: TypeBool, Value: On}, - {Scope: ScopeGlobal, Name: "innodb_online_alter_log_max_size", Value: "134217728"}, - {Scope: ScopeGlobal | ScopeSession, Name: "information_schema_stats_expiry", Value: "86400"}, - {Scope: ScopeGlobal, Name: ThreadPoolSize, Value: "16", Type: TypeUnsigned, MinValue: 1, MaxValue: 64}, - {Scope: ScopeNone, Name: "lower_case_file_system", Value: "1"}, - {Scope: ScopeNone, Name: LowerCaseTableNames, Value: "2"}, + {Scope: vardef.ScopeGlobal, Name: vardef.AvoidTemporalUpgrade, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "key_cache_age_threshold", Value: "300"}, + {Scope: vardef.ScopeGlobal, Name: vardef.InnodbStatusOutput, Value: vardef.Off, Type: vardef.TypeBool, AutoConvertNegativeBool: true}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "min_examined_row_limit", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "sync_frm", Type: vardef.TypeBool, Value: vardef.On}, + {Scope: vardef.ScopeGlobal, Name: "innodb_online_alter_log_max_size", Value: "134217728"}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "information_schema_stats_expiry", Value: "86400"}, + {Scope: vardef.ScopeGlobal, Name: vardef.ThreadPoolSize, Value: "16", Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: 64}, + {Scope: vardef.ScopeNone, Name: "lower_case_file_system", Value: "1"}, + {Scope: vardef.ScopeNone, Name: vardef.LowerCaseTableNames, Value: "2"}, // for compatibility purpose, we should leave them alone. // TODO: Follow the Terminology Updates of MySQL after their changes arrived. // https://mysqlhighavailability.com/mysql-terminology-updates/ - {Scope: ScopeSession, Name: PseudoSlaveMode, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "slave_pending_jobs_size_max", Value: "16777216"}, - {Scope: ScopeGlobal, Name: "slave_transaction_retries", Value: "10"}, - {Scope: ScopeGlobal, Name: "slave_checkpoint_period", Value: "300"}, - {Scope: ScopeGlobal, Name: MasterVerifyChecksum, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_trace_level", Value: ""}, - {Scope: ScopeGlobal, Name: "master_info_repository", Value: "FILE"}, - {Scope: ScopeGlobal, Name: "rpl_stop_slave_timeout", Value: "31536000"}, - {Scope: ScopeGlobal, Name: "slave_net_timeout", Value: "3600"}, - {Scope: ScopeGlobal, Name: "sync_master_info", Value: "10000"}, - {Scope: ScopeGlobal, Name: "init_slave", Value: ""}, - {Scope: ScopeGlobal, Name: SlaveCompressedProtocol, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "rpl_semi_sync_slave_trace_level", Value: ""}, - {Scope: ScopeGlobal, Name: LogSlowSlaveStatements, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "slave_checkpoint_group", Value: "512"}, - {Scope: ScopeNone, Name: "slave_load_tmpdir", Value: "/var/tmp/"}, - {Scope: ScopeGlobal, Name: "slave_parallel_type", Value: ""}, - {Scope: ScopeGlobal, Name: "slave_parallel_workers", Value: "0"}, - {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_timeout", Value: "10000", Type: TypeInt, MaxValue: math.MaxInt64}, - {Scope: ScopeNone, Name: "slave_skip_errors", Value: Off}, - {Scope: ScopeGlobal, Name: "sql_slave_skip_counter", Value: "0"}, - {Scope: ScopeGlobal, Name: "rpl_semi_sync_slave_enabled", Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_enabled", Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "slave_preserve_commit_order", Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "slave_exec_mode", Value: "STRICT"}, - {Scope: ScopeNone, Name: "log_slave_updates", Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_wait_point", Value: "AFTER_SYNC", Type: TypeEnum, PossibleValues: []string{"AFTER_SYNC", "AFTER_COMMIT"}}, - {Scope: ScopeGlobal, Name: "slave_sql_verify_checksum", Value: On, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "slave_max_allowed_packet", Value: "1073741824"}, - {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_wait_for_slave_count", Value: "1", Type: TypeInt, MinValue: 1, MaxValue: 65535}, - {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_wait_no_slave", Value: On, Type: TypeBool}, - {Scope: ScopeGlobal, Name: "slave_rows_search_algorithms", Value: "TABLE_SCAN,INDEX_SCAN"}, - {Scope: ScopeGlobal, Name: SlaveAllowBatching, Value: Off, Type: TypeBool}, + {Scope: vardef.ScopeSession, Name: vardef.PseudoSlaveMode, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "slave_pending_jobs_size_max", Value: "16777216"}, + {Scope: vardef.ScopeGlobal, Name: "slave_transaction_retries", Value: "10"}, + {Scope: vardef.ScopeGlobal, Name: "slave_checkpoint_period", Value: "300"}, + {Scope: vardef.ScopeGlobal, Name: vardef.MasterVerifyChecksum, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "rpl_semi_sync_master_trace_level", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "master_info_repository", Value: "FILE"}, + {Scope: vardef.ScopeGlobal, Name: "rpl_stop_slave_timeout", Value: "31536000"}, + {Scope: vardef.ScopeGlobal, Name: "slave_net_timeout", Value: "3600"}, + {Scope: vardef.ScopeGlobal, Name: "sync_master_info", Value: "10000"}, + {Scope: vardef.ScopeGlobal, Name: "init_slave", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: vardef.SlaveCompressedProtocol, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "rpl_semi_sync_slave_trace_level", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: vardef.LogSlowSlaveStatements, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "slave_checkpoint_group", Value: "512"}, + {Scope: vardef.ScopeNone, Name: "slave_load_tmpdir", Value: "/var/tmp/"}, + {Scope: vardef.ScopeGlobal, Name: "slave_parallel_type", Value: ""}, + {Scope: vardef.ScopeGlobal, Name: "slave_parallel_workers", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "rpl_semi_sync_master_timeout", Value: "10000", Type: vardef.TypeInt, MaxValue: math.MaxInt64}, + {Scope: vardef.ScopeNone, Name: "slave_skip_errors", Value: vardef.Off}, + {Scope: vardef.ScopeGlobal, Name: "sql_slave_skip_counter", Value: "0"}, + {Scope: vardef.ScopeGlobal, Name: "rpl_semi_sync_slave_enabled", Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "rpl_semi_sync_master_enabled", Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "slave_preserve_commit_order", Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "slave_exec_mode", Value: "STRICT"}, + {Scope: vardef.ScopeNone, Name: "log_slave_updates", Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "rpl_semi_sync_master_wait_point", Value: "AFTER_SYNC", Type: vardef.TypeEnum, PossibleValues: []string{"AFTER_SYNC", "AFTER_COMMIT"}}, + {Scope: vardef.ScopeGlobal, Name: "slave_sql_verify_checksum", Value: vardef.On, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "slave_max_allowed_packet", Value: "1073741824"}, + {Scope: vardef.ScopeGlobal, Name: "rpl_semi_sync_master_wait_for_slave_count", Value: "1", Type: vardef.TypeInt, MinValue: 1, MaxValue: 65535}, + {Scope: vardef.ScopeGlobal, Name: "rpl_semi_sync_master_wait_no_slave", Value: vardef.On, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: "slave_rows_search_algorithms", Value: "TABLE_SCAN,INDEX_SCAN"}, + {Scope: vardef.ScopeGlobal, Name: vardef.SlaveAllowBatching, Value: vardef.Off, Type: vardef.TypeBool}, } diff --git a/pkg/sessionctx/variable/removed_test.go b/pkg/sessionctx/variable/removed_test.go index 44d011650a17b..5490a54250bd5 100644 --- a/pkg/sessionctx/variable/removed_test.go +++ b/pkg/sessionctx/variable/removed_test.go @@ -17,12 +17,13 @@ package variable import ( "testing" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/stretchr/testify/require" ) func TestRemovedOpt(t *testing.T) { - require.NoError(t, CheckSysVarIsRemoved(TiDBEnable1PC)) - require.False(t, IsRemovedSysVar(TiDBEnable1PC)) + require.NoError(t, CheckSysVarIsRemoved(vardef.TiDBEnable1PC)) + require.False(t, IsRemovedSysVar(vardef.TiDBEnable1PC)) require.Error(t, CheckSysVarIsRemoved(tiDBEnableAlterPlacement)) require.True(t, IsRemovedSysVar(tiDBEnableAlterPlacement)) } diff --git a/pkg/sessionctx/variable/session.go b/pkg/sessionctx/variable/session.go index af47002f2af40..6863d9b838dea 100644 --- a/pkg/sessionctx/variable/session.go +++ b/pkg/sessionctx/variable/session.go @@ -47,6 +47,7 @@ import ( "github.com/pingcap/tidb/pkg/resourcegroup" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" @@ -657,7 +658,7 @@ func validateReadConsistencyLevel(val string) error { case ReadConsistencyStrict, ReadConsistencyWeak: return nil default: - return ErrWrongTypeForVar.GenWithStackByArgs(TiDBReadConsistency) + return ErrWrongTypeForVar.GenWithStackByArgs(vardef.TiDBReadConsistency) } } @@ -1231,7 +1232,7 @@ type SessionVars struct { mppVersion kv.MppVersion - mppExchangeCompressionMode kv.ExchangeCompressionMode + mppExchangeCompressionMode vardef.ExchangeCompressionMode PlannerSelectBlockAsName atomic.Pointer[[]ast.HintTable] @@ -1281,7 +1282,7 @@ type SessionVars struct { SelectLimit uint64 // EnableClusteredIndex indicates whether to enable clustered index when creating a new table. - EnableClusteredIndex ClusteredIndexDefMode + EnableClusteredIndex vardef.ClusteredIndexDefMode // EnableParallelApply indicates that whether to use parallel apply. EnableParallelApply bool @@ -1844,10 +1845,10 @@ func (s *SessionVars) ChooseMppVersion() kv.MppVersion { } // ChooseMppExchangeCompressionMode indicates the data compression method in mpp exchange operator -func (s *SessionVars) ChooseMppExchangeCompressionMode() kv.ExchangeCompressionMode { - if s.mppExchangeCompressionMode == kv.ExchangeCompressionModeUnspecified { +func (s *SessionVars) ChooseMppExchangeCompressionMode() vardef.ExchangeCompressionMode { + if s.mppExchangeCompressionMode == vardef.ExchangeCompressionModeUnspecified { // If unspecified, use recommended mode - return kv.RecommendedExchangeCompressionMode + return vardef.RecommendedExchangeCompressionMode } return s.mppExchangeCompressionMode } @@ -1868,7 +1869,7 @@ func (s *SessionVars) RaiseWarningWhenMPPEnforced(warning string) { // CheckAndGetTxnScope will return the transaction scope we should use in the current session. func (s *SessionVars) CheckAndGetTxnScope() string { - if s.InRestrictedSQL || !EnableLocalTxn.Load() { + if s.InRestrictedSQL || !vardef.EnableLocalTxn.Load() { return kv.GlobalTxnScope } if s.TxnScope.GetVarValue() == kv.LocalTxnScope { @@ -1914,13 +1915,6 @@ func (s *SessionVars) GetExecuteDuration() time.Duration { return time.Since(s.StartTime) - s.DurationCompile } -const ( - // PlacementModeStrict indicates all placement operations should be checked strictly in ddl - PlacementModeStrict string = "STRICT" - // PlacementModeIgnore indicates ignore all placement operations in ddl - PlacementModeIgnore string = "IGNORE" -) - // PartitionPruneMode presents the prune mode used. type PartitionPruneMode string @@ -2105,147 +2099,147 @@ func NewSessionVars(hctx HookContext) *SessionVars { TxnCtx: &TransactionContext{}, RetryInfo: &RetryInfo{}, ActiveRoles: make([]*auth.RoleIdentity, 0, 10), - AutoIncrementIncrement: DefAutoIncrementIncrement, - AutoIncrementOffset: DefAutoIncrementOffset, + AutoIncrementIncrement: vardef.DefAutoIncrementIncrement, + AutoIncrementOffset: vardef.DefAutoIncrementOffset, StmtCtx: stmtctx.NewStmtCtx(), AllowAggPushDown: false, - AllowCartesianBCJ: DefOptCartesianBCJ, - MPPOuterJoinFixedBuildSide: DefOptMPPOuterJoinFixedBuildSide, - BroadcastJoinThresholdSize: DefBroadcastJoinThresholdSize, - BroadcastJoinThresholdCount: DefBroadcastJoinThresholdSize, - OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, - EnableOuterJoinReorder: DefTiDBEnableOuterJoinReorder, - RetryLimit: DefTiDBRetryLimit, - DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, + AllowCartesianBCJ: vardef.DefOptCartesianBCJ, + MPPOuterJoinFixedBuildSide: vardef.DefOptMPPOuterJoinFixedBuildSide, + BroadcastJoinThresholdSize: vardef.DefBroadcastJoinThresholdSize, + BroadcastJoinThresholdCount: vardef.DefBroadcastJoinThresholdSize, + OptimizerSelectivityLevel: vardef.DefTiDBOptimizerSelectivityLevel, + EnableOuterJoinReorder: vardef.DefTiDBEnableOuterJoinReorder, + RetryLimit: vardef.DefTiDBRetryLimit, + DisableTxnAutoRetry: vardef.DefTiDBDisableTxnAutoRetry, DDLReorgPriority: kv.PriorityLow, - allowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, - preferRangeScan: DefOptPreferRangeScan, - EnableCorrelationAdjustment: DefOptEnableCorrelationAdjustment, - LimitPushDownThreshold: DefOptLimitPushDownThreshold, - CorrelationThreshold: DefOptCorrelationThreshold, - CorrelationExpFactor: DefOptCorrelationExpFactor, - cpuFactor: DefOptCPUFactor, - copCPUFactor: DefOptCopCPUFactor, - CopTiFlashConcurrencyFactor: DefOptTiFlashConcurrencyFactor, - networkFactor: DefOptNetworkFactor, - scanFactor: DefOptScanFactor, - descScanFactor: DefOptDescScanFactor, - seekFactor: DefOptSeekFactor, - memoryFactor: DefOptMemoryFactor, - diskFactor: DefOptDiskFactor, - concurrencyFactor: DefOptConcurrencyFactor, - enableForceInlineCTE: DefOptForceInlineCTE, - EnableVectorizedExpression: DefEnableVectorizedExpression, + allowInSubqToJoinAndAgg: vardef.DefOptInSubqToJoinAndAgg, + preferRangeScan: vardef.DefOptPreferRangeScan, + EnableCorrelationAdjustment: vardef.DefOptEnableCorrelationAdjustment, + LimitPushDownThreshold: vardef.DefOptLimitPushDownThreshold, + CorrelationThreshold: vardef.DefOptCorrelationThreshold, + CorrelationExpFactor: vardef.DefOptCorrelationExpFactor, + cpuFactor: vardef.DefOptCPUFactor, + copCPUFactor: vardef.DefOptCopCPUFactor, + CopTiFlashConcurrencyFactor: vardef.DefOptTiFlashConcurrencyFactor, + networkFactor: vardef.DefOptNetworkFactor, + scanFactor: vardef.DefOptScanFactor, + descScanFactor: vardef.DefOptDescScanFactor, + seekFactor: vardef.DefOptSeekFactor, + memoryFactor: vardef.DefOptMemoryFactor, + diskFactor: vardef.DefOptDiskFactor, + concurrencyFactor: vardef.DefOptConcurrencyFactor, + enableForceInlineCTE: vardef.DefOptForceInlineCTE, + EnableVectorizedExpression: vardef.DefEnableVectorizedExpression, CommandValue: uint32(mysql.ComSleep), - TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, + TiDBOptJoinReorderThreshold: vardef.DefTiDBOptJoinReorderThreshold, SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, - WaitSplitRegionFinish: DefTiDBWaitSplitRegionFinish, - WaitSplitRegionTimeout: DefWaitSplitRegionTimeout, - enableIndexMerge: DefTiDBEnableIndexMerge, - NoopFuncsMode: TiDBOptOnOffWarn(DefTiDBEnableNoopFuncs), + WaitSplitRegionFinish: vardef.DefTiDBWaitSplitRegionFinish, + WaitSplitRegionTimeout: vardef.DefWaitSplitRegionTimeout, + enableIndexMerge: vardef.DefTiDBEnableIndexMerge, + NoopFuncsMode: TiDBOptOnOffWarn(vardef.DefTiDBEnableNoopFuncs), replicaRead: kv.ReplicaReadLeader, - AllowRemoveAutoInc: DefTiDBAllowRemoveAutoInc, - UsePlanBaselines: DefTiDBUsePlanBaselines, - EvolvePlanBaselines: DefTiDBEvolvePlanBaselines, + AllowRemoveAutoInc: vardef.DefTiDBAllowRemoveAutoInc, + UsePlanBaselines: vardef.DefTiDBUsePlanBaselines, + EvolvePlanBaselines: vardef.DefTiDBEvolvePlanBaselines, EnableExtendedStats: false, IsolationReadEngines: make(map[kv.StoreType]struct{}), - LockWaitTimeout: DefInnodbLockWaitTimeout * 1000, - MetricSchemaStep: DefTiDBMetricSchemaStep, - MetricSchemaRangeDuration: DefTiDBMetricSchemaRangeDuration, + LockWaitTimeout: vardef.DefInnodbLockWaitTimeout * 1000, + MetricSchemaStep: vardef.DefTiDBMetricSchemaStep, + MetricSchemaRangeDuration: vardef.DefTiDBMetricSchemaRangeDuration, SequenceState: NewSequenceState(), WindowingUseHighPrecision: true, - PrevFoundInPlanCache: DefTiDBFoundInPlanCache, - FoundInPlanCache: DefTiDBFoundInPlanCache, - PrevFoundInBinding: DefTiDBFoundInBinding, - FoundInBinding: DefTiDBFoundInBinding, + PrevFoundInPlanCache: vardef.DefTiDBFoundInPlanCache, + FoundInPlanCache: vardef.DefTiDBFoundInPlanCache, + PrevFoundInBinding: vardef.DefTiDBFoundInBinding, + FoundInBinding: vardef.DefTiDBFoundInBinding, SelectLimit: math.MaxUint64, - AllowAutoRandExplicitInsert: DefTiDBAllowAutoRandExplicitInsert, - EnableClusteredIndex: DefTiDBEnableClusteredIndex, - EnableParallelApply: DefTiDBEnableParallelApply, - ShardAllocateStep: DefTiDBShardAllocateStep, - PartitionPruneMode: *atomic2.NewString(DefTiDBPartitionPruneMode), + AllowAutoRandExplicitInsert: vardef.DefTiDBAllowAutoRandExplicitInsert, + EnableClusteredIndex: vardef.DefTiDBEnableClusteredIndex, + EnableParallelApply: vardef.DefTiDBEnableParallelApply, + ShardAllocateStep: vardef.DefTiDBShardAllocateStep, + PartitionPruneMode: *atomic2.NewString(vardef.DefTiDBPartitionPruneMode), TxnScope: kv.NewDefaultTxnScopeVar(), - EnabledRateLimitAction: DefTiDBEnableRateLimitAction, - EnableAsyncCommit: DefTiDBEnableAsyncCommit, - Enable1PC: DefTiDBEnable1PC, - GuaranteeLinearizability: DefTiDBGuaranteeLinearizability, - AnalyzeVersion: DefTiDBAnalyzeVersion, - EnableIndexMergeJoin: DefTiDBEnableIndexMergeJoin, + EnabledRateLimitAction: vardef.DefTiDBEnableRateLimitAction, + EnableAsyncCommit: vardef.DefTiDBEnableAsyncCommit, + Enable1PC: vardef.DefTiDBEnable1PC, + GuaranteeLinearizability: vardef.DefTiDBGuaranteeLinearizability, + AnalyzeVersion: vardef.DefTiDBAnalyzeVersion, + EnableIndexMergeJoin: vardef.DefTiDBEnableIndexMergeJoin, AllowFallbackToTiKV: make(map[kv.StoreType]struct{}), - CTEMaxRecursionDepth: DefCTEMaxRecursionDepth, - TMPTableSize: DefTiDBTmpTableMaxSize, - MPPStoreFailTTL: DefTiDBMPPStoreFailTTL, + CTEMaxRecursionDepth: vardef.DefCTEMaxRecursionDepth, + TMPTableSize: vardef.DefTiDBTmpTableMaxSize, + MPPStoreFailTTL: vardef.DefTiDBMPPStoreFailTTL, Rng: mathutil.NewWithTime(), - EnableLegacyInstanceScope: DefEnableLegacyInstanceScope, - RemoveOrderbyInSubquery: DefTiDBRemoveOrderbyInSubquery, - EnableSkewDistinctAgg: DefTiDBSkewDistinctAgg, - Enable3StageDistinctAgg: DefTiDB3StageDistinctAgg, - MaxAllowedPacket: DefMaxAllowedPacket, - TiFlashFastScan: DefTiFlashFastScan, + EnableLegacyInstanceScope: vardef.DefEnableLegacyInstanceScope, + RemoveOrderbyInSubquery: vardef.DefTiDBRemoveOrderbyInSubquery, + EnableSkewDistinctAgg: vardef.DefTiDBSkewDistinctAgg, + Enable3StageDistinctAgg: vardef.DefTiDB3StageDistinctAgg, + MaxAllowedPacket: vardef.DefMaxAllowedPacket, + TiFlashFastScan: vardef.DefTiFlashFastScan, EnableTiFlashReadForWriteStmt: true, - ForeignKeyChecks: DefTiDBForeignKeyChecks, + ForeignKeyChecks: vardef.DefTiDBForeignKeyChecks, HookContext: hctx, - EnableReuseChunk: DefTiDBEnableReusechunk, - preUseChunkAlloc: DefTiDBUseAlloc, + EnableReuseChunk: vardef.DefTiDBEnableReusechunk, + preUseChunkAlloc: vardef.DefTiDBUseAlloc, chunkPool: nil, - mppExchangeCompressionMode: DefaultExchangeCompressionMode, + mppExchangeCompressionMode: vardef.DefaultExchangeCompressionMode, mppVersion: kv.MppVersionUnspecified, - EnableLateMaterialization: DefTiDBOptEnableLateMaterialization, + EnableLateMaterialization: vardef.DefTiDBOptEnableLateMaterialization, TiFlashComputeDispatchPolicy: tiflashcompute.DispatchPolicyConsistentHash, ResourceGroupName: resourcegroup.DefaultResourceGroupName, DefaultCollationForUTF8MB4: mysql.DefaultCollationName, - GroupConcatMaxLen: DefGroupConcatMaxLen, - EnableRedactLog: DefTiDBRedactLog, - EnableWindowFunction: DefEnableWindowFunction, + GroupConcatMaxLen: vardef.DefGroupConcatMaxLen, + EnableRedactLog: vardef.DefTiDBRedactLog, + EnableWindowFunction: vardef.DefEnableWindowFunction, } vars.status.Store(uint32(mysql.ServerStatusAutocommit)) vars.StmtCtx.ResourceGroupName = resourcegroup.DefaultResourceGroupName vars.KVVars = tikvstore.NewVariables(&vars.SQLKiller.Signal) vars.Concurrency = Concurrency{ - indexLookupConcurrency: DefIndexLookupConcurrency, - indexSerialScanConcurrency: DefIndexSerialScanConcurrency, - indexLookupJoinConcurrency: DefIndexLookupJoinConcurrency, - hashJoinConcurrency: DefTiDBHashJoinConcurrency, - projectionConcurrency: DefTiDBProjectionConcurrency, - distSQLScanConcurrency: DefDistSQLScanConcurrency, - analyzeDistSQLScanConcurrency: DefAnalyzeDistSQLScanConcurrency, - hashAggPartialConcurrency: DefTiDBHashAggPartialConcurrency, - hashAggFinalConcurrency: DefTiDBHashAggFinalConcurrency, - windowConcurrency: DefTiDBWindowConcurrency, - mergeJoinConcurrency: DefTiDBMergeJoinConcurrency, - streamAggConcurrency: DefTiDBStreamAggConcurrency, - indexMergeIntersectionConcurrency: DefTiDBIndexMergeIntersectionConcurrency, - ExecutorConcurrency: DefExecutorConcurrency, + indexLookupConcurrency: vardef.DefIndexLookupConcurrency, + indexSerialScanConcurrency: vardef.DefIndexSerialScanConcurrency, + indexLookupJoinConcurrency: vardef.DefIndexLookupJoinConcurrency, + hashJoinConcurrency: vardef.DefTiDBHashJoinConcurrency, + projectionConcurrency: vardef.DefTiDBProjectionConcurrency, + distSQLScanConcurrency: vardef.DefDistSQLScanConcurrency, + analyzeDistSQLScanConcurrency: vardef.DefAnalyzeDistSQLScanConcurrency, + hashAggPartialConcurrency: vardef.DefTiDBHashAggPartialConcurrency, + hashAggFinalConcurrency: vardef.DefTiDBHashAggFinalConcurrency, + windowConcurrency: vardef.DefTiDBWindowConcurrency, + mergeJoinConcurrency: vardef.DefTiDBMergeJoinConcurrency, + streamAggConcurrency: vardef.DefTiDBStreamAggConcurrency, + indexMergeIntersectionConcurrency: vardef.DefTiDBIndexMergeIntersectionConcurrency, + ExecutorConcurrency: vardef.DefExecutorConcurrency, } vars.MemQuota = MemQuota{ - MemQuotaQuery: DefTiDBMemQuotaQuery, - MemQuotaApplyCache: DefTiDBMemQuotaApplyCache, + MemQuotaQuery: vardef.DefTiDBMemQuotaQuery, + MemQuotaApplyCache: vardef.DefTiDBMemQuotaApplyCache, } vars.BatchSize = BatchSize{ - IndexJoinBatchSize: DefIndexJoinBatchSize, - IndexLookupSize: DefIndexLookupSize, - InitChunkSize: DefInitChunkSize, - MaxChunkSize: DefMaxChunkSize, - MinPagingSize: DefMinPagingSize, - MaxPagingSize: DefMaxPagingSize, - } - vars.DMLBatchSize = DefDMLBatchSize - vars.AllowBatchCop = DefTiDBAllowBatchCop - vars.allowMPPExecution = DefTiDBAllowMPPExecution - vars.HashExchangeWithNewCollation = DefTiDBHashExchangeWithNewCollation - vars.enforceMPPExecution = DefTiDBEnforceMPPExecution - vars.TiFlashMaxThreads = DefTiFlashMaxThreads - vars.TiFlashMaxBytesBeforeExternalJoin = DefTiFlashMaxBytesBeforeExternalJoin - vars.TiFlashMaxBytesBeforeExternalGroupBy = DefTiFlashMaxBytesBeforeExternalGroupBy - vars.TiFlashMaxBytesBeforeExternalSort = DefTiFlashMaxBytesBeforeExternalSort - vars.TiFlashMaxQueryMemoryPerNode = DefTiFlashMemQuotaQueryPerNode - vars.TiFlashQuerySpillRatio = DefTiFlashQuerySpillRatio - vars.MPPStoreFailTTL = DefTiDBMPPStoreFailTTL + IndexJoinBatchSize: vardef.DefIndexJoinBatchSize, + IndexLookupSize: vardef.DefIndexLookupSize, + InitChunkSize: vardef.DefInitChunkSize, + MaxChunkSize: vardef.DefMaxChunkSize, + MinPagingSize: vardef.DefMinPagingSize, + MaxPagingSize: vardef.DefMaxPagingSize, + } + vars.DMLBatchSize = vardef.DefDMLBatchSize + vars.AllowBatchCop = vardef.DefTiDBAllowBatchCop + vars.allowMPPExecution = vardef.DefTiDBAllowMPPExecution + vars.HashExchangeWithNewCollation = vardef.DefTiDBHashExchangeWithNewCollation + vars.enforceMPPExecution = vardef.DefTiDBEnforceMPPExecution + vars.TiFlashMaxThreads = vardef.DefTiFlashMaxThreads + vars.TiFlashMaxBytesBeforeExternalJoin = vardef.DefTiFlashMaxBytesBeforeExternalJoin + vars.TiFlashMaxBytesBeforeExternalGroupBy = vardef.DefTiFlashMaxBytesBeforeExternalGroupBy + vars.TiFlashMaxBytesBeforeExternalSort = vardef.DefTiFlashMaxBytesBeforeExternalSort + vars.TiFlashMaxQueryMemoryPerNode = vardef.DefTiFlashMemQuotaQueryPerNode + vars.TiFlashQuerySpillRatio = vardef.DefTiFlashQuerySpillRatio + vars.MPPStoreFailTTL = vardef.DefTiDBMPPStoreFailTTL vars.DiskTracker = disk.NewTracker(memory.LabelForSession, -1) vars.MemTracker = memory.NewTracker(memory.LabelForSession, vars.MemQuotaQuery) vars.MemTracker.IsRootTrackerOfSess = true vars.MemTracker.Killer = &vars.SQLKiller - vars.StatsLoadSyncWait.Store(StatsLoadSyncWait.Load()) + vars.StatsLoadSyncWait.Store(vardef.StatsLoadSyncWait.Load()) for _, engine := range config.GetGlobalConfig().IsolationRead.Engines { switch engine { @@ -2257,13 +2251,13 @@ func NewSessionVars(hctx HookContext) *SessionVars { vars.IsolationReadEngines[kv.TiDB] = struct{}{} } } - if !EnableLocalTxn.Load() { + if !vardef.EnableLocalTxn.Load() { vars.TxnScope = kv.NewGlobalTxnScopeVar() } - if EnableRowLevelChecksum.Load() { + if vardef.EnableRowLevelChecksum.Load() { vars.EnableRowLevelChecksum = true } - vars.systems[CharacterSetConnection], vars.systems[CollationConnection] = charset.GetDefaultCharsetAndCollate() + vars.systems[vardef.CharacterSetConnection], vars.systems[vardef.CollationConnection] = charset.GetDefaultCharsetAndCollate() return vars } @@ -2385,15 +2379,15 @@ func (s *SessionVars) RegisterScalarSubQ(scalarSubQ any) { // have their own collation, which has a higher collation precedence. // See https://dev.mysql.com/doc/refman/5.7/en/charset-connection.html func (s *SessionVars) GetCharsetInfo() (charset, collation string) { - charset = s.systems[CharacterSetConnection] - collation = s.systems[CollationConnection] + charset = s.systems[vardef.CharacterSetConnection] + collation = s.systems[vardef.CollationConnection] return } // GetParseParams gets the parse parameters from session variables. func (s *SessionVars) GetParseParams() []parser.ParseParam { chs, coll := s.GetCharsetInfo() - cli, err := s.GetSessionOrGlobalSystemVar(context.Background(), CharacterSetClient) + cli, err := s.GetSessionOrGlobalSystemVar(context.Background(), vardef.CharacterSetClient) if err != nil { cli = "" } @@ -2487,7 +2481,7 @@ func (s *SessionVars) IsIsolation(isolation string) bool { s.TxnCtx.Isolation = s.txnIsolationLevelOneShot.value } if s.TxnCtx.Isolation == "" { - s.TxnCtx.Isolation, _ = s.GetSystemVar(TxnIsolation) + s.TxnCtx.Isolation, _ = s.GetSystemVar(vardef.TxnIsolation) } return s.TxnCtx.Isolation == isolation } @@ -2505,7 +2499,7 @@ func (s *SessionVars) IsolationLevelForNewTxn() (isolation string) { } if isolation == "" { - isolation, _ = s.GetSystemVar(TxnIsolation) + isolation, _ = s.GetSystemVar(vardef.TxnIsolation) } return @@ -2551,9 +2545,9 @@ func (s *SessionVars) Location() *time.Location { // GetSystemVar gets the string value of a system variable. func (s *SessionVars) GetSystemVar(name string) (string, bool) { - if name == WarningCount { + if name == vardef.WarningCount { return strconv.Itoa(s.SysWarningCount), true - } else if name == ErrorCount { + } else if name == vardef.ErrorCount { return strconv.Itoa(int(s.SysErrorCount)), true } val, ok := s.systems[name] @@ -2600,7 +2594,7 @@ func (s *SessionVars) GetNonPreparedPlanCacheStmt(sql string) any { // AddPreparedStmt adds prepareStmt to current session and count in global. func (s *SessionVars) AddPreparedStmt(stmtID uint32, stmt any) error { if _, exists := s.PreparedStmts[stmtID]; !exists { - maxPreparedStmtCount := MaxPreparedStmtCountValue.Load() + maxPreparedStmtCount := vardef.MaxPreparedStmtCountValue.Load() newPreparedStmtCount := atomic.AddInt64(&PreparedStmtCount, 1) if maxPreparedStmtCount >= 0 && newPreparedStmtCount > maxPreparedStmtCount { atomic.AddInt64(&PreparedStmtCount, -1) @@ -2706,7 +2700,7 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { if sv == nil { return ErrUnknownSystemVar.GenWithStackByArgs(name) } - val, err := sv.Validate(s, val, ScopeSession) + val, err := sv.Validate(s, val, vardef.ScopeSession) if err != nil { return err } @@ -2719,7 +2713,7 @@ func (s *SessionVars) SetSystemVarWithOldValAsRet(name string, val string) (stri if sv == nil { return "", ErrUnknownSystemVar.GenWithStackByArgs(name) } - val, err := sv.Validate(s, val, ScopeSession) + val, err := sv.Validate(s, val, vardef.ScopeSession) if err != nil { return "", err } @@ -2750,7 +2744,7 @@ func (s *SessionVars) SetSystemVarWithRelaxedValidation(name string, val string) if sv == nil { return ErrUnknownSystemVar.GenWithStackByArgs(name) } - val = sv.ValidateWithRelaxedValidation(s, val, ScopeSession) + val = sv.ValidateWithRelaxedValidation(s, val, vardef.ScopeSession) return sv.SetSessionFromHook(s, val) } @@ -2903,9 +2897,6 @@ func (td TableDelta) Clone() TableDelta { } } -// ConcurrencyUnset means the value the of the concurrency related variable is unset. -const ConcurrencyUnset = -1 - // Concurrency defines concurrency values. type Concurrency struct { // indexLookupConcurrency is the number of concurrent index lookup worker. @@ -3036,7 +3027,7 @@ func (c *Concurrency) SetIndexSerialScanConcurrency(n int) { // IndexLookupConcurrency return the number of concurrent index lookup worker. func (c *Concurrency) IndexLookupConcurrency() int { - if c.indexLookupConcurrency != ConcurrencyUnset { + if c.indexLookupConcurrency != vardef.ConcurrencyUnset { return c.indexLookupConcurrency } return c.ExecutorConcurrency @@ -3044,7 +3035,7 @@ func (c *Concurrency) IndexLookupConcurrency() int { // IndexLookupJoinConcurrency return the number of concurrent index lookup join inner worker. func (c *Concurrency) IndexLookupJoinConcurrency() int { - if c.indexLookupJoinConcurrency != ConcurrencyUnset { + if c.indexLookupJoinConcurrency != vardef.ConcurrencyUnset { return c.indexLookupJoinConcurrency } return c.ExecutorConcurrency @@ -3062,7 +3053,7 @@ func (c *Concurrency) AnalyzeDistSQLScanConcurrency() int { // HashJoinConcurrency return the number of concurrent hash join outer worker. func (c *Concurrency) HashJoinConcurrency() int { - if c.hashJoinConcurrency != ConcurrencyUnset { + if c.hashJoinConcurrency != vardef.ConcurrencyUnset { return c.hashJoinConcurrency } return c.ExecutorConcurrency @@ -3070,7 +3061,7 @@ func (c *Concurrency) HashJoinConcurrency() int { // ProjectionConcurrency return the number of concurrent projection worker. func (c *Concurrency) ProjectionConcurrency() int { - if c.projectionConcurrency != ConcurrencyUnset { + if c.projectionConcurrency != vardef.ConcurrencyUnset { return c.projectionConcurrency } return c.ExecutorConcurrency @@ -3078,7 +3069,7 @@ func (c *Concurrency) ProjectionConcurrency() int { // HashAggPartialConcurrency return the number of concurrent hash aggregation partial worker. func (c *Concurrency) HashAggPartialConcurrency() int { - if c.hashAggPartialConcurrency != ConcurrencyUnset { + if c.hashAggPartialConcurrency != vardef.ConcurrencyUnset { return c.hashAggPartialConcurrency } return c.ExecutorConcurrency @@ -3086,7 +3077,7 @@ func (c *Concurrency) HashAggPartialConcurrency() int { // HashAggFinalConcurrency return the number of concurrent hash aggregation final worker. func (c *Concurrency) HashAggFinalConcurrency() int { - if c.hashAggFinalConcurrency != ConcurrencyUnset { + if c.hashAggFinalConcurrency != vardef.ConcurrencyUnset { return c.hashAggFinalConcurrency } return c.ExecutorConcurrency @@ -3094,7 +3085,7 @@ func (c *Concurrency) HashAggFinalConcurrency() int { // WindowConcurrency return the number of concurrent window worker. func (c *Concurrency) WindowConcurrency() int { - if c.windowConcurrency != ConcurrencyUnset { + if c.windowConcurrency != vardef.ConcurrencyUnset { return c.windowConcurrency } return c.ExecutorConcurrency @@ -3102,7 +3093,7 @@ func (c *Concurrency) WindowConcurrency() int { // MergeJoinConcurrency return the number of concurrent merge join worker. func (c *Concurrency) MergeJoinConcurrency() int { - if c.mergeJoinConcurrency != ConcurrencyUnset { + if c.mergeJoinConcurrency != vardef.ConcurrencyUnset { return c.mergeJoinConcurrency } return c.ExecutorConcurrency @@ -3110,7 +3101,7 @@ func (c *Concurrency) MergeJoinConcurrency() int { // StreamAggConcurrency return the number of concurrent stream aggregation worker. func (c *Concurrency) StreamAggConcurrency() int { - if c.streamAggConcurrency != ConcurrencyUnset { + if c.streamAggConcurrency != vardef.ConcurrencyUnset { return c.streamAggConcurrency } return c.ExecutorConcurrency @@ -3118,7 +3109,7 @@ func (c *Concurrency) StreamAggConcurrency() int { // IndexMergeIntersectionConcurrency return the number of concurrent process worker. func (c *Concurrency) IndexMergeIntersectionConcurrency() int { - if c.indexMergeIntersectionConcurrency != ConcurrencyUnset { + if c.indexMergeIntersectionConcurrency != vardef.ConcurrencyUnset { return c.indexMergeIntersectionConcurrency } return c.ExecutorConcurrency @@ -3768,8 +3759,8 @@ func (s *SessionVars) GetStrMatchDefaultSelectivity() float64 { // 0.8 (the default value) is also a special value. For backward compatibility, when the variable is set to 0.8, we // keep the default selectivity of like/regexp and not like/regexp all 0.8. func (s *SessionVars) GetNegateStrMatchDefaultSelectivity() float64 { - if s.DefaultStrMatchSelectivity == DefTiDBDefaultStrMatchSelectivity { - return DefTiDBDefaultStrMatchSelectivity + if s.DefaultStrMatchSelectivity == vardef.DefTiDBDefaultStrMatchSelectivity { + return vardef.DefTiDBDefaultStrMatchSelectivity } return 1 - s.GetStrMatchDefaultSelectivity() } @@ -3945,42 +3936,25 @@ func RuntimeFilterModeStringToMode(name string) (RuntimeFilterMode, bool) { } } -const ( - // OptObjectiveModerate is a possible value and the default value for TiDBOptObjective. - // Please see comments of SessionVars.OptObjective for details. - OptObjectiveModerate string = "moderate" - // OptObjectiveDeterminate is a possible value for TiDBOptObjective. - OptObjectiveDeterminate = "determinate" -) - // GetOptObjective return the session variable "tidb_opt_objective". // Please see comments of SessionVars.OptObjective for details. func (s *SessionVars) GetOptObjective() string { return s.OptObjective } -// ForcePreAggStr means 1st hashagg will be pre aggregated. -// AutoStr means TiFlash will decide which policy for 1st hashagg. -// ForceStreamingStr means 1st hashagg will for pass through all blocks. -const ( - ForcePreAggStr = "force_preagg" - AutoStr = "auto" - ForceStreamingStr = "force_streaming" -) - // ValidTiFlashPreAggMode returns all valid modes. func ValidTiFlashPreAggMode() string { - return ForcePreAggStr + ", " + AutoStr + ", " + ForceStreamingStr + return vardef.ForcePreAggStr + ", " + vardef.AutoStr + ", " + vardef.ForceStreamingStr } // ToTiPBTiFlashPreAggMode return the corresponding tipb value of preaggregation mode. func ToTiPBTiFlashPreAggMode(mode string) (tipb.TiFlashPreAggMode, bool) { switch mode { - case ForcePreAggStr: + case vardef.ForcePreAggStr: return tipb.TiFlashPreAggMode_ForcePreAgg, true - case ForceStreamingStr: + case vardef.ForceStreamingStr: return tipb.TiFlashPreAggMode_ForceStreaming, true - case AutoStr: + case vardef.AutoStr: return tipb.TiFlashPreAggMode_Auto, true default: return tipb.TiFlashPreAggMode_ForcePreAgg, false @@ -4014,12 +3988,3 @@ func (s *SessionVars) PessimisticLockEligible() bool { } return false } - -const ( - // ScatterOff means default, will not scatter region - ScatterOff string = "" - // ScatterTable means scatter region at table level - ScatterTable string = "table" - // ScatterGlobal means scatter region at global level - ScatterGlobal string = "global" -) diff --git a/pkg/sessionctx/variable/session_test.go b/pkg/sessionctx/variable/session_test.go index 3b8351f33e75f..24ec0c814916a 100644 --- a/pkg/sessionctx/variable/session_test.go +++ b/pkg/sessionctx/variable/session_test.go @@ -30,6 +30,7 @@ import ( plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/types" @@ -52,15 +53,15 @@ func TestSetSystemVariable(t *testing.T) { value string err bool }{ - {variable.TxnIsolation, "SERIALIZABLE", true}, - {variable.TimeZone, "xyz", true}, - {variable.TiDBOptAggPushDown, "1", false}, - {variable.TiDBOptDeriveTopN, "1", false}, - {variable.TiDBOptDistinctAggPushDown, "1", false}, - {variable.TiDBMemQuotaQuery, "1024", false}, - {variable.TiDBMemQuotaApplyCache, "1024", false}, - {variable.TiDBEnableStmtSummary, "1", true}, // now global only - {variable.TiDBEnableRowLevelChecksum, "1", true}, + {vardef.TxnIsolation, "SERIALIZABLE", true}, + {vardef.TimeZone, "xyz", true}, + {vardef.TiDBOptAggPushDown, "1", false}, + {vardef.TiDBOptDeriveTopN, "1", false}, + {vardef.TiDBOptDistinctAggPushDown, "1", false}, + {vardef.TiDBMemQuotaQuery, "1024", false}, + {vardef.TiDBMemQuotaApplyCache, "1024", false}, + {vardef.TiDBEnableStmtSummary, "1", true}, // now global only + {vardef.TiDBEnableRowLevelChecksum, "1", true}, } for _, tc := range testCases { @@ -449,7 +450,7 @@ func TestHookContext(t *testing.T) { store := testkit.CreateMockStore(t) ctx := mock.NewContext() ctx.Store = store - sv := variable.SysVar{Scope: variable.ScopeGlobal | variable.ScopeSession, Name: "testhooksysvar", Value: variable.On, Type: variable.TypeBool, SetSession: func(s *variable.SessionVars, val string) error { + sv := variable.SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "testhooksysvar", Value: vardef.On, Type: vardef.TypeBool, SetSession: func(s *variable.SessionVars, val string) error { require.Equal(t, s.GetStore(), store) return nil }} diff --git a/pkg/sessionctx/variable/statusvar.go b/pkg/sessionctx/variable/statusvar.go index f9d790b3fa7c0..45511ad2a4f36 100644 --- a/pkg/sessionctx/variable/statusvar.go +++ b/pkg/sessionctx/variable/statusvar.go @@ -19,6 +19,7 @@ import ( "crypto/tls" "sync" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util" ) @@ -26,18 +27,18 @@ var statisticsList []Statistics var statisticsListLock sync.RWMutex // DefaultStatusVarScopeFlag is the default scope of status variables. -var DefaultStatusVarScopeFlag = ScopeGlobal | ScopeSession +var DefaultStatusVarScopeFlag = vardef.ScopeGlobal | vardef.ScopeSession // StatusVal is the value of the corresponding status variable. type StatusVal struct { - Scope ScopeFlag + Scope vardef.ScopeFlag Value any } // Statistics is the interface of statistics. type Statistics interface { // GetScope gets the status variables scope. - GetScope(status string) ScopeFlag + GetScope(status string) vardef.ScopeFlag // Stats returns the statistics status variables. Stats(*SessionVars) (map[string]any, error) } @@ -129,16 +130,16 @@ var tlsVersionString = map[uint16]string{ } var defaultStatus = map[string]*StatusVal{ - "Ssl_cipher": {ScopeGlobal | ScopeSession, ""}, - "Ssl_cipher_list": {ScopeGlobal | ScopeSession, ""}, - "Ssl_verify_mode": {ScopeGlobal | ScopeSession, 0}, - "Ssl_version": {ScopeGlobal | ScopeSession, ""}, + "Ssl_cipher": {vardef.ScopeGlobal | vardef.ScopeSession, ""}, + "Ssl_cipher_list": {vardef.ScopeGlobal | vardef.ScopeSession, ""}, + "Ssl_verify_mode": {vardef.ScopeGlobal | vardef.ScopeSession, 0}, + "Ssl_version": {vardef.ScopeGlobal | vardef.ScopeSession, ""}, } type defaultStatusStat struct { } -func (s defaultStatusStat) GetScope(status string) ScopeFlag { +func (s defaultStatusStat) GetScope(status string) vardef.ScopeFlag { return defaultStatus[status].Scope } diff --git a/pkg/sessionctx/variable/statusvar_test.go b/pkg/sessionctx/variable/statusvar_test.go index 3cac80df5f0f5..7336d821a228f 100644 --- a/pkg/sessionctx/variable/statusvar_test.go +++ b/pkg/sessionctx/variable/statusvar_test.go @@ -17,6 +17,7 @@ package variable import ( "testing" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/stretchr/testify/require" ) @@ -29,11 +30,11 @@ const ( testStatusVal = "test_status_val" ) -var specificStatusScopes = map[string]ScopeFlag{ - testSessionStatus: ScopeSession, +var specificStatusScopes = map[string]vardef.ScopeFlag{ + testSessionStatus: vardef.ScopeSession, } -func (ms *mockStatistics) GetScope(status string) ScopeFlag { +func (ms *mockStatistics) GetScope(status string) vardef.ScopeFlag { scope, ok := specificStatusScopes[status] if !ok { return DefaultStatusVarScopeFlag @@ -56,7 +57,7 @@ func TestStatusVar(t *testing.T) { scope := ms.GetScope(testStatus) require.Equal(t, DefaultStatusVarScopeFlag, scope) scope = ms.GetScope(testSessionStatus) - require.Equal(t, ScopeSession, scope) + require.Equal(t, vardef.ScopeSession, scope) vars, err := GetStatusVars(nil) require.NoError(t, err) diff --git a/pkg/sessionctx/variable/sysvar.go b/pkg/sessionctx/variable/sysvar.go index c75b278a8a5da..a25cc1c8ba2f8 100644 --- a/pkg/sessionctx/variable/sysvar.go +++ b/pkg/sessionctx/variable/sysvar.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" "github.com/pingcap/tidb/pkg/privilege/privileges/ldap" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" _ "github.com/pingcap/tidb/pkg/types/parser_driver" // for parser driver "github.com/pingcap/tidb/pkg/util" @@ -71,66 +72,66 @@ import ( // If you are adding a new system variable, please put it in the corresponding area. var defaultSysVars = []*SysVar{ /* The system variables below have NONE scope */ - {Scope: ScopeNone, Name: SystemTimeZone, Value: "CST"}, - {Scope: ScopeNone, Name: Hostname, Value: DefHostname}, - {Scope: ScopeNone, Name: Port, Value: "4000", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint16}, - {Scope: ScopeNone, Name: VersionComment, Value: "TiDB Server (Apache License 2.0) " + versioninfo.TiDBEdition + " Edition, MySQL 8.0 compatible"}, - {Scope: ScopeNone, Name: Version, Value: mysql.ServerVersion}, - {Scope: ScopeNone, Name: DataDir, Value: "/usr/local/mysql/data/"}, - {Scope: ScopeNone, Name: Socket, Value: ""}, - {Scope: ScopeNone, Name: "license", Value: "Apache License 2.0"}, - {Scope: ScopeNone, Name: "have_ssl", Value: "DISABLED", Type: TypeBool}, - {Scope: ScopeNone, Name: "have_openssl", Value: "DISABLED", Type: TypeBool}, - {Scope: ScopeNone, Name: "ssl_ca", Value: ""}, - {Scope: ScopeNone, Name: "ssl_cert", Value: ""}, - {Scope: ScopeNone, Name: "ssl_key", Value: ""}, - {Scope: ScopeNone, Name: "version_compile_os", Value: runtime.GOOS}, - {Scope: ScopeNone, Name: "version_compile_machine", Value: runtime.GOARCH}, + {Scope: vardef.ScopeNone, Name: vardef.SystemTimeZone, Value: "CST"}, + {Scope: vardef.ScopeNone, Name: vardef.Hostname, Value: vardef.DefHostname}, + {Scope: vardef.ScopeNone, Name: vardef.Port, Value: "4000", Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint16}, + {Scope: vardef.ScopeNone, Name: vardef.VersionComment, Value: "TiDB Server (Apache License 2.0) " + versioninfo.TiDBEdition + " Edition, MySQL 8.0 compatible"}, + {Scope: vardef.ScopeNone, Name: vardef.Version, Value: mysql.ServerVersion}, + {Scope: vardef.ScopeNone, Name: vardef.DataDir, Value: "/usr/local/mysql/data/"}, + {Scope: vardef.ScopeNone, Name: vardef.Socket, Value: ""}, + {Scope: vardef.ScopeNone, Name: "license", Value: "Apache License 2.0"}, + {Scope: vardef.ScopeNone, Name: "have_ssl", Value: "DISABLED", Type: vardef.TypeBool}, + {Scope: vardef.ScopeNone, Name: "have_openssl", Value: "DISABLED", Type: vardef.TypeBool}, + {Scope: vardef.ScopeNone, Name: "ssl_ca", Value: ""}, + {Scope: vardef.ScopeNone, Name: "ssl_cert", Value: ""}, + {Scope: vardef.ScopeNone, Name: "ssl_key", Value: ""}, + {Scope: vardef.ScopeNone, Name: "version_compile_os", Value: runtime.GOOS}, + {Scope: vardef.ScopeNone, Name: "version_compile_machine", Value: runtime.GOARCH}, /* TiDB specific variables */ - {Scope: ScopeNone, Name: TiDBEnableEnhancedSecurity, Value: Off, Type: TypeBool}, - {Scope: ScopeNone, Name: TiDBAllowFunctionForExpressionIndex, ReadOnly: true, Value: collectAllowFuncName4ExpressionIndex()}, + {Scope: vardef.ScopeNone, Name: vardef.TiDBEnableEnhancedSecurity, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeNone, Name: vardef.TiDBAllowFunctionForExpressionIndex, ReadOnly: true, Value: collectAllowFuncName4ExpressionIndex()}, /* The system variables below have SESSION scope */ - {Scope: ScopeSession, Name: Timestamp, Value: DefTimestamp, MinValue: 0, MaxValue: math.MaxInt32, Type: TypeFloat, GetSession: func(s *SessionVars) (string, error) { - if timestamp, ok := s.systems[Timestamp]; ok && timestamp != DefTimestamp { + {Scope: vardef.ScopeSession, Name: vardef.Timestamp, Value: vardef.DefTimestamp, MinValue: 0, MaxValue: math.MaxInt32, Type: vardef.TypeFloat, GetSession: func(s *SessionVars) (string, error) { + if timestamp, ok := s.systems[vardef.Timestamp]; ok && timestamp != vardef.DefTimestamp { return timestamp, nil } timestamp := s.StmtCtx.GetOrStoreStmtCache(stmtctx.StmtNowTsCacheKey, time.Now()).(time.Time) return types.ToString(float64(timestamp.UnixNano()) / float64(time.Second)) }, GetStateValue: func(s *SessionVars) (string, bool, error) { - timestamp, ok := s.systems[Timestamp] - return timestamp, ok && timestamp != DefTimestamp, nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - val := tidbOptFloat64(originalValue, DefTimestampFloat) + timestamp, ok := s.systems[vardef.Timestamp] + return timestamp, ok && timestamp != vardef.DefTimestamp, nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + val := tidbOptFloat64(originalValue, vardef.DefTimestampFloat) if val > math.MaxInt32 { - return originalValue, ErrWrongValueForVar.GenWithStackByArgs(Timestamp, originalValue) + return originalValue, ErrWrongValueForVar.GenWithStackByArgs(vardef.Timestamp, originalValue) } return normalizedValue, nil }}, - {Scope: ScopeSession, Name: WarningCount, Value: "0", ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.WarningCount, Value: "0", ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return strconv.Itoa(s.SysWarningCount), nil }}, - {Scope: ScopeSession, Name: ErrorCount, Value: "0", ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.ErrorCount, Value: "0", ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return strconv.Itoa(int(s.SysErrorCount)), nil }}, - {Scope: ScopeSession, Name: LastInsertID, Value: "0", Type: TypeUnsigned, AllowEmpty: true, MinValue: 0, MaxValue: math.MaxUint64, GetSession: func(s *SessionVars) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.LastInsertID, Value: "0", Type: vardef.TypeUnsigned, AllowEmpty: true, MinValue: 0, MaxValue: math.MaxUint64, GetSession: func(s *SessionVars) (string, error) { return strconv.FormatUint(s.StmtCtx.PrevLastInsertID, 10), nil }, GetStateValue: func(s *SessionVars) (string, bool, error) { return "", false, nil }}, - {Scope: ScopeSession, Name: Identity, Value: "0", Type: TypeUnsigned, AllowEmpty: true, MinValue: 0, MaxValue: math.MaxUint64, GetSession: func(s *SessionVars) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.Identity, Value: "0", Type: vardef.TypeUnsigned, AllowEmpty: true, MinValue: 0, MaxValue: math.MaxUint64, GetSession: func(s *SessionVars) (string, error) { return strconv.FormatUint(s.StmtCtx.PrevLastInsertID, 10), nil }, GetStateValue: func(s *SessionVars) (string, bool, error) { return "", false, nil }}, /* TiDB specific variables */ // TODO: TiDBTxnScope is hidden because local txn feature is not done. - {Scope: ScopeSession, Name: TiDBTxnScope, skipInit: true, Hidden: true, Value: kv.GlobalTxnScope, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBTxnScope, skipInit: true, Hidden: true, Value: kv.GlobalTxnScope, SetSession: func(s *SessionVars, val string) error { switch val { case kv.GlobalTxnScope: s.TxnScope = kv.NewGlobalTxnScopeVar() case kv.LocalTxnScope: - if !EnableLocalTxn.Load() { + if !vardef.EnableLocalTxn.Load() { return ErrWrongValueForVar.GenWithStack("@@txn_scope can not be set to local when tidb_enable_local_txn is off") } txnScope := config.GetTxnScopeFromConfig() @@ -145,15 +146,15 @@ var defaultSysVars = []*SysVar{ }, GetSession: func(s *SessionVars) (string, error) { return s.TxnScope.GetVarValue(), nil }}, - {Scope: ScopeSession, Name: TiDBTxnReadTS, Value: "", Hidden: true, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBTxnReadTS, Value: "", Hidden: true, SetSession: func(s *SessionVars, val string) error { return setTxnReadTS(s, val) - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { return normalizedValue, nil }}, - {Scope: ScopeSession, Name: TiDBReadStaleness, Value: strconv.Itoa(DefTiDBReadStaleness), Type: TypeInt, MinValue: math.MinInt32, MaxValue: 0, AllowEmpty: true, Hidden: false, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBReadStaleness, Value: strconv.Itoa(vardef.DefTiDBReadStaleness), Type: vardef.TypeInt, MinValue: math.MinInt32, MaxValue: 0, AllowEmpty: true, Hidden: false, SetSession: func(s *SessionVars, val string) error { return setReadStaleness(s, val) }}, - {Scope: ScopeSession, Name: TiDBEnforceMPPExecution, Type: TypeBool, Value: BoolToOnOff(config.GetGlobalConfig().Performance.EnforceMPP), Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.TiDBEnforceMPPExecution, Type: vardef.TypeBool, Value: BoolToOnOff(config.GetGlobalConfig().Performance.EnforceMPP), Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { if TiDBOptOn(normalizedValue) && !vars.allowMPPExecution { return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs("tidb_enforce_mpp", "1' but tidb_allow_mpp is 0, please activate tidb_allow_mpp at first.") } @@ -162,27 +163,27 @@ var defaultSysVars = []*SysVar{ s.enforceMPPExecution = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMaxTiFlashThreads, Type: TypeInt, Value: strconv.Itoa(DefTiFlashMaxThreads), MinValue: -1, MaxValue: MaxConfigurableConcurrency, SetSession: func(s *SessionVars, val string) error { - s.TiFlashMaxThreads = TidbOptInt64(val, DefTiFlashMaxThreads) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBMaxTiFlashThreads, Type: vardef.TypeInt, Value: strconv.Itoa(vardef.DefTiFlashMaxThreads), MinValue: -1, MaxValue: vardef.MaxConfigurableConcurrency, SetSession: func(s *SessionVars, val string) error { + s.TiFlashMaxThreads = TidbOptInt64(val, vardef.DefTiFlashMaxThreads) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMaxBytesBeforeTiFlashExternalJoin, Type: TypeInt, Value: strconv.Itoa(DefTiFlashMaxBytesBeforeExternalJoin), MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - s.TiFlashMaxBytesBeforeExternalJoin = TidbOptInt64(val, DefTiFlashMaxBytesBeforeExternalJoin) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBMaxBytesBeforeTiFlashExternalJoin, Type: vardef.TypeInt, Value: strconv.Itoa(vardef.DefTiFlashMaxBytesBeforeExternalJoin), MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.TiFlashMaxBytesBeforeExternalJoin = TidbOptInt64(val, vardef.DefTiFlashMaxBytesBeforeExternalJoin) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMaxBytesBeforeTiFlashExternalGroupBy, Type: TypeInt, Value: strconv.Itoa(DefTiFlashMaxBytesBeforeExternalGroupBy), MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - s.TiFlashMaxBytesBeforeExternalGroupBy = TidbOptInt64(val, DefTiFlashMaxBytesBeforeExternalGroupBy) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBMaxBytesBeforeTiFlashExternalGroupBy, Type: vardef.TypeInt, Value: strconv.Itoa(vardef.DefTiFlashMaxBytesBeforeExternalGroupBy), MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.TiFlashMaxBytesBeforeExternalGroupBy = TidbOptInt64(val, vardef.DefTiFlashMaxBytesBeforeExternalGroupBy) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMaxBytesBeforeTiFlashExternalSort, Type: TypeInt, Value: strconv.Itoa(DefTiFlashMaxBytesBeforeExternalSort), MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - s.TiFlashMaxBytesBeforeExternalSort = TidbOptInt64(val, DefTiFlashMaxBytesBeforeExternalSort) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBMaxBytesBeforeTiFlashExternalSort, Type: vardef.TypeInt, Value: strconv.Itoa(vardef.DefTiFlashMaxBytesBeforeExternalSort), MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.TiFlashMaxBytesBeforeExternalSort = TidbOptInt64(val, vardef.DefTiFlashMaxBytesBeforeExternalSort) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiFlashMemQuotaQueryPerNode, Type: TypeInt, Value: strconv.Itoa(DefTiFlashMemQuotaQueryPerNode), MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - s.TiFlashMaxQueryMemoryPerNode = TidbOptInt64(val, DefTiFlashMemQuotaQueryPerNode) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiFlashMemQuotaQueryPerNode, Type: vardef.TypeInt, Value: strconv.Itoa(vardef.DefTiFlashMemQuotaQueryPerNode), MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.TiFlashMaxQueryMemoryPerNode = TidbOptInt64(val, vardef.DefTiFlashMemQuotaQueryPerNode) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiFlashQuerySpillRatio, Type: TypeFloat, Value: strconv.FormatFloat(DefTiFlashQuerySpillRatio, 'f', -1, 64), MinValue: 0, MaxValue: 1, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, flag ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiFlashQuerySpillRatio, Type: vardef.TypeFloat, Value: strconv.FormatFloat(vardef.DefTiFlashQuerySpillRatio, 'f', -1, 64), MinValue: 0, MaxValue: 1, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, flag vardef.ScopeFlag) (string, error) { val, err := strconv.ParseFloat(normalizedValue, 64) if err != nil { return "", err @@ -192,92 +193,92 @@ var defaultSysVars = []*SysVar{ } return normalizedValue, nil }, SetSession: func(s *SessionVars, val string) error { - s.TiFlashQuerySpillRatio = tidbOptFloat64(val, DefTiFlashQuerySpillRatio) + s.TiFlashQuerySpillRatio = tidbOptFloat64(val, vardef.DefTiFlashQuerySpillRatio) return nil }}, - {Scope: ScopeGlobal, Name: TiDBEnableTiFlashPipelineMode, Type: TypeBool, Value: BoolToOnOff(DefTiDBEnableTiFlashPipelineMode), SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { - TiFlashEnablePipelineMode.Store(TiDBOptOn(s)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableTiFlashPipelineMode, Type: vardef.TypeBool, Value: BoolToOnOff(vardef.DefTiDBEnableTiFlashPipelineMode), SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + vardef.TiFlashEnablePipelineMode.Store(TiDBOptOn(s)) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return BoolToOnOff(TiFlashEnablePipelineMode.Load()), nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - vars.StmtCtx.AppendWarning(ErrWarnDeprecatedSyntaxSimpleMsg.FastGenByArgs(TiDBEnableTiFlashPipelineMode)) + return BoolToOnOff(vardef.TiFlashEnablePipelineMode.Load()), nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + vars.StmtCtx.AppendWarning(ErrWarnDeprecatedSyntaxSimpleMsg.FastGenByArgs(vardef.TiDBEnableTiFlashPipelineMode)) return normalizedValue, nil }}, - {Scope: ScopeSession, Name: TiDBSnapshot, Value: "", skipInit: true, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBSnapshot, Value: "", skipInit: true, SetSession: func(s *SessionVars, val string) error { err := setSnapshotTS(s, val) if err != nil { return err } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptProjectionPushDown, Value: BoolToOnOff(DefOptEnableProjectionPushDown), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptProjectionPushDown, Value: BoolToOnOff(vardef.DefOptEnableProjectionPushDown), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.AllowProjectionPushDown = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptDeriveTopN, Value: BoolToOnOff(DefOptDeriveTopN), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptDeriveTopN, Value: BoolToOnOff(vardef.DefOptDeriveTopN), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.AllowDeriveTopN = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptAggPushDown, Value: BoolToOnOff(DefOptAggPushDown), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptAggPushDown, Value: BoolToOnOff(vardef.DefOptAggPushDown), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.AllowAggPushDown = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBOptDistinctAggPushDown, Value: BoolToOnOff(config.GetGlobalConfig().Performance.DistinctAggPushDown), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBOptDistinctAggPushDown, Value: BoolToOnOff(config.GetGlobalConfig().Performance.DistinctAggPushDown), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.AllowDistinctAggPushDown = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptSkewDistinctAgg, Value: BoolToOnOff(DefTiDBSkewDistinctAgg), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptSkewDistinctAgg, Value: BoolToOnOff(vardef.DefTiDBSkewDistinctAgg), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableSkewDistinctAgg = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOpt3StageDistinctAgg, Value: BoolToOnOff(DefTiDB3StageDistinctAgg), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOpt3StageDistinctAgg, Value: BoolToOnOff(vardef.DefTiDB3StageDistinctAgg), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.Enable3StageDistinctAgg = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptEnable3StageMultiDistinctAgg, Value: BoolToOnOff(DefTiDB3StageMultiDistinctAgg), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptEnable3StageMultiDistinctAgg, Value: BoolToOnOff(vardef.DefTiDB3StageMultiDistinctAgg), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.Enable3StageMultiDistinctAgg = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptExplainNoEvaledSubQuery, Value: BoolToOnOff(DefTiDBOptExplainEvaledSubquery), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptExplainNoEvaledSubQuery, Value: BoolToOnOff(vardef.DefTiDBOptExplainEvaledSubquery), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.ExplainNonEvaledSubQuery = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBOptWriteRowID, Value: BoolToOnOff(DefOptWriteRowID), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBOptWriteRowID, Value: BoolToOnOff(vardef.DefOptWriteRowID), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.AllowWriteRowID = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBChecksumTableConcurrency, Value: strconv.Itoa(DefChecksumTableConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency}, - {Scope: ScopeSession, Name: TiDBBatchInsert, Value: BoolToOnOff(DefBatchInsert), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBChecksumTableConcurrency, Value: strconv.Itoa(vardef.DefChecksumTableConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency}, + {Scope: vardef.ScopeSession, Name: vardef.TiDBBatchInsert, Value: BoolToOnOff(vardef.DefBatchInsert), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.BatchInsert = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBBatchDelete, Value: BoolToOnOff(DefBatchDelete), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBBatchDelete, Value: BoolToOnOff(vardef.DefBatchDelete), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.BatchDelete = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBBatchCommit, Value: BoolToOnOff(DefBatchCommit), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBBatchCommit, Value: BoolToOnOff(vardef.DefBatchCommit), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.BatchCommit = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBCurrentTS, Value: strconv.Itoa(DefCurretTS), Type: TypeInt, AllowEmpty: true, MinValue: 0, MaxValue: math.MaxInt64, ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.TiDBCurrentTS, Value: strconv.Itoa(vardef.DefCurretTS), Type: vardef.TypeInt, AllowEmpty: true, MinValue: 0, MaxValue: math.MaxInt64, ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return strconv.FormatUint(s.TxnCtx.StartTS, 10), nil }}, - {Scope: ScopeSession, Name: TiDBLastTxnInfo, Value: "", ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.TiDBLastTxnInfo, Value: "", ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return s.LastTxnInfo, nil }}, - {Scope: ScopeSession, Name: TiDBLastQueryInfo, Value: "", ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.TiDBLastQueryInfo, Value: "", ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { info, err := json.Marshal(s.LastQueryInfo) if err != nil { return "", err } return string(info), nil }}, - {Scope: ScopeSession, Name: TiDBEnableChunkRPC, Value: BoolToOnOff(config.GetGlobalConfig().TiKVClient.EnableChunkRPC), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBEnableChunkRPC, Value: BoolToOnOff(config.GetGlobalConfig().TiKVClient.EnableChunkRPC), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableChunkRPC = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TxnIsolationOneShot, Value: "", skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.TxnIsolationOneShot, Value: "", skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { return checkIsolationLevel(vars, normalizedValue, originalValue, scope) }, SetSession: func(s *SessionVars, val string) error { s.txnIsolationLevelOneShot.state = oneShotSet @@ -289,43 +290,43 @@ var defaultSysVars = []*SysVar{ } return "", false, nil }}, - {Scope: ScopeSession, Name: TiDBOptimizerSelectivityLevel, Value: strconv.Itoa(DefTiDBOptimizerSelectivityLevel), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { - s.OptimizerSelectivityLevel = tidbOptPositiveInt32(val, DefTiDBOptimizerSelectivityLevel) + {Scope: vardef.ScopeSession, Name: vardef.TiDBOptimizerSelectivityLevel, Value: strconv.Itoa(vardef.DefTiDBOptimizerSelectivityLevel), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + s.OptimizerSelectivityLevel = tidbOptPositiveInt32(val, vardef.DefTiDBOptimizerSelectivityLevel) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptimizerEnableOuterJoinReorder, Value: BoolToOnOff(DefTiDBEnableOuterJoinReorder), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptimizerEnableOuterJoinReorder, Value: BoolToOnOff(vardef.DefTiDBEnableOuterJoinReorder), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableOuterJoinReorder = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptimizerEnableNAAJ, Value: BoolToOnOff(DefTiDBEnableNAAJ), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptimizerEnableNAAJ, Value: BoolToOnOff(vardef.DefTiDBEnableNAAJ), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.OptimizerEnableNAAJ = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBDDLReorgPriority, Value: "PRIORITY_LOW", Type: TypeEnum, skipInit: true, PossibleValues: []string{"PRIORITY_LOW", "PRIORITY_NORMAL", "PRIORITY_HIGH"}, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBDDLReorgPriority, Value: "PRIORITY_LOW", Type: vardef.TypeEnum, skipInit: true, PossibleValues: []string{"PRIORITY_LOW", "PRIORITY_NORMAL", "PRIORITY_HIGH"}, SetSession: func(s *SessionVars, val string) error { s.setDDLReorgPriority(val) return nil }}, - {Scope: ScopeSession, Name: TiDBSlowQueryFile, Value: "", skipInit: true, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBSlowQueryFile, Value: "", skipInit: true, SetSession: func(s *SessionVars, val string) error { s.SlowQueryFile = val return nil }}, - {Scope: ScopeSession, Name: TiDBWaitSplitRegionFinish, Value: BoolToOnOff(DefTiDBWaitSplitRegionFinish), skipInit: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBWaitSplitRegionFinish, Value: BoolToOnOff(vardef.DefTiDBWaitSplitRegionFinish), skipInit: true, Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.WaitSplitRegionFinish = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBWaitSplitRegionTimeout, Value: strconv.Itoa(DefWaitSplitRegionTimeout), skipInit: true, Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { - s.WaitSplitRegionTimeout = uint64(tidbOptPositiveInt32(val, DefWaitSplitRegionTimeout)) + {Scope: vardef.ScopeSession, Name: vardef.TiDBWaitSplitRegionTimeout, Value: strconv.Itoa(vardef.DefWaitSplitRegionTimeout), skipInit: true, Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + s.WaitSplitRegionTimeout = uint64(tidbOptPositiveInt32(val, vardef.DefWaitSplitRegionTimeout)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBLowResolutionTSO, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBLowResolutionTSO, Value: vardef.Off, Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.lowResolutionTSO = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBAllowRemoveAutoInc, Value: BoolToOnOff(DefTiDBAllowRemoveAutoInc), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBAllowRemoveAutoInc, Value: BoolToOnOff(vardef.DefTiDBAllowRemoveAutoInc), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.AllowRemoveAutoInc = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBIsolationReadEngines, Value: strings.Join(config.GetGlobalConfig().IsolationRead.Engines, ","), Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.TiDBIsolationReadEngines, Value: strings.Join(config.GetGlobalConfig().IsolationRead.Engines, ","), Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { engines := strings.Split(normalizedValue, ",") var formatVal string for i, engine := range engines { @@ -341,7 +342,7 @@ var defaultSysVars = []*SysVar{ case strings.EqualFold(engine, kv.TiDB.Name()): formatVal += kv.TiDB.Name() default: - return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(TiDBIsolationReadEngines, normalizedValue) + return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(vardef.TiDBIsolationReadEngines, normalizedValue) } } return formatVal, nil @@ -362,25 +363,25 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeSession, Name: TiDBMetricSchemaStep, Value: strconv.Itoa(DefTiDBMetricSchemaStep), Type: TypeUnsigned, skipInit: true, MinValue: 10, MaxValue: 60 * 60 * 60, SetSession: func(s *SessionVars, val string) error { - s.MetricSchemaStep = TidbOptInt64(val, DefTiDBMetricSchemaStep) + {Scope: vardef.ScopeSession, Name: vardef.TiDBMetricSchemaStep, Value: strconv.Itoa(vardef.DefTiDBMetricSchemaStep), Type: vardef.TypeUnsigned, skipInit: true, MinValue: 10, MaxValue: 60 * 60 * 60, SetSession: func(s *SessionVars, val string) error { + s.MetricSchemaStep = TidbOptInt64(val, vardef.DefTiDBMetricSchemaStep) return nil }}, - {Scope: ScopeSession, Name: TiDBCDCWriteSource, Value: "0", Type: TypeInt, MinValue: 0, MaxValue: 15, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBCDCWriteSource, Value: "0", Type: vardef.TypeInt, MinValue: 0, MaxValue: 15, SetSession: func(s *SessionVars, val string) error { s.CDCWriteSource = uint64(TidbOptInt(val, 0)) return nil }}, - {Scope: ScopeSession, Name: TiDBMetricSchemaRangeDuration, Value: strconv.Itoa(DefTiDBMetricSchemaRangeDuration), skipInit: true, Type: TypeUnsigned, MinValue: 10, MaxValue: 60 * 60 * 60, SetSession: func(s *SessionVars, val string) error { - s.MetricSchemaRangeDuration = TidbOptInt64(val, DefTiDBMetricSchemaRangeDuration) + {Scope: vardef.ScopeSession, Name: vardef.TiDBMetricSchemaRangeDuration, Value: strconv.Itoa(vardef.DefTiDBMetricSchemaRangeDuration), skipInit: true, Type: vardef.TypeUnsigned, MinValue: 10, MaxValue: 60 * 60 * 60, SetSession: func(s *SessionVars, val string) error { + s.MetricSchemaRangeDuration = TidbOptInt64(val, vardef.DefTiDBMetricSchemaRangeDuration) return nil }}, - {Scope: ScopeSession, Name: TiDBFoundInPlanCache, Value: BoolToOnOff(DefTiDBFoundInPlanCache), Type: TypeBool, ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.TiDBFoundInPlanCache, Value: BoolToOnOff(vardef.DefTiDBFoundInPlanCache), Type: vardef.TypeBool, ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return BoolToOnOff(s.PrevFoundInPlanCache), nil }}, - {Scope: ScopeSession, Name: TiDBFoundInBinding, Value: BoolToOnOff(DefTiDBFoundInBinding), Type: TypeBool, ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.TiDBFoundInBinding, Value: BoolToOnOff(vardef.DefTiDBFoundInBinding), Type: vardef.TypeBool, ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return BoolToOnOff(s.PrevFoundInBinding), nil }}, - {Scope: ScopeSession, Name: RandSeed1, Type: TypeInt, Value: "0", skipInit: true, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.RandSeed1, Type: vardef.TypeInt, Value: "0", skipInit: true, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.Rng.SetSeed1(uint32(tidbOptPositiveInt32(val, 0))) return nil }, GetSession: func(s *SessionVars) (string, error) { @@ -388,7 +389,7 @@ var defaultSysVars = []*SysVar{ }, GetStateValue: func(s *SessionVars) (string, bool, error) { return strconv.FormatUint(uint64(s.Rng.GetSeed1()), 10), true, nil }}, - {Scope: ScopeSession, Name: RandSeed2, Type: TypeInt, Value: "0", skipInit: true, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.RandSeed2, Type: vardef.TypeInt, Value: "0", skipInit: true, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.Rng.SetSeed2(uint32(tidbOptPositiveInt32(val, 0))) return nil }, GetSession: func(s *SessionVars) (string, error) { @@ -396,8 +397,8 @@ var defaultSysVars = []*SysVar{ }, GetStateValue: func(s *SessionVars) (string, bool, error) { return strconv.FormatUint(uint64(s.Rng.GetSeed2()), 10), true, nil }}, - {Scope: ScopeSession, Name: TiDBReadConsistency, Value: string(ReadConsistencyStrict), Type: TypeStr, Hidden: true, - Validation: func(_ *SessionVars, normalized string, _ string, _ ScopeFlag) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.TiDBReadConsistency, Value: string(ReadConsistencyStrict), Type: vardef.TypeStr, Hidden: true, + Validation: func(_ *SessionVars, normalized string, _ string, _ vardef.ScopeFlag) (string, error) { return normalized, validateReadConsistencyLevel(normalized) }, SetSession: func(s *SessionVars, val string) error { @@ -405,34 +406,34 @@ var defaultSysVars = []*SysVar{ return nil }, }, - {Scope: ScopeSession, Name: TiDBLastDDLInfo, Value: "", ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.TiDBLastDDLInfo, Value: "", ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { info, err := json.Marshal(s.LastDDLInfo) if err != nil { return "", err } return string(info), nil }}, - {Scope: ScopeSession, Name: TiDBLastPlanReplayerToken, Value: "", ReadOnly: true, + {Scope: vardef.ScopeSession, Name: vardef.TiDBLastPlanReplayerToken, Value: "", ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return s.LastPlanReplayerToken, nil }, }, - {Scope: ScopeSession, Name: TiDBUseAlloc, Value: BoolToOnOff(DefTiDBUseAlloc), Type: TypeBool, ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.TiDBUseAlloc, Value: BoolToOnOff(vardef.DefTiDBUseAlloc), Type: vardef.TypeBool, ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { return BoolToOnOff(s.preUseChunkAlloc), nil }}, - {Scope: ScopeSession, Name: TiDBExplicitRequestSourceType, Value: "", Type: TypeEnum, PossibleValues: tikvcliutil.ExplicitTypeList, GetSession: func(s *SessionVars) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.TiDBExplicitRequestSourceType, Value: "", Type: vardef.TypeEnum, PossibleValues: tikvcliutil.ExplicitTypeList, GetSession: func(s *SessionVars) (string, error) { return s.ExplicitRequestSourceType, nil }, SetSession: func(s *SessionVars, val string) error { s.ExplicitRequestSourceType = val return nil }}, /* The system variables below have INSTANCE scope */ - {Scope: ScopeInstance, Name: TiDBLogFileMaxDays, Value: strconv.Itoa(config.GetGlobalConfig().Log.File.MaxDays), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeInstance, Name: vardef.TiDBLogFileMaxDays, Value: strconv.Itoa(config.GetGlobalConfig().Log.File.MaxDays), Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt32, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { maxAge, err := strconv.ParseInt(val, 10, 32) if err != nil { return err } - GlobalLogMaxDays.Store(int32(maxAge)) + vardef.GlobalLogMaxDays.Store(int32(maxAge)) cfg := config.GetGlobalConfig().Log.ToLogConfig() cfg.Config.File.MaxDays = int(maxAge) @@ -442,83 +443,83 @@ var defaultSysVars = []*SysVar{ } return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatInt(int64(GlobalLogMaxDays.Load()), 10), nil + return strconv.FormatInt(int64(vardef.GlobalLogMaxDays.Load()), 10), nil }}, - {Scope: ScopeInstance, Name: TiDBConfig, Value: "", ReadOnly: true, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + {Scope: vardef.ScopeInstance, Name: vardef.TiDBConfig, Value: "", ReadOnly: true, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return config.GetJSONConfig() }}, - {Scope: ScopeInstance, Name: TiDBGeneralLog, Value: BoolToOnOff(DefTiDBGeneralLog), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - ProcessGeneralLog.Store(TiDBOptOn(val)) + {Scope: vardef.ScopeInstance, Name: vardef.TiDBGeneralLog, Value: BoolToOnOff(vardef.DefTiDBGeneralLog), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.ProcessGeneralLog.Store(TiDBOptOn(val)) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(ProcessGeneralLog.Load()), nil + return BoolToOnOff(vardef.ProcessGeneralLog.Load()), nil }}, - {Scope: ScopeSession, Name: TiDBSlowTxnLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowTxnThreshold), - Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBSlowTxnLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowTxnThreshold), + Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.SlowTxnThreshold = TidbOptUint64(val, logutil.DefaultSlowTxnThreshold) return nil }, }, - {Scope: ScopeInstance, Name: TiDBSlowLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowThreshold), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeInstance, Name: vardef.TiDBSlowLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowThreshold), Type: vardef.TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { atomic.StoreUint64(&config.GetGlobalConfig().Instance.SlowThreshold, uint64(TidbOptInt64(val, logutil.DefaultSlowThreshold))) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Instance.SlowThreshold), 10), nil }}, - {Scope: ScopeInstance, Name: TiDBRecordPlanInSlowLog, Value: int32ToBoolStr(logutil.DefaultRecordPlanInSlowLog), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeInstance, Name: vardef.TiDBRecordPlanInSlowLog, Value: int32ToBoolStr(logutil.DefaultRecordPlanInSlowLog), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { atomic.StoreUint32(&config.GetGlobalConfig().Instance.RecordPlanInSlowLog, uint32(TidbOptInt64(val, logutil.DefaultRecordPlanInSlowLog))) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { enabled := atomic.LoadUint32(&config.GetGlobalConfig().Instance.RecordPlanInSlowLog) == 1 return BoolToOnOff(enabled), nil }}, - {Scope: ScopeInstance, Name: TiDBEnableSlowLog, Value: BoolToOnOff(logutil.DefaultTiDBEnableSlowLog), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeInstance, Name: vardef.TiDBEnableSlowLog, Value: BoolToOnOff(logutil.DefaultTiDBEnableSlowLog), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { config.GetGlobalConfig().Instance.EnableSlowLog.Store(TiDBOptOn(val)) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return BoolToOnOff(config.GetGlobalConfig().Instance.EnableSlowLog.Load()), nil }}, - {Scope: ScopeInstance, Name: TiDBCheckMb4ValueInUTF8, Value: BoolToOnOff(config.GetGlobalConfig().Instance.CheckMb4ValueInUTF8.Load()), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeInstance, Name: vardef.TiDBCheckMb4ValueInUTF8, Value: BoolToOnOff(config.GetGlobalConfig().Instance.CheckMb4ValueInUTF8.Load()), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { config.GetGlobalConfig().Instance.CheckMb4ValueInUTF8.Store(TiDBOptOn(val)) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return BoolToOnOff(config.GetGlobalConfig().Instance.CheckMb4ValueInUTF8.Load()), nil }}, - {Scope: ScopeInstance, Name: TiDBPProfSQLCPU, Value: strconv.Itoa(DefTiDBPProfSQLCPU), Type: TypeInt, MinValue: 0, MaxValue: 1, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - EnablePProfSQLCPU.Store(uint32(tidbOptPositiveInt32(val, DefTiDBPProfSQLCPU)) > 0) + {Scope: vardef.ScopeInstance, Name: vardef.TiDBPProfSQLCPU, Value: strconv.Itoa(vardef.DefTiDBPProfSQLCPU), Type: vardef.TypeInt, MinValue: 0, MaxValue: 1, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.EnablePProfSQLCPU.Store(uint32(tidbOptPositiveInt32(val, vardef.DefTiDBPProfSQLCPU)) > 0) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { val := "0" - if EnablePProfSQLCPU.Load() { + if vardef.EnablePProfSQLCPU.Load() { val = "1" } return val, nil }}, - {Scope: ScopeInstance, Name: TiDBDDLSlowOprThreshold, Value: strconv.Itoa(DefTiDBDDLSlowOprThreshold), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - atomic.StoreUint32(&DDLSlowOprThreshold, uint32(tidbOptPositiveInt32(val, DefTiDBDDLSlowOprThreshold))) + {Scope: vardef.ScopeInstance, Name: vardef.TiDBDDLSlowOprThreshold, Value: strconv.Itoa(vardef.DefTiDBDDLSlowOprThreshold), Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt32, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + atomic.StoreUint32(&vardef.DDLSlowOprThreshold, uint32(tidbOptPositiveInt32(val, vardef.DefTiDBDDLSlowOprThreshold))) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatUint(uint64(atomic.LoadUint32(&DDLSlowOprThreshold)), 10), nil + return strconv.FormatUint(uint64(atomic.LoadUint32(&vardef.DDLSlowOprThreshold)), 10), nil }}, - {Scope: ScopeInstance, Name: TiDBForcePriority, Value: mysql.Priority2Str[DefTiDBForcePriority], Type: TypeEnum, PossibleValues: []string{"NO_PRIORITY", "LOW_PRIORITY", "HIGH_PRIORITY", "DELAYED"}, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - atomic.StoreInt32(&ForcePriority, int32(mysql.Str2Priority(val))) + {Scope: vardef.ScopeInstance, Name: vardef.TiDBForcePriority, Value: mysql.Priority2Str[vardef.DefTiDBForcePriority], Type: vardef.TypeEnum, PossibleValues: []string{"NO_PRIORITY", "LOW_PRIORITY", "HIGH_PRIORITY", "DELAYED"}, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + atomic.StoreInt32(&vardef.ForcePriority, int32(mysql.Str2Priority(val))) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return mysql.Priority2Str[mysql.PriorityEnum(atomic.LoadInt32(&ForcePriority))], nil + return mysql.Priority2Str[mysql.PriorityEnum(atomic.LoadInt32(&vardef.ForcePriority))], nil }}, - {Scope: ScopeInstance, Name: TiDBExpensiveQueryTimeThreshold, Value: strconv.Itoa(DefTiDBExpensiveQueryTimeThreshold), Type: TypeUnsigned, MinValue: int64(MinExpensiveQueryTimeThreshold), MaxValue: math.MaxInt32, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - atomic.StoreUint64(&ExpensiveQueryTimeThreshold, uint64(tidbOptPositiveInt32(val, DefTiDBExpensiveQueryTimeThreshold))) + {Scope: vardef.ScopeInstance, Name: vardef.TiDBExpensiveQueryTimeThreshold, Value: strconv.Itoa(vardef.DefTiDBExpensiveQueryTimeThreshold), Type: vardef.TypeUnsigned, MinValue: int64(vardef.MinExpensiveQueryTimeThreshold), MaxValue: math.MaxInt32, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + atomic.StoreUint64(&vardef.ExpensiveQueryTimeThreshold, uint64(tidbOptPositiveInt32(val, vardef.DefTiDBExpensiveQueryTimeThreshold))) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatUint(atomic.LoadUint64(&ExpensiveQueryTimeThreshold), 10), nil + return strconv.FormatUint(atomic.LoadUint64(&vardef.ExpensiveQueryTimeThreshold), 10), nil }}, - {Scope: ScopeInstance, Name: TiDBExpensiveTxnTimeThreshold, Value: strconv.Itoa(DefTiDBExpensiveTxnTimeThreshold), Type: TypeUnsigned, MinValue: int64(MinExpensiveTxnTimeThreshold), MaxValue: math.MaxInt32, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - atomic.StoreUint64(&ExpensiveTxnTimeThreshold, uint64(tidbOptPositiveInt32(val, DefTiDBExpensiveTxnTimeThreshold))) + {Scope: vardef.ScopeInstance, Name: vardef.TiDBExpensiveTxnTimeThreshold, Value: strconv.Itoa(vardef.DefTiDBExpensiveTxnTimeThreshold), Type: vardef.TypeUnsigned, MinValue: int64(vardef.MinExpensiveTxnTimeThreshold), MaxValue: math.MaxInt32, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + atomic.StoreUint64(&vardef.ExpensiveTxnTimeThreshold, uint64(tidbOptPositiveInt32(val, vardef.DefTiDBExpensiveTxnTimeThreshold))) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatUint(atomic.LoadUint64(&ExpensiveTxnTimeThreshold), 10), nil + return strconv.FormatUint(atomic.LoadUint64(&vardef.ExpensiveTxnTimeThreshold), 10), nil }}, - {Scope: ScopeInstance, Name: TiDBEnableCollectExecutionInfo, Value: BoolToOnOff(DefTiDBEnableCollectExecutionInfo), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeInstance, Name: vardef.TiDBEnableCollectExecutionInfo, Value: BoolToOnOff(vardef.DefTiDBEnableCollectExecutionInfo), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { oldConfig := config.GetGlobalConfig() newValue := TiDBOptOn(val) if oldConfig.Instance.EnableCollectExecutionInfo.Load() != newValue { @@ -530,19 +531,19 @@ var defaultSysVars = []*SysVar{ }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return BoolToOnOff(config.GetGlobalConfig().Instance.EnableCollectExecutionInfo.Load()), nil }}, - {Scope: ScopeInstance, Name: PluginLoad, Value: "", ReadOnly: true, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + {Scope: vardef.ScopeInstance, Name: vardef.PluginLoad, Value: "", ReadOnly: true, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return config.GetGlobalConfig().Instance.PluginLoad, nil }}, - {Scope: ScopeInstance, Name: PluginDir, Value: "/data/deploy/plugin", ReadOnly: true, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + {Scope: vardef.ScopeInstance, Name: vardef.PluginDir, Value: "/data/deploy/plugin", ReadOnly: true, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return config.GetGlobalConfig().Instance.PluginDir, nil }}, - {Scope: ScopeInstance, Name: MaxConnections, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().Instance.MaxConnections), 10), Type: TypeUnsigned, MinValue: 0, MaxValue: 100000, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeInstance, Name: vardef.MaxConnections, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().Instance.MaxConnections), 10), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: 100000, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { config.GetGlobalConfig().Instance.MaxConnections = uint32(TidbOptInt64(val, 0)) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return strconv.FormatUint(uint64(config.GetGlobalConfig().Instance.MaxConnections), 10), nil }}, - {Scope: ScopeInstance, Name: TiDBEnableDDL, Value: BoolToOnOff(config.GetGlobalConfig().Instance.TiDBEnableDDL.Load()), Type: TypeBool, + {Scope: vardef.ScopeInstance, Name: vardef.TiDBEnableDDL, Value: BoolToOnOff(config.GetGlobalConfig().Instance.TiDBEnableDDL.Load()), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { oldVal, newVal := config.GetGlobalConfig().Instance.TiDBEnableDDL.Load(), TiDBOptOn(val) if oldVal != newVal { @@ -558,7 +559,7 @@ var defaultSysVars = []*SysVar{ return BoolToOnOff(config.GetGlobalConfig().Instance.TiDBEnableDDL.Load()), nil }, }, - {Scope: ScopeInstance, Name: TiDBEnableStatsOwner, Value: BoolToOnOff(config.GetGlobalConfig().Instance.TiDBEnableStatsOwner.Load()), Type: TypeBool, + {Scope: vardef.ScopeInstance, Name: vardef.TiDBEnableStatsOwner, Value: BoolToOnOff(config.GetGlobalConfig().Instance.TiDBEnableStatsOwner.Load()), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { oldVal, newVal := config.GetGlobalConfig().Instance.TiDBEnableStatsOwner.Load(), TiDBOptOn(val) if oldVal != newVal { @@ -574,54 +575,54 @@ var defaultSysVars = []*SysVar{ return BoolToOnOff(config.GetGlobalConfig().Instance.TiDBEnableStatsOwner.Load()), nil }, }, - {Scope: ScopeInstance, Name: TiDBRCReadCheckTS, Value: BoolToOnOff(DefRCReadCheckTS), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - EnableRCReadCheckTS.Store(TiDBOptOn(val)) + {Scope: vardef.ScopeInstance, Name: vardef.TiDBRCReadCheckTS, Value: BoolToOnOff(vardef.DefRCReadCheckTS), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.EnableRCReadCheckTS.Store(TiDBOptOn(val)) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(EnableRCReadCheckTS.Load()), nil + return BoolToOnOff(vardef.EnableRCReadCheckTS.Load()), nil }}, - {Scope: ScopeInstance, Name: TiDBStmtSummaryEnablePersistent, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { + {Scope: vardef.ScopeInstance, Name: vardef.TiDBStmtSummaryEnablePersistent, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { return BoolToOnOff(config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent), nil }}, - {Scope: ScopeInstance, Name: TiDBStmtSummaryFilename, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { + {Scope: vardef.ScopeInstance, Name: vardef.TiDBStmtSummaryFilename, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { return config.GetGlobalConfig().Instance.StmtSummaryFilename, nil }}, - {Scope: ScopeInstance, Name: TiDBStmtSummaryFileMaxDays, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { + {Scope: vardef.ScopeInstance, Name: vardef.TiDBStmtSummaryFileMaxDays, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { return strconv.Itoa(config.GetGlobalConfig().Instance.StmtSummaryFileMaxDays), nil }}, - {Scope: ScopeInstance, Name: TiDBStmtSummaryFileMaxSize, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { + {Scope: vardef.ScopeInstance, Name: vardef.TiDBStmtSummaryFileMaxSize, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { return strconv.Itoa(config.GetGlobalConfig().Instance.StmtSummaryFileMaxSize), nil }}, - {Scope: ScopeInstance, Name: TiDBStmtSummaryFileMaxBackups, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { + {Scope: vardef.ScopeInstance, Name: vardef.TiDBStmtSummaryFileMaxBackups, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { return strconv.Itoa(config.GetGlobalConfig().Instance.StmtSummaryFileMaxBackups), nil }}, /* The system variables below have GLOBAL scope */ - {Scope: ScopeGlobal, Name: MaxPreparedStmtCount, Value: strconv.FormatInt(DefMaxPreparedStmtCount, 10), Type: TypeInt, MinValue: -1, MaxValue: 1048576, + {Scope: vardef.ScopeGlobal, Name: vardef.MaxPreparedStmtCount, Value: strconv.FormatInt(vardef.DefMaxPreparedStmtCount, 10), Type: vardef.TypeInt, MinValue: -1, MaxValue: 1048576, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { num, err := strconv.ParseInt(val, 10, 64) if err != nil { return errors.Trace(err) } - MaxPreparedStmtCountValue.Store(num) + vardef.MaxPreparedStmtCountValue.Store(num) return nil }}, - {Scope: ScopeGlobal, Name: InitConnect, Value: "", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal, Name: vardef.InitConnect, Value: "", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { p := parser.New() p.SetSQLMode(vars.SQLMode) p.SetParserConfig(vars.BuildParserConfig()) _, _, err := p.ParseSQL(normalizedValue) if err != nil { - return normalizedValue, ErrWrongTypeForVar.GenWithStackByArgs(InitConnect) + return normalizedValue, ErrWrongTypeForVar.GenWithStackByArgs(vardef.InitConnect) } return normalizedValue, nil }}, - {Scope: ScopeGlobal, Name: ValidatePasswordEnable, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: ValidatePasswordPolicy, Value: "MEDIUM", Type: TypeEnum, PossibleValues: []string{"LOW", "MEDIUM", "STRONG"}}, - {Scope: ScopeGlobal, Name: ValidatePasswordCheckUserName, Value: On, Type: TypeBool}, - {Scope: ScopeGlobal, Name: ValidatePasswordLength, Value: "8", Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, - Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - numberCount, specialCharCount, mixedCaseCount := PasswordValidtaionNumberCount.Load(), PasswordValidationSpecialCharCount.Load(), PasswordValidationMixedCaseCount.Load() + {Scope: vardef.ScopeGlobal, Name: vardef.ValidatePasswordEnable, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: vardef.ValidatePasswordPolicy, Value: "MEDIUM", Type: vardef.TypeEnum, PossibleValues: []string{"LOW", "MEDIUM", "STRONG"}}, + {Scope: vardef.ScopeGlobal, Name: vardef.ValidatePasswordCheckUserName, Value: vardef.On, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: vardef.ValidatePasswordLength, Value: "8", Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt32, + Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + numberCount, specialCharCount, mixedCaseCount := vardef.PasswordValidtaionNumberCount.Load(), vardef.PasswordValidationSpecialCharCount.Load(), vardef.PasswordValidationMixedCaseCount.Load() length, err := strconv.ParseInt(normalizedValue, 10, 32) if err != nil { return "", err @@ -632,15 +633,15 @@ var defaultSysVars = []*SysVar{ return normalizedValue, nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - PasswordValidationLength.Store(int32(TidbOptInt64(val, 8))) + vardef.PasswordValidationLength.Store(int32(TidbOptInt64(val, 8))) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatInt(int64(PasswordValidationLength.Load()), 10), nil + return strconv.FormatInt(int64(vardef.PasswordValidationLength.Load()), 10), nil }, }, - {Scope: ScopeGlobal, Name: ValidatePasswordMixedCaseCount, Value: "1", Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, - Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - length, numberCount, specialCharCount := PasswordValidationLength.Load(), PasswordValidtaionNumberCount.Load(), PasswordValidationSpecialCharCount.Load() + {Scope: vardef.ScopeGlobal, Name: vardef.ValidatePasswordMixedCaseCount, Value: "1", Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt32, + Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + length, numberCount, specialCharCount := vardef.PasswordValidationLength.Load(), vardef.PasswordValidtaionNumberCount.Load(), vardef.PasswordValidationSpecialCharCount.Load() mixedCaseCount, err := strconv.ParseInt(normalizedValue, 10, 32) if err != nil { return "", err @@ -654,15 +655,15 @@ var defaultSysVars = []*SysVar{ return normalizedValue, nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - PasswordValidationMixedCaseCount.Store(int32(TidbOptInt64(val, 1))) + vardef.PasswordValidationMixedCaseCount.Store(int32(TidbOptInt64(val, 1))) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatInt(int64(PasswordValidationMixedCaseCount.Load()), 10), nil + return strconv.FormatInt(int64(vardef.PasswordValidationMixedCaseCount.Load()), 10), nil }, }, - {Scope: ScopeGlobal, Name: ValidatePasswordNumberCount, Value: "1", Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, - Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - length, specialCharCount, mixedCaseCount := PasswordValidationLength.Load(), PasswordValidationSpecialCharCount.Load(), PasswordValidationMixedCaseCount.Load() + {Scope: vardef.ScopeGlobal, Name: vardef.ValidatePasswordNumberCount, Value: "1", Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt32, + Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + length, specialCharCount, mixedCaseCount := vardef.PasswordValidationLength.Load(), vardef.PasswordValidationSpecialCharCount.Load(), vardef.PasswordValidationMixedCaseCount.Load() numberCount, err := strconv.ParseInt(normalizedValue, 10, 32) if err != nil { return "", err @@ -676,15 +677,15 @@ var defaultSysVars = []*SysVar{ return normalizedValue, nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - PasswordValidtaionNumberCount.Store(int32(TidbOptInt64(val, 1))) + vardef.PasswordValidtaionNumberCount.Store(int32(TidbOptInt64(val, 1))) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatInt(int64(PasswordValidtaionNumberCount.Load()), 10), nil + return strconv.FormatInt(int64(vardef.PasswordValidtaionNumberCount.Load()), 10), nil }, }, - {Scope: ScopeGlobal, Name: ValidatePasswordSpecialCharCount, Value: "1", Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, - Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - length, numberCount, mixedCaseCount := PasswordValidationLength.Load(), PasswordValidtaionNumberCount.Load(), PasswordValidationMixedCaseCount.Load() + {Scope: vardef.ScopeGlobal, Name: vardef.ValidatePasswordSpecialCharCount, Value: "1", Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt32, + Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + length, numberCount, mixedCaseCount := vardef.PasswordValidationLength.Load(), vardef.PasswordValidtaionNumberCount.Load(), vardef.PasswordValidationMixedCaseCount.Load() specialCharCount, err := strconv.ParseInt(normalizedValue, 10, 32) if err != nil { return "", err @@ -698,58 +699,58 @@ var defaultSysVars = []*SysVar{ return normalizedValue, nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - PasswordValidationSpecialCharCount.Store(int32(TidbOptInt64(val, 1))) + vardef.PasswordValidationSpecialCharCount.Store(int32(TidbOptInt64(val, 1))) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatInt(int64(PasswordValidationSpecialCharCount.Load()), 10), nil + return strconv.FormatInt(int64(vardef.PasswordValidationSpecialCharCount.Load()), 10), nil }, }, - {Scope: ScopeGlobal, Name: ValidatePasswordDictionary, Value: "", Type: TypeStr}, - {Scope: ScopeGlobal, Name: DefaultPasswordLifetime, Value: "0", Type: TypeInt, MinValue: 0, MaxValue: math.MaxUint16}, - {Scope: ScopeGlobal, Name: DisconnectOnExpiredPassword, Value: On, Type: TypeBool, ReadOnly: true, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(!IsSandBoxModeEnabled.Load()), nil + {Scope: vardef.ScopeGlobal, Name: vardef.ValidatePasswordDictionary, Value: "", Type: vardef.TypeStr}, + {Scope: vardef.ScopeGlobal, Name: vardef.DefaultPasswordLifetime, Value: "0", Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxUint16}, + {Scope: vardef.ScopeGlobal, Name: vardef.DisconnectOnExpiredPassword, Value: vardef.On, Type: vardef.TypeBool, ReadOnly: true, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + return BoolToOnOff(!vardef.IsSandBoxModeEnabled.Load()), nil }}, /* TiDB specific variables */ - {Scope: ScopeGlobal, Name: TiDBTSOClientBatchMaxWaitTime, Value: strconv.FormatFloat(DefTiDBTSOClientBatchMaxWaitTime, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: 10, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBTSOClientBatchMaxWaitTime, Value: strconv.FormatFloat(vardef.DefTiDBTSOClientBatchMaxWaitTime, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0, MaxValue: 10, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { - return strconv.FormatFloat(MaxTSOBatchWaitInterval.Load(), 'f', -1, 64), nil + return strconv.FormatFloat(vardef.MaxTSOBatchWaitInterval.Load(), 'f', -1, 64), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - return (*SetPDClientDynamicOption.Load())(TiDBTSOClientBatchMaxWaitTime, val) + return (*SetPDClientDynamicOption.Load())(vardef.TiDBTSOClientBatchMaxWaitTime, val) }}, - {Scope: ScopeGlobal, Name: TiDBEnableTSOFollowerProxy, Value: BoolToOnOff(DefTiDBEnableTSOFollowerProxy), Type: TypeBool, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { - return BoolToOnOff(EnableTSOFollowerProxy.Load()), nil + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableTSOFollowerProxy, Value: BoolToOnOff(vardef.DefTiDBEnableTSOFollowerProxy), Type: vardef.TypeBool, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { + return BoolToOnOff(vardef.EnableTSOFollowerProxy.Load()), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - return (*SetPDClientDynamicOption.Load())(TiDBEnableTSOFollowerProxy, val) + return (*SetPDClientDynamicOption.Load())(vardef.TiDBEnableTSOFollowerProxy, val) }}, - {Scope: ScopeGlobal, Name: PDEnableFollowerHandleRegion, Value: BoolToOnOff(DefPDEnableFollowerHandleRegion), Type: TypeBool, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { - return BoolToOnOff(EnablePDFollowerHandleRegion.Load()), nil + {Scope: vardef.ScopeGlobal, Name: vardef.PDEnableFollowerHandleRegion, Value: BoolToOnOff(vardef.DefPDEnableFollowerHandleRegion), Type: vardef.TypeBool, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { + return BoolToOnOff(vardef.EnablePDFollowerHandleRegion.Load()), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - return (*SetPDClientDynamicOption.Load())(PDEnableFollowerHandleRegion, val) + return (*SetPDClientDynamicOption.Load())(vardef.PDEnableFollowerHandleRegion, val) }}, - {Scope: ScopeGlobal, Name: TiDBEnableLocalTxn, Value: BoolToOnOff(DefTiDBEnableLocalTxn), Hidden: true, Type: TypeBool, Depended: true, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { - return BoolToOnOff(EnableLocalTxn.Load()), nil + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableLocalTxn, Value: BoolToOnOff(vardef.DefTiDBEnableLocalTxn), Hidden: true, Type: vardef.TypeBool, Depended: true, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { + return BoolToOnOff(vardef.EnableLocalTxn.Load()), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - oldVal := EnableLocalTxn.Load() + oldVal := vardef.EnableLocalTxn.Load() newVal := TiDBOptOn(val) // Make sure the TxnScope is always Global when disable the Local Txn. // ON -> OFF if oldVal && !newVal { s.TxnScope = kv.NewGlobalTxnScopeVar() } - EnableLocalTxn.Store(newVal) + vardef.EnableLocalTxn.Store(newVal) return nil }}, { - Scope: ScopeGlobal, - Name: TiDBAutoAnalyzeRatio, - Value: strconv.FormatFloat(DefAutoAnalyzeRatio, 'f', -1, 64), - Type: TypeFloat, + Scope: vardef.ScopeGlobal, + Name: vardef.TiDBAutoAnalyzeRatio, + Value: strconv.FormatFloat(vardef.DefAutoAnalyzeRatio, 'f', -1, 64), + Type: vardef.TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, // The value of TiDBAutoAnalyzeRatio should be greater than 0.00001 or equal to 0.00001. - Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { ratio, err := strconv.ParseFloat(normalizedValue, 64) if err != nil { return "", err @@ -757,32 +758,32 @@ var defaultSysVars = []*SysVar{ const minRatio = 0.00001 const tolerance = 1e-9 if ratio < minRatio && math.Abs(ratio-minRatio) > tolerance { - return "", errors.Errorf("the value of %s should be greater than or equal to %f", TiDBAutoAnalyzeRatio, minRatio) + return "", errors.Errorf("the value of %s should be greater than or equal to %f", vardef.TiDBAutoAnalyzeRatio, minRatio) } return normalizedValue, nil }, }, - {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeStartTime, Value: DefAutoAnalyzeStartTime, Type: TypeTime}, - {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeEndTime, Value: DefAutoAnalyzeEndTime, Type: TypeTime}, - {Scope: ScopeGlobal, Name: TiDBMemQuotaBindingCache, Value: strconv.FormatInt(DefTiDBMemQuotaBindingCache, 10), Type: TypeUnsigned, MaxValue: math.MaxInt32, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { - return strconv.FormatInt(MemQuotaBindingCache.Load(), 10), nil + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBAutoAnalyzeStartTime, Value: vardef.DefAutoAnalyzeStartTime, Type: vardef.TypeTime}, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBAutoAnalyzeEndTime, Value: vardef.DefAutoAnalyzeEndTime, Type: vardef.TypeTime}, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBMemQuotaBindingCache, Value: strconv.FormatInt(vardef.DefTiDBMemQuotaBindingCache, 10), Type: vardef.TypeUnsigned, MaxValue: math.MaxInt32, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { + return strconv.FormatInt(vardef.MemQuotaBindingCache.Load(), 10), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - MemQuotaBindingCache.Store(TidbOptInt64(val, DefTiDBMemQuotaBindingCache)) + vardef.MemQuotaBindingCache.Store(TidbOptInt64(val, vardef.DefTiDBMemQuotaBindingCache)) return nil }}, - {Scope: ScopeGlobal, Name: TiDBDDLFlashbackConcurrency, Value: strconv.Itoa(DefTiDBDDLFlashbackConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: MaxConfigurableConcurrency, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - SetDDLFlashbackConcurrency(int32(tidbOptPositiveInt32(val, DefTiDBDDLFlashbackConcurrency))) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBDDLFlashbackConcurrency, Value: strconv.Itoa(vardef.DefTiDBDDLFlashbackConcurrency), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.SetDDLFlashbackConcurrency(int32(tidbOptPositiveInt32(val, vardef.DefTiDBDDLFlashbackConcurrency))) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBDDLReorgWorkerCount, Value: strconv.Itoa(DefTiDBDDLReorgWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: MaxConfigurableConcurrency, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - SetDDLReorgWorkerCounter(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgWorkerCount))) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBDDLReorgWorkerCount, Value: strconv.Itoa(vardef.DefTiDBDDLReorgWorkerCount), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.SetDDLReorgWorkerCounter(int32(tidbOptPositiveInt32(val, vardef.DefTiDBDDLReorgWorkerCount))) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBDDLReorgBatchSize, Value: strconv.Itoa(DefTiDBDDLReorgBatchSize), Type: TypeUnsigned, MinValue: int64(MinDDLReorgBatchSize), MaxValue: uint64(MaxDDLReorgBatchSize), SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - SetDDLReorgBatchSize(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgBatchSize))) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBDDLReorgBatchSize, Value: strconv.Itoa(vardef.DefTiDBDDLReorgBatchSize), Type: vardef.TypeUnsigned, MinValue: int64(vardef.MinDDLReorgBatchSize), MaxValue: uint64(vardef.MaxDDLReorgBatchSize), SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.SetDDLReorgBatchSize(int32(tidbOptPositiveInt32(val, vardef.DefTiDBDDLReorgBatchSize))) return nil }}, - {Scope: ScopeGlobal, Name: TiDBDDLReorgMaxWriteSpeed, Value: strconv.Itoa(DefTiDBDDLReorgMaxWriteSpeed), Type: TypeStr, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBDDLReorgMaxWriteSpeed, Value: strconv.Itoa(vardef.DefTiDBDDLReorgMaxWriteSpeed), Type: vardef.TypeStr, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { i64, err := units.RAMInBytes(val) if err != nil { @@ -794,21 +795,21 @@ var defaultSysVars = []*SysVar{ // 2. units.RAMInBytes would first cast the size to a float, and may lose precision when the size is too large return fmt.Errorf("invalid value for '%d', it should be within [%d, %d]", i64, 0, units.PiB) } - DDLReorgMaxWriteSpeed.Store(i64) + vardef.DDLReorgMaxWriteSpeed.Store(i64) return nil }, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { - return strconv.FormatInt(DDLReorgMaxWriteSpeed.Load(), 10), nil + return strconv.FormatInt(vardef.DDLReorgMaxWriteSpeed.Load(), 10), nil }}, - {Scope: ScopeGlobal, Name: TiDBDDLErrorCountLimit, Value: strconv.Itoa(DefTiDBDDLErrorCountLimit), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt64, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - SetDDLErrorCountLimit(TidbOptInt64(val, DefTiDBDDLErrorCountLimit)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBDDLErrorCountLimit, Value: strconv.Itoa(vardef.DefTiDBDDLErrorCountLimit), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt64, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.SetDDLErrorCountLimit(TidbOptInt64(val, vardef.DefTiDBDDLErrorCountLimit)) return nil }}, - {Scope: ScopeGlobal, Name: TiDBMaxDeltaSchemaCount, Value: strconv.Itoa(DefTiDBMaxDeltaSchemaCount), Type: TypeUnsigned, MinValue: 100, MaxValue: 16384, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBMaxDeltaSchemaCount, Value: strconv.Itoa(vardef.DefTiDBMaxDeltaSchemaCount), Type: vardef.TypeUnsigned, MinValue: 100, MaxValue: 16384, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { // It's a global variable, but it also wants to be cached in server. - SetMaxDeltaSchemaCount(TidbOptInt64(val, DefTiDBMaxDeltaSchemaCount)) + vardef.SetMaxDeltaSchemaCount(TidbOptInt64(val, vardef.DefTiDBMaxDeltaSchemaCount)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBScatterRegion, Value: DefTiDBScatterRegion, PossibleValues: []string{ScatterOff, ScatterTable, ScatterGlobal}, Type: TypeStr, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBScatterRegion, Value: vardef.DefTiDBScatterRegion, PossibleValues: []string{vardef.ScatterOff, vardef.ScatterTable, vardef.ScatterGlobal}, Type: vardef.TypeStr, SetSession: func(vars *SessionVars, val string) error { vars.ScatterRegion = val return nil @@ -816,50 +817,50 @@ var defaultSysVars = []*SysVar{ GetSession: func(vars *SessionVars) (string, error) { return vars.ScatterRegion, nil }, - Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { lowerVal := strings.ToLower(normalizedValue) - if lowerVal != ScatterOff && lowerVal != ScatterTable && lowerVal != ScatterGlobal { - return "", fmt.Errorf("invalid value for '%s', it should be either '%s', '%s' or '%s'", lowerVal, ScatterOff, ScatterTable, ScatterGlobal) + if lowerVal != vardef.ScatterOff && lowerVal != vardef.ScatterTable && lowerVal != vardef.ScatterGlobal { + return "", fmt.Errorf("invalid value for '%s', it should be either '%s', '%s' or '%s'", lowerVal, vardef.ScatterOff, vardef.ScatterTable, vardef.ScatterGlobal) } return lowerVal, nil }, }, - {Scope: ScopeGlobal, Name: TiDBEnableStmtSummary, Value: BoolToOnOff(DefTiDBEnableStmtSummary), Type: TypeBool, AllowEmpty: true, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableStmtSummary, Value: BoolToOnOff(vardef.DefTiDBEnableStmtSummary), Type: vardef.TypeBool, AllowEmpty: true, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { return stmtsummaryv2.SetEnabled(TiDBOptOn(val)) }}, - {Scope: ScopeGlobal, Name: TiDBStmtSummaryInternalQuery, Value: BoolToOnOff(DefTiDBStmtSummaryInternalQuery), Type: TypeBool, AllowEmpty: true, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBStmtSummaryInternalQuery, Value: BoolToOnOff(vardef.DefTiDBStmtSummaryInternalQuery), Type: vardef.TypeBool, AllowEmpty: true, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { return stmtsummaryv2.SetEnableInternalQuery(TiDBOptOn(val)) }}, - {Scope: ScopeGlobal, Name: TiDBStmtSummaryRefreshInterval, Value: strconv.Itoa(DefTiDBStmtSummaryRefreshInterval), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowEmpty: true, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBStmtSummaryRefreshInterval, Value: strconv.Itoa(vardef.DefTiDBStmtSummaryRefreshInterval), Type: vardef.TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowEmpty: true, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { // convert val to int64 - return stmtsummaryv2.SetRefreshInterval(TidbOptInt64(val, DefTiDBStmtSummaryRefreshInterval)) + return stmtsummaryv2.SetRefreshInterval(TidbOptInt64(val, vardef.DefTiDBStmtSummaryRefreshInterval)) }}, - {Scope: ScopeGlobal, Name: TiDBStmtSummaryHistorySize, Value: strconv.Itoa(DefTiDBStmtSummaryHistorySize), Type: TypeInt, MinValue: 0, MaxValue: math.MaxUint8, AllowEmpty: true, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBStmtSummaryHistorySize, Value: strconv.Itoa(vardef.DefTiDBStmtSummaryHistorySize), Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxUint8, AllowEmpty: true, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - return stmtsummaryv2.SetHistorySize(TidbOptInt(val, DefTiDBStmtSummaryHistorySize)) + return stmtsummaryv2.SetHistorySize(TidbOptInt(val, vardef.DefTiDBStmtSummaryHistorySize)) }}, - {Scope: ScopeGlobal, Name: TiDBStmtSummaryMaxStmtCount, Value: strconv.Itoa(DefTiDBStmtSummaryMaxStmtCount), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt16, AllowEmpty: true, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBStmtSummaryMaxStmtCount, Value: strconv.Itoa(vardef.DefTiDBStmtSummaryMaxStmtCount), Type: vardef.TypeInt, MinValue: 1, MaxValue: math.MaxInt16, AllowEmpty: true, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - return stmtsummaryv2.SetMaxStmtCount(TidbOptInt(val, DefTiDBStmtSummaryMaxStmtCount)) + return stmtsummaryv2.SetMaxStmtCount(TidbOptInt(val, vardef.DefTiDBStmtSummaryMaxStmtCount)) }}, - {Scope: ScopeGlobal, Name: TiDBStmtSummaryMaxSQLLength, Value: strconv.Itoa(DefTiDBStmtSummaryMaxSQLLength), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, AllowEmpty: true, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBStmtSummaryMaxSQLLength, Value: strconv.Itoa(vardef.DefTiDBStmtSummaryMaxSQLLength), Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt32, AllowEmpty: true, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - return stmtsummaryv2.SetMaxSQLLength(TidbOptInt(val, DefTiDBStmtSummaryMaxSQLLength)) + return stmtsummaryv2.SetMaxSQLLength(TidbOptInt(val, vardef.DefTiDBStmtSummaryMaxSQLLength)) }}, - {Scope: ScopeGlobal, Name: TiDBCapturePlanBaseline, Value: DefTiDBCapturePlanBaseline, Type: TypeBool, AllowEmptyAll: true}, - {Scope: ScopeGlobal, Name: TiDBEvolvePlanTaskMaxTime, Value: strconv.Itoa(DefTiDBEvolvePlanTaskMaxTime), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, - {Scope: ScopeGlobal, Name: TiDBEvolvePlanTaskStartTime, Value: DefTiDBEvolvePlanTaskStartTime, Type: TypeTime}, - {Scope: ScopeGlobal, Name: TiDBEvolvePlanTaskEndTime, Value: DefTiDBEvolvePlanTaskEndTime, Type: TypeTime}, - {Scope: ScopeGlobal, Name: TiDBStoreLimit, Value: strconv.FormatInt(atomic.LoadInt64(&config.GetGlobalConfig().TiKVClient.StoreLimit), 10), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBCapturePlanBaseline, Value: vardef.DefTiDBCapturePlanBaseline, Type: vardef.TypeBool, AllowEmptyAll: true}, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEvolvePlanTaskMaxTime, Value: strconv.Itoa(vardef.DefTiDBEvolvePlanTaskMaxTime), Type: vardef.TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEvolvePlanTaskStartTime, Value: vardef.DefTiDBEvolvePlanTaskStartTime, Type: vardef.TypeTime}, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEvolvePlanTaskEndTime, Value: vardef.DefTiDBEvolvePlanTaskEndTime, Type: vardef.TypeTime}, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBStoreLimit, Value: strconv.FormatInt(atomic.LoadInt64(&config.GetGlobalConfig().TiKVClient.StoreLimit), 10), Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt64, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return strconv.FormatInt(tikvstore.StoreLimit.Load(), 10), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - tikvstore.StoreLimit.Store(TidbOptInt64(val, DefTiDBStoreLimit)) + tikvstore.StoreLimit.Store(TidbOptInt64(val, vardef.DefTiDBStoreLimit)) return nil }}, - {Scope: ScopeGlobal, Name: TiDBTxnCommitBatchSize, Value: strconv.FormatUint(tikvstore.DefTxnCommitBatchSize, 10), Type: TypeUnsigned, MinValue: 1, MaxValue: 1 << 30, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBTxnCommitBatchSize, Value: strconv.FormatUint(tikvstore.DefTxnCommitBatchSize, 10), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: 1 << 30, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { return strconv.FormatUint(tikvstore.TxnCommitBatchSize.Load(), 10), nil }, @@ -867,39 +868,39 @@ var defaultSysVars = []*SysVar{ tikvstore.TxnCommitBatchSize.Store(uint64(TidbOptInt64(val, int64(tikvstore.DefTxnCommitBatchSize)))) return nil }}, - {Scope: ScopeGlobal, Name: TiDBRestrictedReadOnly, Value: BoolToOnOff(DefTiDBRestrictedReadOnly), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBRestrictedReadOnly, Value: BoolToOnOff(vardef.DefTiDBRestrictedReadOnly), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { on := TiDBOptOn(val) // For user initiated SET GLOBAL, also change the value of TiDBSuperReadOnly if on && s.StmtCtx.StmtType == "Set" { - err := s.GlobalVarsAccessor.SetGlobalSysVarOnly(context.Background(), TiDBSuperReadOnly, "ON", false) + err := s.GlobalVarsAccessor.SetGlobalSysVarOnly(context.Background(), vardef.TiDBSuperReadOnly, "ON", false) if err != nil { return err } - err = GetSysVar(TiDBSuperReadOnly).SetGlobal(context.Background(), s, "ON") + err = GetSysVar(vardef.TiDBSuperReadOnly).SetGlobal(context.Background(), s, "ON") if err != nil { return err } } - RestrictedReadOnly.Store(on) + vardef.RestrictedReadOnly.Store(on) return nil }}, - {Scope: ScopeGlobal, Name: TiDBSuperReadOnly, Value: BoolToOnOff(DefTiDBSuperReadOnly), Type: TypeBool, Validation: func(s *SessionVars, normalizedValue string, _ string, _ ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBSuperReadOnly, Value: BoolToOnOff(vardef.DefTiDBSuperReadOnly), Type: vardef.TypeBool, Validation: func(s *SessionVars, normalizedValue string, _ string, _ vardef.ScopeFlag) (string, error) { on := TiDBOptOn(normalizedValue) if !on && s.StmtCtx.StmtType == "Set" { - result, err := s.GlobalVarsAccessor.GetGlobalSysVar(TiDBRestrictedReadOnly) + result, err := s.GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBRestrictedReadOnly) if err != nil { return normalizedValue, err } if TiDBOptOn(result) { - return normalizedValue, fmt.Errorf("can't turn off %s when %s is on", TiDBSuperReadOnly, TiDBRestrictedReadOnly) + return normalizedValue, fmt.Errorf("can't turn off %s when %s is on", vardef.TiDBSuperReadOnly, vardef.TiDBRestrictedReadOnly) } } return normalizedValue, nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - VarTiDBSuperReadOnly.Store(TiDBOptOn(val)) + vardef.VarTiDBSuperReadOnly.Store(TiDBOptOn(val)) return nil }}, - {Scope: ScopeGlobal, Name: TiDBEnableGOGCTuner, Value: BoolToOnOff(DefTiDBEnableGOGCTuner), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableGOGCTuner, Value: BoolToOnOff(vardef.DefTiDBEnableGOGCTuner), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { on := TiDBOptOn(val) gctuner.EnableGOGCTuner.Store(on) if !on { @@ -908,9 +909,9 @@ var defaultSysVars = []*SysVar{ gctuner.GlobalMemoryLimitTuner.UpdateMemoryLimit() return nil }}, - {Scope: ScopeGlobal, Name: TiDBGOGCTunerMaxValue, Value: strconv.Itoa(DefTiDBGOGCMaxValue), - Type: TypeInt, MinValue: 10, MaxValue: math.MaxInt32, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - maxValue := TidbOptInt64(val, DefTiDBGOGCMaxValue) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBGOGCTunerMaxValue, Value: strconv.Itoa(vardef.DefTiDBGOGCMaxValue), + Type: vardef.TypeInt, MinValue: 10, MaxValue: math.MaxInt32, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + maxValue := TidbOptInt64(val, vardef.DefTiDBGOGCMaxValue) gctuner.SetMaxGCPercent(uint32(maxValue)) gctuner.GlobalMemoryLimitTuner.UpdateMemoryLimit() return nil @@ -918,16 +919,16 @@ var defaultSysVars = []*SysVar{ GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return strconv.FormatInt(int64(gctuner.MaxGCPercent()), 10), nil }, - Validation: func(s *SessionVars, normalizedValue string, origin string, scope ScopeFlag) (string, error) { - maxValue := TidbOptInt64(origin, DefTiDBGOGCMaxValue) + Validation: func(s *SessionVars, normalizedValue string, origin string, scope vardef.ScopeFlag) (string, error) { + maxValue := TidbOptInt64(origin, vardef.DefTiDBGOGCMaxValue) if maxValue <= int64(gctuner.MinGCPercent()) { return "", errors.New("tidb_gogc_tuner_max_value should be more than tidb_gogc_tuner_min_value") } return origin, nil }}, - {Scope: ScopeGlobal, Name: TiDBGOGCTunerMinValue, Value: strconv.Itoa(DefTiDBGOGCMinValue), - Type: TypeInt, MinValue: 10, MaxValue: math.MaxInt32, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - minValue := TidbOptInt64(val, DefTiDBGOGCMinValue) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBGOGCTunerMinValue, Value: strconv.Itoa(vardef.DefTiDBGOGCMinValue), + Type: vardef.TypeInt, MinValue: 10, MaxValue: math.MaxInt32, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + minValue := TidbOptInt64(val, vardef.DefTiDBGOGCMinValue) gctuner.SetMinGCPercent(uint32(minValue)) gctuner.GlobalMemoryLimitTuner.UpdateMemoryLimit() return nil @@ -935,46 +936,46 @@ var defaultSysVars = []*SysVar{ GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return strconv.FormatInt(int64(gctuner.MinGCPercent()), 10), nil }, - Validation: func(s *SessionVars, normalizedValue string, origin string, scope ScopeFlag) (string, error) { - minValue := TidbOptInt64(origin, DefTiDBGOGCMinValue) + Validation: func(s *SessionVars, normalizedValue string, origin string, scope vardef.ScopeFlag) (string, error) { + minValue := TidbOptInt64(origin, vardef.DefTiDBGOGCMinValue) if minValue >= int64(gctuner.MaxGCPercent()) { return "", errors.New("tidb_gogc_tuner_min_value should be less than tidb_gogc_tuner_max_value") } return origin, nil }}, - {Scope: ScopeGlobal, Name: TiDBEnableTelemetry, Value: BoolToOnOff(DefTiDBEnableTelemetry), Type: TypeBool, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableTelemetry, Value: BoolToOnOff(vardef.DefTiDBEnableTelemetry), Type: vardef.TypeBool, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return "OFF", nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { s.StmtCtx.AppendWarning(ErrWarnDeprecatedSyntaxSimpleMsg.FastGen("tidb_enable_telemetry is deprecated since Telemetry has been removed, this variable is 'OFF' always.")) return nil }}, - {Scope: ScopeGlobal, Name: TiDBEnableHistoricalStats, Value: Off, Type: TypeBool, Depended: true}, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableHistoricalStats, Value: vardef.Off, Type: vardef.TypeBool, Depended: true}, /* tikv gc metrics */ - {Scope: ScopeGlobal, Name: TiDBGCEnable, Value: On, Type: TypeBool, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return getTiDBTableValue(s, "tikv_gc_enable", On) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBGCEnable, Value: vardef.On, Type: vardef.TypeBool, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + return getTiDBTableValue(s, "tikv_gc_enable", vardef.On) }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { return setTiDBTableValue(s, "tikv_gc_enable", val, "Current GC enable status") }}, - {Scope: ScopeGlobal, Name: TiDBGCRunInterval, Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: uint64(time.Hour * 24 * 365), GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBGCRunInterval, Value: "10m0s", Type: vardef.TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: uint64(time.Hour * 24 * 365), GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return getTiDBTableValue(s, "tikv_gc_run_interval", "10m0s") }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { return setTiDBTableValue(s, "tikv_gc_run_interval", val, "GC run interval, at least 10m, in Go format.") }}, - {Scope: ScopeGlobal, Name: TiDBGCLifetime, Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: uint64(time.Hour * 24 * 365), GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBGCLifetime, Value: "10m0s", Type: vardef.TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: uint64(time.Hour * 24 * 365), GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return getTiDBTableValue(s, "tikv_gc_life_time", "10m0s") }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { return setTiDBTableValue(s, "tikv_gc_life_time", val, "All versions within life time will not be collected by GC, at least 10m, in Go format.") }}, - {Scope: ScopeGlobal, Name: TiDBGCConcurrency, Value: "-1", Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, AllowAutoValue: true, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - autoConcurrencyVal, err := getTiDBTableValue(s, "tikv_gc_auto_concurrency", On) - if err == nil && autoConcurrencyVal == On { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBGCConcurrency, Value: "-1", Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, AllowAutoValue: true, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + autoConcurrencyVal, err := getTiDBTableValue(s, "tikv_gc_auto_concurrency", vardef.On) + if err == nil && autoConcurrencyVal == vardef.On { return "-1", nil // convention for "AUTO" } return getTiDBTableValue(s, "tikv_gc_concurrency", "-1") }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - autoConcurrency := Off + autoConcurrency := vardef.Off if val == "-1" { - autoConcurrency = On + autoConcurrency = vardef.On } // Update both autoconcurrency and concurrency. if err := setTiDBTableValue(s, "tikv_gc_auto_concurrency", autoConcurrency, "Let TiDB pick the concurrency automatically. If set false, tikv_gc_concurrency will be used"); err != nil { @@ -982,46 +983,46 @@ var defaultSysVars = []*SysVar{ } return setTiDBTableValue(s, "tikv_gc_concurrency", val, "How many goroutines used to do GC parallel, [1, 256], default 2") }}, - {Scope: ScopeGlobal, Name: TiDBGCScanLockMode, Value: "LEGACY", Type: TypeEnum, PossibleValues: []string{"PHYSICAL", "LEGACY"}, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBGCScanLockMode, Value: "LEGACY", Type: vardef.TypeEnum, PossibleValues: []string{"PHYSICAL", "LEGACY"}, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return getTiDBTableValue(s, "tikv_gc_scan_lock_mode", "LEGACY") }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { return setTiDBTableValue(s, "tikv_gc_scan_lock_mode", val, "Mode of scanning locks, \"physical\" or \"legacy\"") }}, - {Scope: ScopeGlobal, Name: TiDBGCMaxWaitTime, Value: strconv.Itoa(DefTiDBGCMaxWaitTime), Type: TypeInt, MinValue: 600, MaxValue: 31536000, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - GCMaxWaitTime.Store(TidbOptInt64(val, DefTiDBGCMaxWaitTime)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBGCMaxWaitTime, Value: strconv.Itoa(vardef.DefTiDBGCMaxWaitTime), Type: vardef.TypeInt, MinValue: 600, MaxValue: 31536000, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.GCMaxWaitTime.Store(TidbOptInt64(val, vardef.DefTiDBGCMaxWaitTime)) return nil }}, - {Scope: ScopeGlobal, Name: TiDBTableCacheLease, Value: strconv.Itoa(DefTiDBTableCacheLease), Type: TypeUnsigned, MinValue: 1, MaxValue: 10, SetGlobal: func(_ context.Context, s *SessionVars, sVal string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBTableCacheLease, Value: strconv.Itoa(vardef.DefTiDBTableCacheLease), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: 10, SetGlobal: func(_ context.Context, s *SessionVars, sVal string) error { var val int64 val, err := strconv.ParseInt(sVal, 10, 64) if err != nil { return errors.Trace(err) } - TableCacheLease.Store(val) + vardef.TableCacheLease.Store(val) return nil }}, - {Scope: ScopeGlobal, Name: TiDBAutoAnalyzePartitionBatchSize, - Value: strconv.Itoa(DefTiDBAutoAnalyzePartitionBatchSize), - Type: TypeUnsigned, MinValue: 1, MaxValue: mysql.PartitionCountLimit, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBAutoAnalyzePartitionBatchSize, + Value: strconv.Itoa(vardef.DefTiDBAutoAnalyzePartitionBatchSize), + Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: mysql.PartitionCountLimit, SetGlobal: func(_ context.Context, vars *SessionVars, s string) error { var val int64 val, err := strconv.ParseInt(s, 10, 64) if err != nil { return errors.Trace(err) } - AutoAnalyzePartitionBatchSize.Store(val) + vardef.AutoAnalyzePartitionBatchSize.Store(val) return nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - vars.StmtCtx.AppendWarning(ErrWarnDeprecatedSyntaxNoReplacement.FastGenByArgs(TiDBAutoAnalyzePartitionBatchSize)) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + vars.StmtCtx.AppendWarning(ErrWarnDeprecatedSyntaxNoReplacement.FastGenByArgs(vardef.TiDBAutoAnalyzePartitionBatchSize)) return normalizedValue, nil }, }, // variable for top SQL feature. // TopSQL enable only be controlled by TopSQL pub/sub sinker. // This global variable only uses to update the global config which store in PD(ETCD). - {Scope: ScopeGlobal, Name: TiDBEnableTopSQL, Value: BoolToOnOff(topsqlstate.DefTiDBTopSQLEnable), Type: TypeBool, AllowEmpty: true, GlobalConfigName: GlobalConfigEnableTopSQL}, - {Scope: ScopeGlobal, Name: TiDBSourceID, Value: "1", Type: TypeInt, MinValue: 1, MaxValue: 15, GlobalConfigName: GlobalConfigSourceID}, - {Scope: ScopeGlobal, Name: TiDBTopSQLMaxTimeSeriesCount, Value: strconv.Itoa(topsqlstate.DefTiDBTopSQLMaxTimeSeriesCount), Type: TypeInt, MinValue: 1, MaxValue: 5000, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableTopSQL, Value: BoolToOnOff(topsqlstate.DefTiDBTopSQLEnable), Type: vardef.TypeBool, AllowEmpty: true, GlobalConfigName: vardef.GlobalConfigEnableTopSQL}, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBSourceID, Value: "1", Type: vardef.TypeInt, MinValue: 1, MaxValue: 15, GlobalConfigName: vardef.GlobalConfigSourceID}, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBTopSQLMaxTimeSeriesCount, Value: strconv.Itoa(topsqlstate.DefTiDBTopSQLMaxTimeSeriesCount), Type: vardef.TypeInt, MinValue: 1, MaxValue: 5000, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return strconv.FormatInt(topsqlstate.GlobalState.MaxStatementCount.Load(), 10), nil }, SetGlobal: func(_ context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) @@ -1031,7 +1032,7 @@ var defaultSysVars = []*SysVar{ topsqlstate.GlobalState.MaxStatementCount.Store(val) return nil }}, - {Scope: ScopeGlobal, Name: TiDBTopSQLMaxMetaCount, Value: strconv.Itoa(topsqlstate.DefTiDBTopSQLMaxMetaCount), Type: TypeInt, MinValue: 1, MaxValue: 10000, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBTopSQLMaxMetaCount, Value: strconv.Itoa(topsqlstate.DefTiDBTopSQLMaxMetaCount), Type: vardef.TypeInt, MinValue: 1, MaxValue: 10000, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return strconv.FormatInt(topsqlstate.GlobalState.MaxCollect.Load(), 10), nil }, SetGlobal: func(_ context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) @@ -1041,50 +1042,50 @@ var defaultSysVars = []*SysVar{ topsqlstate.GlobalState.MaxCollect.Store(val) return nil }}, - {Scope: ScopeGlobal, Name: SkipNameResolve, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal, Name: DefaultAuthPlugin, Value: mysql.AuthNativePassword, Type: TypeEnum, PossibleValues: []string{mysql.AuthNativePassword, mysql.AuthCachingSha2Password, mysql.AuthTiDBSM3Password, mysql.AuthLDAPSASL, mysql.AuthLDAPSimple}}, + {Scope: vardef.ScopeGlobal, Name: vardef.SkipNameResolve, Value: vardef.Off, Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal, Name: vardef.DefaultAuthPlugin, Value: mysql.AuthNativePassword, Type: vardef.TypeEnum, PossibleValues: []string{mysql.AuthNativePassword, mysql.AuthCachingSha2Password, mysql.AuthTiDBSM3Password, mysql.AuthLDAPSASL, mysql.AuthLDAPSimple}}, { - Scope: ScopeGlobal, - Name: TiDBPersistAnalyzeOptions, - Value: BoolToOnOff(DefTiDBPersistAnalyzeOptions), - Type: TypeBool, + Scope: vardef.ScopeGlobal, + Name: vardef.TiDBPersistAnalyzeOptions, + Value: BoolToOnOff(vardef.DefTiDBPersistAnalyzeOptions), + Type: vardef.TypeBool, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(PersistAnalyzeOptions.Load()), nil + return BoolToOnOff(vardef.PersistAnalyzeOptions.Load()), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { persist := TiDBOptOn(val) - PersistAnalyzeOptions.Store(persist) + vardef.PersistAnalyzeOptions.Store(persist) return nil }, }, { - Scope: ScopeGlobal, Name: TiDBEnableAutoAnalyze, Value: BoolToOnOff(DefTiDBEnableAutoAnalyze), Type: TypeBool, + Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableAutoAnalyze, Value: BoolToOnOff(vardef.DefTiDBEnableAutoAnalyze), Type: vardef.TypeBool, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(RunAutoAnalyze.Load()), nil + return BoolToOnOff(vardef.RunAutoAnalyze.Load()), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - RunAutoAnalyze.Store(TiDBOptOn(val)) + vardef.RunAutoAnalyze.Store(TiDBOptOn(val)) return nil }, }, { - Scope: ScopeGlobal, - Name: TiDBAnalyzeColumnOptions, - Value: DefTiDBAnalyzeColumnOptions, - Type: TypeStr, + Scope: vardef.ScopeGlobal, + Name: vardef.TiDBAnalyzeColumnOptions, + Value: vardef.DefTiDBAnalyzeColumnOptions, + Type: vardef.TypeStr, GetGlobal: func(ctx context.Context, s *SessionVars) (string, error) { - return AnalyzeColumnOptions.Load(), nil + return vardef.AnalyzeColumnOptions.Load(), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - AnalyzeColumnOptions.Store(strings.ToUpper(val)) + vardef.AnalyzeColumnOptions.Store(strings.ToUpper(val)) return nil }, - Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { choice := strings.ToUpper(normalizedValue) if choice != ast.AllColumns.String() && choice != ast.PredicateColumns.String() { return "", errors.Errorf( "invalid value for %s, it should be either '%s' or '%s'", - TiDBAnalyzeColumnOptions, + vardef.TiDBAnalyzeColumnOptions, ast.AllColumns.String(), ast.PredicateColumns.String(), ) @@ -1093,28 +1094,28 @@ var defaultSysVars = []*SysVar{ }, }, { - Scope: ScopeGlobal, Name: TiDBEnableAutoAnalyzePriorityQueue, Value: BoolToOnOff(DefTiDBEnableAutoAnalyzePriorityQueue), Type: TypeBool, + Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableAutoAnalyzePriorityQueue, Value: BoolToOnOff(vardef.DefTiDBEnableAutoAnalyzePriorityQueue), Type: vardef.TypeBool, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(EnableAutoAnalyzePriorityQueue.Load()), nil + return BoolToOnOff(vardef.EnableAutoAnalyzePriorityQueue.Load()), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - EnableAutoAnalyzePriorityQueue.Store(TiDBOptOn(val)) + vardef.EnableAutoAnalyzePriorityQueue.Store(TiDBOptOn(val)) return nil }, - Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { s.StmtCtx.AppendWarning(ErrWarnDeprecatedSyntaxSimpleMsg.FastGen("tidb_enable_auto_analyze_priority_queue will be removed in the future and TiDB will always use priority queue to execute auto analyze.")) return normalizedValue, nil }, }, - {Scope: ScopeGlobal, Name: TiDBGOGCTunerThreshold, Value: strconv.FormatFloat(DefTiDBGOGCTunerThreshold, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBGOGCTunerThreshold, Value: strconv.FormatFloat(vardef.DefTiDBGOGCTunerThreshold, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatFloat(GOGCTunerThreshold.Load(), 'f', -1, 64), nil + return strconv.FormatFloat(vardef.GOGCTunerThreshold.Load(), 'f', -1, 64), nil }, - Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - floatValue := tidbOptFloat64(normalizedValue, DefTiDBGOGCTunerThreshold) + Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + floatValue := tidbOptFloat64(normalizedValue, vardef.DefTiDBGOGCTunerThreshold) globalMemoryLimitTuner := gctuner.GlobalMemoryLimitTuner.GetPercentage() if floatValue < 0 && floatValue > 0.9 { - return "", ErrWrongValueForVar.GenWithStackByArgs(TiDBGOGCTunerThreshold, normalizedValue) + return "", ErrWrongValueForVar.GenWithStackByArgs(vardef.TiDBGOGCTunerThreshold, normalizedValue) } // globalMemoryLimitTuner must not be 0. it will be 0 when tidb_server_memory_limit_gc_trigger is not set during startup. if globalMemoryLimitTuner != 0 && globalMemoryLimitTuner < floatValue+0.05 { @@ -1123,8 +1124,8 @@ var defaultSysVars = []*SysVar{ return strconv.FormatFloat(floatValue, 'f', -1, 64), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) (err error) { - factor := tidbOptFloat64(val, DefTiDBGOGCTunerThreshold) - GOGCTunerThreshold.Store(factor) + factor := tidbOptFloat64(val, vardef.DefTiDBGOGCTunerThreshold) + vardef.GOGCTunerThreshold.Store(factor) memTotal := memory.ServerMemoryLimit.Load() if memTotal == 0 { memTotal, err = memory.MemTotal() @@ -1139,11 +1140,11 @@ var defaultSysVars = []*SysVar{ return nil }, }, - {Scope: ScopeGlobal, Name: TiDBServerMemoryLimit, Value: DefTiDBServerMemoryLimit, Type: TypeStr, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBServerMemoryLimit, Value: vardef.DefTiDBServerMemoryLimit, Type: vardef.TypeStr, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return memory.ServerMemoryLimitOriginText.Load(), nil }, - Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { _, str, err := parseMemoryLimit(s, normalizedValue, originalValue) if err != nil { return "", err @@ -1157,17 +1158,17 @@ var defaultSysVars = []*SysVar{ } memory.ServerMemoryLimitOriginText.Store(str) memory.ServerMemoryLimit.Store(bt) - threshold := float64(bt) * GOGCTunerThreshold.Load() + threshold := float64(bt) * vardef.GOGCTunerThreshold.Load() gctuner.Tuning(uint64(threshold)) gctuner.GlobalMemoryLimitTuner.UpdateMemoryLimit() return nil }, }, - {Scope: ScopeGlobal, Name: TiDBServerMemoryLimitSessMinSize, Value: strconv.FormatUint(DefTiDBServerMemoryLimitSessMinSize, 10), Type: TypeStr, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBServerMemoryLimitSessMinSize, Value: strconv.FormatUint(vardef.DefTiDBServerMemoryLimitSessMinSize, 10), Type: vardef.TypeStr, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return memory.ServerMemoryLimitSessMinSize.String(), nil }, - Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { intVal, err := strconv.ParseUint(normalizedValue, 10, 64) if err != nil { bt, str := parseByteSize(normalizedValue) @@ -1177,7 +1178,7 @@ var defaultSysVars = []*SysVar{ intVal = bt } if intVal > 0 && intVal < 128 { // 128 Bytes - s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(TiDBServerMemoryLimitSessMinSize, originalValue)) + s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(vardef.TiDBServerMemoryLimitSessMinSize, originalValue)) intVal = 128 } return strconv.FormatUint(intVal, 10), nil @@ -1191,11 +1192,11 @@ var defaultSysVars = []*SysVar{ return nil }, }, - {Scope: ScopeGlobal, Name: TiDBServerMemoryLimitGCTrigger, Value: strconv.FormatFloat(DefTiDBServerMemoryLimitGCTrigger, 'f', -1, 64), Type: TypeStr, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBServerMemoryLimitGCTrigger, Value: strconv.FormatFloat(vardef.DefTiDBServerMemoryLimitGCTrigger, 'f', -1, 64), Type: vardef.TypeStr, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return strconv.FormatFloat(gctuner.GlobalMemoryLimitTuner.GetPercentage(), 'f', -1, 64), nil }, - Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { floatValue, err := strconv.ParseFloat(normalizedValue, 64) if err != nil { perc, str := parsePercentage(normalizedValue) @@ -1204,9 +1205,9 @@ var defaultSysVars = []*SysVar{ } floatValue = float64(perc) / 100 } - gogcTunerThreshold := GOGCTunerThreshold.Load() + gogcTunerThreshold := vardef.GOGCTunerThreshold.Load() if floatValue < 0.51 || floatValue > 1 { // 51% ~ 100% - return "", ErrWrongValueForVar.GenWithStackByArgs(TiDBServerMemoryLimitGCTrigger, normalizedValue) + return "", ErrWrongValueForVar.GenWithStackByArgs(vardef.TiDBServerMemoryLimitGCTrigger, normalizedValue) } // gogcTunerThreshold must not be 0. it will be 0 when tidb_gogc_tuner_threshold is not set during startup. if gogcTunerThreshold != 0 && floatValue < gogcTunerThreshold+0.05 { @@ -1226,13 +1227,13 @@ var defaultSysVars = []*SysVar{ }, }, { - Scope: ScopeGlobal, Name: TiDBEnableColumnTracking, + Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableColumnTracking, Value: BoolToOnOff(true), - Type: TypeBool, + Type: vardef.TypeBool, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return BoolToOnOff(true), nil }, - Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { // This variable is deprecated and will be removed in the future. vars.StmtCtx.AppendWarning(ErrWarnDeprecatedSyntaxSimpleMsg.FastGen("The 'tidb_enable_column_tracking' variable is deprecated and will be removed in future versions of TiDB. It is always set to 'ON' now.")) return normalizedValue, nil @@ -1240,14 +1241,14 @@ var defaultSysVars = []*SysVar{ SetGlobal: func(_ context.Context, s *SessionVars, val string) error { return nil }}, - {Scope: ScopeGlobal, Name: RequireSecureTransport, Value: BoolToOnOff(DefRequireSecureTransport), Type: TypeBool, + {Scope: vardef.ScopeGlobal, Name: vardef.RequireSecureTransport, Value: BoolToOnOff(vardef.DefRequireSecureTransport), Type: vardef.TypeBool, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return BoolToOnOff(tls.RequireSecureTransport.Load()), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { tls.RequireSecureTransport.Store(TiDBOptOn(val)) return nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { if vars.StmtCtx.StmtType == "Set" && TiDBOptOn(normalizedValue) { // On tidbcloud dedicated cluster with the default configuration, if an user modify // @@global.require_secure_transport=on, he can not login the cluster anymore! @@ -1266,210 +1267,210 @@ var defaultSysVars = []*SysVar{ return normalizedValue, nil }, }, - {Scope: ScopeGlobal, Name: TiDBStatsLoadPseudoTimeout, Value: BoolToOnOff(DefTiDBStatsLoadPseudoTimeout), Type: TypeBool, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBStatsLoadPseudoTimeout, Value: BoolToOnOff(vardef.DefTiDBStatsLoadPseudoTimeout), Type: vardef.TypeBool, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(StatsLoadPseudoTimeout.Load()), nil + return BoolToOnOff(vardef.StatsLoadPseudoTimeout.Load()), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - StatsLoadPseudoTimeout.Store(TiDBOptOn(val)) + vardef.StatsLoadPseudoTimeout.Store(TiDBOptOn(val)) return nil }, }, - {Scope: ScopeGlobal, Name: TiDBEnableBatchDML, Value: BoolToOnOff(DefTiDBEnableBatchDML), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - EnableBatchDML.Store(TiDBOptOn(val)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableBatchDML, Value: BoolToOnOff(vardef.DefTiDBEnableBatchDML), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.EnableBatchDML.Store(TiDBOptOn(val)) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(EnableBatchDML.Load()), nil + return BoolToOnOff(vardef.EnableBatchDML.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBStatsCacheMemQuota, Value: strconv.Itoa(DefTiDBStatsCacheMemQuota), - MinValue: 0, MaxValue: MaxTiDBStatsCacheMemQuota, Type: TypeInt, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBStatsCacheMemQuota, Value: strconv.Itoa(vardef.DefTiDBStatsCacheMemQuota), + MinValue: 0, MaxValue: vardef.MaxTiDBStatsCacheMemQuota, Type: vardef.TypeInt, GetGlobal: func(_ context.Context, vars *SessionVars) (string, error) { - return strconv.FormatInt(StatsCacheMemQuota.Load(), 10), nil + return strconv.FormatInt(vardef.StatsCacheMemQuota.Load(), 10), nil }, SetGlobal: func(_ context.Context, vars *SessionVars, s string) error { - v := TidbOptInt64(s, DefTiDBStatsCacheMemQuota) - oldv := StatsCacheMemQuota.Load() + v := TidbOptInt64(s, vardef.DefTiDBStatsCacheMemQuota) + oldv := vardef.StatsCacheMemQuota.Load() if v != oldv { - StatsCacheMemQuota.Store(v) + vardef.StatsCacheMemQuota.Store(v) SetStatsCacheCapacityFunc := SetStatsCacheCapacity.Load() (*SetStatsCacheCapacityFunc)(v) } return nil }, }, - {Scope: ScopeGlobal, Name: TiDBQueryLogMaxLen, Value: strconv.Itoa(DefTiDBQueryLogMaxLen), Type: TypeInt, MinValue: 0, MaxValue: 1073741824, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - QueryLogMaxLen.Store(int32(TidbOptInt64(val, DefTiDBQueryLogMaxLen))) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBQueryLogMaxLen, Value: strconv.Itoa(vardef.DefTiDBQueryLogMaxLen), Type: vardef.TypeInt, MinValue: 0, MaxValue: 1073741824, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.QueryLogMaxLen.Store(int32(TidbOptInt64(val, vardef.DefTiDBQueryLogMaxLen))) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return fmt.Sprint(QueryLogMaxLen.Load()), nil + return fmt.Sprint(vardef.QueryLogMaxLen.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBCommitterConcurrency, Value: strconv.Itoa(DefTiDBCommitterConcurrency), Type: TypeInt, MinValue: 1, MaxValue: 10000, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - tikvutil.CommitterConcurrency.Store(int32(TidbOptInt64(val, DefTiDBCommitterConcurrency))) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBCommitterConcurrency, Value: strconv.Itoa(vardef.DefTiDBCommitterConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: 10000, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + tikvutil.CommitterConcurrency.Store(int32(TidbOptInt64(val, vardef.DefTiDBCommitterConcurrency))) cfg := config.GetGlobalConfig().GetTiKVConfig() tikvcfg.StoreGlobalConfig(cfg) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return fmt.Sprint(tikvutil.CommitterConcurrency.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBMemQuotaAnalyze, Value: strconv.Itoa(DefTiDBMemQuotaAnalyze), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBMemQuotaAnalyze, Value: strconv.Itoa(vardef.DefTiDBMemQuotaAnalyze), Type: vardef.TypeInt, MinValue: -1, MaxValue: math.MaxInt64, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return strconv.FormatInt(GetMemQuotaAnalyze(), 10), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - SetMemQuotaAnalyze(TidbOptInt64(val, DefTiDBMemQuotaAnalyze)) + SetMemQuotaAnalyze(TidbOptInt64(val, vardef.DefTiDBMemQuotaAnalyze)) return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePrepPlanCache, Value: BoolToOnOff(DefTiDBEnablePrepPlanCache), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnablePrepPlanCache, Value: BoolToOnOff(vardef.DefTiDBEnablePrepPlanCache), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnablePreparedPlanCache = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBPrepPlanCacheSize, Aliases: []string{TiDBSessionPlanCacheSize}, Value: strconv.FormatUint(uint64(DefTiDBPrepPlanCacheSize), 10), Type: TypeUnsigned, MinValue: 1, MaxValue: 100000, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBPrepPlanCacheSize, Aliases: []string{vardef.TiDBSessionPlanCacheSize}, Value: strconv.FormatUint(uint64(vardef.DefTiDBPrepPlanCacheSize), 10), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: 100000, SetSession: func(s *SessionVars, val string) error { uVal, err := strconv.ParseUint(val, 10, 64) if err == nil { s.PreparedPlanCacheSize = uVal } return err - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TiDBPrepPlanCacheSize, TiDBSessionPlanCacheSize) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + appendDeprecationWarning(vars, vardef.TiDBPrepPlanCacheSize, vardef.TiDBSessionPlanCacheSize) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePrepPlanCacheMemoryMonitor, Value: BoolToOnOff(DefTiDBEnablePrepPlanCacheMemoryMonitor), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnablePrepPlanCacheMemoryMonitor, Value: BoolToOnOff(vardef.DefTiDBEnablePrepPlanCacheMemoryMonitor), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnablePreparedPlanCacheMemoryMonitor = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal, Name: TiDBPrepPlanCacheMemoryGuardRatio, Value: strconv.FormatFloat(DefTiDBPrepPlanCacheMemoryGuardRatio, 'f', -1, 64), Type: TypeFloat, MinValue: 0.0, MaxValue: 1.0, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBPrepPlanCacheMemoryGuardRatio, Value: strconv.FormatFloat(vardef.DefTiDBPrepPlanCacheMemoryGuardRatio, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0.0, MaxValue: 1.0, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { f, err := strconv.ParseFloat(val, 64) if err == nil { - PreparedPlanCacheMemoryGuardRatio.Store(f) + vardef.PreparedPlanCacheMemoryGuardRatio.Store(f) } return err }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatFloat(PreparedPlanCacheMemoryGuardRatio.Load(), 'f', -1, 64), nil + return strconv.FormatFloat(vardef.PreparedPlanCacheMemoryGuardRatio.Load(), 'f', -1, 64), nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNonPreparedPlanCache, Value: BoolToOnOff(DefTiDBEnableNonPreparedPlanCache), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableNonPreparedPlanCache, Value: BoolToOnOff(vardef.DefTiDBEnableNonPreparedPlanCache), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableNonPreparedPlanCache = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNonPreparedPlanCacheForDML, Value: BoolToOnOff(DefTiDBEnableNonPreparedPlanCacheForDML), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableNonPreparedPlanCacheForDML, Value: BoolToOnOff(vardef.DefTiDBEnableNonPreparedPlanCacheForDML), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableNonPreparedPlanCacheForDML = TiDBOptOn(val) return nil }}, { - Scope: ScopeGlobal | ScopeSession, - Name: TiDBOptEnableFuzzyBinding, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.TiDBOptEnableFuzzyBinding, Value: BoolToOnOff(false), - Type: TypeBool, + Type: vardef.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 { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBNonPreparedPlanCacheSize, Value: strconv.FormatUint(uint64(vardef.DefTiDBNonPreparedPlanCacheSize), 10), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: 100000, SetSession: func(s *SessionVars, val string) error { uVal, err := strconv.ParseUint(val, 10, 64) if err == nil { s.NonPreparedPlanCacheSize = uVal } return err - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TiDBNonPreparedPlanCacheSize, TiDBSessionPlanCacheSize) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + appendDeprecationWarning(vars, vardef.TiDBNonPreparedPlanCacheSize, vardef.TiDBSessionPlanCacheSize) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBPlanCacheMaxPlanSize, Value: strconv.FormatUint(DefTiDBPlanCacheMaxPlanSize, 10), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBPlanCacheMaxPlanSize, Value: strconv.FormatUint(vardef.DefTiDBPlanCacheMaxPlanSize, 10), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { uVal, err := strconv.ParseUint(val, 10, 64) if err == nil { s.PlanCacheMaxPlanSize = uVal } return err }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSessionPlanCacheSize, Aliases: []string{TiDBPrepPlanCacheSize}, Value: strconv.FormatUint(uint64(DefTiDBSessionPlanCacheSize), 10), Type: TypeUnsigned, MinValue: 1, MaxValue: 100000, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBSessionPlanCacheSize, Aliases: []string{vardef.TiDBPrepPlanCacheSize}, Value: strconv.FormatUint(uint64(vardef.DefTiDBSessionPlanCacheSize), 10), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: 100000, SetSession: func(s *SessionVars, val string) error { uVal, err := strconv.ParseUint(val, 10, 64) if err == nil { s.SessionPlanCacheSize = uVal } return err }}, - {Scope: ScopeGlobal, Name: TiDBEnableInstancePlanCache, Value: Off, Type: TypeBool, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableInstancePlanCache, Value: vardef.Off, Type: vardef.TypeBool, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(EnableInstancePlanCache.Load()), nil + return BoolToOnOff(vardef.EnableInstancePlanCache.Load()), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - EnableInstancePlanCache.Store(TiDBOptOn(val)) + vardef.EnableInstancePlanCache.Store(TiDBOptOn(val)) return nil }}, - {Scope: ScopeGlobal, Name: TiDBInstancePlanCacheReservedPercentage, - Value: strconv.FormatFloat(DefTiDBInstancePlanCacheReservedPercentage, 'f', -1, 64), - Type: TypeFloat, MinValue: 0, MaxValue: 1, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBInstancePlanCacheReservedPercentage, + Value: strconv.FormatFloat(vardef.DefTiDBInstancePlanCacheReservedPercentage, 'f', -1, 64), + Type: vardef.TypeFloat, MinValue: 0, MaxValue: 1, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatFloat(InstancePlanCacheReservedPercentage.Load(), 'f', -1, 64), nil + return strconv.FormatFloat(vardef.InstancePlanCacheReservedPercentage.Load(), 'f', -1, 64), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - v := tidbOptFloat64(val, DefTiDBInstancePlanCacheReservedPercentage) + v := tidbOptFloat64(val, vardef.DefTiDBInstancePlanCacheReservedPercentage) if v < 0 || v > 1 { return errors.Errorf("invalid tidb_instance_plan_cache_reserved_percentage value %s", val) } - InstancePlanCacheReservedPercentage.Store(v) + vardef.InstancePlanCacheReservedPercentage.Store(v) return nil }}, - {Scope: ScopeGlobal, Name: TiDBInstancePlanCacheMaxMemSize, Value: strconv.Itoa(int(DefTiDBInstancePlanCacheMaxMemSize)), Type: TypeStr, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBInstancePlanCacheMaxMemSize, Value: strconv.Itoa(int(vardef.DefTiDBInstancePlanCacheMaxMemSize)), Type: vardef.TypeStr, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatInt(InstancePlanCacheMaxMemSize.Load(), 10), nil + return strconv.FormatInt(vardef.InstancePlanCacheMaxMemSize.Load(), 10), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { v, str := parseByteSize(val) if str == "" || v < 0 { return errors.Errorf("invalid tidb_instance_plan_cache_max_mem_size value %s", val) } - if v < MinTiDBInstancePlanCacheMemSize { + if v < vardef.MinTiDBInstancePlanCacheMemSize { return errors.Errorf("tidb_instance_plan_cache_max_mem_size should be at least 100MiB") } - InstancePlanCacheMaxMemSize.Store(int64(v)) + vardef.InstancePlanCacheMaxMemSize.Store(int64(v)) return nil }}, - {Scope: ScopeGlobal, Name: TiDBMemOOMAction, Value: DefTiDBMemOOMAction, PossibleValues: []string{"CANCEL", "LOG"}, Type: TypeEnum, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBMemOOMAction, Value: vardef.DefTiDBMemOOMAction, PossibleValues: []string{"CANCEL", "LOG"}, Type: vardef.TypeEnum, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return OOMAction.Load(), nil + return vardef.OOMAction.Load(), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - OOMAction.Store(val) + vardef.OOMAction.Store(val) return nil }}, - {Scope: ScopeGlobal, Name: TiDBMaxAutoAnalyzeTime, Value: strconv.Itoa(DefTiDBMaxAutoAnalyzeTime), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBMaxAutoAnalyzeTime, Value: strconv.Itoa(vardef.DefTiDBMaxAutoAnalyzeTime), Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt32, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatInt(MaxAutoAnalyzeTime.Load(), 10), nil + return strconv.FormatInt(vardef.MaxAutoAnalyzeTime.Load(), 10), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { num, err := strconv.ParseInt(val, 10, 64) if err == nil { - MaxAutoAnalyzeTime.Store(num) + vardef.MaxAutoAnalyzeTime.Store(num) } return err }, }, { - Scope: ScopeGlobal, Name: TiDBAutoAnalyzeConcurrency, - Value: strconv.Itoa(DefTiDBAutoAnalyzeConcurrency), - Type: TypeInt, + Scope: vardef.ScopeGlobal, Name: vardef.TiDBAutoAnalyzeConcurrency, + Value: strconv.Itoa(vardef.DefTiDBAutoAnalyzeConcurrency), + Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt32, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatInt(int64(AutoAnalyzeConcurrency.Load()), 10), nil + return strconv.FormatInt(int64(vardef.AutoAnalyzeConcurrency.Load()), 10), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { num, err := strconv.ParseInt(val, 10, 64) if err == nil { - AutoAnalyzeConcurrency.Store(int32(num)) + vardef.AutoAnalyzeConcurrency.Store(int32(num)) } return err }, - Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { // Check if auto-analyze and auto-analyze priority queue are enabled - enableAutoAnalyze := RunAutoAnalyze.Load() - enableAutoAnalyzePriorityQueue := EnableAutoAnalyzePriorityQueue.Load() + enableAutoAnalyze := vardef.RunAutoAnalyze.Load() + enableAutoAnalyzePriorityQueue := vardef.EnableAutoAnalyzePriorityQueue.Load() // Validate that both required settings are enabled if !enableAutoAnalyze || !enableAutoAnalyzePriorityQueue { return originalValue, errors.Errorf( "cannot set %s: requires both tidb_enable_auto_analyze and tidb_enable_auto_analyze_priority_queue to be true. Current values: tidb_enable_auto_analyze=%v, tidb_enable_auto_analyze_priority_queue=%v", - TiDBAutoAnalyzeConcurrency, + vardef.TiDBAutoAnalyzeConcurrency, enableAutoAnalyze, enableAutoAnalyzePriorityQueue, ) @@ -1478,8 +1479,8 @@ var defaultSysVars = []*SysVar{ return normalizedValue, nil }, }, - {Scope: ScopeGlobal, Name: TiDBEnableMDL, Value: BoolToOnOff(DefTiDBEnableMDL), Type: TypeBool, SetGlobal: func(_ context.Context, vars *SessionVars, val string) error { - if EnableMDL.Load() != TiDBOptOn(val) { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableMDL, Value: BoolToOnOff(vardef.DefTiDBEnableMDL), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, vars *SessionVars, val string) error { + if vardef.EnableMDL.Load() != TiDBOptOn(val) { err := SwitchMDL(TiDBOptOn(val)) if err != nil { return err @@ -1487,93 +1488,93 @@ var defaultSysVars = []*SysVar{ } return nil }, GetGlobal: func(_ context.Context, vars *SessionVars) (string, error) { - return BoolToOnOff(EnableMDL.Load()), nil + return BoolToOnOff(vardef.EnableMDL.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBEnableDistTask, Value: BoolToOnOff(DefTiDBEnableDistTask), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - if EnableDistTask.Load() != TiDBOptOn(val) { - EnableDistTask.Store(TiDBOptOn(val)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableDistTask, Value: BoolToOnOff(vardef.DefTiDBEnableDistTask), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + if vardef.EnableDistTask.Load() != TiDBOptOn(val) { + vardef.EnableDistTask.Store(TiDBOptOn(val)) } return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(EnableDistTask.Load()), nil + return BoolToOnOff(vardef.EnableDistTask.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBEnableFastCreateTable, Value: BoolToOnOff(DefTiDBEnableFastCreateTable), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - if EnableFastCreateTable.Load() != TiDBOptOn(val) { - EnableFastCreateTable.Store(TiDBOptOn(val)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableFastCreateTable, Value: BoolToOnOff(vardef.DefTiDBEnableFastCreateTable), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + if vardef.EnableFastCreateTable.Load() != TiDBOptOn(val) { + vardef.EnableFastCreateTable.Store(TiDBOptOn(val)) } return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(EnableFastCreateTable.Load()), nil + return BoolToOnOff(vardef.EnableFastCreateTable.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBEnableNoopVariables, Value: BoolToOnOff(DefTiDBEnableNoopVariables), Type: TypeEnum, PossibleValues: []string{Off, On}, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - EnableNoopVariables.Store(TiDBOptOn(val)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableNoopVariables, Value: BoolToOnOff(vardef.DefTiDBEnableNoopVariables), Type: vardef.TypeEnum, PossibleValues: []string{vardef.Off, vardef.On}, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.EnableNoopVariables.Store(TiDBOptOn(val)) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(EnableNoopVariables.Load()), nil + return BoolToOnOff(vardef.EnableNoopVariables.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBEnableGCAwareMemoryTrack, Value: BoolToOnOff(DefEnableTiDBGCAwareMemoryTrack), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableGCAwareMemoryTrack, Value: BoolToOnOff(vardef.DefEnableTiDBGCAwareMemoryTrack), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { memory.EnableGCAwareMemoryTrack.Store(TiDBOptOn(val)) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return BoolToOnOff(memory.EnableGCAwareMemoryTrack.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBEnableTmpStorageOnOOM, Value: BoolToOnOff(DefTiDBEnableTmpStorageOnOOM), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - EnableTmpStorageOnOOM.Store(TiDBOptOn(val)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableTmpStorageOnOOM, Value: BoolToOnOff(vardef.DefTiDBEnableTmpStorageOnOOM), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.EnableTmpStorageOnOOM.Store(TiDBOptOn(val)) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(EnableTmpStorageOnOOM.Load()), nil + return BoolToOnOff(vardef.EnableTmpStorageOnOOM.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBAutoBuildStatsConcurrency, Value: strconv.Itoa(DefTiDBAutoBuildStatsConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency}, - {Scope: ScopeGlobal, Name: TiDBSysProcScanConcurrency, Value: strconv.Itoa(DefTiDBSysProcScanConcurrency), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32}, - {Scope: ScopeGlobal, Name: TiDBMemoryUsageAlarmRatio, Value: strconv.FormatFloat(DefMemoryUsageAlarmRatio, 'f', -1, 64), Type: TypeFloat, MinValue: 0.0, MaxValue: 1.0, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - MemoryUsageAlarmRatio.Store(tidbOptFloat64(val, DefMemoryUsageAlarmRatio)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBAutoBuildStatsConcurrency, Value: strconv.Itoa(vardef.DefTiDBAutoBuildStatsConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency}, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBSysProcScanConcurrency, Value: strconv.Itoa(vardef.DefTiDBSysProcScanConcurrency), Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt32}, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBMemoryUsageAlarmRatio, Value: strconv.FormatFloat(vardef.DefMemoryUsageAlarmRatio, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0.0, MaxValue: 1.0, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.MemoryUsageAlarmRatio.Store(tidbOptFloat64(val, vardef.DefMemoryUsageAlarmRatio)) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return fmt.Sprintf("%g", MemoryUsageAlarmRatio.Load()), nil + return fmt.Sprintf("%g", vardef.MemoryUsageAlarmRatio.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBMemoryUsageAlarmKeepRecordNum, Value: strconv.Itoa(DefMemoryUsageAlarmKeepRecordNum), Type: TypeInt, MinValue: 1, MaxValue: 10000, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - MemoryUsageAlarmKeepRecordNum.Store(TidbOptInt64(val, DefMemoryUsageAlarmKeepRecordNum)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBMemoryUsageAlarmKeepRecordNum, Value: strconv.Itoa(vardef.DefMemoryUsageAlarmKeepRecordNum), Type: vardef.TypeInt, MinValue: 1, MaxValue: 10000, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.MemoryUsageAlarmKeepRecordNum.Store(TidbOptInt64(val, vardef.DefMemoryUsageAlarmKeepRecordNum)) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatInt(MemoryUsageAlarmKeepRecordNum.Load(), 10), nil + return strconv.FormatInt(vardef.MemoryUsageAlarmKeepRecordNum.Load(), 10), nil }}, - {Scope: ScopeGlobal, Name: PasswordReuseHistory, Value: strconv.Itoa(DefPasswordReuseHistory), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint32, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatInt(PasswordHistory.Load(), 10), nil + {Scope: vardef.ScopeGlobal, Name: vardef.PasswordReuseHistory, Value: strconv.Itoa(vardef.DefPasswordReuseHistory), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint32, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + return strconv.FormatInt(vardef.PasswordHistory.Load(), 10), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - PasswordHistory.Store(TidbOptInt64(val, DefPasswordReuseHistory)) + vardef.PasswordHistory.Store(TidbOptInt64(val, vardef.DefPasswordReuseHistory)) return nil }}, - {Scope: ScopeGlobal, Name: PasswordReuseTime, Value: strconv.Itoa(DefPasswordReuseTime), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint32, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatInt(PasswordReuseInterval.Load(), 10), nil + {Scope: vardef.ScopeGlobal, Name: vardef.PasswordReuseTime, Value: strconv.Itoa(vardef.DefPasswordReuseTime), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint32, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { + return strconv.FormatInt(vardef.PasswordReuseInterval.Load(), 10), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - PasswordReuseInterval.Store(TidbOptInt64(val, DefPasswordReuseTime)) + vardef.PasswordReuseInterval.Store(TidbOptInt64(val, vardef.DefPasswordReuseTime)) return nil }}, - {Scope: ScopeGlobal, Name: TiDBEnableHistoricalStatsForCapture, Value: BoolToOnOff(DefTiDBEnableHistoricalStatsForCapture), Type: TypeBool, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableHistoricalStatsForCapture, Value: BoolToOnOff(vardef.DefTiDBEnableHistoricalStatsForCapture), Type: vardef.TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { - EnableHistoricalStatsForCapture.Store(TiDBOptOn(s)) + vardef.EnableHistoricalStatsForCapture.Store(TiDBOptOn(s)) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return BoolToOnOff(EnableHistoricalStatsForCapture.Load()), nil + return BoolToOnOff(vardef.EnableHistoricalStatsForCapture.Load()), nil }, }, - {Scope: ScopeGlobal, Name: TiDBHistoricalStatsDuration, Value: DefTiDBHistoricalStatsDuration.String(), Type: TypeDuration, MinValue: int64(time.Second), MaxValue: uint64(time.Hour * 24 * 365), + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBHistoricalStatsDuration, Value: vardef.DefTiDBHistoricalStatsDuration.String(), Type: vardef.TypeDuration, MinValue: int64(time.Second), MaxValue: uint64(time.Hour * 24 * 365), GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return HistoricalStatsDuration.Load().String(), nil + return vardef.HistoricalStatsDuration.Load().String(), nil }, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { d, err := time.ParseDuration(s) if err != nil { return err } - HistoricalStatsDuration.Store(d) + vardef.HistoricalStatsDuration.Store(d) return nil }}, - {Scope: ScopeGlobal, Name: TiDBLowResolutionTSOUpdateInterval, Value: strconv.Itoa(DefTiDBLowResolutionTSOUpdateInterval), Type: TypeInt, MinValue: 10, MaxValue: 60000, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBLowResolutionTSOUpdateInterval, Value: strconv.Itoa(vardef.DefTiDBLowResolutionTSOUpdateInterval), Type: vardef.TypeInt, MinValue: 10, MaxValue: 60000, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - LowResolutionTSOUpdateInterval.Store(uint32(TidbOptInt64(val, DefTiDBLowResolutionTSOUpdateInterval))) + vardef.LowResolutionTSOUpdateInterval.Store(uint32(TidbOptInt64(val, vardef.DefTiDBLowResolutionTSOUpdateInterval))) if SetLowResolutionTSOUpdateInterval != nil { - interval := time.Duration(LowResolutionTSOUpdateInterval.Load()) * time.Millisecond + interval := time.Duration(vardef.LowResolutionTSOUpdateInterval.Load()) * time.Millisecond return SetLowResolutionTSOUpdateInterval(interval) } return nil @@ -1581,14 +1582,14 @@ var defaultSysVars = []*SysVar{ }, /* The system variables below have GLOBAL and SESSION scope */ - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePlanReplayerContinuousCapture, Value: BoolToOnOff(false), Type: TypeBool, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnablePlanReplayerContinuousCapture, Value: BoolToOnOff(false), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { - historicalStatsEnabled, err := s.GlobalVarsAccessor.GetGlobalSysVar(TiDBEnableHistoricalStats) + historicalStatsEnabled, err := s.GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBEnableHistoricalStats) if err != nil { return err } if !TiDBOptOn(historicalStatsEnabled) && TiDBOptOn(val) { - return errors.Errorf("%v should be enabled before enabling %v", TiDBEnableHistoricalStats, TiDBEnablePlanReplayerContinuousCapture) + return errors.Errorf("%v should be enabled before enabling %v", vardef.TiDBEnableHistoricalStats, vardef.TiDBEnablePlanReplayerContinuousCapture) } s.EnablePlanReplayedContinuesCapture = TiDBOptOn(val) return nil @@ -1596,18 +1597,18 @@ var defaultSysVars = []*SysVar{ GetSession: func(vars *SessionVars) (string, error) { return BoolToOnOff(vars.EnablePlanReplayedContinuesCapture), nil }, - Validation: func(vars *SessionVars, s string, s2 string, flag ScopeFlag) (string, error) { - historicalStatsEnabled, err := vars.GlobalVarsAccessor.GetGlobalSysVar(TiDBEnableHistoricalStats) + Validation: func(vars *SessionVars, s string, s2 string, flag vardef.ScopeFlag) (string, error) { + historicalStatsEnabled, err := vars.GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBEnableHistoricalStats) if err != nil { return "", err } if !TiDBOptOn(historicalStatsEnabled) && TiDBOptOn(s) { - return "", errors.Errorf("%v should be enabled before enabling %v", TiDBEnableHistoricalStats, TiDBEnablePlanReplayerContinuousCapture) + return "", errors.Errorf("%v should be enabled before enabling %v", vardef.TiDBEnableHistoricalStats, vardef.TiDBEnablePlanReplayerContinuousCapture) } return s, nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePlanReplayerCapture, Value: BoolToOnOff(DefTiDBEnablePlanReplayerCapture), Type: TypeBool, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnablePlanReplayerCapture, Value: BoolToOnOff(vardef.DefTiDBEnablePlanReplayerCapture), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnablePlanReplayerCapture = TiDBOptOn(val) return nil @@ -1616,28 +1617,28 @@ var defaultSysVars = []*SysVar{ return BoolToOnOff(vars.EnablePlanReplayerCapture), nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBRowFormatVersion, Value: strconv.Itoa(DefTiDBRowFormatV1), Type: TypeUnsigned, MinValue: 1, MaxValue: 2, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - SetDDLReorgRowFormat(TidbOptInt64(val, DefTiDBRowFormatV2)) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBRowFormatVersion, Value: strconv.Itoa(vardef.DefTiDBRowFormatV1), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: 2, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.SetDDLReorgRowFormat(TidbOptInt64(val, vardef.DefTiDBRowFormatV2)) return nil }, SetSession: func(s *SessionVars, val string) error { - formatVersion := TidbOptInt64(val, DefTiDBRowFormatV1) - if formatVersion == DefTiDBRowFormatV1 { + formatVersion := TidbOptInt64(val, vardef.DefTiDBRowFormatV1) + if formatVersion == vardef.DefTiDBRowFormatV1 { s.RowEncoder.Enable = false - } else if formatVersion == DefTiDBRowFormatV2 { + } else if formatVersion == vardef.DefTiDBRowFormatV2 { s.RowEncoder.Enable = true } return nil }}, - {Scope: ScopeGlobal, Name: TiDBEnableRowLevelChecksum, Value: BoolToOnOff(DefTiDBEnableRowLevelChecksum), Type: TypeBool, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableRowLevelChecksum, Value: BoolToOnOff(vardef.DefTiDBEnableRowLevelChecksum), Type: vardef.TypeBool, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return BoolToOnOff(EnableRowLevelChecksum.Load()), nil + return BoolToOnOff(vardef.EnableRowLevelChecksum.Load()), nil }, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { - EnableRowLevelChecksum.Store(TiDBOptOn(s)) + vardef.EnableRowLevelChecksum.Store(TiDBOptOn(s)) return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: SQLSelectLimit, Value: "18446744073709551615", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.SQLSelectLimit, Value: "18446744073709551615", Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { result, err := strconv.ParseUint(val, 10, 64) if err != nil { return errors.Trace(err) @@ -1645,14 +1646,14 @@ var defaultSysVars = []*SysVar{ s.SelectLimit = result return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: DefaultWeekFormat, Value: DefDefaultWeekFormat, Type: TypeUnsigned, MinValue: 0, MaxValue: 7}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.DefaultWeekFormat, Value: vardef.DefDefaultWeekFormat, Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: 7}, { - Scope: ScopeGlobal | ScopeSession, - Name: SQLModeVar, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.SQLModeVar, Value: mysql.DefaultSQLMode, IsHintUpdatableVerified: true, Validation: func( - vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag, + vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag, ) (string, error) { // Ensure the SQL mode parses normalizedValue = mysql.FormatSQLModeStr(normalizedValue) @@ -1672,10 +1673,10 @@ var defaultSysVars = []*SysVar{ return nil }}, { - Scope: ScopeGlobal, - Name: TiDBLoadBindingTimeout, + Scope: vardef.ScopeGlobal, + Name: vardef.TiDBLoadBindingTimeout, Value: "200", - Type: TypeUnsigned, + Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, IsHintUpdatableVerified: false, @@ -1685,10 +1686,10 @@ var defaultSysVars = []*SysVar{ return nil }}, { - Scope: ScopeGlobal | ScopeSession, - Name: MaxExecutionTime, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.MaxExecutionTime, Value: "0", - Type: TypeUnsigned, + Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, IsHintUpdatableVerified: true, @@ -1698,10 +1699,10 @@ var defaultSysVars = []*SysVar{ return nil }}, { - Scope: ScopeGlobal | ScopeSession, - Name: TiKVClientReadTimeout, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.TiKVClientReadTimeout, Value: "0", - Type: TypeUnsigned, + Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, IsHintUpdatableVerified: true, @@ -1710,18 +1711,18 @@ var defaultSysVars = []*SysVar{ 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) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.CollationServer, Value: mysql.DefaultCollationName, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { return checkCollation(vars, normalizedValue, originalValue, scope) }, SetSession: func(s *SessionVars, val string) error { if coll, err := collate.GetCollationByName(val); err == nil { - s.systems[CharacterSetServer] = coll.CharsetName + s.systems[vardef.CharacterSetServer] = coll.CharsetName } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: DefaultCollationForUTF8MB4, Value: mysql.DefaultCollationName, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.DefaultCollationForUTF8MB4, Value: mysql.DefaultCollationName, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { coll, err := checkDefaultCollationForUTF8MB4(vars, normalizedValue, originalValue, scope) if err == nil { - vars.StmtCtx.AppendWarning(ErrWarnDeprecatedSyntaxNoReplacement.FastGenByArgs(DefaultCollationForUTF8MB4)) + vars.StmtCtx.AppendWarning(ErrWarnDeprecatedSyntaxNoReplacement.FastGenByArgs(vardef.DefaultCollationForUTF8MB4)) } return coll, err }, SetSession: func(s *SessionVars, val string) error { @@ -1729,13 +1730,13 @@ var defaultSysVars = []*SysVar{ return nil }}, { - Scope: ScopeGlobal | ScopeSession, - Name: TimeZone, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.TimeZone, Value: "SYSTEM", IsHintUpdatableVerified: true, Validation: func( varErrFunctionsNoopImpls *SessionVars, normalizedValue string, originalValue string, - scope ScopeFlag, + scope vardef.ScopeFlag, ) (string, error) { if strings.EqualFold(normalizedValue, "SYSTEM") { return "SYSTEM", nil @@ -1750,67 +1751,67 @@ var defaultSysVars = []*SysVar{ 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) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.ForeignKeyChecks, Value: BoolToOnOff(vardef.DefTiDBForeignKeyChecks), Type: vardef.TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { if TiDBOptOn(normalizedValue) { vars.ForeignKeyChecks = true - return On, nil + return vardef.On, nil } else if !TiDBOptOn(normalizedValue) { vars.ForeignKeyChecks = false - return Off, nil + return vardef.Off, nil } - return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(ForeignKeyChecks, originalValue) + return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(vardef.ForeignKeyChecks, originalValue) }}, - {Scope: ScopeGlobal, Name: TiDBEnableForeignKey, Value: BoolToOnOff(true), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - EnableForeignKey.Store(TiDBOptOn(val)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableForeignKey, Value: BoolToOnOff(true), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + vardef.EnableForeignKey.Store(TiDBOptOn(val)) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return BoolToOnOff(EnableForeignKey.Load()), nil + return BoolToOnOff(vardef.EnableForeignKey.Load()), nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: CollationDatabase, Value: mysql.DefaultCollationName, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.CollationDatabase, Value: mysql.DefaultCollationName, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { return checkCollation(vars, normalizedValue, originalValue, scope) }, SetSession: func(s *SessionVars, val string) error { if coll, err := collate.GetCollationByName(val); err == nil { - s.systems[CharsetDatabase] = coll.CharsetName + s.systems[vardef.CharsetDatabase] = coll.CharsetName } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: AutoIncrementIncrement, Value: strconv.FormatInt(DefAutoIncrementIncrement, 10), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint16, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.AutoIncrementIncrement, Value: strconv.FormatInt(vardef.DefAutoIncrementIncrement, 10), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint16, SetSession: func(s *SessionVars, val string) error { // AutoIncrementIncrement is valid in [1, 65535]. - s.AutoIncrementIncrement = tidbOptPositiveInt32(val, DefAutoIncrementIncrement) + s.AutoIncrementIncrement = tidbOptPositiveInt32(val, vardef.DefAutoIncrementIncrement) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: AutoIncrementOffset, Value: strconv.FormatInt(DefAutoIncrementOffset, 10), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint16, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.AutoIncrementOffset, Value: strconv.FormatInt(vardef.DefAutoIncrementOffset, 10), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint16, SetSession: func(s *SessionVars, val string) error { // AutoIncrementOffset is valid in [1, 65535]. - s.AutoIncrementOffset = tidbOptPositiveInt32(val, DefAutoIncrementOffset) + s.AutoIncrementOffset = tidbOptPositiveInt32(val, vardef.DefAutoIncrementOffset) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetClient, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - return checkCharacterSet(normalizedValue, CharacterSetClient) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.CharacterSetClient, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + return checkCharacterSet(normalizedValue, vardef.CharacterSetClient) }}, - {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetResults, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.CharacterSetResults, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { if normalizedValue == "" { return normalizedValue, nil } return checkCharacterSet(normalizedValue, "") }}, - {Scope: ScopeGlobal | ScopeSession, Name: TxnIsolation, Value: "REPEATABLE-READ", Type: TypeEnum, Aliases: []string{TransactionIsolation}, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TxnIsolation, Value: "REPEATABLE-READ", Type: vardef.TypeEnum, Aliases: []string{vardef.TransactionIsolation}, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { // MySQL appends a warning here for tx_isolation is deprecated // TiDB doesn't currently, but may in future. It is still commonly used by applications // So it might be noisy to do so. return checkIsolationLevel(vars, normalizedValue, originalValue, scope) }}, - {Scope: ScopeGlobal | ScopeSession, Name: TransactionIsolation, Value: "REPEATABLE-READ", Type: TypeEnum, Aliases: []string{TxnIsolation}, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TransactionIsolation, Value: "REPEATABLE-READ", Type: vardef.TypeEnum, Aliases: []string{vardef.TxnIsolation}, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { return checkIsolationLevel(vars, normalizedValue, originalValue, scope) }}, - {Scope: ScopeGlobal | ScopeSession, Name: CollationConnection, Value: mysql.DefaultCollationName, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.CollationConnection, Value: mysql.DefaultCollationName, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { return checkCollation(vars, normalizedValue, originalValue, scope) }, SetSession: func(s *SessionVars, val string) error { if coll, err := collate.GetCollationByName(val); err == nil { - s.systems[CharacterSetConnection] = coll.CharsetName + s.systems[vardef.CharacterSetConnection] = coll.CharsetName } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: AutoCommit, Value: On, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.AutoCommit, Value: vardef.On, Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { isAutocommit := TiDBOptOn(val) // Implicitly commit the possible ongoing transaction if mode is changed from off to on. if !s.IsAutocommit() && isAutocommit { @@ -1819,31 +1820,31 @@ var defaultSysVars = []*SysVar{ s.SetStatusFlag(mysql.ServerStatusAutocommit, isAutocommit) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: CharsetDatabase, Value: mysql.DefaultCharset, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - return checkCharacterSet(normalizedValue, CharsetDatabase) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.CharsetDatabase, Value: mysql.DefaultCharset, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + return checkCharacterSet(normalizedValue, vardef.CharsetDatabase) }, SetSession: func(s *SessionVars, val string) error { if cs, err := charset.GetCharsetInfo(val); err == nil { - s.systems[CollationDatabase] = cs.DefaultCollation + s.systems[vardef.CollationDatabase] = cs.DefaultCollation } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: WaitTimeout, Value: strconv.FormatInt(DefWaitTimeout, 10), Type: TypeUnsigned, MinValue: 0, MaxValue: secondsPerYear}, - {Scope: ScopeGlobal | ScopeSession, Name: InteractiveTimeout, Value: "28800", Type: TypeUnsigned, MinValue: 1, MaxValue: secondsPerYear}, - {Scope: ScopeGlobal | ScopeSession, Name: InnodbLockWaitTimeout, Value: strconv.FormatInt(DefInnodbLockWaitTimeout, 10), Type: TypeUnsigned, MinValue: 1, MaxValue: 3600, SetSession: func(s *SessionVars, val string) error { - lockWaitSec := TidbOptInt64(val, DefInnodbLockWaitTimeout) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.WaitTimeout, Value: strconv.FormatInt(vardef.DefWaitTimeout, 10), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: secondsPerYear}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.InteractiveTimeout, Value: "28800", Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: secondsPerYear}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.InnodbLockWaitTimeout, Value: strconv.FormatInt(vardef.DefInnodbLockWaitTimeout, 10), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: 3600, SetSession: func(s *SessionVars, val string) error { + lockWaitSec := TidbOptInt64(val, vardef.DefInnodbLockWaitTimeout) s.LockWaitTimeout = lockWaitSec * 1000 return nil }}, { - Scope: ScopeGlobal | ScopeSession, - Name: GroupConcatMaxLen, - Value: strconv.FormatUint(DefGroupConcatMaxLen, 10), + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.GroupConcatMaxLen, + Value: strconv.FormatUint(vardef.DefGroupConcatMaxLen, 10), IsHintUpdatableVerified: true, - Type: TypeUnsigned, + Type: vardef.TypeUnsigned, MinValue: 4, MaxValue: math.MaxUint64, Validation: func( - vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag, + vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag, ) (string, error) { // https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_group_concat_max_len // Minimum Value 4 @@ -1852,7 +1853,7 @@ var defaultSysVars = []*SysVar{ 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)) + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(vardef.GroupConcatMaxLen, originalValue)) return strconv.FormatInt(int64(math.MaxUint32), 10), nil } } @@ -1870,26 +1871,26 @@ var defaultSysVars = []*SysVar{ return strconv.FormatUint(sv.GroupConcatMaxLen, 10), 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) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.CharacterSetConnection, Value: mysql.DefaultCharset, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + return checkCharacterSet(normalizedValue, vardef.CharacterSetConnection) }, SetSession: func(s *SessionVars, val string) error { if cs, err := charset.GetCharsetInfo(val); err == nil { - s.systems[CollationConnection] = cs.DefaultCollation + s.systems[vardef.CollationConnection] = cs.DefaultCollation } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetServer, Value: mysql.DefaultCharset, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - return checkCharacterSet(normalizedValue, CharacterSetServer) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.CharacterSetServer, Value: mysql.DefaultCharset, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + return checkCharacterSet(normalizedValue, vardef.CharacterSetServer) }, SetSession: func(s *SessionVars, val string) error { if cs, err := charset.GetCharsetInfo(val); err == nil { - s.systems[CollationServer] = cs.DefaultCollation + s.systems[vardef.CollationServer] = cs.DefaultCollation } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: MaxAllowedPacket, Value: strconv.FormatUint(DefMaxAllowedPacket, 10), Type: TypeUnsigned, MinValue: 1024, MaxValue: MaxOfMaxAllowedPacket, - Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if vars.StmtCtx.StmtType == "Set" && scope == ScopeSession { - err := ErrReadOnly.GenWithStackByArgs("SESSION", MaxAllowedPacket, "GLOBAL") + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.MaxAllowedPacket, Value: strconv.FormatUint(vardef.DefMaxAllowedPacket, 10), Type: vardef.TypeUnsigned, MinValue: 1024, MaxValue: vardef.MaxOfMaxAllowedPacket, + Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + if vars.StmtCtx.StmtType == "Set" && scope == vardef.ScopeSession { + err := ErrReadOnly.GenWithStackByArgs("SESSION", vardef.MaxAllowedPacket, "GLOBAL") return normalizedValue, err } // Truncate the value of max_allowed_packet to be a multiple of 1024, @@ -1900,7 +1901,7 @@ var defaultSysVars = []*SysVar{ } remainder := u % 1024 if remainder != 0 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(MaxAllowedPacket, normalizedValue)) + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(vardef.MaxAllowedPacket, normalizedValue)) u -= remainder } return strconv.FormatUint(u, 10), nil @@ -1917,349 +1918,349 @@ var defaultSysVars = []*SysVar{ }, }, { - Scope: ScopeGlobal | ScopeSession, - Name: WindowingUseHighPrecision, - Value: On, - Type: TypeBool, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.WindowingUseHighPrecision, + Value: vardef.On, + Type: vardef.TypeBool, IsHintUpdatableVerified: true, SetSession: func(s *SessionVars, val string) error { s.WindowingUseHighPrecision = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: BlockEncryptionMode, Value: DefBlockEncryptionMode, 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"}}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.BlockEncryptionMode, Value: vardef.DefBlockEncryptionMode, Type: vardef.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 { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBAllowMPPExecution, Type: vardef.TypeBool, Value: BoolToOnOff(vardef.DefTiDBAllowMPPExecution), Depended: true, SetSession: func(s *SessionVars, val string) error { s.allowMPPExecution = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowTiFlashCop, Type: TypeBool, Value: BoolToOnOff(DefTiDBAllowTiFlashCop), SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBAllowTiFlashCop, Type: vardef.TypeBool, Value: BoolToOnOff(vardef.DefTiDBAllowTiFlashCop), SetSession: func(s *SessionVars, val string) error { s.allowTiFlashCop = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiFlashFastScan, Type: TypeBool, Value: BoolToOnOff(DefTiFlashFastScan), SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiFlashFastScan, Type: vardef.TypeBool, Value: BoolToOnOff(vardef.DefTiFlashFastScan), SetSession: func(s *SessionVars, val string) error { s.TiFlashFastScan = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMPPStoreFailTTL, Type: TypeStr, Value: DefTiDBMPPStoreFailTTL, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBMPPStoreFailTTL, Type: vardef.TypeStr, Value: vardef.DefTiDBMPPStoreFailTTL, SetSession: func(s *SessionVars, val string) error { s.MPPStoreFailTTL = val return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashExchangeWithNewCollation, Type: TypeBool, Value: BoolToOnOff(DefTiDBHashExchangeWithNewCollation), SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBHashExchangeWithNewCollation, Type: vardef.TypeBool, Value: BoolToOnOff(vardef.DefTiDBHashExchangeWithNewCollation), SetSession: func(s *SessionVars, val string) error { s.HashExchangeWithNewCollation = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBCJThresholdCount, Value: strconv.Itoa(DefBroadcastJoinThresholdCount), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - s.BroadcastJoinThresholdCount = TidbOptInt64(val, DefBroadcastJoinThresholdCount) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBBCJThresholdCount, Value: strconv.Itoa(vardef.DefBroadcastJoinThresholdCount), Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.BroadcastJoinThresholdCount = TidbOptInt64(val, vardef.DefBroadcastJoinThresholdCount) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBCJThresholdSize, Value: strconv.Itoa(DefBroadcastJoinThresholdSize), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - s.BroadcastJoinThresholdSize = TidbOptInt64(val, DefBroadcastJoinThresholdSize) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBBCJThresholdSize, Value: strconv.Itoa(vardef.DefBroadcastJoinThresholdSize), Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.BroadcastJoinThresholdSize = TidbOptInt64(val, vardef.DefBroadcastJoinThresholdSize) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBPreferBCJByExchangeDataSize, Type: TypeBool, Value: BoolToOnOff(DefPreferBCJByExchangeDataSize), SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBPreferBCJByExchangeDataSize, Type: vardef.TypeBool, Value: BoolToOnOff(vardef.DefPreferBCJByExchangeDataSize), SetSession: func(s *SessionVars, val string) error { s.PreferBCJByExchangeDataSize = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBuildStatsConcurrency, Value: strconv.Itoa(DefBuildStatsConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBuildSamplingStatsConcurrency, Value: strconv.Itoa(DefBuildSamplingStatsConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCartesianBCJ, Value: strconv.Itoa(DefOptCartesianBCJ), Type: TypeInt, MinValue: 0, MaxValue: 2, SetSession: func(s *SessionVars, val string) error { - s.AllowCartesianBCJ = TidbOptInt(val, DefOptCartesianBCJ) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBBuildStatsConcurrency, Value: strconv.Itoa(vardef.DefBuildStatsConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBBuildSamplingStatsConcurrency, Value: strconv.Itoa(vardef.DefBuildSamplingStatsConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptCartesianBCJ, Value: strconv.Itoa(vardef.DefOptCartesianBCJ), Type: vardef.TypeInt, MinValue: 0, MaxValue: 2, SetSession: func(s *SessionVars, val string) error { + s.AllowCartesianBCJ = TidbOptInt(val, vardef.DefOptCartesianBCJ) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptMPPOuterJoinFixedBuildSide, Value: BoolToOnOff(DefOptMPPOuterJoinFixedBuildSide), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptMPPOuterJoinFixedBuildSide, Value: BoolToOnOff(vardef.DefOptMPPOuterJoinFixedBuildSide), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.MPPOuterJoinFixedBuildSide = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBExecutorConcurrency, Value: strconv.Itoa(DefExecutorConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: MaxConfigurableConcurrency, SetSession: func(s *SessionVars, val string) error { - s.ExecutorConcurrency = tidbOptPositiveInt32(val, DefExecutorConcurrency) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBExecutorConcurrency, Value: strconv.Itoa(vardef.DefExecutorConcurrency), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, SetSession: func(s *SessionVars, val string) error { + s.ExecutorConcurrency = tidbOptPositiveInt32(val, vardef.DefExecutorConcurrency) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBDistSQLScanConcurrency, Value: strconv.Itoa(DefDistSQLScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: MaxConfigurableConcurrency, SetSession: func(s *SessionVars, val string) error { - s.distSQLScanConcurrency = tidbOptPositiveInt32(val, DefDistSQLScanConcurrency) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBDistSQLScanConcurrency, Value: strconv.Itoa(vardef.DefDistSQLScanConcurrency), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, SetSession: func(s *SessionVars, val string) error { + s.distSQLScanConcurrency = tidbOptPositiveInt32(val, vardef.DefDistSQLScanConcurrency) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeDistSQLScanConcurrency, Value: strconv.Itoa(DefAnalyzeDistSQLScanConcurrency), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { - s.analyzeDistSQLScanConcurrency = tidbOptPositiveInt32(val, DefAnalyzeDistSQLScanConcurrency) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBAnalyzeDistSQLScanConcurrency, Value: strconv.Itoa(vardef.DefAnalyzeDistSQLScanConcurrency), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + s.analyzeDistSQLScanConcurrency = tidbOptPositiveInt32(val, vardef.DefAnalyzeDistSQLScanConcurrency) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptInSubqToJoinAndAgg, Value: BoolToOnOff(DefOptInSubqToJoinAndAgg), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptInSubqToJoinAndAgg, Value: BoolToOnOff(vardef.DefOptInSubqToJoinAndAgg), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.SetAllowInSubqToJoinAndAgg(TiDBOptOn(val)) return nil }}, { - Scope: ScopeGlobal | ScopeSession, - Name: TiDBOptPreferRangeScan, - Value: BoolToOnOff(DefOptPreferRangeScan), - Type: TypeBool, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.TiDBOptPreferRangeScan, + Value: BoolToOnOff(vardef.DefOptPreferRangeScan), + Type: vardef.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) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptLimitPushDownThreshold, Value: strconv.Itoa(vardef.DefOptLimitPushDownThreshold), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + s.LimitPushDownThreshold = TidbOptInt64(val, vardef.DefOptLimitPushDownThreshold) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCorrelationThreshold, Value: strconv.FormatFloat(DefOptCorrelationThreshold, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: 1, SetSession: func(s *SessionVars, val string) error { - s.CorrelationThreshold = tidbOptFloat64(val, DefOptCorrelationThreshold) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptCorrelationThreshold, Value: strconv.FormatFloat(vardef.DefOptCorrelationThreshold, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0, MaxValue: 1, SetSession: func(s *SessionVars, val string) error { + s.CorrelationThreshold = tidbOptFloat64(val, vardef.DefOptCorrelationThreshold) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptEnableCorrelationAdjustment, Value: BoolToOnOff(DefOptEnableCorrelationAdjustment), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptEnableCorrelationAdjustment, Value: BoolToOnOff(vardef.DefOptEnableCorrelationAdjustment), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableCorrelationAdjustment = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCorrelationExpFactor, Value: strconv.Itoa(DefOptCorrelationExpFactor), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { - s.CorrelationExpFactor = int(TidbOptInt64(val, DefOptCorrelationExpFactor)) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptCorrelationExpFactor, Value: strconv.Itoa(vardef.DefOptCorrelationExpFactor), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + s.CorrelationExpFactor = int(TidbOptInt64(val, vardef.DefOptCorrelationExpFactor)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCPUFactor, Value: strconv.FormatFloat(DefOptCPUFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.cpuFactor = tidbOptFloat64(val, DefOptCPUFactor) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptCPUFactor, Value: strconv.FormatFloat(vardef.DefOptCPUFactor, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.cpuFactor = tidbOptFloat64(val, vardef.DefOptCPUFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTiFlashConcurrencyFactor, Value: strconv.FormatFloat(DefOptTiFlashConcurrencyFactor, 'f', -1, 64), skipInit: true, Type: TypeFloat, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.CopTiFlashConcurrencyFactor = tidbOptFloat64(val, DefOptTiFlashConcurrencyFactor) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptTiFlashConcurrencyFactor, Value: strconv.FormatFloat(vardef.DefOptTiFlashConcurrencyFactor, 'f', -1, 64), skipInit: true, Type: vardef.TypeFloat, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.CopTiFlashConcurrencyFactor = tidbOptFloat64(val, vardef.DefOptTiFlashConcurrencyFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCopCPUFactor, Value: strconv.FormatFloat(DefOptCopCPUFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.copCPUFactor = tidbOptFloat64(val, DefOptCopCPUFactor) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptCopCPUFactor, Value: strconv.FormatFloat(vardef.DefOptCopCPUFactor, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.copCPUFactor = tidbOptFloat64(val, vardef.DefOptCopCPUFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptNetworkFactor, Value: strconv.FormatFloat(DefOptNetworkFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.networkFactor = tidbOptFloat64(val, DefOptNetworkFactor) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptNetworkFactor, Value: strconv.FormatFloat(vardef.DefOptNetworkFactor, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.networkFactor = tidbOptFloat64(val, vardef.DefOptNetworkFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptScanFactor, Value: strconv.FormatFloat(DefOptScanFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.scanFactor = tidbOptFloat64(val, DefOptScanFactor) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptScanFactor, Value: strconv.FormatFloat(vardef.DefOptScanFactor, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.scanFactor = tidbOptFloat64(val, vardef.DefOptScanFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptDescScanFactor, Value: strconv.FormatFloat(DefOptDescScanFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.descScanFactor = tidbOptFloat64(val, DefOptDescScanFactor) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptDescScanFactor, Value: strconv.FormatFloat(vardef.DefOptDescScanFactor, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.descScanFactor = tidbOptFloat64(val, vardef.DefOptDescScanFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptSeekFactor, Value: strconv.FormatFloat(DefOptSeekFactor, 'f', -1, 64), skipInit: true, Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.seekFactor = tidbOptFloat64(val, DefOptSeekFactor) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptSeekFactor, Value: strconv.FormatFloat(vardef.DefOptSeekFactor, 'f', -1, 64), skipInit: true, Type: vardef.TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.seekFactor = tidbOptFloat64(val, vardef.DefOptSeekFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptMemoryFactor, Value: strconv.FormatFloat(DefOptMemoryFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.memoryFactor = tidbOptFloat64(val, DefOptMemoryFactor) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptMemoryFactor, Value: strconv.FormatFloat(vardef.DefOptMemoryFactor, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.memoryFactor = tidbOptFloat64(val, vardef.DefOptMemoryFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptDiskFactor, Value: strconv.FormatFloat(DefOptDiskFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.diskFactor = tidbOptFloat64(val, DefOptDiskFactor) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptDiskFactor, Value: strconv.FormatFloat(vardef.DefOptDiskFactor, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.diskFactor = tidbOptFloat64(val, vardef.DefOptDiskFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptimizerEnableNewOnlyFullGroupByCheck, Value: BoolToOnOff(DefTiDBOptimizerEnableNewOFGB), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptimizerEnableNewOnlyFullGroupByCheck, Value: BoolToOnOff(vardef.DefTiDBOptimizerEnableNewOFGB), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.OptimizerEnableNewOnlyFullGroupByCheck = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptConcurrencyFactor, Value: strconv.FormatFloat(DefOptConcurrencyFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.concurrencyFactor = tidbOptFloat64(val, DefOptConcurrencyFactor) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptConcurrencyFactor, Value: strconv.FormatFloat(vardef.DefOptConcurrencyFactor, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + s.concurrencyFactor = tidbOptFloat64(val, vardef.DefOptConcurrencyFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptForceInlineCTE, Value: BoolToOnOff(DefOptForceInlineCTE), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptForceInlineCTE, Value: BoolToOnOff(vardef.DefOptForceInlineCTE), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.enableForceInlineCTE = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexJoinBatchSize, Value: strconv.Itoa(DefIndexJoinBatchSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { - s.IndexJoinBatchSize = tidbOptPositiveInt32(val, DefIndexJoinBatchSize) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBIndexJoinBatchSize, Value: strconv.Itoa(vardef.DefIndexJoinBatchSize), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + s.IndexJoinBatchSize = tidbOptPositiveInt32(val, vardef.DefIndexJoinBatchSize) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupSize, Value: strconv.Itoa(DefIndexLookupSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { - s.IndexLookupSize = tidbOptPositiveInt32(val, DefIndexLookupSize) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBIndexLookupSize, Value: strconv.Itoa(vardef.DefIndexLookupSize), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + s.IndexLookupSize = tidbOptPositiveInt32(val, vardef.DefIndexLookupSize) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupConcurrency, Value: strconv.Itoa(DefIndexLookupConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { - s.indexLookupConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBIndexLookupConcurrency, Value: strconv.Itoa(vardef.DefIndexLookupConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + s.indexLookupConcurrency = tidbOptPositiveInt32(val, vardef.ConcurrencyUnset) return nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TiDBIndexLookupConcurrency, TiDBExecutorConcurrency) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + appendDeprecationWarning(vars, vardef.TiDBIndexLookupConcurrency, vardef.TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupJoinConcurrency, Value: strconv.Itoa(DefIndexLookupJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { - s.indexLookupJoinConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBIndexLookupJoinConcurrency, Value: strconv.Itoa(vardef.DefIndexLookupJoinConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + s.indexLookupJoinConcurrency = tidbOptPositiveInt32(val, vardef.ConcurrencyUnset) return nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TiDBIndexLookupJoinConcurrency, TiDBExecutorConcurrency) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + appendDeprecationWarning(vars, vardef.TiDBIndexLookupJoinConcurrency, vardef.TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexSerialScanConcurrency, Value: strconv.Itoa(DefIndexSerialScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: MaxConfigurableConcurrency, SetSession: func(s *SessionVars, val string) error { - s.indexSerialScanConcurrency = tidbOptPositiveInt32(val, DefIndexSerialScanConcurrency) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBIndexSerialScanConcurrency, Value: strconv.Itoa(vardef.DefIndexSerialScanConcurrency), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, SetSession: func(s *SessionVars, val string) error { + s.indexSerialScanConcurrency = tidbOptPositiveInt32(val, vardef.DefIndexSerialScanConcurrency) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipUTF8Check, Value: BoolToOnOff(DefSkipUTF8Check), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBSkipUTF8Check, Value: BoolToOnOff(vardef.DefSkipUTF8Check), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.SkipUTF8Check = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipASCIICheck, Value: BoolToOnOff(DefSkipASCIICheck), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBSkipASCIICheck, Value: BoolToOnOff(vardef.DefSkipASCIICheck), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.SkipASCIICheck = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBDMLBatchSize, Value: strconv.Itoa(DefDMLBatchSize), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { - s.DMLBatchSize = int(TidbOptInt64(val, DefDMLBatchSize)) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBDMLBatchSize, Value: strconv.Itoa(vardef.DefDMLBatchSize), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + s.DMLBatchSize = int(TidbOptInt64(val, vardef.DefDMLBatchSize)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMaxChunkSize, Value: strconv.Itoa(DefMaxChunkSize), Type: TypeUnsigned, MinValue: maxChunkSizeLowerBound, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { - s.MaxChunkSize = tidbOptPositiveInt32(val, DefMaxChunkSize) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBMaxChunkSize, Value: strconv.Itoa(vardef.DefMaxChunkSize), Type: vardef.TypeUnsigned, MinValue: maxChunkSizeLowerBound, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + s.MaxChunkSize = tidbOptPositiveInt32(val, vardef.DefMaxChunkSize) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowBatchCop, Value: strconv.Itoa(DefTiDBAllowBatchCop), Type: TypeInt, MinValue: 0, MaxValue: 2, SetSession: func(s *SessionVars, val string) error { - s.AllowBatchCop = int(TidbOptInt64(val, DefTiDBAllowBatchCop)) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBAllowBatchCop, Value: strconv.Itoa(vardef.DefTiDBAllowBatchCop), Type: vardef.TypeInt, MinValue: 0, MaxValue: 2, SetSession: func(s *SessionVars, val string) error { + s.AllowBatchCop = int(TidbOptInt64(val, vardef.DefTiDBAllowBatchCop)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBShardRowIDBits, Value: strconv.Itoa(DefShardRowIDBits), Type: TypeInt, MinValue: 0, MaxValue: MaxShardRowIDBits, SetSession: func(s *SessionVars, val string) error { - s.ShardRowIDBits = TidbOptUint64(val, DefShardRowIDBits) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBShardRowIDBits, Value: strconv.Itoa(vardef.DefShardRowIDBits), Type: vardef.TypeInt, MinValue: 0, MaxValue: vardef.MaxShardRowIDBits, SetSession: func(s *SessionVars, val string) error { + s.ShardRowIDBits = TidbOptUint64(val, vardef.DefShardRowIDBits) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBPreSplitRegions, Value: strconv.Itoa(DefPreSplitRegions), Type: TypeInt, MinValue: 0, MaxValue: MaxPreSplitRegions, SetSession: func(s *SessionVars, val string) error { - s.PreSplitRegions = TidbOptUint64(val, DefPreSplitRegions) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBPreSplitRegions, Value: strconv.Itoa(vardef.DefPreSplitRegions), Type: vardef.TypeInt, MinValue: 0, MaxValue: vardef.MaxPreSplitRegions, SetSession: func(s *SessionVars, val string) error { + s.PreSplitRegions = TidbOptUint64(val, vardef.DefPreSplitRegions) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBInitChunkSize, Value: strconv.Itoa(DefInitChunkSize), Type: TypeUnsigned, MinValue: 1, MaxValue: initChunkSizeUpperBound, SetSession: func(s *SessionVars, val string) error { - s.InitChunkSize = tidbOptPositiveInt32(val, DefInitChunkSize) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBInitChunkSize, Value: strconv.Itoa(vardef.DefInitChunkSize), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: initChunkSizeUpperBound, SetSession: func(s *SessionVars, val string) error { + s.InitChunkSize = tidbOptPositiveInt32(val, vardef.DefInitChunkSize) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableCascadesPlanner, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableCascadesPlanner, Value: vardef.Off, Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.SetEnableCascadesPlanner(TiDBOptOn(val)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableIndexMerge, Value: BoolToOnOff(DefTiDBEnableIndexMerge), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableIndexMerge, Value: BoolToOnOff(vardef.DefTiDBEnableIndexMerge), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.SetEnableIndexMerge(TiDBOptOn(val)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTablePartition, Value: On, Type: TypeEnum, PossibleValues: []string{Off, On, "AUTO"}, Validation: func(vars *SessionVars, s string, s2 string, flag ScopeFlag) (string, error) { - if s == Off { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableTablePartition, Value: vardef.On, Type: vardef.TypeEnum, PossibleValues: []string{vardef.Off, vardef.On, "AUTO"}, Validation: func(vars *SessionVars, s string, s2 string, flag vardef.ScopeFlag) (string, error) { + if s == vardef.Off { vars.StmtCtx.AppendWarning(errors.NewNoStackError("tidb_enable_table_partition is always turned on. This variable has been deprecated and will be removed in the future releases")) } - return On, nil + return vardef.On, nil }}, // Keeping tidb_enable_list_partition here, to give errors if setting it to anything other than ON - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableListTablePartition, Value: On, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue, _ string, _ ScopeFlag) (string, error) { - vars.StmtCtx.AppendWarning(ErrWarnDeprecatedSyntaxSimpleMsg.FastGenByArgs(TiDBEnableListTablePartition)) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableListTablePartition, Value: vardef.On, Type: vardef.TypeBool, Validation: func(vars *SessionVars, normalizedValue, _ string, _ vardef.ScopeFlag) (string, error) { + vars.StmtCtx.AppendWarning(ErrWarnDeprecatedSyntaxSimpleMsg.FastGenByArgs(vardef.TiDBEnableListTablePartition)) if !TiDBOptOn(normalizedValue) { return normalizedValue, errors.Errorf("tidb_enable_list_partition is now always on, and cannot be turned off") } return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashJoinConcurrency, Value: strconv.Itoa(DefTiDBHashJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { - s.hashJoinConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBHashJoinConcurrency, Value: strconv.Itoa(vardef.DefTiDBHashJoinConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + s.hashJoinConcurrency = tidbOptPositiveInt32(val, vardef.ConcurrencyUnset) return nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TiDBHashJoinConcurrency, TiDBExecutorConcurrency) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + appendDeprecationWarning(vars, vardef.TiDBHashJoinConcurrency, vardef.TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBProjectionConcurrency, Value: strconv.Itoa(DefTiDBProjectionConcurrency), Type: TypeInt, MinValue: -1, MaxValue: MaxConfigurableConcurrency, SetSession: func(s *SessionVars, val string) error { - s.projectionConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBProjectionConcurrency, Value: strconv.Itoa(vardef.DefTiDBProjectionConcurrency), Type: vardef.TypeInt, MinValue: -1, MaxValue: vardef.MaxConfigurableConcurrency, SetSession: func(s *SessionVars, val string) error { + s.projectionConcurrency = tidbOptPositiveInt32(val, vardef.ConcurrencyUnset) return nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TiDBProjectionConcurrency, TiDBExecutorConcurrency) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + appendDeprecationWarning(vars, vardef.TiDBProjectionConcurrency, vardef.TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashAggPartialConcurrency, Value: strconv.Itoa(DefTiDBHashAggPartialConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { - s.hashAggPartialConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBHashAggPartialConcurrency, Value: strconv.Itoa(vardef.DefTiDBHashAggPartialConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + s.hashAggPartialConcurrency = tidbOptPositiveInt32(val, vardef.ConcurrencyUnset) return nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TiDBHashAggPartialConcurrency, TiDBExecutorConcurrency) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + appendDeprecationWarning(vars, vardef.TiDBHashAggPartialConcurrency, vardef.TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashAggFinalConcurrency, Value: strconv.Itoa(DefTiDBHashAggFinalConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { - s.hashAggFinalConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBHashAggFinalConcurrency, Value: strconv.Itoa(vardef.DefTiDBHashAggFinalConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + s.hashAggFinalConcurrency = tidbOptPositiveInt32(val, vardef.ConcurrencyUnset) return nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TiDBHashAggFinalConcurrency, TiDBExecutorConcurrency) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + appendDeprecationWarning(vars, vardef.TiDBHashAggFinalConcurrency, vardef.TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBWindowConcurrency, Value: strconv.Itoa(DefTiDBWindowConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { - s.windowConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBWindowConcurrency, Value: strconv.Itoa(vardef.DefTiDBWindowConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + s.windowConcurrency = tidbOptPositiveInt32(val, vardef.ConcurrencyUnset) return nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TiDBWindowConcurrency, TiDBExecutorConcurrency) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + appendDeprecationWarning(vars, vardef.TiDBWindowConcurrency, vardef.TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMergeJoinConcurrency, Value: strconv.Itoa(DefTiDBMergeJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { - s.mergeJoinConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBMergeJoinConcurrency, Value: strconv.Itoa(vardef.DefTiDBMergeJoinConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + s.mergeJoinConcurrency = tidbOptPositiveInt32(val, vardef.ConcurrencyUnset) return nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TiDBMergeJoinConcurrency, TiDBExecutorConcurrency) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + appendDeprecationWarning(vars, vardef.TiDBMergeJoinConcurrency, vardef.TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStreamAggConcurrency, Value: strconv.Itoa(DefTiDBStreamAggConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { - s.streamAggConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBStreamAggConcurrency, Value: strconv.Itoa(vardef.DefTiDBStreamAggConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + s.streamAggConcurrency = tidbOptPositiveInt32(val, vardef.ConcurrencyUnset) return nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TiDBStreamAggConcurrency, TiDBExecutorConcurrency) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + appendDeprecationWarning(vars, vardef.TiDBStreamAggConcurrency, vardef.TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexMergeIntersectionConcurrency, Value: strconv.Itoa(DefTiDBIndexMergeIntersectionConcurrency), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { - s.indexMergeIntersectionConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBIndexMergeIntersectionConcurrency, Value: strconv.Itoa(vardef.DefTiDBIndexMergeIntersectionConcurrency), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + s.indexMergeIntersectionConcurrency = tidbOptPositiveInt32(val, vardef.ConcurrencyUnset) return nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TiDBIndexMergeIntersectionConcurrency, TiDBExecutorConcurrency) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + appendDeprecationWarning(vars, vardef.TiDBIndexMergeIntersectionConcurrency, vardef.TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableParallelApply, Value: BoolToOnOff(DefTiDBEnableParallelApply), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableParallelApply, Value: BoolToOnOff(vardef.DefTiDBEnableParallelApply), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableParallelApply = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMemQuotaApplyCache, Value: strconv.Itoa(DefTiDBMemQuotaApplyCache), Type: TypeUnsigned, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - s.MemQuotaApplyCache = TidbOptInt64(val, DefTiDBMemQuotaApplyCache) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBMemQuotaApplyCache, Value: strconv.Itoa(vardef.DefTiDBMemQuotaApplyCache), Type: vardef.TypeUnsigned, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.MemQuotaApplyCache = TidbOptInt64(val, vardef.DefTiDBMemQuotaApplyCache) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackoffLockFast, Value: strconv.Itoa(tikvstore.DefBackoffLockFast), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBBackoffLockFast, Value: strconv.Itoa(tikvstore.DefBackoffLockFast), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.KVVars.BackoffLockFast = tidbOptPositiveInt32(val, tikvstore.DefBackoffLockFast) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackOffWeight, Value: strconv.Itoa(tikvstore.DefBackOffWeight), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBBackOffWeight, Value: strconv.Itoa(tikvstore.DefBackOffWeight), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.KVVars.BackOffWeight = tidbOptPositiveInt32(val, tikvstore.DefBackOffWeight) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBTxnEntrySizeLimit, Value: strconv.Itoa(DefTiDBTxnEntrySizeLimit), Type: TypeUnsigned, MinValue: 0, MaxValue: config.MaxTxnEntrySizeLimit, SetSession: func(s *SessionVars, val string) error { - s.TxnEntrySizeLimit = TidbOptUint64(val, DefTiDBTxnEntrySizeLimit) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBTxnEntrySizeLimit, Value: strconv.Itoa(vardef.DefTiDBTxnEntrySizeLimit), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: config.MaxTxnEntrySizeLimit, SetSession: func(s *SessionVars, val string) error { + s.TxnEntrySizeLimit = TidbOptUint64(val, vardef.DefTiDBTxnEntrySizeLimit) return nil }, SetGlobal: func(ctx context.Context, s *SessionVars, val string) error { - TxnEntrySizeLimit.Store(TidbOptUint64(val, DefTiDBTxnEntrySizeLimit)) + vardef.TxnEntrySizeLimit.Store(TidbOptUint64(val, vardef.DefTiDBTxnEntrySizeLimit)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBRetryLimit, Value: strconv.Itoa(DefTiDBRetryLimit), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - s.RetryLimit = TidbOptInt64(val, DefTiDBRetryLimit) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBRetryLimit, Value: strconv.Itoa(vardef.DefTiDBRetryLimit), Type: vardef.TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.RetryLimit = TidbOptInt64(val, vardef.DefTiDBRetryLimit) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBDisableTxnAutoRetry, Value: BoolToOnOff(DefTiDBDisableTxnAutoRetry), Type: TypeBool, - Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if normalizedValue == Off { - vars.StmtCtx.AppendWarning(errWarnDeprecatedSyntax.FastGenByArgs(Off, On)) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBDisableTxnAutoRetry, Value: BoolToOnOff(vardef.DefTiDBDisableTxnAutoRetry), Type: vardef.TypeBool, + Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + if normalizedValue == vardef.Off { + vars.StmtCtx.AppendWarning(errWarnDeprecatedSyntax.FastGenByArgs(vardef.Off, vardef.On)) } - return On, nil + return vardef.On, nil }, SetSession: func(s *SessionVars, val string) error { s.DisableTxnAutoRetry = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBConstraintCheckInPlace, Value: BoolToOnOff(DefTiDBConstraintCheckInPlace), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBConstraintCheckInPlace, Value: BoolToOnOff(vardef.DefTiDBConstraintCheckInPlace), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.ConstraintCheckInPlace = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBTxnMode, Value: DefTiDBTxnMode, AllowEmptyAll: true, Type: TypeEnum, PossibleValues: []string{PessimisticTxnMode, OptimisticTxnMode}, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBTxnMode, Value: vardef.DefTiDBTxnMode, AllowEmptyAll: true, Type: vardef.TypeEnum, PossibleValues: []string{vardef.PessimisticTxnMode, vardef.OptimisticTxnMode}, SetSession: func(s *SessionVars, val string) error { s.TxnMode = strings.ToUpper(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableWindowFunction, Value: BoolToOnOff(DefEnableWindowFunction), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableWindowFunction, Value: BoolToOnOff(vardef.DefEnableWindowFunction), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableWindowFunction = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePipelinedWindowFunction, Value: BoolToOnOff(DefEnablePipelinedWindowFunction), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnablePipelinedWindowFunction, Value: BoolToOnOff(vardef.DefEnablePipelinedWindowFunction), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnablePipelinedWindowExec = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableStrictDoubleTypeCheck, Value: BoolToOnOff(DefEnableStrictDoubleTypeCheck), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableStrictDoubleTypeCheck, Value: BoolToOnOff(vardef.DefEnableStrictDoubleTypeCheck), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableStrictDoubleTypeCheck = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableVectorizedExpression, Value: BoolToOnOff(DefEnableVectorizedExpression), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableVectorizedExpression, Value: BoolToOnOff(vardef.DefEnableVectorizedExpression), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableVectorizedExpression = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableFastAnalyze, Value: BoolToOnOff(DefTiDBUseFastAnalyze), Type: TypeBool, - Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableFastAnalyze, Value: BoolToOnOff(vardef.DefTiDBUseFastAnalyze), Type: vardef.TypeBool, + Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { if TiDBOptOn(normalizedValue) { vars.StmtCtx.AppendWarning(errors.NewNoStackError("the fast analyze feature has already been removed in TiDB v7.5.0, so this will have no effect")) } @@ -2269,12 +2270,12 @@ var defaultSysVars = []*SysVar{ s.EnableFastAnalyze = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipIsolationLevelCheck, Value: BoolToOnOff(DefTiDBSkipIsolationLevelCheck), Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableRateLimitAction, Value: BoolToOnOff(DefTiDBEnableRateLimitAction), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBSkipIsolationLevelCheck, Value: BoolToOnOff(vardef.DefTiDBSkipIsolationLevelCheck), Type: vardef.TypeBool}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableRateLimitAction, Value: BoolToOnOff(vardef.DefTiDBEnableRateLimitAction), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnabledRateLimitAction = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowFallbackToTiKV, Value: "", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBAllowFallbackToTiKV, Value: "", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { if normalizedValue == "" { return "", nil } @@ -2293,7 +2294,7 @@ var defaultSysVars = []*SysVar{ storeTypes[kv.TiFlash] = struct{}{} } default: - return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(TiDBAllowFallbackToTiKV, normalizedValue) + return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(vardef.TiDBAllowFallbackToTiKV, normalizedValue) } } return formatVal, nil @@ -2306,26 +2307,26 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableAutoIncrementInGenerated, Value: BoolToOnOff(DefTiDBEnableAutoIncrementInGenerated), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableAutoIncrementInGenerated, Value: BoolToOnOff(vardef.DefTiDBEnableAutoIncrementInGenerated), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableAutoIncrementInGenerated = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBPlacementMode, Value: DefTiDBPlacementMode, Type: TypeEnum, PossibleValues: []string{PlacementModeStrict, PlacementModeIgnore}, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBPlacementMode, Value: vardef.DefTiDBPlacementMode, Type: vardef.TypeEnum, PossibleValues: []string{vardef.PlacementModeStrict, vardef.PlacementModeIgnore}, SetSession: func(s *SessionVars, val string) error { s.PlacementMode = val return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptJoinReorderThreshold, Value: strconv.Itoa(DefTiDBOptJoinReorderThreshold), Type: TypeUnsigned, MinValue: 0, MaxValue: 63, SetSession: func(s *SessionVars, val string) error { - s.TiDBOptJoinReorderThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinReorderThreshold) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptJoinReorderThreshold, Value: strconv.Itoa(vardef.DefTiDBOptJoinReorderThreshold), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: 63, SetSession: func(s *SessionVars, val string) error { + s.TiDBOptJoinReorderThreshold = tidbOptPositiveInt32(val, vardef.DefTiDBOptJoinReorderThreshold) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNoopFuncs, Value: DefTiDBEnableNoopFuncs, Type: TypeEnum, PossibleValues: []string{Off, On, Warn}, Depended: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableNoopFuncs, Value: vardef.DefTiDBEnableNoopFuncs, Type: vardef.TypeEnum, PossibleValues: []string{vardef.Off, vardef.On, vardef.Warn}, Depended: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { // The behavior is very weird if someone can turn TiDBEnableNoopFuncs OFF, but keep any of the following on: // TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, serverReadOnly, SQLAutoIsNull // To prevent this strange position, prevent setting to OFF when any of these sysVars are ON of the same scope. - if normalizedValue == Off { - for _, potentialIncompatibleSysVar := range []string{TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, ReadOnly, SQLAutoIsNull} { + if normalizedValue == vardef.Off { + for _, potentialIncompatibleSysVar := range []string{vardef.TxReadOnly, vardef.TransactionReadOnly, vardef.OfflineMode, vardef.SuperReadOnly, vardef.ReadOnly, vardef.SQLAutoIsNull} { val, _ := vars.GetSystemVar(potentialIncompatibleSysVar) // session scope - if scope == ScopeGlobal { // global scope + if scope == vardef.ScopeGlobal { // global scope var err error val, err = vars.GlobalVarsAccessor.GetGlobalSysVar(potentialIncompatibleSysVar) if err != nil { @@ -2333,7 +2334,7 @@ var defaultSysVars = []*SysVar{ } } if TiDBOptOn(val) { - return originalValue, errValueNotSupportedWhen.GenWithStackByArgs(TiDBEnableNoopFuncs, potentialIncompatibleSysVar) + return originalValue, errValueNotSupportedWhen.GenWithStackByArgs(vardef.TiDBEnableNoopFuncs, potentialIncompatibleSysVar) } } } @@ -2342,7 +2343,7 @@ var defaultSysVars = []*SysVar{ s.NoopFuncsMode = TiDBOptOnOffWarn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBReplicaRead, Value: "leader", Type: TypeEnum, PossibleValues: []string{"leader", "prefer-leader", "follower", "leader-and-follower", "closest-replicas", "closest-adaptive", "learner"}, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBReplicaRead, Value: "leader", Type: vardef.TypeEnum, PossibleValues: []string{"leader", "prefer-leader", "follower", "leader-and-follower", "closest-replicas", "closest-adaptive", "learner"}, SetSession: func(s *SessionVars, val string) error { if strings.EqualFold(val, "follower") { s.SetReplicaRead(kv.ReplicaReadFollower) } else if strings.EqualFold(val, "leader-and-follower") { @@ -2360,15 +2361,15 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAdaptiveClosestReadThreshold, Value: strconv.Itoa(DefAdaptiveClosestReadThreshold), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - s.ReplicaClosestReadThreshold = TidbOptInt64(val, DefAdaptiveClosestReadThreshold) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBAdaptiveClosestReadThreshold, Value: strconv.Itoa(vardef.DefAdaptiveClosestReadThreshold), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.ReplicaClosestReadThreshold = TidbOptInt64(val, vardef.DefAdaptiveClosestReadThreshold) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBUsePlanBaselines, Value: BoolToOnOff(DefTiDBUsePlanBaselines), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBUsePlanBaselines, Value: BoolToOnOff(vardef.DefTiDBUsePlanBaselines), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.UsePlanBaselines = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEvolvePlanBaselines, Value: BoolToOnOff(DefTiDBEvolvePlanBaselines), Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEvolvePlanBaselines, Value: BoolToOnOff(vardef.DefTiDBEvolvePlanBaselines), Type: vardef.TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { if normalizedValue == "ON" && !config.CheckTableBeforeDrop { return normalizedValue, errors.Errorf("Cannot enable baseline evolution feature, it is not generally available now") } @@ -2377,38 +2378,38 @@ var defaultSysVars = []*SysVar{ s.EvolvePlanBaselines = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableExtendedStats, Value: BoolToOnOff(false), Hidden: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableExtendedStats, Value: BoolToOnOff(false), Hidden: true, Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableExtendedStats = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: CTEMaxRecursionDepth, Value: strconv.Itoa(DefCTEMaxRecursionDepth), Type: TypeInt, MinValue: 0, MaxValue: 4294967295, SetSession: func(s *SessionVars, val string) error { - s.CTEMaxRecursionDepth = TidbOptInt(val, DefCTEMaxRecursionDepth) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.CTEMaxRecursionDepth, Value: strconv.Itoa(vardef.DefCTEMaxRecursionDepth), Type: vardef.TypeInt, MinValue: 0, MaxValue: 4294967295, SetSession: func(s *SessionVars, val string) error { + s.CTEMaxRecursionDepth = TidbOptInt(val, vardef.DefCTEMaxRecursionDepth) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowAutoRandExplicitInsert, Value: BoolToOnOff(DefTiDBAllowAutoRandExplicitInsert), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBAllowAutoRandExplicitInsert, Value: BoolToOnOff(vardef.DefTiDBAllowAutoRandExplicitInsert), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.AllowAutoRandExplicitInsert = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableClusteredIndex, Value: On, Type: TypeEnum, PossibleValues: []string{Off, On, IntOnly}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if normalizedValue == IntOnly { - vars.StmtCtx.AppendWarning(errWarnDeprecatedSyntax.FastGenByArgs(normalizedValue, fmt.Sprintf("'%s' or '%s'", On, Off))) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableClusteredIndex, Value: vardef.On, Type: vardef.TypeEnum, PossibleValues: []string{vardef.Off, vardef.On, vardef.IntOnly}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + if normalizedValue == vardef.IntOnly { + vars.StmtCtx.AppendWarning(errWarnDeprecatedSyntax.FastGenByArgs(normalizedValue, fmt.Sprintf("'%s' or '%s'", vardef.On, vardef.Off))) } return normalizedValue, nil }, SetSession: func(s *SessionVars, val string) error { - s.EnableClusteredIndex = TiDBOptEnableClustered(val) + s.EnableClusteredIndex = vardef.TiDBOptEnableClustered(val) return nil }}, // Keeping tidb_enable_global_index here, to give error if setting it to anything other than ON - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableGlobalIndex, Type: TypeBool, Value: On, Validation: func(vars *SessionVars, normalizedValue, _ string, _ ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableGlobalIndex, Type: vardef.TypeBool, Value: vardef.On, Validation: func(vars *SessionVars, normalizedValue, _ string, _ vardef.ScopeFlag) (string, error) { if !TiDBOptOn(normalizedValue) { vars.StmtCtx.AppendWarning(errors.NewNoStackError("tidb_enable_global_index is always turned on. This variable has been deprecated and will be removed in the future releases")) } return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBPartitionPruneMode, Value: DefTiDBPartitionPruneMode, Type: TypeEnum, PossibleValues: []string{"static", "dynamic", "static-only", "dynamic-only"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBPartitionPruneMode, Value: vardef.DefTiDBPartitionPruneMode, Type: vardef.TypeEnum, PossibleValues: []string{"static", "dynamic", "static-only", "dynamic-only"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { mode := PartitionPruneMode(normalizedValue).Update() if !mode.Valid() { - return normalizedValue, ErrWrongTypeForVar.GenWithStackByArgs(TiDBPartitionPruneMode) + return normalizedValue, ErrWrongTypeForVar.GenWithStackByArgs(vardef.TiDBPartitionPruneMode) } return string(mode), nil }, GetSession: func(s *SessionVars) (string, error) { @@ -2434,36 +2435,36 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBRedactLog, Value: DefTiDBRedactLog, Type: TypeEnum, PossibleValues: []string{Off, On, Marker}, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBRedactLog, Value: vardef.DefTiDBRedactLog, Type: vardef.TypeEnum, PossibleValues: []string{vardef.Off, vardef.On, vardef.Marker}, SetSession: func(s *SessionVars, val string) error { s.EnableRedactLog = val errors.RedactLogEnabled.Store(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBShardAllocateStep, Value: strconv.Itoa(DefTiDBShardAllocateStep), Type: TypeInt, MinValue: 1, MaxValue: uint64(math.MaxInt64), SetSession: func(s *SessionVars, val string) error { - s.ShardAllocateStep = TidbOptInt64(val, DefTiDBShardAllocateStep) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBShardAllocateStep, Value: strconv.Itoa(vardef.DefTiDBShardAllocateStep), Type: vardef.TypeInt, MinValue: 1, MaxValue: uint64(math.MaxInt64), SetSession: func(s *SessionVars, val string) error { + s.ShardAllocateStep = TidbOptInt64(val, vardef.DefTiDBShardAllocateStep) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableAsyncCommit, Value: BoolToOnOff(DefTiDBEnableAsyncCommit), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableAsyncCommit, Value: BoolToOnOff(vardef.DefTiDBEnableAsyncCommit), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableAsyncCommit = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnable1PC, Value: BoolToOnOff(DefTiDBEnable1PC), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnable1PC, Value: BoolToOnOff(vardef.DefTiDBEnable1PC), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.Enable1PC = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBGuaranteeLinearizability, Value: BoolToOnOff(DefTiDBGuaranteeLinearizability), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBGuaranteeLinearizability, Value: BoolToOnOff(vardef.DefTiDBGuaranteeLinearizability), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.GuaranteeLinearizability = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeVersion, Value: strconv.Itoa(DefTiDBAnalyzeVersion), Type: TypeInt, MinValue: 1, MaxValue: 2, SetSession: func(s *SessionVars, val string) error { - s.AnalyzeVersion = tidbOptPositiveInt32(val, DefTiDBAnalyzeVersion) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBAnalyzeVersion, Value: strconv.Itoa(vardef.DefTiDBAnalyzeVersion), Type: vardef.TypeInt, MinValue: 1, MaxValue: 2, SetSession: func(s *SessionVars, val string) error { + s.AnalyzeVersion = tidbOptPositiveInt32(val, vardef.DefTiDBAnalyzeVersion) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashJoinVersion, Value: DefTiDBHashJoinVersion, Type: TypeStr, - Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBHashJoinVersion, Value: vardef.DefTiDBHashJoinVersion, Type: vardef.TypeStr, + Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ vardef.ScopeFlag) (string, error) { lowerValue := strings.ToLower(normalizedValue) if lowerValue != joinversion.HashJoinVersionLegacy && lowerValue != joinversion.HashJoinVersionOptimized { - err := fmt.Errorf("incorrect value: `%s`. %s options: %s", originalValue, TiDBHashJoinVersion, joinversion.HashJoinVersionLegacy+", "+joinversion.HashJoinVersionOptimized) + err := fmt.Errorf("incorrect value: `%s`. %s options: %s", originalValue, vardef.TiDBHashJoinVersion, joinversion.HashJoinVersionLegacy+", "+joinversion.HashJoinVersionOptimized) return normalizedValue, err } return normalizedValue, nil @@ -2473,81 +2474,81 @@ var defaultSysVars = []*SysVar{ return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptEnableHashJoin, Value: BoolToOnOff(DefTiDBOptEnableHashJoin), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptEnableHashJoin, Value: BoolToOnOff(vardef.DefTiDBOptEnableHashJoin), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.DisableHashJoin = !TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableIndexMergeJoin, Value: BoolToOnOff(DefTiDBEnableIndexMergeJoin), Hidden: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableIndexMergeJoin, Value: BoolToOnOff(vardef.DefTiDBEnableIndexMergeJoin), Hidden: true, Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableIndexMergeJoin = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBTrackAggregateMemoryUsage, Value: BoolToOnOff(DefTiDBTrackAggregateMemoryUsage), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBTrackAggregateMemoryUsage, Value: BoolToOnOff(vardef.DefTiDBTrackAggregateMemoryUsage), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.TrackAggregateMemoryUsage = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMultiStatementMode, Value: Off, Type: TypeEnum, PossibleValues: []string{Off, On, Warn}, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBMultiStatementMode, Value: vardef.Off, Type: vardef.TypeEnum, PossibleValues: []string{vardef.Off, vardef.On, vardef.Warn}, SetSession: func(s *SessionVars, val string) error { s.MultiStatementMode = TiDBOptOnOffWarn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableExchangePartition, Value: On, Type: TypeBool, - Validation: func(vars *SessionVars, s string, s2 string, flag ScopeFlag) (string, error) { - if s == Off { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableExchangePartition, Value: vardef.On, Type: vardef.TypeBool, + Validation: func(vars *SessionVars, s string, s2 string, flag vardef.ScopeFlag) (string, error) { + if s == vardef.Off { vars.StmtCtx.AppendWarning(errors.NewNoStackError("tidb_enable_exchange_partition is always turned on. This variable has been deprecated and will be removed in the future releases")) } - return On, nil + return vardef.On, nil }, SetSession: func(s *SessionVars, val string) error { s.TiDBEnableExchangePartition = true return nil }}, // It's different from tmp_table_size or max_heap_table_size. See https://github.com/pingcap/tidb/issues/28691. - {Scope: ScopeGlobal | ScopeSession, Name: TiDBTmpTableMaxSize, Value: strconv.Itoa(DefTiDBTmpTableMaxSize), Type: TypeUnsigned, MinValue: 1 << 20, MaxValue: 1 << 37, SetSession: func(s *SessionVars, val string) error { - s.TMPTableSize = TidbOptInt64(val, DefTiDBTmpTableMaxSize) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBTmpTableMaxSize, Value: strconv.Itoa(vardef.DefTiDBTmpTableMaxSize), Type: vardef.TypeUnsigned, MinValue: 1 << 20, MaxValue: 1 << 37, SetSession: func(s *SessionVars, val string) error { + s.TMPTableSize = TidbOptInt64(val, vardef.DefTiDBTmpTableMaxSize) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableOrderedResultMode, Value: BoolToOnOff(DefTiDBEnableOrderedResultMode), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableOrderedResultMode, Value: BoolToOnOff(vardef.DefTiDBEnableOrderedResultMode), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableStableResultMode = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePseudoForOutdatedStats, Value: BoolToOnOff(DefTiDBEnablePseudoForOutdatedStats), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnablePseudoForOutdatedStats, Value: BoolToOnOff(vardef.DefTiDBEnablePseudoForOutdatedStats), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnablePseudoForOutdatedStats = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBRegardNULLAsPoint, Value: BoolToOnOff(DefTiDBRegardNULLAsPoint), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBRegardNULLAsPoint, Value: BoolToOnOff(vardef.DefTiDBRegardNULLAsPoint), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.RegardNULLAsPoint = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePaging, Value: BoolToOnOff(DefTiDBEnablePaging), Type: TypeBool, Hidden: true, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnablePaging, Value: BoolToOnOff(vardef.DefTiDBEnablePaging), Type: vardef.TypeBool, Hidden: true, SetSession: func(s *SessionVars, val string) error { s.EnablePaging = TiDBOptOn(val) return nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { s.EnablePaging = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableLegacyInstanceScope, Value: BoolToOnOff(DefEnableLegacyInstanceScope), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableLegacyInstanceScope, Value: BoolToOnOff(vardef.DefEnableLegacyInstanceScope), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableLegacyInstanceScope = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStatsLoadSyncWait, Value: strconv.Itoa(DefTiDBStatsLoadSyncWait), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBStatsLoadSyncWait, Value: strconv.Itoa(vardef.DefTiDBStatsLoadSyncWait), Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { - s.StatsLoadSyncWait.Store(TidbOptInt64(val, DefTiDBStatsLoadSyncWait)) + s.StatsLoadSyncWait.Store(TidbOptInt64(val, vardef.DefTiDBStatsLoadSyncWait)) return nil }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { - return strconv.FormatInt(StatsLoadSyncWait.Load(), 10), nil + return strconv.FormatInt(vardef.StatsLoadSyncWait.Load(), 10), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - StatsLoadSyncWait.Store(TidbOptInt64(val, DefTiDBStatsLoadSyncWait)) + vardef.StatsLoadSyncWait.Store(TidbOptInt64(val, vardef.DefTiDBStatsLoadSyncWait)) return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSysdateIsNow, Value: BoolToOnOff(DefSysdateIsNow), Type: TypeBool, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBSysdateIsNow, Value: BoolToOnOff(vardef.DefSysdateIsNow), Type: vardef.TypeBool, SetSession: func(vars *SessionVars, s string) error { vars.SysdateIsNow = TiDBOptOn(s) return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableParallelHashaggSpill, Value: BoolToOnOff(DefTiDBEnableParallelHashaggSpill), Type: TypeBool, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableParallelHashaggSpill, Value: BoolToOnOff(vardef.DefTiDBEnableParallelHashaggSpill), Type: vardef.TypeBool, SetSession: func(vars *SessionVars, s string) error { vars.EnableParallelHashaggSpill = TiDBOptOn(s) if !vars.EnableParallelHashaggSpill { @@ -2556,151 +2557,151 @@ var defaultSysVars = []*SysVar{ return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableMutationChecker, Hidden: true, - Value: BoolToOnOff(DefTiDBEnableMutationChecker), Type: TypeBool, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableMutationChecker, Hidden: true, + Value: BoolToOnOff(vardef.DefTiDBEnableMutationChecker), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableMutationChecker = TiDBOptOn(val) return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBTxnAssertionLevel, Value: DefTiDBTxnAssertionLevel, PossibleValues: []string{AssertionOffStr, AssertionFastStr, AssertionStrictStr}, Hidden: true, Type: TypeEnum, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBTxnAssertionLevel, Value: vardef.DefTiDBTxnAssertionLevel, PossibleValues: []string{vardef.AssertionOffStr, vardef.AssertionFastStr, vardef.AssertionStrictStr}, Hidden: true, Type: vardef.TypeEnum, SetSession: func(s *SessionVars, val string) error { s.AssertionLevel = tidbOptAssertionLevel(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBatchPendingTiFlashCount, Value: strconv.Itoa(DefTiDBBatchPendingTiFlashCount), MinValue: 0, MaxValue: math.MaxUint32, Hidden: false, Type: TypeUnsigned, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBBatchPendingTiFlashCount, Value: strconv.Itoa(vardef.DefTiDBBatchPendingTiFlashCount), MinValue: 0, MaxValue: math.MaxUint32, Hidden: false, Type: vardef.TypeUnsigned, SetSession: func(s *SessionVars, val string) error { b, e := strconv.Atoi(val) if e != nil { - b = DefTiDBBatchPendingTiFlashCount + b = vardef.DefTiDBBatchPendingTiFlashCount } s.BatchPendingTiFlashCount = b return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIgnorePreparedCacheCloseStmt, Value: BoolToOnOff(DefTiDBIgnorePreparedCacheCloseStmt), Type: TypeBool, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBIgnorePreparedCacheCloseStmt, Value: BoolToOnOff(vardef.DefTiDBIgnorePreparedCacheCloseStmt), Type: vardef.TypeBool, SetSession: func(vars *SessionVars, s string) error { vars.IgnorePreparedCacheCloseStmt = TiDBOptOn(s) return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNewCostInterface, Value: BoolToOnOff(true), Hidden: false, Type: TypeBool, - Validation: func(vars *SessionVars, s string, s2 string, flag ScopeFlag) (string, error) { - if s == Off { - vars.StmtCtx.AppendWarning(errWarnDeprecatedSyntax.FastGenByArgs(Off, On)) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableNewCostInterface, Value: BoolToOnOff(true), Hidden: false, Type: vardef.TypeBool, + Validation: func(vars *SessionVars, s string, s2 string, flag vardef.ScopeFlag) (string, error) { + if s == vardef.Off { + vars.StmtCtx.AppendWarning(errWarnDeprecatedSyntax.FastGenByArgs(vardef.Off, vardef.On)) } - return On, nil + return vardef.On, nil }, SetSession: func(vars *SessionVars, s string) error { vars.EnableNewCostInterface = TiDBOptOn(s) return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBCostModelVersion, Value: strconv.Itoa(DefTiDBCostModelVer), Hidden: false, Type: TypeInt, MinValue: 1, MaxValue: 2, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBCostModelVersion, Value: strconv.Itoa(vardef.DefTiDBCostModelVer), Hidden: false, Type: vardef.TypeInt, MinValue: 1, MaxValue: 2, SetSession: func(vars *SessionVars, s string) error { vars.CostModelVersion = int(TidbOptInt64(s, 1)) return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexJoinDoubleReadPenaltyCostRate, Value: strconv.Itoa(0), Hidden: false, Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBIndexJoinDoubleReadPenaltyCostRate, Value: strconv.Itoa(0), Hidden: false, Type: vardef.TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(vars *SessionVars, s string) error { vars.IndexJoinDoubleReadPenaltyCostRate = tidbOptFloat64(s, 0) return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBRCWriteCheckTs, Type: TypeBool, Value: BoolToOnOff(DefTiDBRcWriteCheckTs), SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBRCWriteCheckTs, Type: vardef.TypeBool, Value: BoolToOnOff(vardef.DefTiDBRcWriteCheckTs), SetSession: func(s *SessionVars, val string) error { s.RcWriteCheckTS = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBRemoveOrderbyInSubquery, Value: BoolToOnOff(DefTiDBRemoveOrderbyInSubquery), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBRemoveOrderbyInSubquery, Value: BoolToOnOff(vardef.DefTiDBRemoveOrderbyInSubquery), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.RemoveOrderbyInSubquery = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMemQuotaQuery, Value: strconv.Itoa(DefTiDBMemQuotaQuery), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - s.MemQuotaQuery = TidbOptInt64(val, DefTiDBMemQuotaQuery) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBMemQuotaQuery, Value: strconv.Itoa(vardef.DefTiDBMemQuotaQuery), Type: vardef.TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.MemQuotaQuery = TidbOptInt64(val, vardef.DefTiDBMemQuotaQuery) s.MemTracker.SetBytesLimit(s.MemQuotaQuery) return nil - }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - intVal := TidbOptInt64(normalizedValue, DefTiDBMemQuotaQuery) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { + intVal := TidbOptInt64(normalizedValue, vardef.DefTiDBMemQuotaQuery) if intVal > 0 && intVal < 128 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(TiDBMemQuotaQuery, originalValue)) + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(vardef.TiDBMemQuotaQuery, originalValue)) normalizedValue = "128" } return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBNonTransactionalIgnoreError, Value: BoolToOnOff(DefTiDBBatchDMLIgnoreError), Type: TypeBool, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBNonTransactionalIgnoreError, Value: BoolToOnOff(vardef.DefTiDBBatchDMLIgnoreError), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.NonTransactionalIgnoreError = TiDBOptOn(val) return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiFlashFineGrainedShuffleStreamCount, Value: strconv.Itoa(DefTiFlashFineGrainedShuffleStreamCount), Type: TypeInt, MinValue: -1, MaxValue: 1024, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiFlashFineGrainedShuffleStreamCount, Value: strconv.Itoa(vardef.DefTiFlashFineGrainedShuffleStreamCount), Type: vardef.TypeInt, MinValue: -1, MaxValue: 1024, SetSession: func(s *SessionVars, val string) error { - s.TiFlashFineGrainedShuffleStreamCount = TidbOptInt64(val, DefTiFlashFineGrainedShuffleStreamCount) + s.TiFlashFineGrainedShuffleStreamCount = TidbOptInt64(val, vardef.DefTiFlashFineGrainedShuffleStreamCount) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiFlashFineGrainedShuffleBatchSize, Value: strconv.Itoa(DefTiFlashFineGrainedShuffleBatchSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiFlashFineGrainedShuffleBatchSize, Value: strconv.Itoa(vardef.DefTiFlashFineGrainedShuffleBatchSize), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.TiFlashFineGrainedShuffleBatchSize = uint64(TidbOptInt64(val, DefTiFlashFineGrainedShuffleBatchSize)) + s.TiFlashFineGrainedShuffleBatchSize = uint64(TidbOptInt64(val, vardef.DefTiFlashFineGrainedShuffleBatchSize)) return nil }}, - {Scope: ScopeGlobal, Name: TiDBSimplifiedMetrics, Value: BoolToOnOff(DefTiDBSimplifiedMetrics), Type: TypeBool, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBSimplifiedMetrics, Value: BoolToOnOff(vardef.DefTiDBSimplifiedMetrics), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, vars *SessionVars, s string) error { metrics.ToggleSimplifiedMode(TiDBOptOn(s)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMinPagingSize, Value: strconv.Itoa(DefMinPagingSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - s.MinPagingSize = tidbOptPositiveInt32(val, DefMinPagingSize) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBMinPagingSize, Value: strconv.Itoa(vardef.DefMinPagingSize), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.MinPagingSize = tidbOptPositiveInt32(val, vardef.DefMinPagingSize) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMaxPagingSize, Value: strconv.Itoa(DefMaxPagingSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - s.MaxPagingSize = tidbOptPositiveInt32(val, DefMaxPagingSize) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBMaxPagingSize, Value: strconv.Itoa(vardef.DefMaxPagingSize), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.MaxPagingSize = tidbOptPositiveInt32(val, vardef.DefMaxPagingSize) return nil }}, - {Scope: ScopeSession, Name: TiDBMemoryDebugModeMinHeapInUse, Value: strconv.Itoa(0), Type: TypeInt, MinValue: math.MinInt64, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBMemoryDebugModeMinHeapInUse, Value: strconv.Itoa(0), Type: vardef.TypeInt, MinValue: math.MinInt64, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemoryDebugModeMinHeapInUse = TidbOptInt64(val, 0) return nil }}, - {Scope: ScopeSession, Name: TiDBMemoryDebugModeAlarmRatio, Value: strconv.Itoa(0), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBMemoryDebugModeAlarmRatio, Value: strconv.Itoa(0), Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemoryDebugModeAlarmRatio = TidbOptInt64(val, 0) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLRequirePrimaryKey, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.SQLRequirePrimaryKey, Value: vardef.Off, Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.PrimaryKeyRequired = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableAnalyzeSnapshot, Value: BoolToOnOff(DefTiDBEnableAnalyzeSnapshot), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableAnalyzeSnapshot, Value: BoolToOnOff(vardef.DefTiDBEnableAnalyzeSnapshot), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableAnalyzeSnapshot = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal, Name: TiDBGenerateBinaryPlan, Value: BoolToOnOff(DefTiDBGenerateBinaryPlan), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBGenerateBinaryPlan, Value: BoolToOnOff(vardef.DefTiDBGenerateBinaryPlan), Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { GenerateBinaryPlan.Store(TiDBOptOn(val)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBDefaultStrMatchSelectivity, Value: strconv.FormatFloat(DefTiDBDefaultStrMatchSelectivity, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: 1, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBDefaultStrMatchSelectivity, Value: strconv.FormatFloat(vardef.DefTiDBDefaultStrMatchSelectivity, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0, MaxValue: 1, SetSession: func(s *SessionVars, val string) error { - s.DefaultStrMatchSelectivity = tidbOptFloat64(val, DefTiDBDefaultStrMatchSelectivity) + s.DefaultStrMatchSelectivity = tidbOptFloat64(val, vardef.DefTiDBDefaultStrMatchSelectivity) return nil }}, - {Scope: ScopeGlobal, Name: TiDBDDLEnableFastReorg, Value: BoolToOnOff(DefTiDBEnableFastReorg), Type: TypeBool, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { - return BoolToOnOff(EnableFastReorg.Load()), nil + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBDDLEnableFastReorg, Value: BoolToOnOff(vardef.DefTiDBEnableFastReorg), Type: vardef.TypeBool, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { + return BoolToOnOff(vardef.EnableFastReorg.Load()), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - EnableFastReorg.Store(TiDBOptOn(val)) + vardef.EnableFastReorg.Store(TiDBOptOn(val)) return nil }}, // This system var is set disk quota for lightning sort dir, from 100 GB to 1PB. - {Scope: ScopeGlobal, Name: TiDBDDLDiskQuota, Value: strconv.Itoa(DefTiDBDDLDiskQuota), Type: TypeInt, MinValue: DefTiDBDDLDiskQuota, MaxValue: 1024 * 1024 * DefTiDBDDLDiskQuota / 100, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { - return strconv.FormatUint(DDLDiskQuota.Load(), 10), nil + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBDDLDiskQuota, Value: strconv.Itoa(vardef.DefTiDBDDLDiskQuota), Type: vardef.TypeInt, MinValue: vardef.DefTiDBDDLDiskQuota, MaxValue: 1024 * 1024 * vardef.DefTiDBDDLDiskQuota / 100, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) { + return strconv.FormatUint(vardef.DDLDiskQuota.Load(), 10), nil }, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - DDLDiskQuota.Store(TidbOptUint64(val, DefTiDBDDLDiskQuota)) + vardef.DDLDiskQuota.Store(TidbOptUint64(val, vardef.DefTiDBDDLDiskQuota)) return nil }}, // can't assign validate function here. Because validation function will run after GetGlobal function - {Scope: ScopeGlobal, Name: TiDBCloudStorageURI, Value: "", Type: TypeStr, GetGlobal: func(ctx context.Context, sv *SessionVars) (string, error) { - cloudStorageURI := CloudStorageURI.Load() + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBCloudStorageURI, Value: "", Type: vardef.TypeStr, GetGlobal: func(ctx context.Context, sv *SessionVars) (string, error) { + cloudStorageURI := vardef.CloudStorageURI.Load() if len(cloudStorageURI) > 0 { cloudStorageURI = ast.RedactURL(cloudStorageURI) } return cloudStorageURI, nil }, SetGlobal: func(ctx context.Context, s *SessionVars, val string) error { - if len(val) > 0 && val != CloudStorageURI.Load() { + if len(val) > 0 && val != vardef.CloudStorageURI.Load() { if err := ValidateCloudStorageURI(ctx, val); err != nil { // convert annotations (second-level message) to message so clientConn.writeError // will print friendly error. @@ -2710,10 +2711,10 @@ var defaultSysVars = []*SysVar{ return err } } - CloudStorageURI.Store(val) + vardef.CloudStorageURI.Store(val) return nil }}, - {Scope: ScopeSession, Name: TiDBConstraintCheckInPlacePessimistic, Value: BoolToOnOff(config.GetGlobalConfig().PessimisticTxn.ConstraintCheckInPlacePessimistic), Type: TypeBool, + {Scope: vardef.ScopeSession, Name: vardef.TiDBConstraintCheckInPlacePessimistic, Value: BoolToOnOff(config.GetGlobalConfig().PessimisticTxn.ConstraintCheckInPlacePessimistic), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.ConstraintCheckInPlacePessimistic = TiDBOptOn(val) if !s.ConstraintCheckInPlacePessimistic { @@ -2721,63 +2722,63 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTiFlashReadForWriteStmt, Value: On, Type: TypeBool, - Validation: func(vars *SessionVars, s string, s2 string, flag ScopeFlag) (string, error) { - if s == Off { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableTiFlashReadForWriteStmt, Value: vardef.On, Type: vardef.TypeBool, + Validation: func(vars *SessionVars, s string, s2 string, flag vardef.ScopeFlag) (string, error) { + if s == vardef.Off { vars.StmtCtx.AppendWarning(errors.NewNoStackError("tidb_enable_tiflash_read_for_write_stmt is always turned on. This variable has been deprecated and will be removed in the future releases")) } - return On, nil + return vardef.On, nil }, SetSession: func(s *SessionVars, val string) error { s.EnableTiFlashReadForWriteStmt = true return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableUnsafeSubstitute, Value: BoolToOnOff(false), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableUnsafeSubstitute, Value: BoolToOnOff(false), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableUnsafeSubstitute = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptRangeMaxSize, Value: strconv.FormatInt(DefTiDBOptRangeMaxSize, 10), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - s.RangeMaxSize = TidbOptInt64(val, DefTiDBOptRangeMaxSize) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptRangeMaxSize, Value: strconv.FormatInt(vardef.DefTiDBOptRangeMaxSize, 10), Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + s.RangeMaxSize = TidbOptInt64(val, vardef.DefTiDBOptRangeMaxSize) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptAdvancedJoinHint, Value: BoolToOnOff(DefTiDBOptAdvancedJoinHint), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptAdvancedJoinHint, Value: BoolToOnOff(vardef.DefTiDBOptAdvancedJoinHint), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableAdvancedJoinHint = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBOptUseInvisibleIndexes, Value: BoolToOnOff(false), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeSession, Name: vardef.TiDBOptUseInvisibleIndexes, Value: BoolToOnOff(false), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.OptimizerUseInvisibleIndexes = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, - Name: TiDBAnalyzePartitionConcurrency, - Value: strconv.FormatInt(DefTiDBAnalyzePartitionConcurrency, 10), - Type: TypeInt, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.TiDBAnalyzePartitionConcurrency, + Value: strconv.FormatInt(vardef.DefTiDBAnalyzePartitionConcurrency, 10), + Type: vardef.TypeInt, MinValue: 1, MaxValue: 128, SetSession: func(s *SessionVars, val string) error { - s.AnalyzePartitionConcurrency = int(TidbOptInt64(val, DefTiDBAnalyzePartitionConcurrency)) + s.AnalyzePartitionConcurrency = int(TidbOptInt64(val, vardef.DefTiDBAnalyzePartitionConcurrency)) return nil }, }, { - Scope: ScopeGlobal | ScopeSession, Name: TiDBMergePartitionStatsConcurrency, Value: strconv.FormatInt(DefTiDBMergePartitionStatsConcurrency, 10), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBMergePartitionStatsConcurrency, Value: strconv.FormatInt(vardef.DefTiDBMergePartitionStatsConcurrency, 10), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, SetSession: func(s *SessionVars, val string) error { - s.AnalyzePartitionMergeConcurrency = TidbOptInt(val, DefTiDBMergePartitionStatsConcurrency) + s.AnalyzePartitionMergeConcurrency = TidbOptInt(val, vardef.DefTiDBMergePartitionStatsConcurrency) return nil }, }, { - Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableAsyncMergeGlobalStats, Value: BoolToOnOff(DefTiDBEnableAsyncMergeGlobalStats), Type: TypeBool, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableAsyncMergeGlobalStats, Value: BoolToOnOff(vardef.DefTiDBEnableAsyncMergeGlobalStats), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableAsyncMergeGlobalStats = TiDBOptOn(val) return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptPrefixIndexSingleScan, Value: BoolToOnOff(DefTiDBOptPrefixIndexSingleScan), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptPrefixIndexSingleScan, Value: BoolToOnOff(vardef.DefTiDBOptPrefixIndexSingleScan), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.OptPrefixIndexSingleScan = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal, Name: TiDBExternalTS, Value: strconv.FormatInt(DefTiDBExternalTS, 10), SetGlobal: func(ctx context.Context, s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBExternalTS, Value: strconv.FormatInt(vardef.DefTiDBExternalTS, 10), SetGlobal: func(ctx context.Context, s *SessionVars, val string) error { ts, err := parseTSFromNumberOrTime(s, val) if err != nil { return err @@ -2790,73 +2791,73 @@ var defaultSysVars = []*SysVar{ } return strconv.Itoa(int(ts)), err }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableExternalTSRead, Value: BoolToOnOff(DefTiDBEnableExternalTSRead), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableExternalTSRead, Value: BoolToOnOff(vardef.DefTiDBEnableExternalTSRead), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableExternalTSRead = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableReusechunk, Value: BoolToOnOff(DefTiDBEnableReusechunk), Type: TypeBool, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableReusechunk, Value: BoolToOnOff(vardef.DefTiDBEnableReusechunk), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableReuseChunk = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal, Name: TiDBIgnoreInlistPlanDigest, Value: BoolToOnOff(DefTiDBIgnoreInlistPlanDigest), Type: TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { - IgnoreInlistPlanDigest.Store(TiDBOptOn(s)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBIgnoreInlistPlanDigest, Value: BoolToOnOff(vardef.DefTiDBIgnoreInlistPlanDigest), Type: vardef.TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + vardef.IgnoreInlistPlanDigest.Store(TiDBOptOn(s)) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return BoolToOnOff(IgnoreInlistPlanDigest.Load()), nil + return BoolToOnOff(vardef.IgnoreInlistPlanDigest.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBTTLJobEnable, Value: BoolToOnOff(DefTiDBTTLJobEnable), Type: TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { - EnableTTLJob.Store(TiDBOptOn(s)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBTTLJobEnable, Value: BoolToOnOff(vardef.DefTiDBTTLJobEnable), Type: vardef.TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + vardef.EnableTTLJob.Store(TiDBOptOn(s)) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return BoolToOnOff(EnableTTLJob.Load()), nil + return BoolToOnOff(vardef.EnableTTLJob.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBTTLScanBatchSize, Value: strconv.Itoa(DefTiDBTTLScanBatchSize), Type: TypeInt, MinValue: DefTiDBTTLScanBatchMinSize, MaxValue: DefTiDBTTLScanBatchMaxSize, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBTTLScanBatchSize, Value: strconv.Itoa(vardef.DefTiDBTTLScanBatchSize), Type: vardef.TypeInt, MinValue: vardef.DefTiDBTTLScanBatchMinSize, MaxValue: vardef.DefTiDBTTLScanBatchMaxSize, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err } - TTLScanBatchSize.Store(val) + vardef.TTLScanBatchSize.Store(val) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - val := TTLScanBatchSize.Load() + val := vardef.TTLScanBatchSize.Load() return strconv.FormatInt(val, 10), nil }}, - {Scope: ScopeGlobal, Name: TiDBTTLDeleteBatchSize, Value: strconv.Itoa(DefTiDBTTLDeleteBatchSize), Type: TypeInt, MinValue: DefTiDBTTLDeleteBatchMinSize, MaxValue: DefTiDBTTLDeleteBatchMaxSize, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBTTLDeleteBatchSize, Value: strconv.Itoa(vardef.DefTiDBTTLDeleteBatchSize), Type: vardef.TypeInt, MinValue: vardef.DefTiDBTTLDeleteBatchMinSize, MaxValue: vardef.DefTiDBTTLDeleteBatchMaxSize, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err } - TTLDeleteBatchSize.Store(val) + vardef.TTLDeleteBatchSize.Store(val) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - val := TTLDeleteBatchSize.Load() + val := vardef.TTLDeleteBatchSize.Load() return strconv.FormatInt(val, 10), nil }}, - {Scope: ScopeGlobal, Name: TiDBTTLDeleteRateLimit, Value: strconv.Itoa(DefTiDBTTLDeleteRateLimit), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBTTLDeleteRateLimit, Value: strconv.Itoa(vardef.DefTiDBTTLDeleteRateLimit), Type: vardef.TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err } - TTLDeleteRateLimit.Store(val) + vardef.TTLDeleteRateLimit.Store(val) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - val := TTLDeleteRateLimit.Load() + val := vardef.TTLDeleteRateLimit.Load() return strconv.FormatInt(val, 10), nil }}, { - Scope: ScopeGlobal | ScopeSession, Name: TiDBStoreBatchSize, Value: strconv.FormatInt(DefTiDBStoreBatchSize, 10), - Type: TypeInt, MinValue: 0, MaxValue: 25000, SetSession: func(s *SessionVars, val string) error { - s.StoreBatchSize = TidbOptInt(val, DefTiDBStoreBatchSize) + Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBStoreBatchSize, Value: strconv.FormatInt(vardef.DefTiDBStoreBatchSize, 10), + Type: vardef.TypeInt, MinValue: 0, MaxValue: 25000, SetSession: func(s *SessionVars, val string) error { + s.StoreBatchSize = TidbOptInt(val, vardef.DefTiDBStoreBatchSize) return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: MppExchangeCompressionMode, Type: TypeStr, Value: DefaultExchangeCompressionMode.Name(), - Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ ScopeFlag) (string, error) { - _, ok := kv.ToExchangeCompressionMode(normalizedValue) + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.MppExchangeCompressionMode, Type: vardef.TypeStr, Value: vardef.DefaultExchangeCompressionMode.Name(), + Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ vardef.ScopeFlag) (string, error) { + _, ok := vardef.ToExchangeCompressionMode(normalizedValue) if !ok { var msg string - for m := kv.ExchangeCompressionModeNONE; m <= kv.ExchangeCompressionModeUnspecified; m += 1 { + for m := vardef.ExchangeCompressionModeNONE; m <= vardef.ExchangeCompressionModeUnspecified; m += 1 { if m == 0 { msg = m.Name() } else { @@ -2865,28 +2866,28 @@ var defaultSysVars = []*SysVar{ } err := fmt.Errorf("incorrect value: `%s`. %s options: %s", originalValue, - MppExchangeCompressionMode, msg) + vardef.MppExchangeCompressionMode, msg) return normalizedValue, err } return normalizedValue, nil }, SetSession: func(s *SessionVars, val string) error { - s.mppExchangeCompressionMode, _ = kv.ToExchangeCompressionMode(val) - if s.ChooseMppVersion() == kv.MppVersionV0 && s.mppExchangeCompressionMode != kv.ExchangeCompressionModeUnspecified { + s.mppExchangeCompressionMode, _ = vardef.ToExchangeCompressionMode(val) + if s.ChooseMppVersion() == kv.MppVersionV0 && s.mppExchangeCompressionMode != vardef.ExchangeCompressionModeUnspecified { s.StmtCtx.AppendWarning(fmt.Errorf("mpp exchange compression won't work under current mpp version %d", kv.MppVersionV0)) } return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: MppVersion, Type: TypeStr, Value: kv.MppVersionUnspecifiedName, - Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.MppVersion, Type: vardef.TypeStr, Value: kv.MppVersionUnspecifiedName, + Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ vardef.ScopeFlag) (string, error) { _, ok := kv.ToMppVersion(normalizedValue) if ok { return normalizedValue, nil } errMsg := fmt.Sprintf("incorrect value: %s. %s options: %d (unspecified)", - originalValue, MppVersion, kv.MppVersionUnspecified) + originalValue, vardef.MppVersion, kv.MppVersionUnspecified) for i := kv.MppVersionV0; i <= kv.GetNewestMppVersion(); i += 1 { errMsg = fmt.Sprintf("%s, %d", errMsg, i) } @@ -2900,84 +2901,84 @@ var defaultSysVars = []*SysVar{ }, }, { - Scope: ScopeGlobal, Name: TiDBTTLJobScheduleWindowStartTime, Value: DefTiDBTTLJobScheduleWindowStartTime, Type: TypeTime, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { - startTime, err := time.ParseInLocation(FullDayTimeFormat, s, time.UTC) + Scope: vardef.ScopeGlobal, Name: vardef.TiDBTTLJobScheduleWindowStartTime, Value: vardef.DefTiDBTTLJobScheduleWindowStartTime, Type: vardef.TypeTime, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + startTime, err := time.ParseInLocation(vardef.FullDayTimeFormat, s, time.UTC) if err != nil { return err } - TTLJobScheduleWindowStartTime.Store(startTime) + vardef.TTLJobScheduleWindowStartTime.Store(startTime) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - startTime := TTLJobScheduleWindowStartTime.Load() - return startTime.Format(FullDayTimeFormat), nil + startTime := vardef.TTLJobScheduleWindowStartTime.Load() + return startTime.Format(vardef.FullDayTimeFormat), nil }, }, { - Scope: ScopeGlobal, Name: TiDBTTLJobScheduleWindowEndTime, Value: DefTiDBTTLJobScheduleWindowEndTime, Type: TypeTime, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { - endTime, err := time.ParseInLocation(FullDayTimeFormat, s, time.UTC) + Scope: vardef.ScopeGlobal, Name: vardef.TiDBTTLJobScheduleWindowEndTime, Value: vardef.DefTiDBTTLJobScheduleWindowEndTime, Type: vardef.TypeTime, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + endTime, err := time.ParseInLocation(vardef.FullDayTimeFormat, s, time.UTC) if err != nil { return err } - TTLJobScheduleWindowEndTime.Store(endTime) + vardef.TTLJobScheduleWindowEndTime.Store(endTime) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - endTime := TTLJobScheduleWindowEndTime.Load() - return endTime.Format(FullDayTimeFormat), nil + endTime := vardef.TTLJobScheduleWindowEndTime.Load() + return endTime.Format(vardef.FullDayTimeFormat), nil }, }, { - Scope: ScopeGlobal, Name: TiDBTTLScanWorkerCount, Value: strconv.Itoa(DefTiDBTTLScanWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: 256, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + Scope: vardef.ScopeGlobal, Name: vardef.TiDBTTLScanWorkerCount, Value: strconv.Itoa(vardef.DefTiDBTTLScanWorkerCount), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: 256, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err } - TTLScanWorkerCount.Store(int32(val)) + vardef.TTLScanWorkerCount.Store(int32(val)) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return strconv.Itoa(int(TTLScanWorkerCount.Load())), nil + return strconv.Itoa(int(vardef.TTLScanWorkerCount.Load())), nil }, }, { - Scope: ScopeGlobal, Name: TiDBTTLDeleteWorkerCount, Value: strconv.Itoa(DefTiDBTTLDeleteWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: 256, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + Scope: vardef.ScopeGlobal, Name: vardef.TiDBTTLDeleteWorkerCount, Value: strconv.Itoa(vardef.DefTiDBTTLDeleteWorkerCount), Type: vardef.TypeUnsigned, MinValue: 1, MaxValue: 256, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err } - TTLDeleteWorkerCount.Store(int32(val)) + vardef.TTLDeleteWorkerCount.Store(int32(val)) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return strconv.Itoa(int(TTLDeleteWorkerCount.Load())), nil + return strconv.Itoa(int(vardef.TTLDeleteWorkerCount.Load())), nil }, }, - {Scope: ScopeGlobal, Name: TiDBEnableResourceControl, Value: BoolToOnOff(DefTiDBEnableResourceControl), Type: TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { - if TiDBOptOn(s) != EnableResourceControl.Load() { - EnableResourceControl.Store(TiDBOptOn(s)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableResourceControl, Value: BoolToOnOff(vardef.DefTiDBEnableResourceControl), Type: vardef.TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + if TiDBOptOn(s) != vardef.EnableResourceControl.Load() { + vardef.EnableResourceControl.Store(TiDBOptOn(s)) (*SetGlobalResourceControl.Load())(TiDBOptOn(s)) logutil.BgLogger().Info("set resource control", zap.Bool("enable", TiDBOptOn(s))) } return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return BoolToOnOff(EnableResourceControl.Load()), nil + return BoolToOnOff(vardef.EnableResourceControl.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBResourceControlStrictMode, Value: BoolToOnOff(DefTiDBResourceControlStrictMode), Type: TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBResourceControlStrictMode, Value: BoolToOnOff(vardef.DefTiDBResourceControlStrictMode), Type: vardef.TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { opOn := TiDBOptOn(s) - if opOn != EnableResourceControlStrictMode.Load() { - EnableResourceControlStrictMode.Store(opOn) + if opOn != vardef.EnableResourceControlStrictMode.Load() { + vardef.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 + return BoolToOnOff(vardef.EnableResourceControlStrictMode.Load()), nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBPessimisticTransactionFairLocking, Value: BoolToOnOff(DefTiDBPessimisticTransactionFairLocking), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBPessimisticTransactionFairLocking, Value: BoolToOnOff(vardef.DefTiDBPessimisticTransactionFairLocking), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.PessimisticTransactionFairLocking = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePlanCacheForParamLimit, Value: BoolToOnOff(DefTiDBEnablePlanCacheForParamLimit), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnablePlanCacheForParamLimit, Value: BoolToOnOff(vardef.DefTiDBEnablePlanCacheForParamLimit), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnablePlanCacheForParamLimit = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableINLJoinInnerMultiPattern, Value: BoolToOnOff(DefTiDBEnableINLJoinMultiPattern), Type: TypeBool, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableINLJoinInnerMultiPattern, Value: BoolToOnOff(vardef.DefTiDBEnableINLJoinMultiPattern), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableINLJoinInnerMultiPattern = TiDBOptOn(val) return nil @@ -2986,20 +2987,20 @@ var defaultSysVars = []*SysVar{ return BoolToOnOff(s.EnableINLJoinInnerMultiPattern), nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiFlashComputeDispatchPolicy, Value: string(DefTiFlashComputeDispatchPolicy), Type: TypeStr, SetSession: setTiFlashComputeDispatchPolicy, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiFlashComputeDispatchPolicy, Value: string(vardef.DefTiFlashComputeDispatchPolicy), Type: vardef.TypeStr, SetSession: setTiFlashComputeDispatchPolicy, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { return setTiFlashComputeDispatchPolicy(vars, s) }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePlanCacheForSubquery, Value: BoolToOnOff(DefTiDBEnablePlanCacheForSubquery), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnablePlanCacheForSubquery, Value: BoolToOnOff(vardef.DefTiDBEnablePlanCacheForSubquery), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnablePlanCacheForSubquery = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptEnableLateMaterialization, Value: BoolToOnOff(DefTiDBOptEnableLateMaterialization), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptEnableLateMaterialization, Value: BoolToOnOff(vardef.DefTiDBOptEnableLateMaterialization), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableLateMaterialization = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBLoadBasedReplicaReadThreshold, Value: DefTiDBLoadBasedReplicaReadThreshold.String(), Type: TypeDuration, MaxValue: uint64(time.Hour), SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBLoadBasedReplicaReadThreshold, Value: vardef.DefTiDBLoadBasedReplicaReadThreshold.String(), Type: vardef.TypeDuration, MaxValue: uint64(time.Hour), SetSession: func(s *SessionVars, val string) error { d, err := time.ParseDuration(val) if err != nil { return err @@ -3007,35 +3008,35 @@ var defaultSysVars = []*SysVar{ s.LoadBasedReplicaReadThreshold = d return nil }}, - {Scope: ScopeGlobal, Name: TiDBTTLRunningTasks, Value: strconv.Itoa(DefTiDBTTLRunningTasks), Type: TypeInt, MinValue: 1, MaxValue: MaxConfigurableConcurrency, AllowAutoValue: true, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBTTLRunningTasks, Value: strconv.Itoa(vardef.DefTiDBTTLRunningTasks), Type: vardef.TypeInt, MinValue: 1, MaxValue: vardef.MaxConfigurableConcurrency, AllowAutoValue: true, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err } - TTLRunningTasks.Store(int32(val)) + vardef.TTLRunningTasks.Store(int32(val)) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return strconv.Itoa(int(TTLRunningTasks.Load())), nil + return strconv.Itoa(int(vardef.TTLRunningTasks.Load())), nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptOrderingIdxSelThresh, Value: strconv.FormatFloat(DefTiDBOptOrderingIdxSelThresh, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: 1, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptOrderingIdxSelThresh, Value: strconv.FormatFloat(vardef.DefTiDBOptOrderingIdxSelThresh, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: 0, MaxValue: 1, SetSession: func(s *SessionVars, val string) error { - s.OptOrderingIdxSelThresh = tidbOptFloat64(val, DefTiDBOptOrderingIdxSelThresh) + s.OptOrderingIdxSelThresh = tidbOptFloat64(val, vardef.DefTiDBOptOrderingIdxSelThresh) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptOrderingIdxSelRatio, Value: strconv.FormatFloat(DefTiDBOptOrderingIdxSelRatio, 'f', -1, 64), Type: TypeFloat, MinValue: -1, MaxValue: 1, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptOrderingIdxSelRatio, Value: strconv.FormatFloat(vardef.DefTiDBOptOrderingIdxSelRatio, 'f', -1, 64), Type: vardef.TypeFloat, MinValue: -1, MaxValue: 1, SetSession: func(s *SessionVars, val string) error { - s.OptOrderingIdxSelRatio = tidbOptFloat64(val, DefTiDBOptOrderingIdxSelRatio) + s.OptOrderingIdxSelRatio = tidbOptFloat64(val, vardef.DefTiDBOptOrderingIdxSelRatio) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptEnableMPPSharedCTEExecution, Value: BoolToOnOff(DefTiDBOptEnableMPPSharedCTEExecution), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBOptEnableMPPSharedCTEExecution, Value: BoolToOnOff(vardef.DefTiDBOptEnableMPPSharedCTEExecution), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableMPPSharedCTEExecution = TiDBOptOn(val) return nil }}, { - Scope: ScopeGlobal | ScopeSession, - Name: TiDBOptFixControl, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.TiDBOptFixControl, Value: "", - Type: TypeStr, + Type: vardef.TypeStr, IsHintUpdatableVerified: true, SetGlobal: func(ctx context.Context, vars *SessionVars, val string) error { // validation logic for setting global @@ -3060,19 +3061,19 @@ var defaultSysVars = []*SysVar{ s.OptimizerFixControl = newMap return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeSkipColumnTypes, Value: "json,blob,mediumblob,longblob,mediumtext,longtext", Type: TypeStr, - Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBAnalyzeSkipColumnTypes, Value: "json,blob,mediumblob,longblob,mediumtext,longtext", Type: vardef.TypeStr, + Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { return ValidAnalyzeSkipColumnTypes(normalizedValue) }, SetSession: func(s *SessionVars, val string) error { s.AnalyzeSkipColumnTypes = ParseAnalyzeSkipColumnTypes(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBPlanCacheInvalidationOnFreshStats, Value: BoolToOnOff(DefTiDBPlanCacheInvalidationOnFreshStats), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBPlanCacheInvalidationOnFreshStats, Value: BoolToOnOff(vardef.DefTiDBPlanCacheInvalidationOnFreshStats), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.PlanCacheInvalidationOnFreshStats = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiFlashReplicaRead, Value: DefTiFlashReplicaRead, Type: TypeEnum, PossibleValues: []string{DefTiFlashReplicaRead, tiflash.ClosestAdaptiveStr, tiflash.ClosestReplicasStr}, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiFlashReplicaRead, Value: vardef.DefTiFlashReplicaRead, Type: vardef.TypeEnum, PossibleValues: []string{vardef.DefTiFlashReplicaRead, vardef.ClosestAdaptiveStr, vardef.ClosestReplicasStr}, SetSession: func(s *SessionVars, val string) error { s.TiFlashReplicaRead = tiflash.GetTiFlashReplicaReadByStr(val) return nil @@ -3081,28 +3082,28 @@ var defaultSysVars = []*SysVar{ return tiflash.GetTiFlashReplicaRead(s.TiFlashReplicaRead), nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBFastCheckTable, Value: BoolToOnOff(DefTiDBEnableFastCheckTable), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBFastCheckTable, Value: BoolToOnOff(vardef.DefTiDBEnableFastCheckTable), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.FastCheckTable = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipMissingPartitionStats, Value: BoolToOnOff(DefTiDBSkipMissingPartitionStats), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBSkipMissingPartitionStats, Value: BoolToOnOff(vardef.DefTiDBSkipMissingPartitionStats), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.SkipMissingPartitionStats = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSASLAuthMethodName, Value: DefAuthenticationLDAPSASLAuthMethodName, Type: TypeEnum, PossibleValues: []string{ldap.SASLAuthMethodSCRAMSHA1, ldap.SASLAuthMethodSCRAMSHA256, ldap.SASLAuthMethodGSSAPI}, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSASLAuthMethodName, Value: vardef.DefAuthenticationLDAPSASLAuthMethodName, Type: vardef.TypeEnum, PossibleValues: []string{ldap.SASLAuthMethodSCRAMSHA1, ldap.SASLAuthMethodSCRAMSHA256, ldap.SASLAuthMethodGSSAPI}, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { ldap.LDAPSASLAuthImpl.SetSASLAuthMethod(s) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return ldap.LDAPSASLAuthImpl.GetSASLAuthMethod(), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSASLServerHost, Value: "", Type: TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSASLServerHost, Value: "", Type: vardef.TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { // TODO: validate the ip/hostname ldap.LDAPSASLAuthImpl.SetLDAPServerHost(s) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return ldap.LDAPSASLAuthImpl.GetLDAPServerHost(), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSASLServerPort, Value: strconv.Itoa(DefAuthenticationLDAPSASLServerPort), Type: TypeInt, MinValue: 1, MaxValue: math.MaxUint16, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSASLServerPort, Value: strconv.Itoa(vardef.DefAuthenticationLDAPSASLServerPort), Type: vardef.TypeInt, MinValue: 1, MaxValue: math.MaxUint16, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err @@ -3112,47 +3113,47 @@ var defaultSysVars = []*SysVar{ }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return strconv.Itoa(ldap.LDAPSASLAuthImpl.GetLDAPServerPort()), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSASLTLS, Value: BoolToOnOff(DefAuthenticationLDAPSASLTLS), Type: TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSASLTLS, Value: BoolToOnOff(vardef.DefAuthenticationLDAPSASLTLS), Type: vardef.TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { ldap.LDAPSASLAuthImpl.SetEnableTLS(TiDBOptOn(s)) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return BoolToOnOff(ldap.LDAPSASLAuthImpl.GetEnableTLS()), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSASLCAPath, Value: "", Type: TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSASLCAPath, Value: "", Type: vardef.TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { return ldap.LDAPSASLAuthImpl.SetCAPath(s) }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return ldap.LDAPSASLAuthImpl.GetCAPath(), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSASLUserSearchAttr, Value: DefAuthenticationLDAPSASLUserSearchAttr, Type: TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSASLUserSearchAttr, Value: vardef.DefAuthenticationLDAPSASLUserSearchAttr, Type: vardef.TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { // TODO: validate the ip/hostname ldap.LDAPSASLAuthImpl.SetSearchAttr(s) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return ldap.LDAPSASLAuthImpl.GetSearchAttr(), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSASLBindBaseDN, Value: "", Type: TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSASLBindBaseDN, Value: "", Type: vardef.TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { ldap.LDAPSASLAuthImpl.SetBindBaseDN(s) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return ldap.LDAPSASLAuthImpl.GetBindBaseDN(), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSASLBindRootDN, Value: "", Type: TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSASLBindRootDN, Value: "", Type: vardef.TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { ldap.LDAPSASLAuthImpl.SetBindRootDN(s) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return ldap.LDAPSASLAuthImpl.GetBindRootDN(), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSASLBindRootPWD, Value: "", Type: TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSASLBindRootPWD, Value: "", Type: vardef.TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { ldap.LDAPSASLAuthImpl.SetBindRootPW(s) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { if ldap.LDAPSASLAuthImpl.GetBindRootPW() == "" { return "", nil } - return MaskPwd, nil + return vardef.MaskPwd, nil }}, // TODO: allow setting init_pool_size to 0 to disable pooling - {Scope: ScopeGlobal, Name: AuthenticationLDAPSASLInitPoolSize, Value: strconv.Itoa(DefAuthenticationLDAPSASLInitPoolSize), Type: TypeInt, MinValue: 1, MaxValue: 32767, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSASLInitPoolSize, Value: strconv.Itoa(vardef.DefAuthenticationLDAPSASLInitPoolSize), Type: vardef.TypeInt, MinValue: 1, MaxValue: 32767, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err @@ -3162,7 +3163,7 @@ var defaultSysVars = []*SysVar{ }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return strconv.Itoa(ldap.LDAPSASLAuthImpl.GetInitCapacity()), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSASLMaxPoolSize, Value: strconv.Itoa(DefAuthenticationLDAPSASLMaxPoolSize), Type: TypeInt, MinValue: 1, MaxValue: 32767, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSASLMaxPoolSize, Value: strconv.Itoa(vardef.DefAuthenticationLDAPSASLMaxPoolSize), Type: vardef.TypeInt, MinValue: 1, MaxValue: 32767, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err @@ -3172,7 +3173,7 @@ var defaultSysVars = []*SysVar{ }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return strconv.Itoa(ldap.LDAPSASLAuthImpl.GetMaxCapacity()), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSimpleAuthMethodName, Value: DefAuthenticationLDAPSimpleAuthMethodName, Type: TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSimpleAuthMethodName, Value: vardef.DefAuthenticationLDAPSimpleAuthMethodName, Type: vardef.TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { s = strings.ToUpper(s) // Only "SIMPLE" is supported if s != "SIMPLE" { @@ -3180,14 +3181,14 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSimpleServerHost, Value: "", Type: TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSimpleServerHost, Value: "", Type: vardef.TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { // TODO: validate the ip/hostname ldap.LDAPSimpleAuthImpl.SetLDAPServerHost(s) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return ldap.LDAPSimpleAuthImpl.GetLDAPServerHost(), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSimpleServerPort, Value: strconv.Itoa(DefAuthenticationLDAPSimpleServerPort), Type: TypeInt, MinValue: 1, MaxValue: math.MaxUint16, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSimpleServerPort, Value: strconv.Itoa(vardef.DefAuthenticationLDAPSimpleServerPort), Type: vardef.TypeInt, MinValue: 1, MaxValue: math.MaxUint16, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err @@ -3197,47 +3198,47 @@ var defaultSysVars = []*SysVar{ }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return strconv.Itoa(ldap.LDAPSimpleAuthImpl.GetLDAPServerPort()), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSimpleTLS, Value: BoolToOnOff(DefAuthenticationLDAPSimpleTLS), Type: TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSimpleTLS, Value: BoolToOnOff(vardef.DefAuthenticationLDAPSimpleTLS), Type: vardef.TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { ldap.LDAPSimpleAuthImpl.SetEnableTLS(TiDBOptOn(s)) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return BoolToOnOff(ldap.LDAPSimpleAuthImpl.GetEnableTLS()), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSimpleCAPath, Value: "", Type: TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSimpleCAPath, Value: "", Type: vardef.TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { return ldap.LDAPSimpleAuthImpl.SetCAPath(s) }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return ldap.LDAPSimpleAuthImpl.GetCAPath(), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSimpleUserSearchAttr, Value: DefAuthenticationLDAPSimpleUserSearchAttr, Type: TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSimpleUserSearchAttr, Value: vardef.DefAuthenticationLDAPSimpleUserSearchAttr, Type: vardef.TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { // TODO: validate the ip/hostname ldap.LDAPSimpleAuthImpl.SetSearchAttr(s) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return ldap.LDAPSimpleAuthImpl.GetSearchAttr(), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSimpleBindBaseDN, Value: "", Type: TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSimpleBindBaseDN, Value: "", Type: vardef.TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { ldap.LDAPSimpleAuthImpl.SetBindBaseDN(s) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return ldap.LDAPSimpleAuthImpl.GetBindBaseDN(), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSimpleBindRootDN, Value: "", Type: TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSimpleBindRootDN, Value: "", Type: vardef.TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { ldap.LDAPSimpleAuthImpl.SetBindRootDN(s) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return ldap.LDAPSimpleAuthImpl.GetBindRootDN(), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSimpleBindRootPWD, Value: "", Type: TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSimpleBindRootPWD, Value: "", Type: vardef.TypeStr, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { ldap.LDAPSimpleAuthImpl.SetBindRootPW(s) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { if ldap.LDAPSimpleAuthImpl.GetBindRootPW() == "" { return "", nil } - return MaskPwd, nil + return vardef.MaskPwd, nil }}, // TODO: allow setting init_pool_size to 0 to disable pooling - {Scope: ScopeGlobal, Name: AuthenticationLDAPSimpleInitPoolSize, Value: strconv.Itoa(DefAuthenticationLDAPSimpleInitPoolSize), Type: TypeInt, MinValue: 1, MaxValue: 32767, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSimpleInitPoolSize, Value: strconv.Itoa(vardef.DefAuthenticationLDAPSimpleInitPoolSize), Type: vardef.TypeInt, MinValue: 1, MaxValue: 32767, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err @@ -3247,7 +3248,7 @@ var defaultSysVars = []*SysVar{ }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return strconv.Itoa(ldap.LDAPSimpleAuthImpl.GetInitCapacity()), nil }}, - {Scope: ScopeGlobal, Name: AuthenticationLDAPSimpleMaxPoolSize, Value: strconv.Itoa(DefAuthenticationLDAPSimpleMaxPoolSize), Type: TypeInt, MinValue: 1, MaxValue: 32767, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + {Scope: vardef.ScopeGlobal, Name: vardef.AuthenticationLDAPSimpleMaxPoolSize, Value: strconv.Itoa(vardef.DefAuthenticationLDAPSimpleMaxPoolSize), Type: vardef.TypeInt, MinValue: 1, MaxValue: 32767, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err @@ -3258,14 +3259,14 @@ var defaultSysVars = []*SysVar{ return strconv.Itoa(ldap.LDAPSimpleAuthImpl.GetMaxCapacity()), nil }}, // runtime filter variables group - {Scope: ScopeGlobal | ScopeSession, Name: TiDBRuntimeFilterTypeName, Value: DefRuntimeFilterType, Type: TypeStr, - Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBRuntimeFilterTypeName, Value: vardef.DefRuntimeFilterType, Type: vardef.TypeStr, + Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ vardef.ScopeFlag) (string, error) { _, ok := ToRuntimeFilterType(normalizedValue) if ok { return normalizedValue, nil } errMsg := fmt.Sprintf("incorrect value: %s. %s should be sepreated by , such as %s, also we only support IN and MIN_MAX now. ", - originalValue, TiDBRuntimeFilterTypeName, DefRuntimeFilterType) + originalValue, vardef.TiDBRuntimeFilterTypeName, vardef.DefRuntimeFilterType) return normalizedValue, errors.New(errMsg) }, SetSession: func(s *SessionVars, val string) error { @@ -3273,14 +3274,14 @@ var defaultSysVars = []*SysVar{ return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBRuntimeFilterModeName, Value: DefRuntimeFilterMode, Type: TypeStr, - Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBRuntimeFilterModeName, Value: vardef.DefRuntimeFilterMode, Type: vardef.TypeStr, + Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ vardef.ScopeFlag) (string, error) { _, ok := RuntimeFilterModeStringToMode(normalizedValue) if ok { return normalizedValue, nil } errMsg := fmt.Sprintf("incorrect value: %s. %s options: %s ", - originalValue, TiDBRuntimeFilterModeName, DefRuntimeFilterMode) + originalValue, vardef.TiDBRuntimeFilterModeName, vardef.DefRuntimeFilterMode) return normalizedValue, errors.New(errMsg) }, SetSession: func(s *SessionVars, val string) error { @@ -3289,23 +3290,23 @@ var defaultSysVars = []*SysVar{ }, }, { - Scope: ScopeGlobal | ScopeSession, - Name: TiDBLockUnchangedKeys, - Value: BoolToOnOff(DefTiDBLockUnchangedKeys), - Type: TypeBool, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.TiDBLockUnchangedKeys, + Value: BoolToOnOff(vardef.DefTiDBLockUnchangedKeys), + Type: vardef.TypeBool, SetSession: func(vars *SessionVars, s string) error { vars.LockUnchangedKeys = TiDBOptOn(s) return nil }, }, - {Scope: ScopeGlobal, Name: TiDBEnableCheckConstraint, Value: BoolToOnOff(DefTiDBEnableCheckConstraint), Type: TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { - EnableCheckConstraint.Store(TiDBOptOn(s)) + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBEnableCheckConstraint, Value: BoolToOnOff(vardef.DefTiDBEnableCheckConstraint), Type: vardef.TypeBool, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + vardef.EnableCheckConstraint.Store(TiDBOptOn(s)) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return BoolToOnOff(EnableCheckConstraint.Load()), nil + return BoolToOnOff(vardef.EnableCheckConstraint.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBSchemaCacheSize, Value: strconv.Itoa(DefTiDBSchemaCacheSize), Type: TypeStr, - Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBSchemaCacheSize, Value: strconv.Itoa(vardef.DefTiDBSchemaCacheSize), Type: vardef.TypeStr, + Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { _, str, err := parseSchemaCacheSize(s, normalizedValue, originalValue) if err != nil { return "", err @@ -3318,24 +3319,24 @@ var defaultSysVars = []*SysVar{ if err != nil { return err } - if SchemaCacheSize.Load() != bt && ChangeSchemaCacheSize != nil { + if vardef.SchemaCacheSize.Load() != bt && ChangeSchemaCacheSize != nil { if err := ChangeSchemaCacheSize(ctx, bt); err != nil { return err } } - SchemaCacheSize.Store(bt) - SchemaCacheSizeOriginText.Store(str) + vardef.SchemaCacheSize.Store(bt) + vardef.SchemaCacheSizeOriginText.Store(str) return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return SchemaCacheSizeOriginText.Load(), nil + return vardef.SchemaCacheSizeOriginText.Load(), nil }}, - {Scope: ScopeSession, Name: TiDBSessionAlias, Value: "", Type: TypeStr, - Validation: func(s *SessionVars, normalizedValue string, originalValue string, _ ScopeFlag) (string, error) { + {Scope: vardef.ScopeSession, Name: vardef.TiDBSessionAlias, Value: "", Type: vardef.TypeStr, + Validation: func(s *SessionVars, normalizedValue string, originalValue string, _ vardef.ScopeFlag) (string, error) { chars := []rune(normalizedValue) warningAdded := false if len(chars) > 64 { - s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(TiDBSessionAlias, originalValue)) + s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(vardef.TiDBSessionAlias, originalValue)) warningAdded = true chars = chars[:64] normalizedValue = string(chars) @@ -3344,7 +3345,7 @@ var defaultSysVars = []*SysVar{ // truncate to a valid identifier for normalizedValue != "" && util.IsInCorrectIdentifierName(normalizedValue) { if !warningAdded { - s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(TiDBSessionAlias, originalValue)) + s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(vardef.TiDBSessionAlias, originalValue)) warningAdded = true } chars = chars[:len(chars)-1] @@ -3360,23 +3361,23 @@ var defaultSysVars = []*SysVar{ return vars.SessionAlias, nil }}, { - Scope: ScopeGlobal | ScopeSession, - Name: TiDBOptObjective, - Value: DefTiDBOptObjective, - Type: TypeEnum, - PossibleValues: []string{OptObjectiveModerate, OptObjectiveDeterminate}, + Scope: vardef.ScopeGlobal | vardef.ScopeSession, + Name: vardef.TiDBOptObjective, + Value: vardef.DefTiDBOptObjective, + Type: vardef.TypeEnum, + PossibleValues: []string{vardef.OptObjectiveModerate, vardef.OptObjectiveDeterminate}, SetSession: func(vars *SessionVars, s string) error { vars.OptObjective = s return nil }, }, - {Scope: ScopeInstance, Name: TiDBServiceScope, Value: "", Type: TypeStr, - Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ ScopeFlag) (string, error) { + {Scope: vardef.ScopeInstance, Name: vardef.TiDBServiceScope, Value: "", Type: vardef.TypeStr, + Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ vardef.ScopeFlag) (string, error) { return normalizedValue, servicescope.CheckServiceScope(originalValue) }, SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { newValue := strings.ToLower(s) - ServiceScope.Store(newValue) + vardef.ServiceScope.Store(newValue) oldConfig := config.GetGlobalConfig() if oldConfig.Instance.TiDBServiceScope != newValue { newConfig := *oldConfig @@ -3385,24 +3386,24 @@ var defaultSysVars = []*SysVar{ } return nil }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return ServiceScope.Load(), nil + return vardef.ServiceScope.Load(), nil }}, - {Scope: ScopeGlobal, Name: TiDBSchemaVersionCacheLimit, Value: strconv.Itoa(DefTiDBSchemaVersionCacheLimit), Type: TypeInt, MinValue: 2, MaxValue: math.MaxUint8, AllowEmpty: true, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBSchemaVersionCacheLimit, Value: strconv.Itoa(vardef.DefTiDBSchemaVersionCacheLimit), Type: vardef.TypeInt, MinValue: 2, MaxValue: math.MaxUint8, AllowEmpty: true, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - SchemaVersionCacheLimit.Store(TidbOptInt64(val, DefTiDBSchemaVersionCacheLimit)) + vardef.SchemaVersionCacheLimit.Store(TidbOptInt64(val, vardef.DefTiDBSchemaVersionCacheLimit)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIdleTransactionTimeout, Value: strconv.Itoa(DefTiDBIdleTransactionTimeout), Type: TypeUnsigned, MinValue: 0, MaxValue: secondsPerYear, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBIdleTransactionTimeout, Value: strconv.Itoa(vardef.DefTiDBIdleTransactionTimeout), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: secondsPerYear, SetSession: func(s *SessionVars, val string) error { - s.IdleTransactionTimeout = tidbOptPositiveInt32(val, DefTiDBIdleTransactionTimeout) + s.IdleTransactionTimeout = tidbOptPositiveInt32(val, vardef.DefTiDBIdleTransactionTimeout) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: DivPrecisionIncrement, Value: strconv.Itoa(DefDivPrecisionIncrement), Type: TypeUnsigned, MinValue: 0, MaxValue: 30, + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.DivPrecisionIncrement, Value: strconv.Itoa(vardef.DefDivPrecisionIncrement), Type: vardef.TypeUnsigned, MinValue: 0, MaxValue: 30, SetSession: func(s *SessionVars, val string) error { - s.DivPrecisionIncrement = tidbOptPositiveInt32(val, DefDivPrecisionIncrement) + s.DivPrecisionIncrement = tidbOptPositiveInt32(val, vardef.DefDivPrecisionIncrement) return nil }}, - {Scope: ScopeSession, Name: TiDBDMLType, Value: DefTiDBDMLType, Type: TypeStr, + {Scope: vardef.ScopeSession, Name: vardef.TiDBDMLType, Value: vardef.DefTiDBDMLType, Type: vardef.TypeStr, SetSession: func(s *SessionVars, val string) error { lowerVal := strings.ToLower(val) if strings.EqualFold(lowerVal, "standard") { @@ -3417,13 +3418,13 @@ var defaultSysVars = []*SysVar{ }, IsHintUpdatableVerified: true, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiFlashHashAggPreAggMode, Value: DefTiFlashPreAggMode, Type: TypeStr, - Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ ScopeFlag) (string, error) { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiFlashHashAggPreAggMode, Value: vardef.DefTiFlashPreAggMode, Type: vardef.TypeStr, + Validation: func(_ *SessionVars, normalizedValue string, originalValue string, _ vardef.ScopeFlag) (string, error) { if _, ok := ToTiPBTiFlashPreAggMode(normalizedValue); ok { return normalizedValue, nil } errMsg := fmt.Sprintf("incorrect value: `%s`. %s options: %s", - originalValue, TiFlashHashAggPreAggMode, ValidTiFlashPreAggMode()) + originalValue, vardef.TiFlashHashAggPreAggMode, ValidTiFlashPreAggMode()) return normalizedValue, errors.New(errMsg) }, SetSession: func(s *SessionVars, val string) error { @@ -3431,20 +3432,20 @@ var defaultSysVars = []*SysVar{ return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableLazyCursorFetch, Value: BoolToOnOff(DefTiDBEnableLazyCursorFetch), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableLazyCursorFetch, Value: BoolToOnOff(vardef.DefTiDBEnableLazyCursorFetch), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableLazyCursorFetch = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableSharedLockPromotion, Value: BoolToOnOff(DefTiDBEnableSharedLockPromotion), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: vardef.TiDBEnableSharedLockPromotion, Value: BoolToOnOff(vardef.DefTiDBEnableSharedLockPromotion), Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { if s.NoopFuncsMode != OffInt && TiDBOptOn(val) { logutil.BgLogger().Warn("tidb_enable_shared_lock_promotion set to on would override tidb_enable_noop_functions on") } s.SharedLockPromotion = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal, Name: TiDBTSOClientRPCMode, Value: DefTiDBTSOClientRPCMode, Type: TypeEnum, PossibleValues: []string{TSOClientRPCModeDefault, TSOClientRPCModeParallel, TSOClientRPCModeParallelFast}, + {Scope: vardef.ScopeGlobal, Name: vardef.TiDBTSOClientRPCMode, Value: vardef.DefTiDBTSOClientRPCMode, Type: vardef.TypeEnum, PossibleValues: []string{vardef.TSOClientRPCModeDefault, vardef.TSOClientRPCModeParallel, vardef.TSOClientRPCModeParallelFast}, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - return (*SetPDClientDynamicOption.Load())(TiDBTSOClientRPCMode, val) + return (*SetPDClientDynamicOption.Load())(vardef.TiDBTSOClientRPCMode, val) }, }, } @@ -3452,30 +3453,30 @@ var defaultSysVars = []*SysVar{ // GlobalSystemVariableInitialValue gets the default value for a system variable including ones that are dynamically set (e.g. based on the store) func GlobalSystemVariableInitialValue(varName, varVal string) string { switch varName { - case TiDBEnableAsyncCommit, TiDBEnable1PC: + case vardef.TiDBEnableAsyncCommit, vardef.TiDBEnable1PC: if config.GetGlobalConfig().Store == config.StoreTypeTiKV { - varVal = On + varVal = vardef.On } - case TiDBMemOOMAction: + case vardef.TiDBMemOOMAction: if intest.InTest { - varVal = OOMActionLog + varVal = vardef.OOMActionLog } - case TiDBEnableAutoAnalyze: + case vardef.TiDBEnableAutoAnalyze: if intest.InTest { - varVal = Off + varVal = vardef.Off } // For the following sysvars, we change the default // FOR NEW INSTALLS ONLY. In most cases you don't want to do this. // It is better to change the value in the Sysvar struct, so that // all installs will have the same value. - case TiDBRowFormatVersion: - varVal = strconv.Itoa(DefTiDBRowFormatV2) - case TiDBTxnAssertionLevel: - varVal = AssertionFastStr - case TiDBEnableMutationChecker: - varVal = On - case TiDBPessimisticTransactionFairLocking: - varVal = On + case vardef.TiDBRowFormatVersion: + varVal = strconv.Itoa(vardef.DefTiDBRowFormatV2) + case vardef.TiDBTxnAssertionLevel: + varVal = vardef.AssertionFastStr + case vardef.TiDBEnableMutationChecker: + varVal = vardef.On + case vardef.TiDBPessimisticTransactionFairLocking: + varVal = vardef.On } return varVal } @@ -3488,330 +3489,3 @@ func setTiFlashComputeDispatchPolicy(s *SessionVars, val string) error { s.TiFlashComputeDispatchPolicy = p return nil } - -// SetNamesVariables is the system variable names related to set names statements. -var SetNamesVariables = []string{ - CharacterSetClient, - CharacterSetConnection, - CharacterSetResults, -} - -// SetCharsetVariables is the system variable names related to set charset statements. -var SetCharsetVariables = []string{ - CharacterSetClient, - CharacterSetResults, -} - -const ( - // MaskPwd is the mask of password for LDAP variables. - MaskPwd = "******" - - // PessimisticTxnMode is the name for tidb_txn_mode system variable. - PessimisticTxnMode = "pessimistic" - // OptimisticTxnMode is the name for tidb_txn_mode system variable. - OptimisticTxnMode = "optimistic" -) - -const ( - // CharacterSetConnection is the name for character_set_connection system variable. - CharacterSetConnection = "character_set_connection" - // CollationConnection is the name for collation_connection system variable. - CollationConnection = "collation_connection" - // CharsetDatabase is the name for character_set_database system variable. - CharsetDatabase = "character_set_database" - // CollationDatabase is the name for collation_database system variable. - CollationDatabase = "collation_database" - // CharacterSetFilesystem is the name for character_set_filesystem system variable. - CharacterSetFilesystem = "character_set_filesystem" - // CharacterSetClient is the name for character_set_client system variable. - CharacterSetClient = "character_set_client" - // CharacterSetSystem is the name for character_set_system system variable. - CharacterSetSystem = "character_set_system" - // GeneralLog is the name for 'general_log' system variable. - GeneralLog = "general_log" - // AvoidTemporalUpgrade is the name for 'avoid_temporal_upgrade' system variable. - AvoidTemporalUpgrade = "avoid_temporal_upgrade" - // MaxPreparedStmtCount is the name for 'max_prepared_stmt_count' system variable. - MaxPreparedStmtCount = "max_prepared_stmt_count" - // BigTables is the name for 'big_tables' system variable. - BigTables = "big_tables" - // CheckProxyUsers is the name for 'check_proxy_users' system variable. - CheckProxyUsers = "check_proxy_users" - // CoreFile is the name for 'core_file' system variable. - CoreFile = "core_file" - // DefaultWeekFormat is the name for 'default_week_format' system variable. - DefaultWeekFormat = "default_week_format" - // GroupConcatMaxLen is the name for 'group_concat_max_len' system variable. - GroupConcatMaxLen = "group_concat_max_len" - // DelayKeyWrite is the name for 'delay_key_write' system variable. - DelayKeyWrite = "delay_key_write" - // EndMarkersInJSON is the name for 'end_markers_in_json' system variable. - EndMarkersInJSON = "end_markers_in_json" - // Hostname is the name for 'hostname' system variable. - Hostname = "hostname" - // InnodbCommitConcurrency is the name for 'innodb_commit_concurrency' system variable. - InnodbCommitConcurrency = "innodb_commit_concurrency" - // InnodbFastShutdown is the name for 'innodb_fast_shutdown' system variable. - InnodbFastShutdown = "innodb_fast_shutdown" - // InnodbLockWaitTimeout is the name for 'innodb_lock_wait_timeout' system variable. - InnodbLockWaitTimeout = "innodb_lock_wait_timeout" - // MaxSortLength is the name for 'max_sort_length' system variable. - MaxSortLength = "max_sort_length" - // MaxSpRecursionDepth is the name for 'max_sp_recursion_depth' system variable. - MaxSpRecursionDepth = "max_sp_recursion_depth" - // MaxUserConnections is the name for 'max_user_connections' system variable. - MaxUserConnections = "max_user_connections" - // OfflineMode is the name for 'offline_mode' system variable. - OfflineMode = "offline_mode" - // InteractiveTimeout is the name for 'interactive_timeout' system variable. - InteractiveTimeout = "interactive_timeout" - // FlushTime is the name for 'flush_time' system variable. - FlushTime = "flush_time" - // PseudoSlaveMode is the name for 'pseudo_slave_mode' system variable. - PseudoSlaveMode = "pseudo_slave_mode" - // LowPriorityUpdates is the name for 'low_priority_updates' system variable. - LowPriorityUpdates = "low_priority_updates" - // LowerCaseTableNames is the name for 'lower_case_table_names' system variable. - LowerCaseTableNames = "lower_case_table_names" - // SessionTrackGtids is the name for 'session_track_gtids' system variable. - SessionTrackGtids = "session_track_gtids" - // OldPasswords is the name for 'old_passwords' system variable. - OldPasswords = "old_passwords" - // MaxConnections is the name for 'max_connections' system variable. - MaxConnections = "max_connections" - // SkipNameResolve is the name for 'skip_name_resolve' system variable. - SkipNameResolve = "skip_name_resolve" - // ForeignKeyChecks is the name for 'foreign_key_checks' system variable. - ForeignKeyChecks = "foreign_key_checks" - // SQLSafeUpdates is the name for 'sql_safe_updates' system variable. - SQLSafeUpdates = "sql_safe_updates" - // WarningCount is the name for 'warning_count' system variable. - WarningCount = "warning_count" - // ErrorCount is the name for 'error_count' system variable. - ErrorCount = "error_count" - // DefaultPasswordLifetime is the name for 'default_password_lifetime' system variable. - DefaultPasswordLifetime = "default_password_lifetime" - // DisconnectOnExpiredPassword is the name for 'disconnect_on_expired_password' system variable. - DisconnectOnExpiredPassword = "disconnect_on_expired_password" - // SQLSelectLimit is the name for 'sql_select_limit' system variable. - SQLSelectLimit = "sql_select_limit" - // MaxConnectErrors is the name for 'max_connect_errors' system variable. - MaxConnectErrors = "max_connect_errors" - // TableDefinitionCache is the name for 'table_definition_cache' system variable. - TableDefinitionCache = "table_definition_cache" - // Timestamp is the name for 'timestamp' system variable. - Timestamp = "timestamp" - // ConnectTimeout is the name for 'connect_timeout' system variable. - ConnectTimeout = "connect_timeout" - // SyncBinlog is the name for 'sync_binlog' system variable. - SyncBinlog = "sync_binlog" - // BlockEncryptionMode is the name for 'block_encryption_mode' system variable. - BlockEncryptionMode = "block_encryption_mode" - // WaitTimeout is the name for 'wait_timeout' system variable. - WaitTimeout = "wait_timeout" - // Version is the name of 'version' system variable. - Version = "version" - // VersionComment is the name of 'version_comment' system variable. - VersionComment = "version_comment" - // PluginDir is the name of 'plugin_dir' system variable. - PluginDir = "plugin_dir" - // PluginLoad is the name of 'plugin_load' system variable. - PluginLoad = "plugin_load" - // TiDBEnableDDL indicates whether the tidb-server campaigns the DDL owner, - TiDBEnableDDL = "tidb_enable_ddl" - // TiDBEnableStatsOwner indicates whether the tidb-server campaigns the Stats owner, - TiDBEnableStatsOwner = "tidb_enable_stats_owner" - // Port is the name for 'port' system variable. - Port = "port" - // DataDir is the name for 'datadir' system variable. - DataDir = "datadir" - // Profiling is the name for 'Profiling' system variable. - Profiling = "profiling" - // Socket is the name for 'socket' system variable. - Socket = "socket" - // BinlogOrderCommits is the name for 'binlog_order_commits' system variable. - BinlogOrderCommits = "binlog_order_commits" - // MasterVerifyChecksum is the name for 'master_verify_checksum' system variable. - MasterVerifyChecksum = "master_verify_checksum" - // SuperReadOnly is the name for 'super_read_only' system variable. - SuperReadOnly = "super_read_only" - // SQLNotes is the name for 'sql_notes' system variable. - SQLNotes = "sql_notes" - // SlaveCompressedProtocol is the name for 'slave_compressed_protocol' system variable. - SlaveCompressedProtocol = "slave_compressed_protocol" - // BinlogRowQueryLogEvents is the name for 'binlog_rows_query_log_events' system variable. - BinlogRowQueryLogEvents = "binlog_rows_query_log_events" - // LogSlowSlaveStatements is the name for 'log_slow_slave_statements' system variable. - LogSlowSlaveStatements = "log_slow_slave_statements" - // LogSlowAdminStatements is the name for 'log_slow_admin_statements' system variable. - LogSlowAdminStatements = "log_slow_admin_statements" - // LogQueriesNotUsingIndexes is the name for 'log_queries_not_using_indexes' system variable. - LogQueriesNotUsingIndexes = "log_queries_not_using_indexes" - // SQLAutoIsNull is the name for 'sql_auto_is_null' system variable. - SQLAutoIsNull = "sql_auto_is_null" - // RelayLogPurge is the name for 'relay_log_purge' system variable. - RelayLogPurge = "relay_log_purge" - // AutomaticSpPrivileges is the name for 'automatic_sp_privileges' system variable. - AutomaticSpPrivileges = "automatic_sp_privileges" - // SQLQuoteShowCreate is the name for 'sql_quote_show_create' system variable. - SQLQuoteShowCreate = "sql_quote_show_create" - // SlowQueryLog is the name for 'slow_query_log' system variable. - SlowQueryLog = "slow_query_log" - // BinlogDirectNonTransactionalUpdates is the name for 'binlog_direct_non_transactional_updates' system variable. - BinlogDirectNonTransactionalUpdates = "binlog_direct_non_transactional_updates" - // SQLBigSelects is the name for 'sql_big_selects' system variable. - SQLBigSelects = "sql_big_selects" - // LogBinTrustFunctionCreators is the name for 'log_bin_trust_function_creators' system variable. - LogBinTrustFunctionCreators = "log_bin_trust_function_creators" - // OldAlterTable is the name for 'old_alter_table' system variable. - OldAlterTable = "old_alter_table" - // EnforceGtidConsistency is the name for 'enforce_gtid_consistency' system variable. - EnforceGtidConsistency = "enforce_gtid_consistency" - // SecureAuth is the name for 'secure_auth' system variable. - SecureAuth = "secure_auth" - // UniqueChecks is the name for 'unique_checks' system variable. - UniqueChecks = "unique_checks" - // SQLWarnings is the name for 'sql_warnings' system variable. - SQLWarnings = "sql_warnings" - // AutoCommit is the name for 'autocommit' system variable. - AutoCommit = "autocommit" - // KeepFilesOnCreate is the name for 'keep_files_on_create' system variable. - KeepFilesOnCreate = "keep_files_on_create" - // ShowOldTemporals is the name for 'show_old_temporals' system variable. - ShowOldTemporals = "show_old_temporals" - // LocalInFile is the name for 'local_infile' system variable. - LocalInFile = "local_infile" - // PerformanceSchema is the name for 'performance_schema' system variable. - PerformanceSchema = "performance_schema" - // Flush is the name for 'flush' system variable. - Flush = "flush" - // SlaveAllowBatching is the name for 'slave_allow_batching' system variable. - SlaveAllowBatching = "slave_allow_batching" - // MyISAMUseMmap is the name for 'myisam_use_mmap' system variable. - MyISAMUseMmap = "myisam_use_mmap" - // InnodbFilePerTable is the name for 'innodb_file_per_table' system variable. - InnodbFilePerTable = "innodb_file_per_table" - // InnodbLogCompressedPages is the name for 'innodb_log_compressed_pages' system variable. - InnodbLogCompressedPages = "innodb_log_compressed_pages" - // InnodbPrintAllDeadlocks is the name for 'innodb_print_all_deadlocks' system variable. - InnodbPrintAllDeadlocks = "innodb_print_all_deadlocks" - // InnodbStrictMode is the name for 'innodb_strict_mode' system variable. - InnodbStrictMode = "innodb_strict_mode" - // InnodbCmpPerIndexEnabled is the name for 'innodb_cmp_per_index_enabled' system variable. - InnodbCmpPerIndexEnabled = "innodb_cmp_per_index_enabled" - // InnodbBufferPoolDumpAtShutdown is the name for 'innodb_buffer_pool_dump_at_shutdown' system variable. - InnodbBufferPoolDumpAtShutdown = "innodb_buffer_pool_dump_at_shutdown" - // InnodbAdaptiveHashIndex is the name for 'innodb_adaptive_hash_index' system variable. - InnodbAdaptiveHashIndex = "innodb_adaptive_hash_index" - // InnodbFtEnableStopword is the name for 'innodb_ft_enable_stopword' system variable. - InnodbFtEnableStopword = "innodb_ft_enable_stopword" // #nosec G101 - // InnodbOptimizeFullTextOnly is the name for 'innodb_optimize_fulltext_only' system variable. - InnodbOptimizeFullTextOnly = "innodb_optimize_fulltext_only" - // InnodbStatusOutputLocks is the name for 'innodb_status_output_locks' system variable. - InnodbStatusOutputLocks = "innodb_status_output_locks" - // InnodbBufferPoolDumpNow is the name for 'innodb_buffer_pool_dump_now' system variable. - InnodbBufferPoolDumpNow = "innodb_buffer_pool_dump_now" - // InnodbBufferPoolLoadNow is the name for 'innodb_buffer_pool_load_now' system variable. - InnodbBufferPoolLoadNow = "innodb_buffer_pool_load_now" - // InnodbStatsOnMetadata is the name for 'innodb_stats_on_metadata' system variable. - InnodbStatsOnMetadata = "innodb_stats_on_metadata" - // InnodbDisableSortFileCache is the name for 'innodb_disable_sort_file_cache' system variable. - InnodbDisableSortFileCache = "innodb_disable_sort_file_cache" - // InnodbStatsAutoRecalc is the name for 'innodb_stats_auto_recalc' system variable. - InnodbStatsAutoRecalc = "innodb_stats_auto_recalc" - // InnodbBufferPoolLoadAbort is the name for 'innodb_buffer_pool_load_abort' system variable. - InnodbBufferPoolLoadAbort = "innodb_buffer_pool_load_abort" - // InnodbStatsPersistent is the name for 'innodb_stats_persistent' system variable. - InnodbStatsPersistent = "innodb_stats_persistent" - // InnodbRandomReadAhead is the name for 'innodb_random_read_ahead' system variable. - InnodbRandomReadAhead = "innodb_random_read_ahead" - // InnodbAdaptiveFlushing is the name for 'innodb_adaptive_flushing' system variable. - InnodbAdaptiveFlushing = "innodb_adaptive_flushing" - // InnodbTableLocks is the name for 'innodb_table_locks' system variable. - InnodbTableLocks = "innodb_table_locks" - // InnodbStatusOutput is the name for 'innodb_status_output' system variable. - InnodbStatusOutput = "innodb_status_output" - // NetBufferLength is the name for 'net_buffer_length' system variable. - NetBufferLength = "net_buffer_length" - // TxReadOnly is the name of 'tx_read_only' system variable. - TxReadOnly = "tx_read_only" - // TransactionReadOnly is the name of 'transaction_read_only' system variable. - TransactionReadOnly = "transaction_read_only" - // CharacterSetServer is the name of 'character_set_server' system variable. - CharacterSetServer = "character_set_server" - // AutoIncrementIncrement is the name of 'auto_increment_increment' system variable. - AutoIncrementIncrement = "auto_increment_increment" - // AutoIncrementOffset is the name of 'auto_increment_offset' system variable. - AutoIncrementOffset = "auto_increment_offset" - // InitConnect is the name of 'init_connect' system variable. - InitConnect = "init_connect" - // CollationServer is the name of 'collation_server' variable. - CollationServer = "collation_server" - // DefaultCollationForUTF8MB4 is the name of 'default_collation_for_utf8mb4' variable. - DefaultCollationForUTF8MB4 = "default_collation_for_utf8mb4" - // NetWriteTimeout is the name of 'net_write_timeout' variable. - NetWriteTimeout = "net_write_timeout" - // ThreadPoolSize is the name of 'thread_pool_size' variable. - ThreadPoolSize = "thread_pool_size" - // WindowingUseHighPrecision is the name of 'windowing_use_high_precision' system variable. - WindowingUseHighPrecision = "windowing_use_high_precision" - // OptimizerSwitch is the name of 'optimizer_switch' system variable. - OptimizerSwitch = "optimizer_switch" - // SystemTimeZone is the name of 'system_time_zone' system variable. - SystemTimeZone = "system_time_zone" - // CTEMaxRecursionDepth is the name of 'cte_max_recursion_depth' system variable. - CTEMaxRecursionDepth = "cte_max_recursion_depth" - // SQLModeVar is the name of the 'sql_mode' system variable. - SQLModeVar = "sql_mode" - // CharacterSetResults is the name of the 'character_set_results' system variable. - CharacterSetResults = "character_set_results" - // MaxAllowedPacket is the name of the 'max_allowed_packet' system variable. - MaxAllowedPacket = "max_allowed_packet" - // TimeZone is the name of the 'time_zone' system variable. - TimeZone = "time_zone" - // TxnIsolation is the name of the 'tx_isolation' system variable. - TxnIsolation = "tx_isolation" - // TransactionIsolation is the name of the 'transaction_isolation' system variable. - TransactionIsolation = "transaction_isolation" - // TxnIsolationOneShot is the name of the 'tx_isolation_one_shot' system variable. - TxnIsolationOneShot = "tx_isolation_one_shot" - // MaxExecutionTime is the name of the 'max_execution_time' system variable. - MaxExecutionTime = "max_execution_time" - // TiKVClientReadTimeout is the name of the 'tikv_client_read_timeout' system variable. - TiKVClientReadTimeout = "tikv_client_read_timeout" - // TiDBLoadBindingTimeout is the name of the 'tidb_load_binding_timeout' system variable. - TiDBLoadBindingTimeout = "tidb_load_binding_timeout" - // ReadOnly is the name of the 'read_only' system variable. - ReadOnly = "read_only" - // DefaultAuthPlugin is the name of 'default_authentication_plugin' system variable. - DefaultAuthPlugin = "default_authentication_plugin" - // LastInsertID is the name of 'last_insert_id' system variable. - LastInsertID = "last_insert_id" - // Identity is the name of 'identity' system variable. - Identity = "identity" - // TiDBAllowFunctionForExpressionIndex is the name of `TiDBAllowFunctionForExpressionIndex` system variable. - TiDBAllowFunctionForExpressionIndex = "tidb_allow_function_for_expression_index" - // RandSeed1 is the name of 'rand_seed1' system variable. - RandSeed1 = "rand_seed1" - // RandSeed2 is the name of 'rand_seed2' system variable. - RandSeed2 = "rand_seed2" - // SQLRequirePrimaryKey is the name of `sql_require_primary_key` system variable. - SQLRequirePrimaryKey = "sql_require_primary_key" - // ValidatePasswordEnable turns on/off the validation of password. - ValidatePasswordEnable = "validate_password.enable" - // ValidatePasswordPolicy specifies the password policy enforced by validate_password. - ValidatePasswordPolicy = "validate_password.policy" - // ValidatePasswordCheckUserName controls whether validate_password compares passwords to the user name part of - // the effective user account for the current session - ValidatePasswordCheckUserName = "validate_password.check_user_name" - // ValidatePasswordLength specified the minimum number of characters that validate_password requires passwords to have - ValidatePasswordLength = "validate_password.length" - // ValidatePasswordMixedCaseCount specified the minimum number of lowercase and uppercase characters that validate_password requires - ValidatePasswordMixedCaseCount = "validate_password.mixed_case_count" - // ValidatePasswordNumberCount specified the minimum number of numeric (digit) characters that validate_password requires - ValidatePasswordNumberCount = "validate_password.number_count" - // ValidatePasswordSpecialCharCount specified the minimum number of nonalphanumeric characters that validate_password requires - ValidatePasswordSpecialCharCount = "validate_password.special_char_count" - // ValidatePasswordDictionary specified the dictionary that validate_password uses for checking passwords. Each word is separated by semicolon (;). - ValidatePasswordDictionary = "validate_password.dictionary" -) diff --git a/pkg/sessionctx/variable/sysvar_test.go b/pkg/sessionctx/variable/sysvar_test.go index 5dd0b43367f6c..ddc4867ddb65b 100644 --- a/pkg/sessionctx/variable/sysvar_test.go +++ b/pkg/sessionctx/variable/sysvar_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util/gctuner" "github.com/pingcap/tidb/pkg/util/memory" "github.com/pingcap/tidb/pkg/util/timeutil" @@ -38,13 +39,13 @@ import ( ) func TestSQLSelectLimit(t *testing.T) { - sv := GetSysVar(SQLSelectLimit) + sv := GetSysVar(vardef.SQLSelectLimit) vars := NewSessionVars(nil) - val, err := sv.Validate(vars, "-10", ScopeSession) + val, err := sv.Validate(vars, "-10", vardef.ScopeSession) require.NoError(t, err) // it has autoconvert out of range. require.Equal(t, "0", val) - val, err = sv.Validate(vars, "9999", ScopeSession) + val, err = sv.Validate(vars, "9999", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "9999", val) @@ -53,16 +54,16 @@ func TestSQLSelectLimit(t *testing.T) { } func TestSQLModeVar(t *testing.T) { - sv := GetSysVar(SQLModeVar) + sv := GetSysVar(vardef.SQLModeVar) vars := NewSessionVars(nil) - val, err := sv.Validate(vars, "strict_trans_tabLES ", ScopeSession) + val, err := sv.Validate(vars, "strict_trans_tabLES ", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "STRICT_TRANS_TABLES", val) - _, err = sv.Validate(vars, "strict_trans_tabLES,nonsense_option", ScopeSession) + _, err = sv.Validate(vars, "strict_trans_tabLES,nonsense_option", vardef.ScopeSession) require.Equal(t, "ERROR 1231 (42000): Variable 'sql_mode' can't be set to the value of 'NONSENSE_OPTION'", err.Error()) - val, err = sv.Validate(vars, "ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", ScopeSession) + val, err = sv.Validate(vars, "ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", val) @@ -74,7 +75,7 @@ func TestSQLModeVar(t *testing.T) { require.Equal(t, sqlMode, vars.SQLMode) // Set it to non strict. - val, err = sv.Validate(vars, "ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", ScopeSession) + val, err = sv.Validate(vars, "ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", val) @@ -86,14 +87,14 @@ func TestSQLModeVar(t *testing.T) { } func TestMaxExecutionTime(t *testing.T) { - sv := GetSysVar(MaxExecutionTime) + sv := GetSysVar(vardef.MaxExecutionTime) vars := NewSessionVars(nil) - val, err := sv.Validate(vars, "-10", ScopeSession) + val, err := sv.Validate(vars, "-10", vardef.ScopeSession) require.NoError(t, err) // it has autoconvert out of range. require.Equal(t, "0", val) - val, err = sv.Validate(vars, "99999", ScopeSession) + val, err = sv.Validate(vars, "99999", vardef.ScopeSession) require.NoError(t, err) // it has autoconvert out of range. require.Equal(t, "99999", val) @@ -102,21 +103,21 @@ func TestMaxExecutionTime(t *testing.T) { } func TestTiFlashMaxBytes(t *testing.T) { - varNames := []string{TiDBMaxBytesBeforeTiFlashExternalJoin, TiDBMaxBytesBeforeTiFlashExternalGroupBy, TiDBMaxBytesBeforeTiFlashExternalSort} + varNames := []string{vardef.TiDBMaxBytesBeforeTiFlashExternalJoin, vardef.TiDBMaxBytesBeforeTiFlashExternalGroupBy, vardef.TiDBMaxBytesBeforeTiFlashExternalSort} for index, varName := range varNames { sv := GetSysVar(varName) vars := NewSessionVars(nil) - val, err := sv.Validate(vars, "-10", ScopeSession) + val, err := sv.Validate(vars, "-10", vardef.ScopeSession) require.NoError(t, err) // it has autoconvert out of range. require.Equal(t, "-1", val) - val, err = sv.Validate(vars, "-10", ScopeGlobal) + val, err = sv.Validate(vars, "-10", vardef.ScopeGlobal) require.NoError(t, err) // it has autoconvert out of range. require.Equal(t, "-1", val) - val, err = sv.Validate(vars, "100", ScopeSession) + val, err = sv.Validate(vars, "100", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "100", val) - _, err = sv.Validate(vars, strconv.FormatUint(uint64(math.MaxInt64)+1, 10), ScopeSession) + _, err = sv.Validate(vars, strconv.FormatUint(uint64(math.MaxInt64)+1, 10), vardef.ScopeSession) // can not autoconvert because the input is out of the range of Int64 require.Error(t, err) @@ -134,18 +135,18 @@ func TestTiFlashMaxBytes(t *testing.T) { func TestTiFlashMemQuotaQueryPerNode(t *testing.T) { // test TiFlash query memory threshold - sv := GetSysVar(TiFlashMemQuotaQueryPerNode) + sv := GetSysVar(vardef.TiFlashMemQuotaQueryPerNode) vars := NewSessionVars(nil) - val, err := sv.Validate(vars, "-10", ScopeSession) + val, err := sv.Validate(vars, "-10", vardef.ScopeSession) require.NoError(t, err) // it has been auto converted if out of range require.Equal(t, "-1", val) - val, err = sv.Validate(vars, "-10", ScopeGlobal) + val, err = sv.Validate(vars, "-10", vardef.ScopeGlobal) require.NoError(t, err) // it has been auto converted if out of range require.Equal(t, "-1", val) - val, err = sv.Validate(vars, "100", ScopeSession) + val, err = sv.Validate(vars, "100", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "100", val) - _, err = sv.Validate(vars, strconv.FormatUint(uint64(math.MaxInt64)+1, 10), ScopeSession) + _, err = sv.Validate(vars, strconv.FormatUint(uint64(math.MaxInt64)+1, 10), vardef.ScopeSession) // can not autoconvert because the input is out of the range of Int64 require.Error(t, err) require.Nil(t, sv.SetSessionFromHook(vars, "10000")) // sets @@ -154,19 +155,19 @@ func TestTiFlashMemQuotaQueryPerNode(t *testing.T) { func TestTiFlashQuerySpillRatio(t *testing.T) { // test TiFlash auto spill ratio - sv := GetSysVar(TiFlashQuerySpillRatio) + sv := GetSysVar(vardef.TiFlashQuerySpillRatio) vars := NewSessionVars(nil) - val, err := sv.Validate(vars, "-10", ScopeSession) + val, err := sv.Validate(vars, "-10", vardef.ScopeSession) require.NoError(t, err) // it has been auto converted if out of range require.Equal(t, "0", val) - val, err = sv.Validate(vars, "-10", ScopeGlobal) + val, err = sv.Validate(vars, "-10", vardef.ScopeGlobal) require.NoError(t, err) // it has been auto converted if out of range require.Equal(t, "0", val) - _, err = sv.Validate(vars, "100", ScopeSession) + _, err = sv.Validate(vars, "100", vardef.ScopeSession) require.Error(t, err) - _, err = sv.Validate(vars, "0.9", ScopeSession) + _, err = sv.Validate(vars, "0.9", vardef.ScopeSession) require.Error(t, err) - val, err = sv.Validate(vars, "0.85", ScopeSession) + val, err = sv.Validate(vars, "0.85", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "0.85", val) require.Nil(t, sv.SetSessionFromHook(vars, "0.75")) // sets @@ -174,50 +175,50 @@ func TestTiFlashQuerySpillRatio(t *testing.T) { } func TestCollationServer(t *testing.T) { - sv := GetSysVar(CollationServer) + sv := GetSysVar(vardef.CollationServer) vars := NewSessionVars(nil) - val, err := sv.Validate(vars, "LATIN1_bin", ScopeSession) + val, err := sv.Validate(vars, "LATIN1_bin", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "latin1_bin", val) // test normalization - _, err = sv.Validate(vars, "BOGUSCOLLation", ScopeSession) + _, err = sv.Validate(vars, "BOGUSCOLLation", vardef.ScopeSession) require.Equal(t, "[ddl:1273]Unknown collation: 'BOGUSCOLLation'", err.Error()) require.Nil(t, sv.SetSessionFromHook(vars, "latin1_bin")) - require.Equal(t, "latin1", vars.systems[CharacterSetServer]) // check it also changes charset. + require.Equal(t, "latin1", vars.systems[vardef.CharacterSetServer]) // check it also changes charset. require.Nil(t, sv.SetSessionFromHook(vars, "utf8mb4_bin")) - require.Equal(t, "utf8mb4", vars.systems[CharacterSetServer]) // check it also changes charset. + require.Equal(t, "utf8mb4", vars.systems[vardef.CharacterSetServer]) // check it also changes charset. } func TestDefaultCollationForUTF8MB4(t *testing.T) { - sv := GetSysVar(DefaultCollationForUTF8MB4) + sv := GetSysVar(vardef.DefaultCollationForUTF8MB4) vars := NewSessionVars(nil) // test normalization - val, err := sv.Validate(vars, "utf8mb4_BIN", ScopeSession) + val, err := sv.Validate(vars, "utf8mb4_BIN", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "utf8mb4_bin", val) warn := vars.StmtCtx.GetWarnings()[0].Err require.Equal(t, "[variable:1681]Updating 'default_collation_for_utf8mb4' is deprecated. It will be made read-only in a future release.", warn.Error()) - val, err = sv.Validate(vars, "utf8mb4_GENeral_CI", ScopeGlobal) + val, err = sv.Validate(vars, "utf8mb4_GENeral_CI", vardef.ScopeGlobal) require.NoError(t, err) require.Equal(t, "utf8mb4_general_ci", val) warn = vars.StmtCtx.GetWarnings()[0].Err require.Equal(t, "[variable:1681]Updating 'default_collation_for_utf8mb4' is deprecated. It will be made read-only in a future release.", warn.Error()) - val, err = sv.Validate(vars, "utf8mb4_0900_AI_CI", ScopeSession) + val, err = sv.Validate(vars, "utf8mb4_0900_AI_CI", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "utf8mb4_0900_ai_ci", val) warn = vars.StmtCtx.GetWarnings()[0].Err require.Equal(t, "[variable:1681]Updating 'default_collation_for_utf8mb4' is deprecated. It will be made read-only in a future release.", warn.Error()) // test set variable failed - _, err = sv.Validate(vars, "LATIN1_bin", ScopeSession) + _, err = sv.Validate(vars, "LATIN1_bin", vardef.ScopeSession) require.EqualError(t, err, ErrInvalidDefaultUTF8MB4Collation.GenWithStackByArgs("latin1_bin").Error()) } func TestTimeZone(t *testing.T) { - sv := GetSysVar(TimeZone) + sv := GetSysVar(vardef.TimeZone) vars := NewSessionVars(nil) // TiDB uses the Golang TZ library, so TZs are case-sensitive. @@ -226,19 +227,19 @@ func TestTimeZone(t *testing.T) { // val, err := sv.Validate(vars, "America/EDMONTON", ScopeSession) // See: https://github.com/pingcap/tidb/issues/8087 - val, err := sv.Validate(vars, "America/Edmonton", ScopeSession) + val, err := sv.Validate(vars, "America/Edmonton", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "America/Edmonton", val) - val, err = sv.Validate(vars, "+10:00", ScopeSession) + val, err = sv.Validate(vars, "+10:00", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "+10:00", val) - val, err = sv.Validate(vars, "UTC", ScopeSession) + val, err = sv.Validate(vars, "UTC", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "UTC", val) - val, err = sv.Validate(vars, "+00:00", ScopeSession) + val, err = sv.Validate(vars, "+00:00", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "+00:00", val) @@ -249,59 +250,59 @@ func TestTimeZone(t *testing.T) { } func TestTxnIsolation(t *testing.T) { - sv := GetSysVar(TxnIsolation) + sv := GetSysVar(vardef.TxnIsolation) vars := NewSessionVars(nil) - _, err := sv.Validate(vars, "on", ScopeSession) + _, err := sv.Validate(vars, "on", vardef.ScopeSession) require.Equal(t, "[variable:1231]Variable 'tx_isolation' can't be set to the value of 'on'", err.Error()) - val, err := sv.Validate(vars, "read-COMMitted", ScopeSession) + val, err := sv.Validate(vars, "read-COMMitted", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "READ-COMMITTED", val) - _, err = sv.Validate(vars, "Serializable", ScopeSession) + _, err = sv.Validate(vars, "Serializable", vardef.ScopeSession) require.Equal(t, "[variable:8048]The isolation level 'SERIALIZABLE' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error", err.Error()) - _, err = sv.Validate(vars, "read-uncommitted", ScopeSession) + _, err = sv.Validate(vars, "read-uncommitted", vardef.ScopeSession) require.Equal(t, "[variable:8048]The isolation level 'READ-UNCOMMITTED' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error", err.Error()) // Enable global skip isolation check doesn't affect current session - require.Nil(t, GetSysVar(TiDBSkipIsolationLevelCheck).SetGlobalFromHook(context.Background(), vars, "ON", true)) - _, err = sv.Validate(vars, "Serializable", ScopeSession) + require.Nil(t, GetSysVar(vardef.TiDBSkipIsolationLevelCheck).SetGlobalFromHook(context.Background(), vars, "ON", true)) + _, err = sv.Validate(vars, "Serializable", vardef.ScopeSession) require.Equal(t, "[variable:8048]The isolation level 'SERIALIZABLE' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error", err.Error()) // Enable session skip isolation check - require.Nil(t, GetSysVar(TiDBSkipIsolationLevelCheck).SetSessionFromHook(vars, "ON")) + require.Nil(t, GetSysVar(vardef.TiDBSkipIsolationLevelCheck).SetSessionFromHook(vars, "ON")) - val, err = sv.Validate(vars, "Serializable", ScopeSession) + val, err = sv.Validate(vars, "Serializable", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "SERIALIZABLE", val) // Init TiDBSkipIsolationLevelCheck like what loadCommonGlobalVariables does vars = NewSessionVars(nil) - require.NoError(t, vars.SetSystemVarWithRelaxedValidation(TiDBSkipIsolationLevelCheck, "1")) - val, err = sv.Validate(vars, "Serializable", ScopeSession) + require.NoError(t, vars.SetSystemVarWithRelaxedValidation(vardef.TiDBSkipIsolationLevelCheck, "1")) + val, err = sv.Validate(vars, "Serializable", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "SERIALIZABLE", val) } func TestTiDBMultiStatementMode(t *testing.T) { - sv := GetSysVar(TiDBMultiStatementMode) + sv := GetSysVar(vardef.TiDBMultiStatementMode) vars := NewSessionVars(nil) - val, err := sv.Validate(vars, "on", ScopeSession) + val, err := sv.Validate(vars, "on", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "ON", val) require.Nil(t, sv.SetSessionFromHook(vars, val)) require.Equal(t, 1, vars.MultiStatementMode) - val, err = sv.Validate(vars, "0", ScopeSession) + val, err = sv.Validate(vars, "0", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "OFF", val) require.Nil(t, sv.SetSessionFromHook(vars, val)) require.Equal(t, 0, vars.MultiStatementMode) - val, err = sv.Validate(vars, "Warn", ScopeSession) + val, err = sv.Validate(vars, "Warn", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "WARN", val) require.Nil(t, sv.SetSessionFromHook(vars, val)) @@ -313,220 +314,220 @@ func TestReadOnlyNoop(t *testing.T) { mock := NewMockGlobalAccessor4Tests() mock.SessionVars = vars vars.GlobalVarsAccessor = mock - noopFuncs := GetSysVar(TiDBEnableNoopFuncs) + noopFuncs := GetSysVar(vardef.TiDBEnableNoopFuncs) // For session scope - for _, name := range []string{TxReadOnly, TransactionReadOnly} { + for _, name := range []string{vardef.TxReadOnly, vardef.TransactionReadOnly} { sv := GetSysVar(name) - val, err := sv.Validate(vars, "on", ScopeSession) + val, err := sv.Validate(vars, "on", vardef.ScopeSession) require.Equal(t, "[variable:1235]function READ ONLY has only noop implementation in tidb now, use tidb_enable_noop_functions to enable these functions", err.Error()) require.Equal(t, "OFF", val) require.NoError(t, noopFuncs.SetSessionFromHook(vars, "ON")) - _, err = sv.Validate(vars, "on", ScopeSession) + _, err = sv.Validate(vars, "on", vardef.ScopeSession) require.NoError(t, err) require.NoError(t, noopFuncs.SetSessionFromHook(vars, "OFF")) // restore default. } // For global scope - for _, name := range []string{TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, ReadOnly} { + for _, name := range []string{vardef.TxReadOnly, vardef.TransactionReadOnly, vardef.OfflineMode, vardef.SuperReadOnly, vardef.ReadOnly} { sv := GetSysVar(name) - val, err := sv.Validate(vars, "on", ScopeGlobal) - if name == OfflineMode { + val, err := sv.Validate(vars, "on", vardef.ScopeGlobal) + if name == vardef.OfflineMode { require.Equal(t, "[variable:1235]function OFFLINE MODE has only noop implementation in tidb now, use tidb_enable_noop_functions to enable these functions", err.Error()) } else { require.Equal(t, "[variable:1235]function READ ONLY has only noop implementation in tidb now, use tidb_enable_noop_functions to enable these functions", err.Error()) } require.Equal(t, "OFF", val) - require.NoError(t, vars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), TiDBEnableNoopFuncs, "ON")) - _, err = sv.Validate(vars, "on", ScopeGlobal) + require.NoError(t, vars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.TiDBEnableNoopFuncs, "ON")) + _, err = sv.Validate(vars, "on", vardef.ScopeGlobal) require.NoError(t, err) - require.NoError(t, vars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), TiDBEnableNoopFuncs, "OFF")) + require.NoError(t, vars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.TiDBEnableNoopFuncs, "OFF")) } } func TestSkipInit(t *testing.T) { - sv := SysVar{Scope: ScopeGlobal, Name: "skipinit1", Value: On, Type: TypeBool} + sv := SysVar{Scope: vardef.ScopeGlobal, Name: "skipinit1", Value: vardef.On, Type: vardef.TypeBool} require.True(t, sv.SkipInit()) - sv = SysVar{Scope: ScopeGlobal | ScopeSession, Name: "skipinit1", Value: On, Type: TypeBool} + sv = SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "skipinit1", Value: vardef.On, Type: vardef.TypeBool} require.False(t, sv.SkipInit()) - sv = SysVar{Scope: ScopeSession, Name: "skipinit1", Value: On, Type: TypeBool} + sv = SysVar{Scope: vardef.ScopeSession, Name: "skipinit1", Value: vardef.On, Type: vardef.TypeBool} require.False(t, sv.SkipInit()) - sv = SysVar{Scope: ScopeSession, Name: "skipinit1", Value: On, Type: TypeBool, skipInit: true} + sv = SysVar{Scope: vardef.ScopeSession, Name: "skipinit1", Value: vardef.On, Type: vardef.TypeBool, skipInit: true} require.True(t, sv.SkipInit()) } func TestSessionGetterFuncs(t *testing.T) { vars := NewSessionVars(nil) - val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBCurrentTS) + val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBCurrentTS) require.NoError(t, err) require.Equal(t, fmt.Sprintf("%d", vars.TxnCtx.StartTS), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBLastTxnInfo) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBLastTxnInfo) require.NoError(t, err) require.Equal(t, vars.LastTxnInfo, val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBLastQueryInfo) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBLastQueryInfo) require.NoError(t, err) info, err := json.Marshal(vars.LastQueryInfo) require.NoError(t, err) require.Equal(t, string(info), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBFoundInPlanCache) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBFoundInPlanCache) require.NoError(t, err) require.Equal(t, BoolToOnOff(vars.PrevFoundInPlanCache), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBFoundInBinding) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBFoundInBinding) require.NoError(t, err) require.Equal(t, BoolToOnOff(vars.PrevFoundInBinding), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBTxnScope) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBTxnScope) require.NoError(t, err) require.Equal(t, vars.TxnScope.GetVarValue(), val) } func TestInstanceScopedVars(t *testing.T) { vars := NewSessionVars(nil) - val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBGeneralLog) + val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBGeneralLog) require.NoError(t, err) - require.Equal(t, BoolToOnOff(ProcessGeneralLog.Load()), val) + require.Equal(t, BoolToOnOff(vardef.ProcessGeneralLog.Load()), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBPProfSQLCPU) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBPProfSQLCPU) require.NoError(t, err) expected := "0" - if EnablePProfSQLCPU.Load() { + if vardef.EnablePProfSQLCPU.Load() { expected = "1" } require.Equal(t, expected, val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBExpensiveQueryTimeThreshold) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBExpensiveQueryTimeThreshold) require.NoError(t, err) - require.Equal(t, fmt.Sprintf("%d", atomic.LoadUint64(&ExpensiveQueryTimeThreshold)), val) + require.Equal(t, fmt.Sprintf("%d", atomic.LoadUint64(&vardef.ExpensiveQueryTimeThreshold)), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBExpensiveTxnTimeThreshold) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBExpensiveTxnTimeThreshold) require.NoError(t, err) - require.Equal(t, fmt.Sprintf("%d", atomic.LoadUint64(&ExpensiveTxnTimeThreshold)), val) + require.Equal(t, fmt.Sprintf("%d", atomic.LoadUint64(&vardef.ExpensiveTxnTimeThreshold)), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBMemoryUsageAlarmRatio) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBMemoryUsageAlarmRatio) require.NoError(t, err) - require.Equal(t, fmt.Sprintf("%g", MemoryUsageAlarmRatio.Load()), val) + require.Equal(t, fmt.Sprintf("%g", vardef.MemoryUsageAlarmRatio.Load()), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBMemoryUsageAlarmKeepRecordNum) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBMemoryUsageAlarmKeepRecordNum) require.NoError(t, err) - require.Equal(t, fmt.Sprintf("%d", MemoryUsageAlarmKeepRecordNum.Load()), val) + require.Equal(t, fmt.Sprintf("%d", vardef.MemoryUsageAlarmKeepRecordNum.Load()), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBForcePriority) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBForcePriority) require.NoError(t, err) - require.Equal(t, mysql.Priority2Str[mysql.PriorityEnum(atomic.LoadInt32(&ForcePriority))], val) + require.Equal(t, mysql.Priority2Str[mysql.PriorityEnum(atomic.LoadInt32(&vardef.ForcePriority))], val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBDDLSlowOprThreshold) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBDDLSlowOprThreshold) require.NoError(t, err) - require.Equal(t, strconv.FormatUint(uint64(atomic.LoadUint32(&DDLSlowOprThreshold)), 10), val) + require.Equal(t, strconv.FormatUint(uint64(atomic.LoadUint32(&vardef.DDLSlowOprThreshold)), 10), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), PluginDir) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.PluginDir) require.NoError(t, err) require.Equal(t, config.GetGlobalConfig().Instance.PluginDir, val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), PluginLoad) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.PluginLoad) require.NoError(t, err) require.Equal(t, config.GetGlobalConfig().Instance.PluginLoad, val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBSlowLogThreshold) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBSlowLogThreshold) require.NoError(t, err) require.Equal(t, strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Instance.SlowThreshold), 10), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBRecordPlanInSlowLog) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBRecordPlanInSlowLog) require.NoError(t, err) enabled := atomic.LoadUint32(&config.GetGlobalConfig().Instance.RecordPlanInSlowLog) == 1 require.Equal(t, BoolToOnOff(enabled), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBEnableSlowLog) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBEnableSlowLog) require.NoError(t, err) require.Equal(t, BoolToOnOff(config.GetGlobalConfig().Instance.EnableSlowLog.Load()), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBCheckMb4ValueInUTF8) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBCheckMb4ValueInUTF8) require.NoError(t, err) require.Equal(t, BoolToOnOff(config.GetGlobalConfig().Instance.CheckMb4ValueInUTF8.Load()), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBEnableCollectExecutionInfo) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBEnableCollectExecutionInfo) require.NoError(t, err) require.Equal(t, BoolToOnOff(config.GetGlobalConfig().Instance.EnableCollectExecutionInfo.Load()), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBConfig) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBConfig) require.NoError(t, err) expected, err = config.GetJSONConfig() require.NoError(t, err) require.Equal(t, expected, val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBLogFileMaxDays) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBLogFileMaxDays) require.NoError(t, err) - require.Equal(t, fmt.Sprint(GlobalLogMaxDays.Load()), val) + require.Equal(t, fmt.Sprint(vardef.GlobalLogMaxDays.Load()), val) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBRCReadCheckTS) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBRCReadCheckTS) require.NoError(t, err) - require.Equal(t, BoolToOnOff(EnableRCReadCheckTS.Load()), val) + require.Equal(t, BoolToOnOff(vardef.EnableRCReadCheckTS.Load()), val) } func TestSecureAuth(t *testing.T) { - sv := GetSysVar(SecureAuth) + sv := GetSysVar(vardef.SecureAuth) vars := NewSessionVars(nil) - _, err := sv.Validate(vars, "OFF", ScopeGlobal) + _, err := sv.Validate(vars, "OFF", vardef.ScopeGlobal) require.Equal(t, "[variable:1231]Variable 'secure_auth' can't be set to the value of 'OFF'", err.Error()) - val, err := sv.Validate(vars, "ON", ScopeGlobal) + val, err := sv.Validate(vars, "ON", vardef.ScopeGlobal) require.NoError(t, err) require.Equal(t, "ON", val) } func TestTiDBReplicaRead(t *testing.T) { - sv := GetSysVar(TiDBReplicaRead) + sv := GetSysVar(vardef.TiDBReplicaRead) vars := NewSessionVars(nil) - val, err := sv.Validate(vars, "follower", ScopeGlobal) + val, err := sv.Validate(vars, "follower", vardef.ScopeGlobal) require.Equal(t, val, "follower") require.NoError(t, err) } func TestSQLAutoIsNull(t *testing.T) { - svSQL, svNoop := GetSysVar(SQLAutoIsNull), GetSysVar(TiDBEnableNoopFuncs) + svSQL, svNoop := GetSysVar(vardef.SQLAutoIsNull), GetSysVar(vardef.TiDBEnableNoopFuncs) vars := NewSessionVars(nil) vars.GlobalVarsAccessor = NewMockGlobalAccessor4Tests() - _, err := svSQL.Validate(vars, "ON", ScopeSession) + _, err := svSQL.Validate(vars, "ON", vardef.ScopeSession) require.True(t, terror.ErrorEqual(err, ErrFunctionsNoopImpl)) // change tidb_enable_noop_functions to 1, it will success require.NoError(t, svNoop.SetSessionFromHook(vars, "ON")) - _, err = svSQL.Validate(vars, "ON", ScopeSession) + _, err = svSQL.Validate(vars, "ON", vardef.ScopeSession) require.NoError(t, err) require.NoError(t, svSQL.SetSessionFromHook(vars, "ON")) - res, ok := vars.GetSystemVar(SQLAutoIsNull) + res, ok := vars.GetSystemVar(vardef.SQLAutoIsNull) require.True(t, ok) require.Equal(t, "ON", res) // restore tidb_enable_noop_functions to 0 failed, as sql_auto_is_null is 1 - _, err = svNoop.Validate(vars, "OFF", ScopeSession) + _, err = svNoop.Validate(vars, "OFF", vardef.ScopeSession) require.True(t, terror.ErrorEqual(err, errValueNotSupportedWhen)) // after set sql_auto_is_null to 0, restore success require.NoError(t, svSQL.SetSessionFromHook(vars, "OFF")) require.NoError(t, svNoop.SetSessionFromHook(vars, "OFF")) // Only test validate as MockGlobalAccessor do not support SetGlobalSysVar - _, err = svSQL.Validate(vars, "ON", ScopeGlobal) + _, err = svSQL.Validate(vars, "ON", vardef.ScopeGlobal) require.True(t, terror.ErrorEqual(err, ErrFunctionsNoopImpl)) } func TestLastInsertID(t *testing.T) { vars := NewSessionVars(nil) - val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), LastInsertID) + val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.LastInsertID) require.NoError(t, err) require.Equal(t, val, "0") vars.StmtCtx.PrevLastInsertID = 21 - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), LastInsertID) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.LastInsertID) require.NoError(t, err) require.Equal(t, val, "21") vars.StmtCtx.PrevLastInsertID = 9223372036854775809 - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), LastInsertID) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.LastInsertID) require.NoError(t, err) require.Equal(t, val, "9223372036854775809") @@ -539,48 +540,48 @@ func TestLastInsertID(t *testing.T) { func TestTimestamp(t *testing.T) { vars := NewSessionVars(nil) - val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), Timestamp) + val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.Timestamp) require.NoError(t, err) require.NotEqual(t, "", val) - vars.systems[Timestamp] = "10" - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), Timestamp) + vars.systems[vardef.Timestamp] = "10" + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.Timestamp) require.NoError(t, err) require.Equal(t, "10", val) - vars.systems[Timestamp] = "0" // set to default - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), Timestamp) + vars.systems[vardef.Timestamp] = "0" // set to default + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.Timestamp) require.NoError(t, err) require.NotEqual(t, "", val) require.NotEqual(t, "10", val) // Test validating a value that less than the minimum one. - sv := GetSysVar(Timestamp) - _, err = sv.Validate(vars, "-5", ScopeSession) + sv := GetSysVar(vardef.Timestamp) + _, err = sv.Validate(vars, "-5", vardef.ScopeSession) require.NoError(t, err) warn := vars.StmtCtx.GetWarnings()[0].Err require.Equal(t, "[variable:1292]Truncated incorrect timestamp value: '-5'", warn.Error()) // Test validating values that larger than the maximum one. - _, err = sv.Validate(vars, "3147483698", ScopeSession) + _, err = sv.Validate(vars, "3147483698", vardef.ScopeSession) require.Equal(t, "[variable:1231]Variable 'timestamp' can't be set to the value of '3147483698'", err.Error()) - _, err = sv.Validate(vars, "2147483648", ScopeSession) + _, err = sv.Validate(vars, "2147483648", vardef.ScopeSession) require.Equal(t, "[variable:1231]Variable 'timestamp' can't be set to the value of '2147483648'", err.Error()) // Test validating the maximum value. - _, err = sv.Validate(vars, "2147483647", ScopeSession) + _, err = sv.Validate(vars, "2147483647", vardef.ScopeSession) require.NoError(t, err) } func TestIdentity(t *testing.T) { vars := NewSessionVars(nil) - val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), Identity) + val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.Identity) require.NoError(t, err) require.Equal(t, val, "0") vars.StmtCtx.PrevLastInsertID = 21 - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), Identity) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.Identity) require.NoError(t, err) require.Equal(t, val, "21") } @@ -589,7 +590,7 @@ func TestLcTimeNamesReadOnly(t *testing.T) { sv := GetSysVar("lc_time_names") vars := NewSessionVars(nil) vars.GlobalVarsAccessor = NewMockGlobalAccessor4Tests() - _, err := sv.Validate(vars, "newvalue", ScopeGlobal) + _, err := sv.Validate(vars, "newvalue", vardef.ScopeGlobal) require.Error(t, err) } @@ -597,7 +598,7 @@ func TestLcMessages(t *testing.T) { sv := GetSysVar("lc_messages") vars := NewSessionVars(nil) vars.GlobalVarsAccessor = NewMockGlobalAccessor4Tests() - _, err := sv.Validate(vars, "zh_CN", ScopeGlobal) + _, err := sv.Validate(vars, "zh_CN", vardef.ScopeGlobal) require.NoError(t, err) err = sv.SetSessionFromHook(vars, "zh_CN") require.NoError(t, err) @@ -607,37 +608,37 @@ func TestLcMessages(t *testing.T) { } func TestDDLWorkers(t *testing.T) { - svWorkerCount, svBatchSize := GetSysVar(TiDBDDLReorgWorkerCount), GetSysVar(TiDBDDLReorgBatchSize) + svWorkerCount, svBatchSize := GetSysVar(vardef.TiDBDDLReorgWorkerCount), GetSysVar(vardef.TiDBDDLReorgBatchSize) vars := NewSessionVars(nil) vars.GlobalVarsAccessor = NewMockGlobalAccessor4Tests() - val, err := svWorkerCount.Validate(vars, "-100", ScopeGlobal) + val, err := svWorkerCount.Validate(vars, "-100", vardef.ScopeGlobal) require.NoError(t, err) require.Equal(t, val, "1") // converts it to min value - val, err = svWorkerCount.Validate(vars, "1234", ScopeGlobal) + val, err = svWorkerCount.Validate(vars, "1234", vardef.ScopeGlobal) require.NoError(t, err) require.Equal(t, val, "256") // converts it to max value - val, err = svWorkerCount.Validate(vars, "100", ScopeGlobal) + val, err = svWorkerCount.Validate(vars, "100", vardef.ScopeGlobal) require.NoError(t, err) require.Equal(t, val, "100") // unchanged - val, err = svBatchSize.Validate(vars, "10", ScopeGlobal) + val, err = svBatchSize.Validate(vars, "10", vardef.ScopeGlobal) require.NoError(t, err) - require.Equal(t, val, fmt.Sprint(MinDDLReorgBatchSize)) // converts it to min value - val, err = svBatchSize.Validate(vars, "999999", ScopeGlobal) + require.Equal(t, val, fmt.Sprint(vardef.MinDDLReorgBatchSize)) // converts it to min value + val, err = svBatchSize.Validate(vars, "999999", vardef.ScopeGlobal) require.NoError(t, err) - require.Equal(t, val, fmt.Sprint(MaxDDLReorgBatchSize)) // converts it to max value - val, err = svBatchSize.Validate(vars, "100", ScopeGlobal) + require.Equal(t, val, fmt.Sprint(vardef.MaxDDLReorgBatchSize)) // converts it to max value + val, err = svBatchSize.Validate(vars, "100", vardef.ScopeGlobal) require.NoError(t, err) require.Equal(t, val, "100") // unchanged } func TestDefaultCharsetAndCollation(t *testing.T) { vars := NewSessionVars(nil) - val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), CharacterSetConnection) + val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.CharacterSetConnection) require.NoError(t, err) require.Equal(t, val, mysql.DefaultCharset) - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), CollationConnection) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.CollationConnection) require.NoError(t, err) require.Equal(t, val, mysql.DefaultCollationName) } @@ -645,49 +646,49 @@ func TestDefaultCharsetAndCollation(t *testing.T) { func TestIndexMergeSwitcher(t *testing.T) { vars := NewSessionVars(nil) vars.GlobalVarsAccessor = NewMockGlobalAccessor4Tests() - val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBEnableIndexMerge) + val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBEnableIndexMerge) require.NoError(t, err) - require.Equal(t, DefTiDBEnableIndexMerge, true) - require.Equal(t, BoolToOnOff(DefTiDBEnableIndexMerge), val) + require.Equal(t, vardef.DefTiDBEnableIndexMerge, true) + require.Equal(t, BoolToOnOff(vardef.DefTiDBEnableIndexMerge), val) } func TestNetBufferLength(t *testing.T) { - netBufferLength := GetSysVar(NetBufferLength) + netBufferLength := GetSysVar(vardef.NetBufferLength) vars := NewSessionVars(nil) vars.GlobalVarsAccessor = NewMockGlobalAccessor4Tests() - val, err := netBufferLength.Validate(vars, "1", ScopeGlobal) + val, err := netBufferLength.Validate(vars, "1", vardef.ScopeGlobal) require.NoError(t, err) require.Equal(t, "1024", val) // converts it to min value - val, err = netBufferLength.Validate(vars, "10485760", ScopeGlobal) + val, err = netBufferLength.Validate(vars, "10485760", vardef.ScopeGlobal) require.NoError(t, err) require.Equal(t, "1048576", val) // converts it to max value - val, err = netBufferLength.Validate(vars, "524288", ScopeGlobal) + val, err = netBufferLength.Validate(vars, "524288", vardef.ScopeGlobal) require.NoError(t, err) require.Equal(t, "524288", val) // unchanged } func TestTiDBBatchPendingTiFlashCount(t *testing.T) { - sv := GetSysVar(TiDBBatchPendingTiFlashCount) + sv := GetSysVar(vardef.TiDBBatchPendingTiFlashCount) vars := NewSessionVars(nil) - val, err := sv.Validate(vars, "-10", ScopeSession) + val, err := sv.Validate(vars, "-10", vardef.ScopeSession) require.NoError(t, err) // it has autoconvert out of range. require.Equal(t, "0", val) - val, err = sv.Validate(vars, "9999", ScopeSession) + val, err = sv.Validate(vars, "9999", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "9999", val) - _, err = sv.Validate(vars, "1.5", ScopeSession) + _, err = sv.Validate(vars, "1.5", vardef.ScopeSession) require.Error(t, err) require.EqualError(t, err, "[variable:1232]Incorrect argument type to variable 'tidb_batch_pending_tiflash_count'") } func TestTiDBMemQuotaQuery(t *testing.T) { - sv := GetSysVar(TiDBMemQuotaQuery) + sv := GetSysVar(vardef.TiDBMemQuotaQuery) vars := NewSessionVars(nil) - for _, scope := range []ScopeFlag{ScopeGlobal, ScopeSession} { + for _, scope := range []vardef.ScopeFlag{vardef.ScopeGlobal, vardef.ScopeSession} { newVal := 32 * 1024 * 1024 val, err := sv.Validate(vars, fmt.Sprintf("%d", newVal), scope) require.Equal(t, val, "33554432") @@ -704,18 +705,18 @@ func TestTiDBMemQuotaQuery(t *testing.T) { } func TestTiDBQueryLogMaxLen(t *testing.T) { - sv := GetSysVar(TiDBQueryLogMaxLen) + sv := GetSysVar(vardef.TiDBQueryLogMaxLen) vars := NewSessionVars(nil) newVal := 32 * 1024 * 1024 - val, err := sv.Validate(vars, fmt.Sprintf("%d", newVal), ScopeGlobal) + val, err := sv.Validate(vars, fmt.Sprintf("%d", newVal), vardef.ScopeGlobal) require.Equal(t, val, "33554432") require.NoError(t, err) // out of range newVal = 1073741825 expected := 1073741824 - val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), ScopeGlobal) + val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), vardef.ScopeGlobal) // expected to truncate require.Equal(t, val, fmt.Sprintf("%d", expected)) require.NoError(t, err) @@ -723,25 +724,25 @@ func TestTiDBQueryLogMaxLen(t *testing.T) { // min value out of range newVal = -2 expected = 0 - val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), ScopeGlobal) + val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), vardef.ScopeGlobal) // expected to set to min value require.Equal(t, val, fmt.Sprintf("%d", expected)) require.NoError(t, err) } func TestTiDBCommitterConcurrency(t *testing.T) { - sv := GetSysVar(TiDBCommitterConcurrency) + sv := GetSysVar(vardef.TiDBCommitterConcurrency) vars := NewSessionVars(nil) newVal := 1024 - val, err := sv.Validate(vars, fmt.Sprintf("%d", newVal), ScopeGlobal) + val, err := sv.Validate(vars, fmt.Sprintf("%d", newVal), vardef.ScopeGlobal) require.Equal(t, val, "1024") require.NoError(t, err) // out of range newVal = 10001 expected := 10000 - val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), ScopeGlobal) + val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), vardef.ScopeGlobal) // expected to truncate require.Equal(t, val, fmt.Sprintf("%d", expected)) require.NoError(t, err) @@ -749,25 +750,25 @@ func TestTiDBCommitterConcurrency(t *testing.T) { // min value out of range newVal = 0 expected = 1 - val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), ScopeGlobal) + val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), vardef.ScopeGlobal) // expected to set to min value require.Equal(t, val, fmt.Sprintf("%d", expected)) require.NoError(t, err) } func TestTiDBDDLFlashbackConcurrency(t *testing.T) { - sv := GetSysVar(TiDBDDLFlashbackConcurrency) + sv := GetSysVar(vardef.TiDBDDLFlashbackConcurrency) vars := NewSessionVars(nil) newVal := 128 - val, err := sv.Validate(vars, fmt.Sprintf("%d", newVal), ScopeGlobal) + val, err := sv.Validate(vars, fmt.Sprintf("%d", newVal), vardef.ScopeGlobal) require.Equal(t, val, "128") require.NoError(t, err) // out of range - newVal = MaxConfigurableConcurrency + 1 - expected := MaxConfigurableConcurrency - val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), ScopeGlobal) + newVal = vardef.MaxConfigurableConcurrency + 1 + expected := vardef.MaxConfigurableConcurrency + val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), vardef.ScopeGlobal) // expected to truncate require.Equal(t, val, fmt.Sprintf("%d", expected)) require.NoError(t, err) @@ -775,7 +776,7 @@ func TestTiDBDDLFlashbackConcurrency(t *testing.T) { // min value out of range newVal = 0 expected = 1 - val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), ScopeGlobal) + val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), vardef.ScopeGlobal) // expected to set to min value require.Equal(t, val, fmt.Sprintf("%d", expected)) require.NoError(t, err) @@ -783,10 +784,10 @@ func TestTiDBDDLFlashbackConcurrency(t *testing.T) { func TestDefaultMemoryDebugModeValue(t *testing.T) { vars := NewSessionVars(nil) - val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBMemoryDebugModeMinHeapInUse) + val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBMemoryDebugModeMinHeapInUse) require.NoError(t, err) require.Equal(t, val, "0") - val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBMemoryDebugModeAlarmRatio) + val, err = vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBMemoryDebugModeAlarmRatio) require.NoError(t, err) require.Equal(t, val, "0") } @@ -798,18 +799,18 @@ func TestSetTIDBDistributeReorg(t *testing.T) { vars.GlobalVarsAccessor = mock // Set to on - err := mock.SetGlobalSysVar(context.Background(), TiDBEnableDistTask, On) + err := mock.SetGlobalSysVar(context.Background(), vardef.TiDBEnableDistTask, vardef.On) require.NoError(t, err) - val, err := mock.GetGlobalSysVar(TiDBEnableDistTask) + val, err := mock.GetGlobalSysVar(vardef.TiDBEnableDistTask) require.NoError(t, err) - require.Equal(t, On, val) + require.Equal(t, vardef.On, val) // Set to off - err = mock.SetGlobalSysVar(context.Background(), TiDBEnableDistTask, Off) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBEnableDistTask, vardef.Off) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBEnableDistTask) + val, err = mock.GetGlobalSysVar(vardef.TiDBEnableDistTask) require.NoError(t, err) - require.Equal(t, Off, val) + require.Equal(t, vardef.Off, val) } func TestDefaultPartitionPruneMode(t *testing.T) { @@ -817,10 +818,10 @@ func TestDefaultPartitionPruneMode(t *testing.T) { mock := NewMockGlobalAccessor4Tests() mock.SessionVars = vars vars.GlobalVarsAccessor = mock - val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), TiDBPartitionPruneMode) + val, err := vars.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBPartitionPruneMode) require.NoError(t, err) require.Equal(t, "dynamic", val) - require.Equal(t, "dynamic", DefTiDBPartitionPruneMode) + require.Equal(t, "dynamic", vardef.DefTiDBPartitionPruneMode) } func TestSetTIDBFastDDL(t *testing.T) { @@ -828,24 +829,24 @@ func TestSetTIDBFastDDL(t *testing.T) { mock := NewMockGlobalAccessor4Tests() mock.SessionVars = vars vars.GlobalVarsAccessor = mock - fastDDL := GetSysVar(TiDBDDLEnableFastReorg) + fastDDL := GetSysVar(vardef.TiDBDDLEnableFastReorg) // Default true - require.Equal(t, fastDDL.Value, On) + require.Equal(t, fastDDL.Value, vardef.On) // Set to On - err := mock.SetGlobalSysVar(context.Background(), TiDBDDLEnableFastReorg, On) + err := mock.SetGlobalSysVar(context.Background(), vardef.TiDBDDLEnableFastReorg, vardef.On) require.NoError(t, err) - val, err1 := mock.GetGlobalSysVar(TiDBDDLEnableFastReorg) + val, err1 := mock.GetGlobalSysVar(vardef.TiDBDDLEnableFastReorg) require.NoError(t, err1) - require.Equal(t, On, val) + require.Equal(t, vardef.On, val) // Set to off - err = mock.SetGlobalSysVar(context.Background(), TiDBDDLEnableFastReorg, Off) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBDDLEnableFastReorg, vardef.Off) require.NoError(t, err) - val, err1 = mock.GetGlobalSysVar(TiDBDDLEnableFastReorg) + val, err1 = mock.GetGlobalSysVar(vardef.TiDBDDLEnableFastReorg) require.NoError(t, err1) - require.Equal(t, Off, val) + require.Equal(t, vardef.Off, val) } func TestSetTIDBDiskQuota(t *testing.T) { @@ -853,7 +854,7 @@ func TestSetTIDBDiskQuota(t *testing.T) { mock := NewMockGlobalAccessor4Tests() mock.SessionVars = vars vars.GlobalVarsAccessor = mock - diskQuota := GetSysVar(TiDBDDLDiskQuota) + diskQuota := GetSysVar(vardef.TiDBDDLDiskQuota) var ( gb int64 = 1024 * 1024 * 1024 pb int64 = 1024 * 1024 * 1024 * 1024 * 1024 @@ -864,37 +865,37 @@ func TestSetTIDBDiskQuota(t *testing.T) { require.Equal(t, diskQuota.Value, strconv.FormatInt(100*gb, 10)) // MinValue is 100 GB, set to 50 Gb is not allowed - err = mock.SetGlobalSysVar(context.Background(), TiDBDDLDiskQuota, strconv.FormatInt(50*gb, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBDDLDiskQuota, strconv.FormatInt(50*gb, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBDDLDiskQuota) + val, err = mock.GetGlobalSysVar(vardef.TiDBDDLDiskQuota) require.NoError(t, err) require.Equal(t, strconv.FormatInt(100*gb, 10), val) // Set to 100 GB - err = mock.SetGlobalSysVar(context.Background(), TiDBDDLDiskQuota, strconv.FormatInt(100*gb, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBDDLDiskQuota, strconv.FormatInt(100*gb, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBDDLDiskQuota) + val, err = mock.GetGlobalSysVar(vardef.TiDBDDLDiskQuota) require.NoError(t, err) require.Equal(t, strconv.FormatInt(100*gb, 10), val) // Set to 200 GB - err = mock.SetGlobalSysVar(context.Background(), TiDBDDLDiskQuota, strconv.FormatInt(200*gb, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBDDLDiskQuota, strconv.FormatInt(200*gb, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBDDLDiskQuota) + val, err = mock.GetGlobalSysVar(vardef.TiDBDDLDiskQuota) require.NoError(t, err) require.Equal(t, strconv.FormatInt(200*gb, 10), val) // Set to 1 Pb - err = mock.SetGlobalSysVar(context.Background(), TiDBDDLDiskQuota, strconv.FormatInt(pb, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBDDLDiskQuota, strconv.FormatInt(pb, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBDDLDiskQuota) + val, err = mock.GetGlobalSysVar(vardef.TiDBDDLDiskQuota) require.NoError(t, err) require.Equal(t, strconv.FormatInt(pb, 10), val) // MaxValue is 1 PB, set to 2 Pb is not allowed, it will set back to 1 PB max allowed value. - err = mock.SetGlobalSysVar(context.Background(), TiDBDDLDiskQuota, strconv.FormatInt(2*pb, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBDDLDiskQuota, strconv.FormatInt(2*pb, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBDDLDiskQuota) + val, err = mock.GetGlobalSysVar(vardef.TiDBDDLDiskQuota) require.NoError(t, err) require.Equal(t, strconv.FormatInt(pb, 10), val) } @@ -910,68 +911,68 @@ func TestTiDBServerMemoryLimit(t *testing.T) { val string ) // Test tidb_server_memory_limit - serverMemoryLimit := GetSysVar(TiDBServerMemoryLimit) + serverMemoryLimit := GetSysVar(vardef.TiDBServerMemoryLimit) // Check default value - require.Equal(t, serverMemoryLimit.Value, DefTiDBServerMemoryLimit) + require.Equal(t, serverMemoryLimit.Value, vardef.DefTiDBServerMemoryLimit) // MinValue is 512 MB - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, strconv.FormatUint(100*mb, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, strconv.FormatUint(100*mb, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) require.Equal(t, "512MB", val) // Test Close - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, strconv.FormatUint(0, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, strconv.FormatUint(0, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) require.Equal(t, "0", val) // Test MaxValue - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, strconv.FormatUint(math.MaxUint64, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, strconv.FormatUint(math.MaxUint64, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) require.Equal(t, strconv.FormatUint(math.MaxUint64, 10), val) // Test Normal Value - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, strconv.FormatUint(1024*mb, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, strconv.FormatUint(1024*mb, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) require.Equal(t, strconv.FormatUint(1024*mb, 10), val) // Test tidb_server_memory_limit_sess_min_size - serverMemoryLimitSessMinSize := GetSysVar(TiDBServerMemoryLimitSessMinSize) + serverMemoryLimitSessMinSize := GetSysVar(vardef.TiDBServerMemoryLimitSessMinSize) // Check default value - require.Equal(t, serverMemoryLimitSessMinSize.Value, strconv.FormatUint(DefTiDBServerMemoryLimitSessMinSize, 10)) + require.Equal(t, serverMemoryLimitSessMinSize.Value, strconv.FormatUint(vardef.DefTiDBServerMemoryLimitSessMinSize, 10)) // MinValue is 128 Bytes - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitSessMinSize, strconv.FormatUint(100, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitSessMinSize, strconv.FormatUint(100, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimitSessMinSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimitSessMinSize) require.NoError(t, err) require.Equal(t, strconv.FormatUint(128, 10), val) // Test Close - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitSessMinSize, strconv.FormatUint(0, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitSessMinSize, strconv.FormatUint(0, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimitSessMinSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimitSessMinSize) require.NoError(t, err) require.Equal(t, strconv.FormatUint(0, 10), val) // Test MaxValue - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitSessMinSize, strconv.FormatUint(math.MaxUint64, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitSessMinSize, strconv.FormatUint(math.MaxUint64, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimitSessMinSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimitSessMinSize) require.NoError(t, err) require.Equal(t, strconv.FormatUint(math.MaxUint64, 10), val) // Test Normal Value - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitSessMinSize, strconv.FormatUint(200*mb, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitSessMinSize, strconv.FormatUint(200*mb, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimitSessMinSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimitSessMinSize) require.NoError(t, err) require.Equal(t, strconv.FormatUint(200*mb, 10), val) } @@ -986,17 +987,17 @@ func TestTiDBServerMemoryLimit2(t *testing.T) { val string ) // Test tidb_server_memory_limit - serverMemoryLimit := GetSysVar(TiDBServerMemoryLimit) + serverMemoryLimit := GetSysVar(vardef.TiDBServerMemoryLimit) // Check default value - require.Equal(t, serverMemoryLimit.Value, DefTiDBServerMemoryLimit) + require.Equal(t, serverMemoryLimit.Value, vardef.DefTiDBServerMemoryLimit) total := memory.GetMemTotalIgnoreErr() if total > 0 { // Can use percentage format when TiDB can obtain physical memory // Test Percentage Format - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "1%") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "1%") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) if total/100 > uint64(512<<20) { require.Equal(t, memory.ServerMemoryLimit.Load(), total/100) @@ -1006,86 +1007,86 @@ func TestTiDBServerMemoryLimit2(t *testing.T) { require.Equal(t, "512MB", val) } - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "0%") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "0%") require.Error(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "100%") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "100%") require.Error(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "75%") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "75%") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) require.Equal(t, "75%", val) require.Equal(t, memory.ServerMemoryLimit.Load(), total/100*75) } // Test can't obtain physical memory require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/pkg/util/memory/GetMemTotalError", `return(true)`)) - require.Error(t, mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "75%")) + require.Error(t, mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "75%")) require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/pkg/util/memory/GetMemTotalError")) // Test byteSize format - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "1234") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "1234") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) require.Equal(t, memory.ServerMemoryLimit.Load(), uint64(512<<20)) require.Equal(t, "512MB", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "1234567890123") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "1234567890123") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) require.Equal(t, memory.ServerMemoryLimit.Load(), uint64(1234567890123)) require.Equal(t, "1234567890123", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "10KB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "10KB") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) require.Equal(t, memory.ServerMemoryLimit.Load(), uint64(512<<20)) require.Equal(t, "512MB", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "12345678KB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "12345678KB") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) require.Equal(t, memory.ServerMemoryLimit.Load(), uint64(12345678<<10)) require.Equal(t, "12345678KB", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "10MB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "10MB") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) require.Equal(t, memory.ServerMemoryLimit.Load(), uint64(512<<20)) require.Equal(t, "512MB", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "700MB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "700MB") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) require.Equal(t, memory.ServerMemoryLimit.Load(), uint64(700<<20)) require.Equal(t, "700MB", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "20GB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "20GB") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) require.Equal(t, memory.ServerMemoryLimit.Load(), uint64(20<<30)) require.Equal(t, "20GB", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "2TB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "2TB") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimit) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimit) require.NoError(t, err) require.Equal(t, memory.ServerMemoryLimit.Load(), uint64(2<<40)) require.Equal(t, "2TB", val) // Test error - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "123aaa123") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "123aaa123") require.Error(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "700MBaa") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "700MBaa") require.Error(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimit, "a700MB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimit, "a700MB") require.Error(t, err) } @@ -1100,27 +1101,27 @@ func TestTiDBServerMemoryLimitSessMinSize(t *testing.T) { val string ) - serverMemroyLimitSessMinSize := GetSysVar(TiDBServerMemoryLimitSessMinSize) + serverMemroyLimitSessMinSize := GetSysVar(vardef.TiDBServerMemoryLimitSessMinSize) // Check default value - require.Equal(t, serverMemroyLimitSessMinSize.Value, strconv.FormatInt(DefTiDBServerMemoryLimitSessMinSize, 10)) + require.Equal(t, serverMemroyLimitSessMinSize.Value, strconv.FormatInt(vardef.DefTiDBServerMemoryLimitSessMinSize, 10)) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitSessMinSize, "123456") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitSessMinSize, "123456") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimitSessMinSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimitSessMinSize) require.NoError(t, err) require.Equal(t, memory.ServerMemoryLimitSessMinSize.Load(), uint64(123456)) require.Equal(t, "123456", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitSessMinSize, "100") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitSessMinSize, "100") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimitSessMinSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimitSessMinSize) require.NoError(t, err) require.Equal(t, memory.ServerMemoryLimitSessMinSize.Load(), uint64(128)) require.Equal(t, "128", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitSessMinSize, "123MB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitSessMinSize, "123MB") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimitSessMinSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimitSessMinSize) require.NoError(t, err) require.Equal(t, memory.ServerMemoryLimitSessMinSize.Load(), uint64(123<<20)) require.Equal(t, "128974848", val) @@ -1137,51 +1138,51 @@ func TestTiDBServerMemoryLimitGCTrigger(t *testing.T) { val string ) - serverMemroyLimitGCTrigger := GetSysVar(TiDBServerMemoryLimitGCTrigger) + serverMemroyLimitGCTrigger := GetSysVar(vardef.TiDBServerMemoryLimitGCTrigger) // Check default value - require.Equal(t, serverMemroyLimitGCTrigger.Value, strconv.FormatFloat(DefTiDBServerMemoryLimitGCTrigger, 'f', -1, 64)) + require.Equal(t, serverMemroyLimitGCTrigger.Value, strconv.FormatFloat(vardef.DefTiDBServerMemoryLimitGCTrigger, 'f', -1, 64)) defer func() { - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitGCTrigger, strconv.FormatFloat(DefTiDBServerMemoryLimitGCTrigger, 'f', -1, 64)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitGCTrigger, strconv.FormatFloat(vardef.DefTiDBServerMemoryLimitGCTrigger, 'f', -1, 64)) require.NoError(t, err) }() - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitGCTrigger, "0.8") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitGCTrigger, "0.8") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimitGCTrigger) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimitGCTrigger) require.NoError(t, err) require.Equal(t, gctuner.GlobalMemoryLimitTuner.GetPercentage(), 0.8) require.Equal(t, "0.8", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitGCTrigger, "90%") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitGCTrigger, "90%") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBServerMemoryLimitGCTrigger) + val, err = mock.GetGlobalSysVar(vardef.TiDBServerMemoryLimitGCTrigger) require.NoError(t, err) require.Equal(t, gctuner.GlobalMemoryLimitTuner.GetPercentage(), 0.9) require.Equal(t, "0.9", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitGCTrigger, "100%") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitGCTrigger, "100%") require.Error(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitGCTrigger, "101%") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitGCTrigger, "101%") require.Error(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitGCTrigger, "99%") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitGCTrigger, "99%") require.NoError(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBGOGCTunerThreshold, "0.4") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBGOGCTunerThreshold, "0.4") require.NoError(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitGCTrigger, "49%") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitGCTrigger, "49%") require.Error(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBServerMemoryLimitGCTrigger, "51%") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBServerMemoryLimitGCTrigger, "51%") require.NoError(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBGOGCTunerMaxValue, "50") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBGOGCTunerMaxValue, "50") require.Error(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBGOGCTunerMinValue, "200") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBGOGCTunerMinValue, "200") require.NoError(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBGOGCTunerMinValue, "1000") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBGOGCTunerMinValue, "1000") require.Error(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBGOGCTunerMinValue, "100") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBGOGCTunerMinValue, "100") require.NoError(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBGOGCTunerMaxValue, "200") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBGOGCTunerMaxValue, "200") require.NoError(t, err) } @@ -1191,12 +1192,12 @@ func TestSetAggPushDownGlobally(t *testing.T) { mock.SessionVars = vars vars.GlobalVarsAccessor = mock - val, err := mock.GetGlobalSysVar(TiDBOptAggPushDown) + val, err := mock.GetGlobalSysVar(vardef.TiDBOptAggPushDown) require.NoError(t, err) require.Equal(t, "OFF", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBOptAggPushDown, "ON") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBOptAggPushDown, "ON") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBOptAggPushDown) + val, err = mock.GetGlobalSysVar(vardef.TiDBOptAggPushDown) require.NoError(t, err) require.Equal(t, "ON", val) } @@ -1207,12 +1208,12 @@ func TestSetDeriveTopNGlobally(t *testing.T) { mock.SessionVars = vars vars.GlobalVarsAccessor = mock - val, err := mock.GetGlobalSysVar(TiDBOptDeriveTopN) + val, err := mock.GetGlobalSysVar(vardef.TiDBOptDeriveTopN) require.NoError(t, err) require.Equal(t, "OFF", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBOptDeriveTopN, "ON") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBOptDeriveTopN, "ON") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBOptDeriveTopN) + val, err = mock.GetGlobalSysVar(vardef.TiDBOptDeriveTopN) require.NoError(t, err) require.Equal(t, "ON", val) } @@ -1224,35 +1225,35 @@ func TestSetJobScheduleWindow(t *testing.T) { vars.GlobalVarsAccessor = mock // default value - val, err := mock.GetGlobalSysVar(TiDBTTLJobScheduleWindowStartTime) + val, err := mock.GetGlobalSysVar(vardef.TiDBTTLJobScheduleWindowStartTime) require.NoError(t, err) require.Equal(t, "00:00 +0000", val) // set and get variable in UTC vars.TimeZone = time.UTC - err = mock.SetGlobalSysVar(context.Background(), TiDBTTLJobScheduleWindowStartTime, "16:11") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBTTLJobScheduleWindowStartTime, "16:11") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBTTLJobScheduleWindowStartTime) + val, err = mock.GetGlobalSysVar(vardef.TiDBTTLJobScheduleWindowStartTime) require.NoError(t, err) require.Equal(t, "16:11 +0000", val) // set variable in UTC, get it in Asia/Shanghai vars.TimeZone = time.UTC - err = mock.SetGlobalSysVar(context.Background(), TiDBTTLJobScheduleWindowStartTime, "16:11") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBTTLJobScheduleWindowStartTime, "16:11") require.NoError(t, err) vars.TimeZone, err = time.LoadLocation("Asia/Shanghai") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBTTLJobScheduleWindowStartTime) + val, err = mock.GetGlobalSysVar(vardef.TiDBTTLJobScheduleWindowStartTime) require.NoError(t, err) require.Equal(t, "16:11 +0000", val) // set variable in Asia/Shanghai, get it it UTC vars.TimeZone, err = time.LoadLocation("Asia/Shanghai") require.NoError(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBTTLJobScheduleWindowStartTime, "16:11") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBTTLJobScheduleWindowStartTime, "16:11") require.NoError(t, err) vars.TimeZone = time.UTC - val, err = mock.GetGlobalSysVar(TiDBTTLJobScheduleWindowStartTime) + val, err = mock.GetGlobalSysVar(vardef.TiDBTTLJobScheduleWindowStartTime) require.NoError(t, err) require.Equal(t, "16:11 +0800", val) } @@ -1262,15 +1263,15 @@ func TestTiDBIgnoreInlistPlanDigest(t *testing.T) { mock := NewMockGlobalAccessor4Tests() mock.SessionVars = vars vars.GlobalVarsAccessor = mock - initValue, err := mock.GetGlobalSysVar(TiDBIgnoreInlistPlanDigest) + initValue, err := mock.GetGlobalSysVar(vardef.TiDBIgnoreInlistPlanDigest) require.NoError(t, err) - require.Equal(t, initValue, Off) + require.Equal(t, initValue, vardef.Off) // Set to On(init at start) - err1 := mock.SetGlobalSysVar(context.Background(), TiDBIgnoreInlistPlanDigest, On) + err1 := mock.SetGlobalSysVar(context.Background(), vardef.TiDBIgnoreInlistPlanDigest, vardef.On) require.NoError(t, err1) - NewVal, err2 := mock.GetGlobalSysVar(TiDBIgnoreInlistPlanDigest) + NewVal, err2 := mock.GetGlobalSysVar(vardef.TiDBIgnoreInlistPlanDigest) require.NoError(t, err2) - require.Equal(t, NewVal, On) + require.Equal(t, NewVal, vardef.On) } func TestTiDBEnableResourceControl(t *testing.T) { @@ -1289,40 +1290,40 @@ func TestTiDBEnableResourceControl(t *testing.T) { } SetGlobalResourceControl.Store(&setGlobalResourceControlFunc) // Reset the switch. It may be set by other tests. - EnableResourceControl.Store(false) + vardef.EnableResourceControl.Store(false) vars := NewSessionVars(nil) mock := NewMockGlobalAccessor4Tests() mock.SessionVars = vars vars.GlobalVarsAccessor = mock - resourceControlEnabled := GetSysVar(TiDBEnableResourceControl) + resourceControlEnabled := GetSysVar(vardef.TiDBEnableResourceControl) // Default true - require.Equal(t, resourceControlEnabled.Value, On) + require.Equal(t, resourceControlEnabled.Value, vardef.On) require.Equal(t, enable, false) // Set to On(init at start) - err := mock.SetGlobalSysVar(context.Background(), TiDBEnableResourceControl, On) + err := mock.SetGlobalSysVar(context.Background(), vardef.TiDBEnableResourceControl, vardef.On) require.NoError(t, err) - val, err1 := mock.GetGlobalSysVar(TiDBEnableResourceControl) + val, err1 := mock.GetGlobalSysVar(vardef.TiDBEnableResourceControl) require.NoError(t, err1) - require.Equal(t, On, val) + require.Equal(t, vardef.On, val) require.Equal(t, enable, true) // Set to Off - err = mock.SetGlobalSysVar(context.Background(), TiDBEnableResourceControl, Off) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBEnableResourceControl, vardef.Off) require.NoError(t, err) - val, err1 = mock.GetGlobalSysVar(TiDBEnableResourceControl) + val, err1 = mock.GetGlobalSysVar(vardef.TiDBEnableResourceControl) require.NoError(t, err1) - require.Equal(t, Off, val) + require.Equal(t, vardef.Off, val) require.Equal(t, enable, false) // Set to On again - err = mock.SetGlobalSysVar(context.Background(), TiDBEnableResourceControl, On) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBEnableResourceControl, vardef.On) require.NoError(t, err) - val, err1 = mock.GetGlobalSysVar(TiDBEnableResourceControl) + val, err1 = mock.GetGlobalSysVar(vardef.TiDBEnableResourceControl) require.NoError(t, err1) - require.Equal(t, On, val) + require.Equal(t, vardef.On, val) require.Equal(t, enable, true) } @@ -1331,25 +1332,25 @@ func TestTiDBResourceControlStrictMode(t *testing.T) { mock := NewMockGlobalAccessor4Tests() mock.SessionVars = vars vars.GlobalVarsAccessor = mock - resourceControlStrictMode := GetSysVar(TiDBResourceControlStrictMode) + resourceControlStrictMode := GetSysVar(vardef.TiDBResourceControlStrictMode) // Default true - require.Equal(t, resourceControlStrictMode.Value, On) - require.Equal(t, EnableResourceControlStrictMode.Load(), true) + require.Equal(t, resourceControlStrictMode.Value, vardef.On) + require.Equal(t, vardef.EnableResourceControlStrictMode.Load(), true) // Set to Off - err := mock.SetGlobalSysVar(context.Background(), TiDBResourceControlStrictMode, Off) + err := mock.SetGlobalSysVar(context.Background(), vardef.TiDBResourceControlStrictMode, vardef.Off) require.NoError(t, err) - val, err1 := mock.GetGlobalSysVar(TiDBResourceControlStrictMode) + val, err1 := mock.GetGlobalSysVar(vardef.TiDBResourceControlStrictMode) require.NoError(t, err1) - require.Equal(t, Off, val) + require.Equal(t, vardef.Off, val) // Set to On again - err = mock.SetGlobalSysVar(context.Background(), TiDBResourceControlStrictMode, On) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBResourceControlStrictMode, vardef.On) require.NoError(t, err) - val, err1 = mock.GetGlobalSysVar(TiDBResourceControlStrictMode) + val, err1 = mock.GetGlobalSysVar(vardef.TiDBResourceControlStrictMode) require.NoError(t, err1) - require.Equal(t, On, val) + require.Equal(t, vardef.On, val) } func TestTiDBEnableRowLevelChecksum(t *testing.T) { @@ -1360,23 +1361,23 @@ func TestTiDBEnableRowLevelChecksum(t *testing.T) { vars.GlobalVarsAccessor = mock // default to false - val, err := mock.GetGlobalSysVar(TiDBEnableRowLevelChecksum) + val, err := mock.GetGlobalSysVar(vardef.TiDBEnableRowLevelChecksum) require.NoError(t, err) - require.Equal(t, Off, val) + require.Equal(t, vardef.Off, val) // enable - err = mock.SetGlobalSysVar(ctx, TiDBEnableRowLevelChecksum, On) + err = mock.SetGlobalSysVar(ctx, vardef.TiDBEnableRowLevelChecksum, vardef.On) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBEnableRowLevelChecksum) + val, err = mock.GetGlobalSysVar(vardef.TiDBEnableRowLevelChecksum) require.NoError(t, err) - require.Equal(t, On, val) + require.Equal(t, vardef.On, val) // disable - err = mock.SetGlobalSysVar(ctx, TiDBEnableRowLevelChecksum, Off) + err = mock.SetGlobalSysVar(ctx, vardef.TiDBEnableRowLevelChecksum, vardef.Off) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBEnableRowLevelChecksum) + val, err = mock.GetGlobalSysVar(vardef.TiDBEnableRowLevelChecksum) require.NoError(t, err) - require.Equal(t, Off, val) + require.Equal(t, vardef.Off, val) } func TestTiDBAutoAnalyzeRatio(t *testing.T) { @@ -1387,49 +1388,49 @@ func TestTiDBAutoAnalyzeRatio(t *testing.T) { vars.GlobalVarsAccessor = mock // default to 0.5 - val, err := mock.GetGlobalSysVar(TiDBAutoAnalyzeRatio) + val, err := mock.GetGlobalSysVar(vardef.TiDBAutoAnalyzeRatio) require.NoError(t, err) require.Equal(t, "0.5", val) // set to 0.1 - err = mock.SetGlobalSysVar(ctx, TiDBAutoAnalyzeRatio, "0.1") + err = mock.SetGlobalSysVar(ctx, vardef.TiDBAutoAnalyzeRatio, "0.1") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBAutoAnalyzeRatio) + val, err = mock.GetGlobalSysVar(vardef.TiDBAutoAnalyzeRatio) require.NoError(t, err) require.Equal(t, "0.1", val) // set to 1.1 - err = mock.SetGlobalSysVar(ctx, TiDBAutoAnalyzeRatio, "1.1") + err = mock.SetGlobalSysVar(ctx, vardef.TiDBAutoAnalyzeRatio, "1.1") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBAutoAnalyzeRatio) + val, err = mock.GetGlobalSysVar(vardef.TiDBAutoAnalyzeRatio) require.NoError(t, err) require.Equal(t, "1.1", val) // set to 0 - err = mock.SetGlobalSysVar(ctx, TiDBAutoAnalyzeRatio, "0") + err = mock.SetGlobalSysVar(ctx, vardef.TiDBAutoAnalyzeRatio, "0") require.Error(t, err) - val, err = mock.GetGlobalSysVar(TiDBAutoAnalyzeRatio) + val, err = mock.GetGlobalSysVar(vardef.TiDBAutoAnalyzeRatio) require.NoError(t, err) require.Equal(t, "1.1", val) // set to 0.0000000001 - err = mock.SetGlobalSysVar(ctx, TiDBAutoAnalyzeRatio, "0.0000000001") + err = mock.SetGlobalSysVar(ctx, vardef.TiDBAutoAnalyzeRatio, "0.0000000001") require.Error(t, err) - val, err = mock.GetGlobalSysVar(TiDBAutoAnalyzeRatio) + val, err = mock.GetGlobalSysVar(vardef.TiDBAutoAnalyzeRatio) require.NoError(t, err) require.Equal(t, "1.1", val) // set to 0.00001 - err = mock.SetGlobalSysVar(ctx, TiDBAutoAnalyzeRatio, "0.00001") + err = mock.SetGlobalSysVar(ctx, vardef.TiDBAutoAnalyzeRatio, "0.00001") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBAutoAnalyzeRatio) + val, err = mock.GetGlobalSysVar(vardef.TiDBAutoAnalyzeRatio) require.NoError(t, err) require.Equal(t, "0.00001", val) // set to 0.000009999 - err = mock.SetGlobalSysVar(ctx, TiDBAutoAnalyzeRatio, "0.000009999") + err = mock.SetGlobalSysVar(ctx, vardef.TiDBAutoAnalyzeRatio, "0.000009999") require.Error(t, err) - val, err = mock.GetGlobalSysVar(TiDBAutoAnalyzeRatio) + val, err = mock.GetGlobalSysVar(vardef.TiDBAutoAnalyzeRatio) require.NoError(t, err) require.Equal(t, "0.00001", val) } @@ -1439,35 +1440,35 @@ func TestTiDBTiFlashReplicaRead(t *testing.T) { mock := NewMockGlobalAccessor4Tests() mock.SessionVars = vars vars.GlobalVarsAccessor = mock - tidbTiFlashReplicaRead := GetSysVar(TiFlashReplicaRead) + tidbTiFlashReplicaRead := GetSysVar(vardef.TiFlashReplicaRead) // Check default value - require.Equal(t, DefTiFlashReplicaRead, tidbTiFlashReplicaRead.Value) + require.Equal(t, vardef.DefTiFlashReplicaRead, tidbTiFlashReplicaRead.Value) - err := mock.SetGlobalSysVar(context.Background(), TiFlashReplicaRead, "all_replicas") + err := mock.SetGlobalSysVar(context.Background(), vardef.TiFlashReplicaRead, "all_replicas") require.NoError(t, err) - val, err := mock.GetGlobalSysVar(TiFlashReplicaRead) + val, err := mock.GetGlobalSysVar(vardef.TiFlashReplicaRead) require.NoError(t, err) require.Equal(t, "all_replicas", val) - err = mock.SetGlobalSysVar(context.Background(), TiFlashReplicaRead, "closest_adaptive") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiFlashReplicaRead, "closest_adaptive") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiFlashReplicaRead) + val, err = mock.GetGlobalSysVar(vardef.TiFlashReplicaRead) require.NoError(t, err) require.Equal(t, "closest_adaptive", val) - err = mock.SetGlobalSysVar(context.Background(), TiFlashReplicaRead, "closest_replicas") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiFlashReplicaRead, "closest_replicas") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiFlashReplicaRead) + val, err = mock.GetGlobalSysVar(vardef.TiFlashReplicaRead) require.NoError(t, err) require.Equal(t, "closest_replicas", val) - err = mock.SetGlobalSysVar(context.Background(), TiFlashReplicaRead, DefTiFlashReplicaRead) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiFlashReplicaRead, vardef.DefTiFlashReplicaRead) require.NoError(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiFlashReplicaRead, "random") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiFlashReplicaRead, "random") require.Error(t, err) - val, err = mock.GetGlobalSysVar(TiFlashReplicaRead) + val, err = mock.GetGlobalSysVar(vardef.TiFlashReplicaRead) require.NoError(t, err) - require.Equal(t, DefTiFlashReplicaRead, val) + require.Equal(t, vardef.DefTiFlashReplicaRead, val) } func TestSetTiDBCloudStorageURI(t *testing.T) { @@ -1475,10 +1476,10 @@ func TestSetTiDBCloudStorageURI(t *testing.T) { mock := NewMockGlobalAccessor4Tests() mock.SessionVars = vars vars.GlobalVarsAccessor = mock - cloudStorageURI := GetSysVar(TiDBCloudStorageURI) - require.Len(t, CloudStorageURI.Load(), 0) + cloudStorageURI := GetSysVar(vardef.TiDBCloudStorageURI) + require.Len(t, vardef.CloudStorageURI.Load(), 0) defer func() { - CloudStorageURI.Store("") + vardef.CloudStorageURI.Store("") }() // Default empty @@ -1488,15 +1489,15 @@ func TestSetTiDBCloudStorageURI(t *testing.T) { defer cancel() // Set to noop noopURI := "noop://blackhole?access-key=hello&secret-access-key=world" - err := mock.SetGlobalSysVar(ctx, TiDBCloudStorageURI, noopURI) + err := mock.SetGlobalSysVar(ctx, vardef.TiDBCloudStorageURI, noopURI) require.NoError(t, err) - val, err1 := mock.SessionVars.GetSessionOrGlobalSystemVar(ctx, TiDBCloudStorageURI) + val, err1 := mock.SessionVars.GetSessionOrGlobalSystemVar(ctx, vardef.TiDBCloudStorageURI) require.NoError(t, err1) require.Equal(t, noopURI, val) - require.Equal(t, noopURI, CloudStorageURI.Load()) + require.Equal(t, noopURI, vardef.CloudStorageURI.Load()) // Set to s3, should fail - err = mock.SetGlobalSysVar(ctx, TiDBCloudStorageURI, "s3://blackhole") + err = mock.SetGlobalSysVar(ctx, vardef.TiDBCloudStorageURI, "s3://blackhole") require.Error(t, err, "unreachable storage URI") s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { @@ -1506,32 +1507,32 @@ func TestSetTiDBCloudStorageURI(t *testing.T) { // Set to s3, should return uri without variable s3URI := "s3://tiflow-test/?access-key=testid&secret-access-key=testkey8&session-token=testtoken&endpoint=" + s.URL - err = mock.SetGlobalSysVar(ctx, TiDBCloudStorageURI, s3URI) + err = mock.SetGlobalSysVar(ctx, vardef.TiDBCloudStorageURI, s3URI) require.NoError(t, err) - val, err1 = mock.SessionVars.GetSessionOrGlobalSystemVar(ctx, TiDBCloudStorageURI) + val, err1 = mock.SessionVars.GetSessionOrGlobalSystemVar(ctx, vardef.TiDBCloudStorageURI) require.NoError(t, err1) require.True(t, strings.HasPrefix(val, "s3://tiflow-test/")) require.Contains(t, val, "access-key=xxxxxx") require.Contains(t, val, "secret-access-key=xxxxxx") require.Contains(t, val, "session-token=xxxxxx") - require.Equal(t, s3URI, CloudStorageURI.Load()) + require.Equal(t, s3URI, vardef.CloudStorageURI.Load()) // ks3 is like s3 ks3URI := "ks3://tiflow-test/?region=test&access-key=testid&secret-access-key=testkey8&session-token=testtoken&endpoint=" + s.URL - err = mock.SetGlobalSysVar(ctx, TiDBCloudStorageURI, ks3URI) + err = mock.SetGlobalSysVar(ctx, vardef.TiDBCloudStorageURI, ks3URI) require.NoError(t, err) - val, err1 = mock.SessionVars.GetSessionOrGlobalSystemVar(ctx, TiDBCloudStorageURI) + val, err1 = mock.SessionVars.GetSessionOrGlobalSystemVar(ctx, vardef.TiDBCloudStorageURI) require.NoError(t, err1) require.True(t, strings.HasPrefix(val, "ks3://tiflow-test/")) require.Contains(t, val, "access-key=xxxxxx") require.Contains(t, val, "secret-access-key=xxxxxx") require.Contains(t, val, "session-token=xxxxxx") - require.Equal(t, ks3URI, CloudStorageURI.Load()) + require.Equal(t, ks3URI, vardef.CloudStorageURI.Load()) // Set to empty, should return no error - err = mock.SetGlobalSysVar(ctx, TiDBCloudStorageURI, "") + err = mock.SetGlobalSysVar(ctx, vardef.TiDBCloudStorageURI, "") require.NoError(t, err) - val, err1 = mock.SessionVars.GetSessionOrGlobalSystemVar(ctx, TiDBCloudStorageURI) + val, err1 = mock.SessionVars.GetSessionOrGlobalSystemVar(ctx, vardef.TiDBCloudStorageURI) require.NoError(t, err1) require.Len(t, val, 0) cancel() @@ -1544,49 +1545,49 @@ func TestGlobalSystemVariableInitialValue(t *testing.T) { initVal string }{ { - TiDBTxnMode, - DefTiDBTxnMode, + vardef.TiDBTxnMode, + vardef.DefTiDBTxnMode, "pessimistic", }, { - TiDBEnableAsyncCommit, - BoolToOnOff(DefTiDBEnableAsyncCommit), - BoolToOnOff(DefTiDBEnableAsyncCommit), + vardef.TiDBEnableAsyncCommit, + BoolToOnOff(vardef.DefTiDBEnableAsyncCommit), + BoolToOnOff(vardef.DefTiDBEnableAsyncCommit), }, { - TiDBEnable1PC, - BoolToOnOff(DefTiDBEnable1PC), - BoolToOnOff(DefTiDBEnable1PC), + vardef.TiDBEnable1PC, + BoolToOnOff(vardef.DefTiDBEnable1PC), + BoolToOnOff(vardef.DefTiDBEnable1PC), }, { - TiDBMemOOMAction, - DefTiDBMemOOMAction, - OOMActionLog, + vardef.TiDBMemOOMAction, + vardef.DefTiDBMemOOMAction, + vardef.OOMActionLog, }, { - TiDBEnableAutoAnalyze, - BoolToOnOff(DefTiDBEnableAutoAnalyze), - Off, + vardef.TiDBEnableAutoAnalyze, + BoolToOnOff(vardef.DefTiDBEnableAutoAnalyze), + vardef.Off, }, { - TiDBRowFormatVersion, - strconv.Itoa(DefTiDBRowFormatV1), - strconv.Itoa(DefTiDBRowFormatV2), + vardef.TiDBRowFormatVersion, + strconv.Itoa(vardef.DefTiDBRowFormatV1), + strconv.Itoa(vardef.DefTiDBRowFormatV2), }, { - TiDBTxnAssertionLevel, - DefTiDBTxnAssertionLevel, - AssertionFastStr, + vardef.TiDBTxnAssertionLevel, + vardef.DefTiDBTxnAssertionLevel, + vardef.AssertionFastStr, }, { - TiDBEnableMutationChecker, - BoolToOnOff(DefTiDBEnableMutationChecker), - On, + vardef.TiDBEnableMutationChecker, + BoolToOnOff(vardef.DefTiDBEnableMutationChecker), + vardef.On, }, { - TiDBPessimisticTransactionFairLocking, - BoolToOnOff(DefTiDBPessimisticTransactionFairLocking), - On, + vardef.TiDBPessimisticTransactionFairLocking, + BoolToOnOff(vardef.DefTiDBPessimisticTransactionFairLocking), + vardef.On, }, } for _, v := range vars { @@ -1596,10 +1597,10 @@ func TestGlobalSystemVariableInitialValue(t *testing.T) { } func TestTiDBOptTxnAutoRetry(t *testing.T) { - sv := GetSysVar(TiDBDisableTxnAutoRetry) + sv := GetSysVar(vardef.TiDBDisableTxnAutoRetry) vars := NewSessionVars(nil) - for _, scope := range []ScopeFlag{ScopeSession, ScopeGlobal} { + for _, scope := range []vardef.ScopeFlag{vardef.ScopeSession, vardef.ScopeGlobal} { val, err := sv.Validate(vars, "OFF", scope) require.NoError(t, err) require.Equal(t, "ON", val) @@ -1609,25 +1610,25 @@ func TestTiDBOptTxnAutoRetry(t *testing.T) { } func TestTiDBLowResTSOUpdateInterval(t *testing.T) { - sv := GetSysVar(TiDBLowResolutionTSOUpdateInterval) + sv := GetSysVar(vardef.TiDBLowResolutionTSOUpdateInterval) vars := NewSessionVars(nil) // Too low, will get raised to the min value - val, err := sv.Validate(vars, "0", ScopeGlobal) + val, err := sv.Validate(vars, "0", vardef.ScopeGlobal) require.NoError(t, err) - require.Equal(t, strconv.FormatInt(GetSysVar(TiDBLowResolutionTSOUpdateInterval).MinValue, 10), val) + require.Equal(t, strconv.FormatInt(GetSysVar(vardef.TiDBLowResolutionTSOUpdateInterval).MinValue, 10), val) warn := vars.StmtCtx.GetWarnings()[0].Err require.Equal(t, "[variable:1292]Truncated incorrect tidb_low_resolution_tso_update_interval value: '0'", warn.Error()) // Too high, will get lowered to the max value - val, err = sv.Validate(vars, "100000", ScopeGlobal) + val, err = sv.Validate(vars, "100000", vardef.ScopeGlobal) require.NoError(t, err) - require.Equal(t, strconv.FormatUint(GetSysVar(TiDBLowResolutionTSOUpdateInterval).MaxValue, 10), val) + require.Equal(t, strconv.FormatUint(GetSysVar(vardef.TiDBLowResolutionTSOUpdateInterval).MaxValue, 10), val) warn = vars.StmtCtx.GetWarnings()[1].Err require.Equal(t, "[variable:1292]Truncated incorrect tidb_low_resolution_tso_update_interval value: '100000'", warn.Error()) // valid - val, err = sv.Validate(vars, "1000", ScopeGlobal) + val, err = sv.Validate(vars, "1000", vardef.ScopeGlobal) require.NoError(t, err) require.Equal(t, "1000", val) } @@ -1644,133 +1645,133 @@ func TestTiDBSchemaCacheSize(t *testing.T) { maxValue uint64 = math.MaxInt64 ) // Test tidb_schema_cache_size - schemaCacheSize := GetSysVar(TiDBSchemaCacheSize) + schemaCacheSize := GetSysVar(vardef.TiDBSchemaCacheSize) // Check default value - require.Equal(t, schemaCacheSize.Value, strconv.Itoa(DefTiDBSchemaCacheSize)) + require.Equal(t, schemaCacheSize.Value, strconv.Itoa(vardef.DefTiDBSchemaCacheSize)) // MinValue is 64 MB - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, strconv.FormatUint(63*mb, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, strconv.FormatUint(63*mb, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBSchemaCacheSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBSchemaCacheSize) require.NoError(t, err) require.Equal(t, "64MB", val) // MaxValue is 9223372036854775807 - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, strconv.FormatUint(maxValue, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, strconv.FormatUint(maxValue, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBSchemaCacheSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBSchemaCacheSize) require.NoError(t, err) require.Equal(t, strconv.FormatUint(maxValue, 10), val) // test MinValue-1 - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, strconv.FormatUint(64*mb-1, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, strconv.FormatUint(64*mb-1, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBSchemaCacheSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBSchemaCacheSize) require.NoError(t, err) require.Equal(t, "64MB", val) // test MaxValue+1 - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, strconv.FormatUint(maxValue+1, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, strconv.FormatUint(maxValue+1, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBSchemaCacheSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBSchemaCacheSize) require.NoError(t, err) require.Equal(t, strconv.FormatUint(maxValue, 10), val) // Test Normal Value - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, strconv.FormatUint(1024*mb, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, strconv.FormatUint(1024*mb, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBSchemaCacheSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBSchemaCacheSize) require.NoError(t, err) require.Equal(t, strconv.FormatUint(1024*mb, 10), val) // Test Close - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, strconv.FormatUint(0, 10)) + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, strconv.FormatUint(0, 10)) require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBSchemaCacheSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBSchemaCacheSize) require.NoError(t, err) require.Equal(t, "0", val) // Test byteSize format - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, "1234567890123") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, "1234567890123") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBSchemaCacheSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBSchemaCacheSize) require.NoError(t, err) - require.Equal(t, SchemaCacheSize.Load(), uint64(1234567890123)) + require.Equal(t, vardef.SchemaCacheSize.Load(), uint64(1234567890123)) require.Equal(t, "1234567890123", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, "10KB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, "10KB") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBSchemaCacheSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBSchemaCacheSize) require.NoError(t, err) - require.Equal(t, SchemaCacheSize.Load(), uint64(64<<20)) + require.Equal(t, vardef.SchemaCacheSize.Load(), uint64(64<<20)) require.Equal(t, "64MB", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, "12345678KB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, "12345678KB") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBSchemaCacheSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBSchemaCacheSize) require.NoError(t, err) - require.Equal(t, SchemaCacheSize.Load(), uint64(12345678<<10)) + require.Equal(t, vardef.SchemaCacheSize.Load(), uint64(12345678<<10)) require.Equal(t, "12345678KB", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, "700MB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, "700MB") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBSchemaCacheSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBSchemaCacheSize) require.NoError(t, err) - require.Equal(t, SchemaCacheSize.Load(), uint64(700<<20)) + require.Equal(t, vardef.SchemaCacheSize.Load(), uint64(700<<20)) require.Equal(t, "700MB", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, "20GB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, "20GB") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBSchemaCacheSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBSchemaCacheSize) require.NoError(t, err) - require.Equal(t, SchemaCacheSize.Load(), uint64(20<<30)) + require.Equal(t, vardef.SchemaCacheSize.Load(), uint64(20<<30)) require.Equal(t, "20GB", val) - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, "2TB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, "2TB") require.NoError(t, err) - val, err = mock.GetGlobalSysVar(TiDBSchemaCacheSize) + val, err = mock.GetGlobalSysVar(vardef.TiDBSchemaCacheSize) require.NoError(t, err) - require.Equal(t, SchemaCacheSize.Load(), uint64(2<<40)) + require.Equal(t, vardef.SchemaCacheSize.Load(), uint64(2<<40)) require.Equal(t, "2TB", val) // Test error - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, "123aaa123") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, "123aaa123") require.Error(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, "700MBaa") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, "700MBaa") require.Error(t, err) - err = mock.SetGlobalSysVar(context.Background(), TiDBSchemaCacheSize, "a700MB") + err = mock.SetGlobalSysVar(context.Background(), vardef.TiDBSchemaCacheSize, "a700MB") require.Error(t, err) } func TestEnableWindowFunction(t *testing.T) { vars := NewSessionVars(nil) - require.Equal(t, vars.EnableWindowFunction, DefEnableWindowFunction) - require.NoError(t, vars.SetSystemVar(TiDBEnableWindowFunction, "on")) + require.Equal(t, vars.EnableWindowFunction, vardef.DefEnableWindowFunction) + require.NoError(t, vars.SetSystemVar(vardef.TiDBEnableWindowFunction, "on")) require.Equal(t, vars.EnableWindowFunction, true) - require.NoError(t, vars.SetSystemVar(TiDBEnableWindowFunction, "0")) + require.NoError(t, vars.SetSystemVar(vardef.TiDBEnableWindowFunction, "0")) require.Equal(t, vars.EnableWindowFunction, false) - require.NoError(t, vars.SetSystemVar(TiDBEnableWindowFunction, "1")) + require.NoError(t, vars.SetSystemVar(vardef.TiDBEnableWindowFunction, "1")) require.Equal(t, vars.EnableWindowFunction, true) } func TestTiDBHashJoinVersion(t *testing.T) { vars := NewSessionVars(nil) - sv := GetSysVar(TiDBHashJoinVersion) + sv := GetSysVar(vardef.TiDBHashJoinVersion) // set error value - _, err := sv.Validation(vars, "invalid", "invalid", ScopeSession) + _, err := sv.Validation(vars, "invalid", "invalid", vardef.ScopeSession) require.NotNil(t, err) // set valid value - _, err = sv.Validation(vars, "legacy", "legacy", ScopeSession) + _, err = sv.Validation(vars, "legacy", "legacy", vardef.ScopeSession) require.NoError(t, err) - _, err = sv.Validation(vars, "optimized", "optimized", ScopeSession) + _, err = sv.Validation(vars, "optimized", "optimized", vardef.ScopeSession) require.NoError(t, err) - _, err = sv.Validation(vars, "Legacy", "Legacy", ScopeSession) + _, err = sv.Validation(vars, "Legacy", "Legacy", vardef.ScopeSession) require.NoError(t, err) - _, err = sv.Validation(vars, "Optimized", "Optimized", ScopeSession) + _, err = sv.Validation(vars, "Optimized", "Optimized", vardef.ScopeSession) require.NoError(t, err) - _, err = sv.Validation(vars, "LegaCy", "LegaCy", ScopeSession) + _, err = sv.Validation(vars, "LegaCy", "LegaCy", vardef.ScopeSession) require.NoError(t, err) - _, err = sv.Validation(vars, "OptimiZed", "OptimiZed", ScopeSession) + _, err = sv.Validation(vars, "OptimiZed", "OptimiZed", vardef.ScopeSession) require.NoError(t, err) } @@ -1816,13 +1817,13 @@ func TestTiDBAutoAnalyzeConcurrencyValidation(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - RunAutoAnalyze.Store(tt.autoAnalyze) - EnableAutoAnalyzePriorityQueue.Store(tt.autoAnalyzePriority) + vardef.RunAutoAnalyze.Store(tt.autoAnalyze) + vardef.EnableAutoAnalyzePriorityQueue.Store(tt.autoAnalyzePriority) - sysVar := GetSysVar(TiDBAutoAnalyzeConcurrency) + sysVar := GetSysVar(vardef.TiDBAutoAnalyzeConcurrency) require.NotNil(t, sysVar) - _, err := sysVar.Validate(vars, tt.input, ScopeGlobal) + _, err := sysVar.Validate(vars, tt.input, vardef.ScopeGlobal) if tt.expectError { require.Error(t, err) } else { diff --git a/pkg/sessionctx/variable/tidb_vars.go b/pkg/sessionctx/variable/tidb_vars.go index 89bb6a47d79cc..8a71f9f22d3a5 100644 --- a/pkg/sessionctx/variable/tidb_vars.go +++ b/pkg/sessionctx/variable/tidb_vars.go @@ -16,1687 +16,11 @@ package variable import ( "context" - "fmt" - "math" - "strconv" "time" - "github.com/pingcap/tidb/pkg/config" - "github.com/pingcap/tidb/pkg/executor/join/joinversion" - "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/util/memory" - "github.com/pingcap/tidb/pkg/util/paging" - "github.com/pingcap/tidb/pkg/util/size" - "github.com/pingcap/tidb/pkg/util/tiflash" - "github.com/pingcap/tidb/pkg/util/tiflashcompute" "go.uber.org/atomic" ) -/* - Steps to add a new TiDB specific system variable: - - 1. Add a new variable name with comment in this file. - 2. Add the default value of the new variable in this file. - 3. Add SysVar instance in 'defaultSysVars' slice. -*/ - -// TiDB system variable names that only in session scope. -const ( - TiDBDDLSlowOprThreshold = "ddl_slow_threshold" - - // TiDBSnapshot is used for reading history data, the default value is empty string. - // The value can be a datetime string like '2017-11-11 20:20:20' or a tso string. When this variable is set, the session reads history data of that time. - TiDBSnapshot = "tidb_snapshot" - - // TiDBOptAggPushDown is used to enable/disable the optimizer rule of aggregation push down. - TiDBOptAggPushDown = "tidb_opt_agg_push_down" - - // TiDBOptDeriveTopN is used to enable/disable the optimizer rule of deriving topN. - TiDBOptDeriveTopN = "tidb_opt_derive_topn" - - // TiDBOptCartesianBCJ is used to disable/enable broadcast cartesian join in MPP mode - TiDBOptCartesianBCJ = "tidb_opt_broadcast_cartesian_join" - - TiDBOptMPPOuterJoinFixedBuildSide = "tidb_opt_mpp_outer_join_fixed_build_side" - - // TiDBOptDistinctAggPushDown is used to decide whether agg with distinct should be pushed to tikv/tiflash. - TiDBOptDistinctAggPushDown = "tidb_opt_distinct_agg_push_down" - - // TiDBOptSkewDistinctAgg is used to indicate the distinct agg has data skew - TiDBOptSkewDistinctAgg = "tidb_opt_skew_distinct_agg" - - // TiDBOpt3StageDistinctAgg is used to indicate whether to plan and execute the distinct agg in 3 stages - TiDBOpt3StageDistinctAgg = "tidb_opt_three_stage_distinct_agg" - - // TiDBOptEnable3StageMultiDistinctAgg is used to indicate whether to plan and execute the multi distinct agg in 3 stages - TiDBOptEnable3StageMultiDistinctAgg = "tidb_opt_enable_three_stage_multi_distinct_agg" - - TiDBOptExplainNoEvaledSubQuery = "tidb_opt_enable_non_eval_scalar_subquery" - - // TiDBBCJThresholdSize is used to limit the size of small table for mpp broadcast join. - // Its unit is bytes, if the size of small table is larger than it, we will not use bcj. - TiDBBCJThresholdSize = "tidb_broadcast_join_threshold_size" - - // TiDBBCJThresholdCount is used to limit the count of small table for mpp broadcast join. - // If we can't estimate the size of one side of join child, we will check if its row number exceeds this limitation. - TiDBBCJThresholdCount = "tidb_broadcast_join_threshold_count" - - // TiDBPreferBCJByExchangeDataSize indicates the method used to choose mpp broadcast join - TiDBPreferBCJByExchangeDataSize = "tidb_prefer_broadcast_join_by_exchange_data_size" - - // TiDBOptWriteRowID is used to enable/disable the operations of insert、replace and update to _tidb_rowid. - TiDBOptWriteRowID = "tidb_opt_write_row_id" - - // TiDBAutoAnalyzeRatio will run if (table modify count)/(table row count) is greater than this value. - TiDBAutoAnalyzeRatio = "tidb_auto_analyze_ratio" - - // TiDBAutoAnalyzeStartTime will run if current time is within start time and end time. - TiDBAutoAnalyzeStartTime = "tidb_auto_analyze_start_time" - TiDBAutoAnalyzeEndTime = "tidb_auto_analyze_end_time" - - // TiDBChecksumTableConcurrency is used to speed up the ADMIN CHECKSUM TABLE - // statement, when a table has multiple indices, those indices can be - // scanned concurrently, with the cost of higher system performance impact. - TiDBChecksumTableConcurrency = "tidb_checksum_table_concurrency" - - // TiDBCurrentTS is used to get the current transaction timestamp. - // It is read-only. - TiDBCurrentTS = "tidb_current_ts" - - // TiDBLastTxnInfo is used to get the last transaction info within the current session. - TiDBLastTxnInfo = "tidb_last_txn_info" - - // TiDBLastQueryInfo is used to get the last query info within the current session. - TiDBLastQueryInfo = "tidb_last_query_info" - - // TiDBLastDDLInfo is used to get the last ddl info within the current session. - TiDBLastDDLInfo = "tidb_last_ddl_info" - - // TiDBLastPlanReplayerToken is used to get the last plan replayer token within the current session - TiDBLastPlanReplayerToken = "tidb_last_plan_replayer_token" - - // TiDBConfig is a read-only variable that shows the config of the current server. - TiDBConfig = "tidb_config" - - // TiDBBatchInsert is used to enable/disable auto-split insert data. If set this option on, insert executor will automatically - // insert data into multiple batches and use a single txn for each batch. This will be helpful when inserting large data. - TiDBBatchInsert = "tidb_batch_insert" - - // TiDBBatchDelete is used to enable/disable auto-split delete data. If set this option on, delete executor will automatically - // split data into multiple batches and use a single txn for each batch. This will be helpful when deleting large data. - TiDBBatchDelete = "tidb_batch_delete" - - // TiDBBatchCommit is used to enable/disable auto-split the transaction. - // If set this option on, the transaction will be committed when it reaches stmt-count-limit and starts a new transaction. - TiDBBatchCommit = "tidb_batch_commit" - - // TiDBDMLBatchSize is used to split the insert/delete data into small batches. - // It only takes effort when tidb_batch_insert/tidb_batch_delete is on. - // Its default value is 20000. When the row size is large, 20k rows could be larger than 100MB. - // User could change it to a smaller one to avoid breaking the transaction size limitation. - TiDBDMLBatchSize = "tidb_dml_batch_size" - - // The following session variables controls the memory quota during query execution. - - // TiDBMemQuotaQuery controls the memory quota of a query. - TiDBMemQuotaQuery = "tidb_mem_quota_query" // Bytes. - // TiDBMemQuotaApplyCache controls the memory quota of a query. - TiDBMemQuotaApplyCache = "tidb_mem_quota_apply_cache" - - // TiDBGeneralLog is used to log every query in the server in info level. - TiDBGeneralLog = "tidb_general_log" - - // TiDBLogFileMaxDays is used to log every query in the server in info level. - TiDBLogFileMaxDays = "tidb_log_file_max_days" - - // TiDBPProfSQLCPU is used to add label sql label to pprof result. - TiDBPProfSQLCPU = "tidb_pprof_sql_cpu" - - // TiDBRetryLimit is the maximum number of retries when committing a transaction. - TiDBRetryLimit = "tidb_retry_limit" - - // TiDBDisableTxnAutoRetry disables transaction auto retry. - // Deprecated: This variable is deprecated, please do not use this variable. - TiDBDisableTxnAutoRetry = "tidb_disable_txn_auto_retry" - - // TiDBEnableChunkRPC enables TiDB to use Chunk format for coprocessor requests. - TiDBEnableChunkRPC = "tidb_enable_chunk_rpc" - - // TiDBOptimizerSelectivityLevel is used to control the selectivity estimation level. - TiDBOptimizerSelectivityLevel = "tidb_optimizer_selectivity_level" - - // TiDBOptimizerEnableNewOnlyFullGroupByCheck is used to open the newly only_full_group_by check by maintaining functional dependency. - TiDBOptimizerEnableNewOnlyFullGroupByCheck = "tidb_enable_new_only_full_group_by_check" - - TiDBOptimizerEnableOuterJoinReorder = "tidb_enable_outer_join_reorder" - - // TiDBOptimizerEnableNAAJ is used to open the newly null-aware anti join - TiDBOptimizerEnableNAAJ = "tidb_enable_null_aware_anti_join" - - // TiDBTxnMode is used to control the transaction behavior. - TiDBTxnMode = "tidb_txn_mode" - - // TiDBRowFormatVersion is used to control tidb row format version current. - TiDBRowFormatVersion = "tidb_row_format_version" - - // TiDBEnableRowLevelChecksum is used to control whether to append checksum to row values. - TiDBEnableRowLevelChecksum = "tidb_enable_row_level_checksum" - - // TiDBEnableTablePartition is used to control table partition feature. - // The valid value include auto/on/off: - // on or auto: enable table partition if the partition type is implemented. - // off: always disable table partition. - TiDBEnableTablePartition = "tidb_enable_table_partition" - - // TiDBEnableListTablePartition is used to control list table partition feature. - // Deprecated: This variable is deprecated, please do not use this variable. - TiDBEnableListTablePartition = "tidb_enable_list_partition" - - // TiDBSkipIsolationLevelCheck is used to control whether to return error when set unsupported transaction - // isolation level. - TiDBSkipIsolationLevelCheck = "tidb_skip_isolation_level_check" - - // TiDBLowResolutionTSO is used for reading data with low resolution TSO which is updated once every two seconds - TiDBLowResolutionTSO = "tidb_low_resolution_tso" - - // TiDBReplicaRead is used for reading data from replicas, followers for example. - TiDBReplicaRead = "tidb_replica_read" - - // TiDBAdaptiveClosestReadThreshold is for reading data from closest replicas(with same 'zone' label). - // TiKV client should send read request to the closest replica(leader/follower) if the estimated response - // size exceeds this threshold; otherwise, this request should be sent to leader. - // This variable only take effect when `tidb_replica_read` is 'closest-adaptive'. - TiDBAdaptiveClosestReadThreshold = "tidb_adaptive_closest_read_threshold" - - // TiDBAllowRemoveAutoInc indicates whether a user can drop the auto_increment column attribute or not. - TiDBAllowRemoveAutoInc = "tidb_allow_remove_auto_inc" - - // TiDBMultiStatementMode enables multi statement at the risk of SQL injection - // provides backwards compatibility - TiDBMultiStatementMode = "tidb_multi_statement_mode" - - // TiDBEvolvePlanTaskMaxTime controls the max time of a single evolution task. - TiDBEvolvePlanTaskMaxTime = "tidb_evolve_plan_task_max_time" - - // TiDBEvolvePlanTaskStartTime is the start time of evolution task. - TiDBEvolvePlanTaskStartTime = "tidb_evolve_plan_task_start_time" - // TiDBEvolvePlanTaskEndTime is the end time of evolution task. - TiDBEvolvePlanTaskEndTime = "tidb_evolve_plan_task_end_time" - - // TiDBSlowLogThreshold is used to set the slow log threshold in the server. - TiDBSlowLogThreshold = "tidb_slow_log_threshold" - - // TiDBSlowTxnLogThreshold is used to set the slow transaction log threshold in the server. - TiDBSlowTxnLogThreshold = "tidb_slow_txn_log_threshold" - - // TiDBRecordPlanInSlowLog is used to log the plan of the slow query. - TiDBRecordPlanInSlowLog = "tidb_record_plan_in_slow_log" - - // TiDBEnableSlowLog enables TiDB to log slow queries. - TiDBEnableSlowLog = "tidb_enable_slow_log" - - // TiDBCheckMb4ValueInUTF8 is used to control whether to enable the check wrong utf8 value. - TiDBCheckMb4ValueInUTF8 = "tidb_check_mb4_value_in_utf8" - - // TiDBFoundInPlanCache indicates whether the last statement was found in plan cache - TiDBFoundInPlanCache = "last_plan_from_cache" - - // TiDBFoundInBinding indicates whether the last statement was matched with the hints in the binding. - TiDBFoundInBinding = "last_plan_from_binding" - - // TiDBAllowAutoRandExplicitInsert indicates whether explicit insertion on auto_random column is allowed. - TiDBAllowAutoRandExplicitInsert = "allow_auto_random_explicit_insert" - - // TiDBTxnScope indicates whether using global transactions or local transactions. - TiDBTxnScope = "txn_scope" - - // TiDBTxnReadTS indicates the next transaction should be staleness transaction and provide the startTS - TiDBTxnReadTS = "tx_read_ts" - - // TiDBReadStaleness indicates the staleness duration for following statement - TiDBReadStaleness = "tidb_read_staleness" - - // TiDBEnablePaging indicates whether paging is enabled in coprocessor requests. - TiDBEnablePaging = "tidb_enable_paging" - - // TiDBReadConsistency indicates whether the autocommit read statement goes through TiKV RC. - TiDBReadConsistency = "tidb_read_consistency" - - // TiDBSysdateIsNow is the name of the `tidb_sysdate_is_now` system variable - TiDBSysdateIsNow = "tidb_sysdate_is_now" - - // RequireSecureTransport indicates the secure mode for data transport - RequireSecureTransport = "require_secure_transport" - - // TiFlashFastScan indicates whether use fast scan in tiflash. - TiFlashFastScan = "tiflash_fastscan" - - // TiDBEnableUnsafeSubstitute indicates whether to enable generate column takes unsafe substitute. - TiDBEnableUnsafeSubstitute = "tidb_enable_unsafe_substitute" - - // TiDBEnableTiFlashReadForWriteStmt indicates whether to enable TiFlash to read for write statements. - TiDBEnableTiFlashReadForWriteStmt = "tidb_enable_tiflash_read_for_write_stmt" - - // TiDBUseAlloc indicates whether the last statement used chunk alloc - TiDBUseAlloc = "last_sql_use_alloc" - - // TiDBExplicitRequestSourceType indicates the source of the request, it's a complement of RequestSourceType. - // The value maybe "lightning", "br", "dumpling" etc. - TiDBExplicitRequestSourceType = "tidb_request_source_type" -) - -// TiDB system variable names that both in session and global scope. -const ( - // TiDBBuildStatsConcurrency specifies the number of concurrent workers used for analyzing tables or partitions. - // When multiple tables or partitions are specified in the analyze statement, TiDB will process them concurrently. - // Additionally, this setting controls the concurrency for building NDV (Number of Distinct Values) for special indexes, - // such as generated columns composed indexes. - TiDBBuildStatsConcurrency = "tidb_build_stats_concurrency" - - // TiDBBuildSamplingStatsConcurrency is used to control the concurrency of building stats using sampling. - // 1. The number of concurrent workers to merge FMSketches and Sample Data from different regions. - // 2. The number of concurrent workers to build TopN and Histogram concurrently. - TiDBBuildSamplingStatsConcurrency = "tidb_build_sampling_stats_concurrency" - - // TiDBDistSQLScanConcurrency is used to set the concurrency of a distsql scan task. - // A distsql scan task can be a table scan or a index scan, which may be distributed to many TiKV nodes. - // Higher concurrency may reduce latency, but with the cost of higher memory usage and system performance impact. - // If the query has a LIMIT clause, high concurrency makes the system do much more work than needed. - TiDBDistSQLScanConcurrency = "tidb_distsql_scan_concurrency" - - // TiDBAnalyzeDistSQLScanConcurrency is the number of concurrent workers to scan regions to collect statistics (FMSketch, Samples). - // For auto analyze, the value is controlled by tidb_sysproc_scan_concurrency variable. - TiDBAnalyzeDistSQLScanConcurrency = "tidb_analyze_distsql_scan_concurrency" - - // TiDBOptInSubqToJoinAndAgg is used to enable/disable the optimizer rule of rewriting IN subquery. - TiDBOptInSubqToJoinAndAgg = "tidb_opt_insubq_to_join_and_agg" - - // TiDBOptPreferRangeScan is used to enable/disable the optimizer to always prefer range scan over table scan, ignoring their costs. - TiDBOptPreferRangeScan = "tidb_opt_prefer_range_scan" - - // TiDBOptEnableCorrelationAdjustment is used to indicates if enable correlation adjustment. - TiDBOptEnableCorrelationAdjustment = "tidb_opt_enable_correlation_adjustment" - - // TiDBOptLimitPushDownThreshold determines if push Limit or TopN down to TiKV forcibly. - TiDBOptLimitPushDownThreshold = "tidb_opt_limit_push_down_threshold" - - // TiDBOptCorrelationThreshold is a guard to enable row count estimation using column order correlation. - TiDBOptCorrelationThreshold = "tidb_opt_correlation_threshold" - - // TiDBOptCorrelationExpFactor is an exponential factor to control heuristic approach when tidb_opt_correlation_threshold is not satisfied. - TiDBOptCorrelationExpFactor = "tidb_opt_correlation_exp_factor" - - // TiDBOptCPUFactor is the CPU cost of processing one expression for one row. - TiDBOptCPUFactor = "tidb_opt_cpu_factor" - // TiDBOptCopCPUFactor is the CPU cost of processing one expression for one row in coprocessor. - TiDBOptCopCPUFactor = "tidb_opt_copcpu_factor" - // TiDBOptTiFlashConcurrencyFactor is concurrency number of tiflash computation. - TiDBOptTiFlashConcurrencyFactor = "tidb_opt_tiflash_concurrency_factor" - // TiDBOptNetworkFactor is the network cost of transferring 1 byte data. - TiDBOptNetworkFactor = "tidb_opt_network_factor" - // TiDBOptScanFactor is the IO cost of scanning 1 byte data on TiKV. - TiDBOptScanFactor = "tidb_opt_scan_factor" - // TiDBOptDescScanFactor is the IO cost of scanning 1 byte data on TiKV in desc order. - TiDBOptDescScanFactor = "tidb_opt_desc_factor" - // TiDBOptSeekFactor is the IO cost of seeking the start value in a range on TiKV or TiFlash. - TiDBOptSeekFactor = "tidb_opt_seek_factor" - // TiDBOptMemoryFactor is the memory cost of storing one tuple. - TiDBOptMemoryFactor = "tidb_opt_memory_factor" - // TiDBOptDiskFactor is the IO cost of reading/writing one byte to temporary disk. - TiDBOptDiskFactor = "tidb_opt_disk_factor" - // TiDBOptConcurrencyFactor is the CPU cost of additional one goroutine. - TiDBOptConcurrencyFactor = "tidb_opt_concurrency_factor" - // TiDBOptForceInlineCTE is used to enable/disable inline CTE - TiDBOptForceInlineCTE = "tidb_opt_force_inline_cte" - - // TiDBIndexJoinBatchSize is used to set the batch size of an index lookup join. - // The index lookup join fetches batches of data from outer executor and constructs ranges for inner executor. - // This value controls how much of data in a batch to do the index join. - // Large value may reduce the latency but consumes more system resource. - TiDBIndexJoinBatchSize = "tidb_index_join_batch_size" - - // TiDBIndexLookupSize is used for index lookup executor. - // The index lookup executor first scan a batch of handles from a index, then use those handles to lookup the table - // rows, this value controls how much of handles in a batch to do a lookup task. - // Small value sends more RPCs to TiKV, consume more system resource. - // Large value may do more work than needed if the query has a limit. - TiDBIndexLookupSize = "tidb_index_lookup_size" - - // TiDBIndexLookupConcurrency is used for index lookup executor. - // A lookup task may have 'tidb_index_lookup_size' of handles at maximum, the handles may be distributed - // in many TiKV nodes, we execute multiple concurrent index lookup tasks concurrently to reduce the time - // waiting for a task to finish. - // Set this value higher may reduce the latency but consumes more system resource. - // tidb_index_lookup_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBIndexLookupConcurrency = "tidb_index_lookup_concurrency" - - // TiDBIndexLookupJoinConcurrency is used for index lookup join executor. - // IndexLookUpJoin starts "tidb_index_lookup_join_concurrency" inner workers - // to fetch inner rows and join the matched (outer, inner) row pairs. - // tidb_index_lookup_join_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBIndexLookupJoinConcurrency = "tidb_index_lookup_join_concurrency" - - // TiDBIndexSerialScanConcurrency is used for controlling the concurrency of index scan operation - // when we need to keep the data output order the same as the order of index data. - TiDBIndexSerialScanConcurrency = "tidb_index_serial_scan_concurrency" - - // TiDBMaxChunkSize is used to control the max chunk size during query execution. - TiDBMaxChunkSize = "tidb_max_chunk_size" - - // TiDBAllowBatchCop means if we should send batch coprocessor to TiFlash. It can be set to 0, 1 and 2. - // 0 means never use batch cop, 1 means use batch cop in case of aggregation and join, 2, means to force sending batch cop for any query. - // The default value is 0 - TiDBAllowBatchCop = "tidb_allow_batch_cop" - - // TiDBShardRowIDBits means all the tables created in the current session will be sharded. - // The default value is 0 - TiDBShardRowIDBits = "tidb_shard_row_id_bits" - - // TiDBPreSplitRegions means all the tables created in the current session will be pre-splited. - // The default value is 0 - TiDBPreSplitRegions = "tidb_pre_split_regions" - - // TiDBAllowMPPExecution means if we should use mpp way to execute query or not. - // Default value is `true`, means to be determined by the optimizer. - // Value set to `false` means never use mpp. - TiDBAllowMPPExecution = "tidb_allow_mpp" - - // TiDBAllowTiFlashCop means we only use MPP mode to query data. - // Default value is `true`, means to be determined by the optimizer. - // Value set to `false` means we may fall back to TiFlash cop plan if possible. - TiDBAllowTiFlashCop = "tidb_allow_tiflash_cop" - - // TiDBHashExchangeWithNewCollation means if hash exchange is supported when new collation is on. - // Default value is `true`, means support hash exchange when new collation is on. - // Value set to `false` means not support hash exchange when new collation is on. - TiDBHashExchangeWithNewCollation = "tidb_hash_exchange_with_new_collation" - - // TiDBEnforceMPPExecution means if we should enforce mpp way to execute query or not. - // Default value is `false`, means to be determined by variable `tidb_allow_mpp`. - // Value set to `true` means enforce use mpp. - // Note if you want to set `tidb_enforce_mpp` to `true`, you must set `tidb_allow_mpp` to `true` first. - TiDBEnforceMPPExecution = "tidb_enforce_mpp" - - // TiDBMaxTiFlashThreads is the maximum number of threads to execute the request which is pushed down to tiflash. - // Default value is -1, means it will not be pushed down to tiflash. - // If the value is bigger than -1, it will be pushed down to tiflash and used to create db context in tiflash. - TiDBMaxTiFlashThreads = "tidb_max_tiflash_threads" - - // TiDBMaxBytesBeforeTiFlashExternalJoin is the maximum bytes used by a TiFlash join before spill to disk - TiDBMaxBytesBeforeTiFlashExternalJoin = "tidb_max_bytes_before_tiflash_external_join" - - // TiDBMaxBytesBeforeTiFlashExternalGroupBy is the maximum bytes used by a TiFlash hash aggregation before spill to disk - TiDBMaxBytesBeforeTiFlashExternalGroupBy = "tidb_max_bytes_before_tiflash_external_group_by" - - // TiDBMaxBytesBeforeTiFlashExternalSort is the maximum bytes used by a TiFlash sort/TopN before spill to disk - TiDBMaxBytesBeforeTiFlashExternalSort = "tidb_max_bytes_before_tiflash_external_sort" - - // TiFlashMemQuotaQueryPerNode is the maximum bytes used by a TiFlash Query on each TiFlash node - TiFlashMemQuotaQueryPerNode = "tiflash_mem_quota_query_per_node" - - // TiFlashQuerySpillRatio is the threshold that TiFlash will trigger auto spill when the memory usage is above this percentage - TiFlashQuerySpillRatio = "tiflash_query_spill_ratio" - - // TiDBMPPStoreFailTTL is the unavailable time when a store is detected failed. During that time, tidb will not send any task to - // TiFlash even though the failed TiFlash node has been recovered. - TiDBMPPStoreFailTTL = "tidb_mpp_store_fail_ttl" - - // TiDBInitChunkSize is used to control the init chunk size during query execution. - TiDBInitChunkSize = "tidb_init_chunk_size" - - // TiDBMinPagingSize is used to control the min paging size in the coprocessor paging protocol. - TiDBMinPagingSize = "tidb_min_paging_size" - - // TiDBMaxPagingSize is used to control the max paging size in the coprocessor paging protocol. - TiDBMaxPagingSize = "tidb_max_paging_size" - - // TiDBEnableCascadesPlanner is used to control whether to enable the cascades planner. - TiDBEnableCascadesPlanner = "tidb_enable_cascades_planner" - - // TiDBSkipUTF8Check skips the UTF8 validate process, validate UTF8 has performance cost, if we can make sure - // the input string values are valid, we can skip the check. - TiDBSkipUTF8Check = "tidb_skip_utf8_check" - - // TiDBSkipASCIICheck skips the ASCII validate process - // old tidb may already have fields with invalid ASCII bytes - // disable ASCII validate can guarantee a safe replication - TiDBSkipASCIICheck = "tidb_skip_ascii_check" - - // TiDBHashJoinConcurrency is used for hash join executor. - // The hash join outer executor starts multiple concurrent join workers to probe the hash table. - // tidb_hash_join_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBHashJoinConcurrency = "tidb_hash_join_concurrency" - - // TiDBProjectionConcurrency is used for projection operator. - // This variable controls the worker number of projection operator. - // tidb_projection_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBProjectionConcurrency = "tidb_projection_concurrency" - - // TiDBHashAggPartialConcurrency is used for hash agg executor. - // The hash agg executor starts multiple concurrent partial workers to do partial aggregate works. - // tidb_hashagg_partial_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBHashAggPartialConcurrency = "tidb_hashagg_partial_concurrency" - - // TiDBHashAggFinalConcurrency is used for hash agg executor. - // The hash agg executor starts multiple concurrent final workers to do final aggregate works. - // tidb_hashagg_final_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBHashAggFinalConcurrency = "tidb_hashagg_final_concurrency" - - // TiDBWindowConcurrency is used for window parallel executor. - // tidb_window_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBWindowConcurrency = "tidb_window_concurrency" - - // TiDBMergeJoinConcurrency is used for merge join parallel executor - TiDBMergeJoinConcurrency = "tidb_merge_join_concurrency" - - // TiDBStreamAggConcurrency is used for stream aggregation parallel executor. - // tidb_stream_agg_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBStreamAggConcurrency = "tidb_streamagg_concurrency" - - // TiDBIndexMergeIntersectionConcurrency is used for parallel worker of index merge intersection. - TiDBIndexMergeIntersectionConcurrency = "tidb_index_merge_intersection_concurrency" - - // TiDBEnableParallelApply is used for parallel apply. - TiDBEnableParallelApply = "tidb_enable_parallel_apply" - - // TiDBBackoffLockFast is used for tikv backoff base time in milliseconds. - TiDBBackoffLockFast = "tidb_backoff_lock_fast" - - // TiDBBackOffWeight is used to control the max back off time in TiDB. - // The default maximum back off time is a small value. - // BackOffWeight could multiply it to let the user adjust the maximum time for retrying. - // Only positive integers can be accepted, which means that the maximum back off time can only grow. - TiDBBackOffWeight = "tidb_backoff_weight" - - // TiDBDDLReorgWorkerCount defines the count of ddl reorg workers. - TiDBDDLReorgWorkerCount = "tidb_ddl_reorg_worker_cnt" - - // TiDBDDLFlashbackConcurrency defines the count of ddl flashback workers. - TiDBDDLFlashbackConcurrency = "tidb_ddl_flashback_concurrency" - - // TiDBDDLReorgBatchSize defines the transaction batch size of ddl reorg workers. - TiDBDDLReorgBatchSize = "tidb_ddl_reorg_batch_size" - - // TiDBDDLErrorCountLimit defines the count of ddl error limit. - TiDBDDLErrorCountLimit = "tidb_ddl_error_count_limit" - - // TiDBDDLReorgPriority defines the operations' priority of adding indices. - // It can be: PRIORITY_LOW, PRIORITY_NORMAL, PRIORITY_HIGH - TiDBDDLReorgPriority = "tidb_ddl_reorg_priority" - - // TiDBDDLReorgMaxWriteSpeed defines the max write limitation for the lightning local backend - TiDBDDLReorgMaxWriteSpeed = "tidb_ddl_reorg_max_write_speed" - - // TiDBEnableAutoIncrementInGenerated disables the mysql compatibility check on using auto-incremented columns in - // expression indexes and generated columns described here https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html for details. - TiDBEnableAutoIncrementInGenerated = "tidb_enable_auto_increment_in_generated" - - // TiDBPlacementMode is used to control the mode for placement - TiDBPlacementMode = "tidb_placement_mode" - - // TiDBMaxDeltaSchemaCount defines the max length of deltaSchemaInfos. - // deltaSchemaInfos is a queue that maintains the history of schema changes. - TiDBMaxDeltaSchemaCount = "tidb_max_delta_schema_count" - - // TiDBScatterRegion will scatter the regions for DDLs when it is "table" or "global", "" indicates not trigger scatter. - TiDBScatterRegion = "tidb_scatter_region" - - // TiDBWaitSplitRegionFinish defines the split region behaviour is sync or async. - TiDBWaitSplitRegionFinish = "tidb_wait_split_region_finish" - - // TiDBWaitSplitRegionTimeout uses to set the split and scatter region back off time. - TiDBWaitSplitRegionTimeout = "tidb_wait_split_region_timeout" - - // TiDBForcePriority defines the operations' priority of all statements. - // It can be "NO_PRIORITY", "LOW_PRIORITY", "HIGH_PRIORITY", "DELAYED" - TiDBForcePriority = "tidb_force_priority" - - // TiDBConstraintCheckInPlace indicates to check the constraint when the SQL executing. - // It could hurt the performance of bulking insert when it is ON. - TiDBConstraintCheckInPlace = "tidb_constraint_check_in_place" - - // TiDBEnableWindowFunction is used to control whether to enable the window function. - TiDBEnableWindowFunction = "tidb_enable_window_function" - - // TiDBEnablePipelinedWindowFunction is used to control whether to use pipelined window function, it only works when tidb_enable_window_function = true. - TiDBEnablePipelinedWindowFunction = "tidb_enable_pipelined_window_function" - - // TiDBEnableStrictDoubleTypeCheck is used to control table field double type syntax check. - TiDBEnableStrictDoubleTypeCheck = "tidb_enable_strict_double_type_check" - - // TiDBOptProjectionPushDown is used to control whether to pushdown projection to coprocessor. - TiDBOptProjectionPushDown = "tidb_opt_projection_push_down" - - // TiDBEnableVectorizedExpression is used to control whether to enable the vectorized expression evaluation. - TiDBEnableVectorizedExpression = "tidb_enable_vectorized_expression" - - // TiDBOptJoinReorderThreshold defines the threshold less than which - // we'll choose a rather time-consuming algorithm to calculate the join order. - TiDBOptJoinReorderThreshold = "tidb_opt_join_reorder_threshold" - - // TiDBSlowQueryFile indicates which slow query log file for SLOW_QUERY table to parse. - TiDBSlowQueryFile = "tidb_slow_query_file" - - // TiDBEnableFastAnalyze indicates to use fast analyze. - // Deprecated: This variable is deprecated, please do not use this variable. - TiDBEnableFastAnalyze = "tidb_enable_fast_analyze" - - // TiDBExpensiveQueryTimeThreshold indicates the time threshold of expensive query. - TiDBExpensiveQueryTimeThreshold = "tidb_expensive_query_time_threshold" - - // TiDBExpensiveTxnTimeThreshold indicates the time threshold of expensive transaction. - TiDBExpensiveTxnTimeThreshold = "tidb_expensive_txn_time_threshold" - - // TiDBEnableIndexMerge indicates to generate IndexMergePath. - TiDBEnableIndexMerge = "tidb_enable_index_merge" - - // TiDBEnableNoopFuncs set true will enable using fake funcs(like get_lock release_lock) - TiDBEnableNoopFuncs = "tidb_enable_noop_functions" - - // TiDBEnableStmtSummary indicates whether the statement summary is enabled. - TiDBEnableStmtSummary = "tidb_enable_stmt_summary" - - // TiDBStmtSummaryInternalQuery indicates whether the statement summary contain internal query. - TiDBStmtSummaryInternalQuery = "tidb_stmt_summary_internal_query" - - // TiDBStmtSummaryRefreshInterval indicates the refresh interval in seconds for each statement summary. - TiDBStmtSummaryRefreshInterval = "tidb_stmt_summary_refresh_interval" - - // TiDBStmtSummaryHistorySize indicates the history size of each statement summary. - TiDBStmtSummaryHistorySize = "tidb_stmt_summary_history_size" - - // TiDBStmtSummaryMaxStmtCount indicates the max number of statements kept in memory. - TiDBStmtSummaryMaxStmtCount = "tidb_stmt_summary_max_stmt_count" - - // TiDBStmtSummaryMaxSQLLength indicates the max length of displayed normalized sql and sample sql. - TiDBStmtSummaryMaxSQLLength = "tidb_stmt_summary_max_sql_length" - - // TiDBIgnoreInlistPlanDigest enables TiDB to generate the same plan digest with SQL using different in-list arguments. - TiDBIgnoreInlistPlanDigest = "tidb_ignore_inlist_plan_digest" - - // TiDBCapturePlanBaseline indicates whether the capture of plan baselines is enabled. - TiDBCapturePlanBaseline = "tidb_capture_plan_baselines" - - // TiDBUsePlanBaselines indicates whether the use of plan baselines is enabled. - TiDBUsePlanBaselines = "tidb_use_plan_baselines" - - // TiDBEvolvePlanBaselines indicates whether the evolution of plan baselines is enabled. - TiDBEvolvePlanBaselines = "tidb_evolve_plan_baselines" - - // TiDBOptEnableFuzzyBinding indicates whether to enable the universal binding. - TiDBOptEnableFuzzyBinding = "tidb_opt_enable_fuzzy_binding" - - // TiDBEnableExtendedStats indicates whether the extended statistics feature is enabled. - TiDBEnableExtendedStats = "tidb_enable_extended_stats" - - // TiDBIsolationReadEngines indicates the tidb only read from the stores whose engine type is involved in IsolationReadEngines. - // Now, only support TiKV and TiFlash. - TiDBIsolationReadEngines = "tidb_isolation_read_engines" - - // TiDBStoreLimit indicates the limit of sending request to a store, 0 means without limit. - TiDBStoreLimit = "tidb_store_limit" - - // TiDBMetricSchemaStep indicates the step when query metric schema. - TiDBMetricSchemaStep = "tidb_metric_query_step" - - // TiDBCDCWriteSource indicates the following data is written by TiCDC if it is not 0. - TiDBCDCWriteSource = "tidb_cdc_write_source" - - // TiDBMetricSchemaRangeDuration indicates the range duration when query metric schema. - TiDBMetricSchemaRangeDuration = "tidb_metric_query_range_duration" - - // TiDBEnableCollectExecutionInfo indicates that whether execution info is collected. - TiDBEnableCollectExecutionInfo = "tidb_enable_collect_execution_info" - - // TiDBExecutorConcurrency is used for controlling the concurrency of all types of executors. - TiDBExecutorConcurrency = "tidb_executor_concurrency" - - // TiDBEnableClusteredIndex indicates if clustered index feature is enabled. - TiDBEnableClusteredIndex = "tidb_enable_clustered_index" - - // TiDBEnableGlobalIndex means if we could create an global index on a partition table or not. - // Deprecated, will always be ON - TiDBEnableGlobalIndex = "tidb_enable_global_index" - - // TiDBPartitionPruneMode indicates the partition prune mode used. - TiDBPartitionPruneMode = "tidb_partition_prune_mode" - - // TiDBRedactLog indicates that whether redact log. - TiDBRedactLog = "tidb_redact_log" - - // TiDBRestrictedReadOnly is meant for the cloud admin to toggle the cluster read only - TiDBRestrictedReadOnly = "tidb_restricted_read_only" - - // TiDBSuperReadOnly is tidb's variant of mysql's super_read_only, which has some differences from mysql's super_read_only. - TiDBSuperReadOnly = "tidb_super_read_only" - - // TiDBShardAllocateStep indicates the max size of continuous rowid shard in one transaction. - TiDBShardAllocateStep = "tidb_shard_allocate_step" - // TiDBEnableTelemetry indicates that whether usage data report to PingCAP is enabled. - // Deprecated: it is 'off' always since Telemetry has been removed from TiDB. - TiDBEnableTelemetry = "tidb_enable_telemetry" - - // TiDBMemoryUsageAlarmRatio indicates the alarm threshold when memory usage of the tidb-server exceeds. - TiDBMemoryUsageAlarmRatio = "tidb_memory_usage_alarm_ratio" - - // TiDBMemoryUsageAlarmKeepRecordNum indicates the number of saved alarm files. - TiDBMemoryUsageAlarmKeepRecordNum = "tidb_memory_usage_alarm_keep_record_num" - - // TiDBEnableRateLimitAction indicates whether enabled ratelimit action - TiDBEnableRateLimitAction = "tidb_enable_rate_limit_action" - - // TiDBEnableAsyncCommit indicates whether to enable the async commit feature. - TiDBEnableAsyncCommit = "tidb_enable_async_commit" - - // TiDBEnable1PC indicates whether to enable the one-phase commit feature. - TiDBEnable1PC = "tidb_enable_1pc" - - // TiDBGuaranteeLinearizability indicates whether to guarantee linearizability. - TiDBGuaranteeLinearizability = "tidb_guarantee_linearizability" - - // TiDBAnalyzeVersion indicates how tidb collects the analyzed statistics and how use to it. - TiDBAnalyzeVersion = "tidb_analyze_version" - - // TiDBAutoAnalyzePartitionBatchSize indicates the batch size for partition tables for auto analyze in dynamic mode - // Deprecated: This variable is deprecated, please do not use this variable. - TiDBAutoAnalyzePartitionBatchSize = "tidb_auto_analyze_partition_batch_size" - - // TiDBEnableIndexMergeJoin indicates whether to enable index merge join. - TiDBEnableIndexMergeJoin = "tidb_enable_index_merge_join" - - // TiDBTrackAggregateMemoryUsage indicates whether track the memory usage of aggregate function. - TiDBTrackAggregateMemoryUsage = "tidb_track_aggregate_memory_usage" - - // TiDBEnableExchangePartition indicates whether to enable exchange partition. - TiDBEnableExchangePartition = "tidb_enable_exchange_partition" - - // TiDBAllowFallbackToTiKV indicates the engine types whose unavailability triggers fallback to TiKV. - // Now we only support TiFlash. - TiDBAllowFallbackToTiKV = "tidb_allow_fallback_to_tikv" - - // TiDBEnableTopSQL indicates whether the top SQL is enabled. - TiDBEnableTopSQL = "tidb_enable_top_sql" - - // TiDBSourceID indicates the source ID of the TiDB server. - TiDBSourceID = "tidb_source_id" - - // TiDBTopSQLMaxTimeSeriesCount indicates the max number of statements been collected in each time series. - TiDBTopSQLMaxTimeSeriesCount = "tidb_top_sql_max_time_series_count" - - // TiDBTopSQLMaxMetaCount indicates the max capacity of the collect meta per second. - TiDBTopSQLMaxMetaCount = "tidb_top_sql_max_meta_count" - - // TiDBEnableLocalTxn indicates whether to enable Local Txn. - TiDBEnableLocalTxn = "tidb_enable_local_txn" - - // TiDBEnableMDL indicates whether to enable MDL. - TiDBEnableMDL = "tidb_enable_metadata_lock" - - // TiDBTSOClientBatchMaxWaitTime indicates the max value of the TSO Batch Wait interval time of PD client. - TiDBTSOClientBatchMaxWaitTime = "tidb_tso_client_batch_max_wait_time" - - // TiDBTxnCommitBatchSize is used to control the batch size of transaction commit related requests sent by TiDB to TiKV. - // If a single transaction has a large amount of writes, you can increase the batch size to improve the batch effect, - // setting too large will exceed TiKV's raft-entry-max-size limit and cause commit failure. - TiDBTxnCommitBatchSize = "tidb_txn_commit_batch_size" - - // TiDBEnableTSOFollowerProxy indicates whether to enable the TSO Follower Proxy feature of PD client. - TiDBEnableTSOFollowerProxy = "tidb_enable_tso_follower_proxy" - - // PDEnableFollowerHandleRegion indicates whether to enable the PD Follower handle region API. - PDEnableFollowerHandleRegion = "pd_enable_follower_handle_region" - - // TiDBEnableOrderedResultMode indicates if stabilize query results. - TiDBEnableOrderedResultMode = "tidb_enable_ordered_result_mode" - - // TiDBRemoveOrderbyInSubquery indicates whether to remove ORDER BY in subquery. - TiDBRemoveOrderbyInSubquery = "tidb_remove_orderby_in_subquery" - - // TiDBEnablePseudoForOutdatedStats indicates whether use pseudo for outdated stats - TiDBEnablePseudoForOutdatedStats = "tidb_enable_pseudo_for_outdated_stats" - - // TiDBRegardNULLAsPoint indicates whether regard NULL as point when optimizing - TiDBRegardNULLAsPoint = "tidb_regard_null_as_point" - - // TiDBTmpTableMaxSize indicates the max memory size of temporary tables. - TiDBTmpTableMaxSize = "tidb_tmp_table_max_size" - - // TiDBEnableLegacyInstanceScope indicates if instance scope can be set with SET SESSION. - TiDBEnableLegacyInstanceScope = "tidb_enable_legacy_instance_scope" - - // TiDBTableCacheLease indicates the read lock lease of a cached table. - TiDBTableCacheLease = "tidb_table_cache_lease" - - // TiDBStatsLoadSyncWait indicates the time sql execution will sync-wait for stats load. - TiDBStatsLoadSyncWait = "tidb_stats_load_sync_wait" - - // TiDBEnableMutationChecker indicates whether to check data consistency for mutations - TiDBEnableMutationChecker = "tidb_enable_mutation_checker" - // TiDBTxnAssertionLevel indicates how strict the assertion will be, which helps to detect and preventing data & - // index inconsistency problems. - TiDBTxnAssertionLevel = "tidb_txn_assertion_level" - - // TiDBIgnorePreparedCacheCloseStmt indicates whether to ignore close-stmt commands for prepared statements. - TiDBIgnorePreparedCacheCloseStmt = "tidb_ignore_prepared_cache_close_stmt" - - // TiDBEnableNewCostInterface is a internal switch to indicates whether to use the new cost calculation interface. - TiDBEnableNewCostInterface = "tidb_enable_new_cost_interface" - - // TiDBCostModelVersion is a internal switch to indicates the cost model version. - TiDBCostModelVersion = "tidb_cost_model_version" - - // TiDBIndexJoinDoubleReadPenaltyCostRate indicates whether to add some penalty cost to IndexJoin and how much of it. - // IndexJoin can cause plenty of extra double read tasks, which consume lots of resources and take a long time. - // Since the number of double read tasks is hard to estimated accurately, we leave this variable to let us can adjust this - // part of cost manually. - TiDBIndexJoinDoubleReadPenaltyCostRate = "tidb_index_join_double_read_penalty_cost_rate" - - // TiDBBatchPendingTiFlashCount indicates the maximum count of non-available TiFlash tables. - TiDBBatchPendingTiFlashCount = "tidb_batch_pending_tiflash_count" - - // TiDBQueryLogMaxLen is used to set the max length of the query in the log. - TiDBQueryLogMaxLen = "tidb_query_log_max_len" - - // TiDBEnableNoopVariables is used to indicate if noops appear in SHOW [GLOBAL] VARIABLES - TiDBEnableNoopVariables = "tidb_enable_noop_variables" - - // TiDBNonTransactionalIgnoreError is used to ignore error in non-transactional DMLs. - // When set to false, a non-transactional DML returns when it meets the first error. - // When set to true, a non-transactional DML finishes all batches even if errors are met in some batches. - TiDBNonTransactionalIgnoreError = "tidb_nontransactional_ignore_error" - - // Fine grained shuffle is disabled when TiFlashFineGrainedShuffleStreamCount is zero. - TiFlashFineGrainedShuffleStreamCount = "tiflash_fine_grained_shuffle_stream_count" - TiFlashFineGrainedShuffleBatchSize = "tiflash_fine_grained_shuffle_batch_size" - - // TiDBSimplifiedMetrics controls whether to unregister some unused metrics. - TiDBSimplifiedMetrics = "tidb_simplified_metrics" - - // TiDBMemoryDebugModeMinHeapInUse is used to set tidb memory debug mode trigger threshold. - // When set to 0, the function is disabled. - // When set to a negative integer, use memory debug mode to detect the issue of frequent allocation and release of memory. - // We do not actively trigger gc, and check whether the `tracker memory * (1+bias ratio) > heap in use` each 5s. - // When set to a positive integer, use memory debug mode to detect the issue of memory tracking inaccurate. - // We trigger runtime.GC() each 5s, and check whether the `tracker memory * (1+bias ratio) > heap in use`. - TiDBMemoryDebugModeMinHeapInUse = "tidb_memory_debug_mode_min_heap_inuse" - // TiDBMemoryDebugModeAlarmRatio is used set tidb memory debug mode bias ratio. Treat memory bias less than this ratio as noise. - TiDBMemoryDebugModeAlarmRatio = "tidb_memory_debug_mode_alarm_ratio" - - // TiDBEnableAnalyzeSnapshot indicates whether to read data on snapshot when collecting statistics. - // When set to false, ANALYZE reads the latest data. - // When set to true, ANALYZE reads data on the snapshot at the beginning of ANALYZE. - TiDBEnableAnalyzeSnapshot = "tidb_enable_analyze_snapshot" - - // TiDBDefaultStrMatchSelectivity controls some special cardinality estimation strategy for string match functions (like and regexp). - // When set to 0, Selectivity() will try to evaluate those functions with TopN and NULL in the stats to estimate, - // and the default selectivity and the selectivity for the histogram part will be 0.1. - // When set to (0, 1], Selectivity() will use the value of this variable as the default selectivity of those - // functions instead of the selectionFactor (0.8). - TiDBDefaultStrMatchSelectivity = "tidb_default_string_match_selectivity" - - // TiDBEnablePrepPlanCache indicates whether to enable prepared plan cache - TiDBEnablePrepPlanCache = "tidb_enable_prepared_plan_cache" - // TiDBPrepPlanCacheSize indicates the number of cached statements. - // This variable is deprecated, use tidb_session_plan_cache_size instead. - TiDBPrepPlanCacheSize = "tidb_prepared_plan_cache_size" - // TiDBEnablePrepPlanCacheMemoryMonitor indicates whether to enable prepared plan cache monitor - TiDBEnablePrepPlanCacheMemoryMonitor = "tidb_enable_prepared_plan_cache_memory_monitor" - - // TiDBEnableNonPreparedPlanCache indicates whether to enable non-prepared plan cache. - TiDBEnableNonPreparedPlanCache = "tidb_enable_non_prepared_plan_cache" - // TiDBEnableNonPreparedPlanCacheForDML indicates whether to enable non-prepared plan cache for DML statements. - TiDBEnableNonPreparedPlanCacheForDML = "tidb_enable_non_prepared_plan_cache_for_dml" - // TiDBNonPreparedPlanCacheSize controls the size of non-prepared plan cache. - // This variable is deprecated, use tidb_session_plan_cache_size instead. - TiDBNonPreparedPlanCacheSize = "tidb_non_prepared_plan_cache_size" - // TiDBPlanCacheMaxPlanSize controls the maximum size of a plan that can be cached. - TiDBPlanCacheMaxPlanSize = "tidb_plan_cache_max_plan_size" - // TiDBPlanCacheInvalidationOnFreshStats controls if plan cache will be invalidated automatically when - // related stats are analyzed after the plan cache is generated. - TiDBPlanCacheInvalidationOnFreshStats = "tidb_plan_cache_invalidation_on_fresh_stats" - // TiDBSessionPlanCacheSize controls the size of session plan cache. - TiDBSessionPlanCacheSize = "tidb_session_plan_cache_size" - - // TiDBEnableInstancePlanCache indicates whether to enable instance plan cache. - // If this variable is false, session-level plan cache will be used. - TiDBEnableInstancePlanCache = "tidb_enable_instance_plan_cache" - // TiDBInstancePlanCacheReservedPercentage indicates the percentage memory to evict. - TiDBInstancePlanCacheReservedPercentage = "tidb_instance_plan_cache_reserved_percentage" - // TiDBInstancePlanCacheMaxMemSize indicates the maximum memory size of instance plan cache. - TiDBInstancePlanCacheMaxMemSize = "tidb_instance_plan_cache_max_size" - - // TiDBConstraintCheckInPlacePessimistic controls whether to skip certain kinds of pessimistic locks. - TiDBConstraintCheckInPlacePessimistic = "tidb_constraint_check_in_place_pessimistic" - - // TiDBEnableForeignKey indicates whether to enable foreign key feature. - // TODO(crazycs520): remove this after foreign key GA. - TiDBEnableForeignKey = "tidb_enable_foreign_key" - - // TiDBOptRangeMaxSize is the max memory limit for ranges. When the optimizer estimates that the memory usage of complete - // ranges would exceed the limit, it chooses less accurate ranges such as full range. 0 indicates that there is no memory - // limit for ranges. - TiDBOptRangeMaxSize = "tidb_opt_range_max_size" - - // TiDBOptAdvancedJoinHint indicates whether the join method hint is compatible with join order hint. - TiDBOptAdvancedJoinHint = "tidb_opt_advanced_join_hint" - // TiDBOptUseInvisibleIndexes indicates whether to use invisible indexes. - TiDBOptUseInvisibleIndexes = "tidb_opt_use_invisible_indexes" - // TiDBAnalyzePartitionConcurrency is the number of concurrent workers to save statistics to the system tables. - TiDBAnalyzePartitionConcurrency = "tidb_analyze_partition_concurrency" - // TiDBMergePartitionStatsConcurrency indicates the concurrency when merge partition stats into global stats - TiDBMergePartitionStatsConcurrency = "tidb_merge_partition_stats_concurrency" - // TiDBEnableAsyncMergeGlobalStats indicates whether to enable async merge global stats - TiDBEnableAsyncMergeGlobalStats = "tidb_enable_async_merge_global_stats" - // TiDBOptPrefixIndexSingleScan indicates whether to do some optimizations to avoid double scan for prefix index. - // When set to true, `col is (not) null`(`col` is index prefix column) is regarded as index filter rather than table filter. - TiDBOptPrefixIndexSingleScan = "tidb_opt_prefix_index_single_scan" - - // TiDBEnableExternalTSRead indicates whether to enable read through an external ts - TiDBEnableExternalTSRead = "tidb_enable_external_ts_read" - - // TiDBEnablePlanReplayerCapture indicates whether to enable plan replayer capture - TiDBEnablePlanReplayerCapture = "tidb_enable_plan_replayer_capture" - - // TiDBEnablePlanReplayerContinuousCapture indicates whether to enable continuous capture - TiDBEnablePlanReplayerContinuousCapture = "tidb_enable_plan_replayer_continuous_capture" - // TiDBEnableReusechunk indicates whether to enable chunk alloc - TiDBEnableReusechunk = "tidb_enable_reuse_chunk" - - // TiDBStoreBatchSize indicates the batch size of coprocessor in the same store. - TiDBStoreBatchSize = "tidb_store_batch_size" - - // MppExchangeCompressionMode indicates the data compression method in mpp exchange operator - MppExchangeCompressionMode = "mpp_exchange_compression_mode" - - // MppVersion indicates the mpp-version used to build mpp plan - MppVersion = "mpp_version" - - // TiDBPessimisticTransactionFairLocking controls whether fair locking for pessimistic transaction - // is enabled. - TiDBPessimisticTransactionFairLocking = "tidb_pessimistic_txn_fair_locking" - - // TiDBEnablePlanCacheForParamLimit controls whether prepare statement with parameterized limit can be cached - TiDBEnablePlanCacheForParamLimit = "tidb_enable_plan_cache_for_param_limit" - - // TiDBEnableINLJoinInnerMultiPattern indicates whether enable multi pattern for inner side of inl join - TiDBEnableINLJoinInnerMultiPattern = "tidb_enable_inl_join_inner_multi_pattern" - - // TiFlashComputeDispatchPolicy indicates how to dispatch task to tiflash_compute nodes. - TiFlashComputeDispatchPolicy = "tiflash_compute_dispatch_policy" - - // TiDBEnablePlanCacheForSubquery controls whether prepare statement with subquery can be cached - TiDBEnablePlanCacheForSubquery = "tidb_enable_plan_cache_for_subquery" - - // TiDBOptEnableLateMaterialization indicates whether to enable late materialization - TiDBOptEnableLateMaterialization = "tidb_opt_enable_late_materialization" - // TiDBLoadBasedReplicaReadThreshold is the wait duration threshold to enable replica read automatically. - TiDBLoadBasedReplicaReadThreshold = "tidb_load_based_replica_read_threshold" - - // TiDBOptOrderingIdxSelThresh is the threshold for optimizer to consider the ordering index. - TiDBOptOrderingIdxSelThresh = "tidb_opt_ordering_index_selectivity_threshold" - - // TiDBOptOrderingIdxSelRatio is the ratio the optimizer will assume applies when non indexed filtering rows are found - // via the ordering index. - TiDBOptOrderingIdxSelRatio = "tidb_opt_ordering_index_selectivity_ratio" - - // TiDBOptEnableMPPSharedCTEExecution indicates whether the optimizer try to build shared CTE scan during MPP execution. - TiDBOptEnableMPPSharedCTEExecution = "tidb_opt_enable_mpp_shared_cte_execution" - // TiDBOptFixControl makes the user able to control some details of the optimizer behavior. - TiDBOptFixControl = "tidb_opt_fix_control" - - // TiFlashReplicaRead is used to set the policy of TiFlash replica read when the query needs the TiFlash engine. - TiFlashReplicaRead = "tiflash_replica_read" - - // TiDBLockUnchangedKeys indicates whether to lock duplicate keys in INSERT IGNORE and REPLACE statements, - // or unchanged unique keys in UPDATE statements, see PR #42210 and #42713 - TiDBLockUnchangedKeys = "tidb_lock_unchanged_keys" - - // TiDBFastCheckTable enables fast check table. - TiDBFastCheckTable = "tidb_enable_fast_table_check" - - // TiDBAnalyzeSkipColumnTypes indicates the column types whose statistics would not be collected when executing the ANALYZE command. - TiDBAnalyzeSkipColumnTypes = "tidb_analyze_skip_column_types" - - // TiDBEnableCheckConstraint indicates whether to enable check constraint feature. - TiDBEnableCheckConstraint = "tidb_enable_check_constraint" - - // TiDBOptEnableHashJoin indicates whether to enable hash join. - TiDBOptEnableHashJoin = "tidb_opt_enable_hash_join" - - // TiDBHashJoinVersion indicates whether to use hash join implementation v2. - TiDBHashJoinVersion = "tidb_hash_join_version" - - // TiDBOptObjective indicates whether the optimizer should be more stable, predictable or more aggressive. - // Please see comments of SessionVars.OptObjective for details. - TiDBOptObjective = "tidb_opt_objective" - - // TiDBEnableParallelHashaggSpill is the name of the `tidb_enable_parallel_hashagg_spill` system variable - TiDBEnableParallelHashaggSpill = "tidb_enable_parallel_hashagg_spill" - - // TiDBTxnEntrySizeLimit indicates the max size of a entry in membuf. - TiDBTxnEntrySizeLimit = "tidb_txn_entry_size_limit" - - // TiDBSchemaCacheSize indicates the size of infoschema meta data which are cached in V2 implementation. - TiDBSchemaCacheSize = "tidb_schema_cache_size" - - // DivPrecisionIncrement indicates the number of digits by which to increase the scale of the result of - // division operations performed with the / operator. - DivPrecisionIncrement = "div_precision_increment" - - // TiDBEnableSharedLockPromotion indicates whether the `select for share` statement would be executed - // as `select for update` statements which do acquire pessimistic locks. - TiDBEnableSharedLockPromotion = "tidb_enable_shared_lock_promotion" -) - -// TiDB vars that have only global scope - -const ( - // TiDBGCEnable turns garbage collection on or OFF - TiDBGCEnable = "tidb_gc_enable" - // TiDBGCRunInterval sets the interval that GC runs - TiDBGCRunInterval = "tidb_gc_run_interval" - // TiDBGCLifetime sets the retention window of older versions - TiDBGCLifetime = "tidb_gc_life_time" - // TiDBGCConcurrency sets the concurrency of garbage collection. -1 = AUTO value - TiDBGCConcurrency = "tidb_gc_concurrency" - // TiDBGCScanLockMode enables the green GC feature (deprecated) - TiDBGCScanLockMode = "tidb_gc_scan_lock_mode" - // TiDBGCMaxWaitTime sets max time for gc advances the safepoint delayed by active transactions - TiDBGCMaxWaitTime = "tidb_gc_max_wait_time" - // TiDBEnableEnhancedSecurity restricts SUPER users from certain operations. - TiDBEnableEnhancedSecurity = "tidb_enable_enhanced_security" - // TiDBEnableHistoricalStats enables the historical statistics feature (default off) - TiDBEnableHistoricalStats = "tidb_enable_historical_stats" - // TiDBPersistAnalyzeOptions persists analyze options for later analyze and auto-analyze - TiDBPersistAnalyzeOptions = "tidb_persist_analyze_options" - // TiDBEnableColumnTracking enables collecting predicate columns. - // DEPRECATED: This variable is deprecated, please do not use this variable. - TiDBEnableColumnTracking = "tidb_enable_column_tracking" - // TiDBAnalyzeColumnOptions specifies the default column selection strategy for both manual and automatic analyze operations. - // It accepts two values: - // `PREDICATE`: Analyze only the columns that are used in the predicates of the query. - // `ALL`: Analyze all columns in the table. - TiDBAnalyzeColumnOptions = "tidb_analyze_column_options" - // TiDBDisableColumnTrackingTime records the last time TiDBEnableColumnTracking is set off. - // It is used to invalidate the collected predicate columns after turning off TiDBEnableColumnTracking, which avoids physical deletion. - // It doesn't have cache in memory, and we directly get/set the variable value from/to mysql.tidb. - // DEPRECATED: This variable is deprecated, please do not use this variable. - TiDBDisableColumnTrackingTime = "tidb_disable_column_tracking_time" - // TiDBStatsLoadPseudoTimeout indicates whether to fallback to pseudo stats after load timeout. - TiDBStatsLoadPseudoTimeout = "tidb_stats_load_pseudo_timeout" - // TiDBMemQuotaBindingCache indicates the memory quota for the bind cache. - TiDBMemQuotaBindingCache = "tidb_mem_quota_binding_cache" - // TiDBRCReadCheckTS indicates the tso optimization for read-consistency read is enabled. - TiDBRCReadCheckTS = "tidb_rc_read_check_ts" - // TiDBRCWriteCheckTs indicates whether some special write statements don't get latest tso from PD at RC - TiDBRCWriteCheckTs = "tidb_rc_write_check_ts" - // TiDBCommitterConcurrency controls the number of running concurrent requests in the commit phase. - TiDBCommitterConcurrency = "tidb_committer_concurrency" - // TiDBEnableBatchDML enables batch dml. - TiDBEnableBatchDML = "tidb_enable_batch_dml" - // TiDBStatsCacheMemQuota records stats cache quota - TiDBStatsCacheMemQuota = "tidb_stats_cache_mem_quota" - // TiDBMemQuotaAnalyze indicates the memory quota for all analyze jobs. - TiDBMemQuotaAnalyze = "tidb_mem_quota_analyze" - // TiDBEnableAutoAnalyze determines whether TiDB executes automatic analysis. - // In test, we disable it by default. See GlobalSystemVariableInitialValue for details. - TiDBEnableAutoAnalyze = "tidb_enable_auto_analyze" - // TiDBEnableAutoAnalyzePriorityQueue determines whether TiDB executes automatic analysis with priority queue. - TiDBEnableAutoAnalyzePriorityQueue = "tidb_enable_auto_analyze_priority_queue" - // TiDBMemOOMAction indicates what operation TiDB perform when a single SQL statement exceeds - // the memory quota specified by tidb_mem_quota_query and cannot be spilled to disk. - TiDBMemOOMAction = "tidb_mem_oom_action" - // TiDBPrepPlanCacheMemoryGuardRatio is used to prevent [performance.max-memory] from being exceeded - TiDBPrepPlanCacheMemoryGuardRatio = "tidb_prepared_plan_cache_memory_guard_ratio" - // TiDBMaxAutoAnalyzeTime is the max time that auto analyze can run. If auto analyze runs longer than the value, it - // will be killed. 0 indicates that there is no time limit. - TiDBMaxAutoAnalyzeTime = "tidb_max_auto_analyze_time" - // TiDBAutoAnalyzeConcurrency is the concurrency of the auto analyze - TiDBAutoAnalyzeConcurrency = "tidb_auto_analyze_concurrency" - // TiDBEnableDistTask indicates whether to enable the distributed execute background tasks(For example DDL, Import etc). - TiDBEnableDistTask = "tidb_enable_dist_task" - // TiDBEnableFastCreateTable indicates whether to enable the fast create table feature. - TiDBEnableFastCreateTable = "tidb_enable_fast_create_table" - // TiDBGenerateBinaryPlan indicates whether binary plan should be generated in slow log and statements summary. - TiDBGenerateBinaryPlan = "tidb_generate_binary_plan" - // TiDBEnableGCAwareMemoryTrack indicates whether to turn-on GC-aware memory track. - TiDBEnableGCAwareMemoryTrack = "tidb_enable_gc_aware_memory_track" - // TiDBEnableTmpStorageOnOOM controls whether to enable the temporary storage for some operators - // when a single SQL statement exceeds the memory quota specified by the memory quota. - TiDBEnableTmpStorageOnOOM = "tidb_enable_tmp_storage_on_oom" - // TiDBDDLEnableFastReorg indicates whether to use lighting backfill process for adding index. - TiDBDDLEnableFastReorg = "tidb_ddl_enable_fast_reorg" - // TiDBDDLDiskQuota used to set disk quota for lightning add index. - TiDBDDLDiskQuota = "tidb_ddl_disk_quota" - // TiDBCloudStorageURI used to set a cloud storage uri for ddl add index and import into. - TiDBCloudStorageURI = "tidb_cloud_storage_uri" - // TiDBAutoBuildStatsConcurrency is the number of concurrent workers to automatically analyze tables or partitions. - // It is very similar to the `tidb_build_stats_concurrency` variable, but it is used for the auto analyze feature. - TiDBAutoBuildStatsConcurrency = "tidb_auto_build_stats_concurrency" - // TiDBSysProcScanConcurrency is used to set the scan concurrency of for backend system processes, like auto-analyze. - // For now, it controls the number of concurrent workers to scan regions to collect statistics (FMSketch, Samples). - TiDBSysProcScanConcurrency = "tidb_sysproc_scan_concurrency" - // TiDBServerMemoryLimit indicates the memory limit of the tidb-server instance. - TiDBServerMemoryLimit = "tidb_server_memory_limit" - // TiDBServerMemoryLimitSessMinSize indicates the minimal memory used of a session, that becomes a candidate for session kill. - TiDBServerMemoryLimitSessMinSize = "tidb_server_memory_limit_sess_min_size" - // TiDBServerMemoryLimitGCTrigger indicates the gc percentage of the TiDBServerMemoryLimit. - TiDBServerMemoryLimitGCTrigger = "tidb_server_memory_limit_gc_trigger" - // TiDBEnableGOGCTuner is to enable GOGC tuner. it can tuner GOGC - TiDBEnableGOGCTuner = "tidb_enable_gogc_tuner" - // TiDBGOGCTunerThreshold is to control the threshold of GOGC tuner. - TiDBGOGCTunerThreshold = "tidb_gogc_tuner_threshold" - // TiDBGOGCTunerMaxValue is the max value of GOGC that GOGC tuner can change to. - TiDBGOGCTunerMaxValue = "tidb_gogc_tuner_max_value" - // TiDBGOGCTunerMinValue is the min value of GOGC that GOGC tuner can change to. - TiDBGOGCTunerMinValue = "tidb_gogc_tuner_min_value" - // TiDBExternalTS is the ts to read through when the `TiDBEnableExternalTsRead` is on - TiDBExternalTS = "tidb_external_ts" - // TiDBTTLJobEnable is used to enable/disable scheduling ttl job - TiDBTTLJobEnable = "tidb_ttl_job_enable" - // TiDBTTLScanBatchSize is used to control the batch size in the SELECT statement for TTL jobs - TiDBTTLScanBatchSize = "tidb_ttl_scan_batch_size" - // TiDBTTLDeleteBatchSize is used to control the batch size in the DELETE statement for TTL jobs - TiDBTTLDeleteBatchSize = "tidb_ttl_delete_batch_size" - // TiDBTTLDeleteRateLimit is used to control the delete rate limit for TTL jobs in each node - TiDBTTLDeleteRateLimit = "tidb_ttl_delete_rate_limit" - // TiDBTTLJobScheduleWindowStartTime is used to restrict the start time of the time window of scheduling the ttl jobs. - TiDBTTLJobScheduleWindowStartTime = "tidb_ttl_job_schedule_window_start_time" - // TiDBTTLJobScheduleWindowEndTime is used to restrict the end time of the time window of scheduling the ttl jobs. - TiDBTTLJobScheduleWindowEndTime = "tidb_ttl_job_schedule_window_end_time" - // TiDBTTLScanWorkerCount indicates the count of the scan workers in each TiDB node - TiDBTTLScanWorkerCount = "tidb_ttl_scan_worker_count" - // TiDBTTLDeleteWorkerCount indicates the count of the delete workers in each TiDB node - TiDBTTLDeleteWorkerCount = "tidb_ttl_delete_worker_count" - // PasswordReuseHistory limit a few passwords to reuse. - PasswordReuseHistory = "password_history" - // PasswordReuseTime limit how long passwords can be reused. - PasswordReuseTime = "password_reuse_interval" - // TiDBHistoricalStatsDuration indicates the duration to remain tidb historical stats - TiDBHistoricalStatsDuration = "tidb_historical_stats_duration" - // TiDBEnableHistoricalStatsForCapture indicates whether use historical stats in plan replayer capture - 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. - TiDBStmtSummaryFilename = "tidb_stmt_summary_filename" - // TiDBStmtSummaryFileMaxDays indicates how many days the files written by stmtsummary will be kept. - TiDBStmtSummaryFileMaxDays = "tidb_stmt_summary_file_max_days" - // TiDBStmtSummaryFileMaxSize indicates the maximum size (in mb) of a single file written by stmtsummary. - TiDBStmtSummaryFileMaxSize = "tidb_stmt_summary_file_max_size" - // TiDBStmtSummaryFileMaxBackups indicates the maximum number of files written by stmtsummary. - TiDBStmtSummaryFileMaxBackups = "tidb_stmt_summary_file_max_backups" - // TiDBTTLRunningTasks limits the count of running ttl tasks. Default to 0, means 3 times the count of TiKV (or no - // limitation, if the storage is not TiKV). - TiDBTTLRunningTasks = "tidb_ttl_running_tasks" - // AuthenticationLDAPSASLAuthMethodName defines the authentication method used by LDAP SASL authentication plugin - AuthenticationLDAPSASLAuthMethodName = "authentication_ldap_sasl_auth_method_name" - // AuthenticationLDAPSASLCAPath defines the ca certificate to verify LDAP connection in LDAP SASL authentication plugin - AuthenticationLDAPSASLCAPath = "authentication_ldap_sasl_ca_path" - // AuthenticationLDAPSASLTLS defines whether to use TLS connection in LDAP SASL authentication plugin - AuthenticationLDAPSASLTLS = "authentication_ldap_sasl_tls" - // AuthenticationLDAPSASLServerHost defines the server host of LDAP server for LDAP SASL authentication plugin - AuthenticationLDAPSASLServerHost = "authentication_ldap_sasl_server_host" - // AuthenticationLDAPSASLServerPort defines the port of LDAP server for LDAP SASL authentication plugin - AuthenticationLDAPSASLServerPort = "authentication_ldap_sasl_server_port" - // AuthenticationLDAPSASLReferral defines whether to enable LDAP referral for LDAP SASL authentication plugin - AuthenticationLDAPSASLReferral = "authentication_ldap_sasl_referral" - // AuthenticationLDAPSASLUserSearchAttr defines the attribute of username in LDAP server - AuthenticationLDAPSASLUserSearchAttr = "authentication_ldap_sasl_user_search_attr" - // AuthenticationLDAPSASLBindBaseDN defines the `dn` to search the users in. It's used to limit the search scope of TiDB. - AuthenticationLDAPSASLBindBaseDN = "authentication_ldap_sasl_bind_base_dn" - // AuthenticationLDAPSASLBindRootDN defines the `dn` of the user to login the LDAP server and perform search. - AuthenticationLDAPSASLBindRootDN = "authentication_ldap_sasl_bind_root_dn" - // AuthenticationLDAPSASLBindRootPWD defines the password of the user to login the LDAP server and perform search. - AuthenticationLDAPSASLBindRootPWD = "authentication_ldap_sasl_bind_root_pwd" - // AuthenticationLDAPSASLInitPoolSize defines the init size of connection pool to LDAP server for SASL plugin. - AuthenticationLDAPSASLInitPoolSize = "authentication_ldap_sasl_init_pool_size" - // AuthenticationLDAPSASLMaxPoolSize defines the max size of connection pool to LDAP server for SASL plugin. - AuthenticationLDAPSASLMaxPoolSize = "authentication_ldap_sasl_max_pool_size" - // AuthenticationLDAPSimpleAuthMethodName defines the authentication method used by LDAP Simple authentication plugin - AuthenticationLDAPSimpleAuthMethodName = "authentication_ldap_simple_auth_method_name" - // AuthenticationLDAPSimpleCAPath defines the ca certificate to verify LDAP connection in LDAP Simple authentication plugin - AuthenticationLDAPSimpleCAPath = "authentication_ldap_simple_ca_path" - // AuthenticationLDAPSimpleTLS defines whether to use TLS connection in LDAP Simple authentication plugin - AuthenticationLDAPSimpleTLS = "authentication_ldap_simple_tls" - // AuthenticationLDAPSimpleServerHost defines the server host of LDAP server for LDAP Simple authentication plugin - AuthenticationLDAPSimpleServerHost = "authentication_ldap_simple_server_host" - // AuthenticationLDAPSimpleServerPort defines the port of LDAP server for LDAP Simple authentication plugin - AuthenticationLDAPSimpleServerPort = "authentication_ldap_simple_server_port" - // AuthenticationLDAPSimpleReferral defines whether to enable LDAP referral for LDAP Simple authentication plugin - AuthenticationLDAPSimpleReferral = "authentication_ldap_simple_referral" - // AuthenticationLDAPSimpleUserSearchAttr defines the attribute of username in LDAP server - AuthenticationLDAPSimpleUserSearchAttr = "authentication_ldap_simple_user_search_attr" - // AuthenticationLDAPSimpleBindBaseDN defines the `dn` to search the users in. It's used to limit the search scope of TiDB. - AuthenticationLDAPSimpleBindBaseDN = "authentication_ldap_simple_bind_base_dn" - // AuthenticationLDAPSimpleBindRootDN defines the `dn` of the user to login the LDAP server and perform search. - AuthenticationLDAPSimpleBindRootDN = "authentication_ldap_simple_bind_root_dn" - // AuthenticationLDAPSimpleBindRootPWD defines the password of the user to login the LDAP server and perform search. - AuthenticationLDAPSimpleBindRootPWD = "authentication_ldap_simple_bind_root_pwd" - // AuthenticationLDAPSimpleInitPoolSize defines the init size of connection pool to LDAP server for SASL plugin. - AuthenticationLDAPSimpleInitPoolSize = "authentication_ldap_simple_init_pool_size" - // AuthenticationLDAPSimpleMaxPoolSize defines the max size of connection pool to LDAP server for SASL plugin. - AuthenticationLDAPSimpleMaxPoolSize = "authentication_ldap_simple_max_pool_size" - // TiDBRuntimeFilterTypeName the value of is string, a runtime filter type list split by ",", such as: "IN,MIN_MAX" - TiDBRuntimeFilterTypeName = "tidb_runtime_filter_type" - // TiDBRuntimeFilterModeName the mode of runtime filter, such as "OFF", "LOCAL" - TiDBRuntimeFilterModeName = "tidb_runtime_filter_mode" - // TiDBSkipMissingPartitionStats controls how to handle missing partition stats when merging partition stats to global stats. - // When set to true, skip missing partition stats and continue to merge other partition stats to global stats. - // When set to false, give up merging partition stats to global stats. - TiDBSkipMissingPartitionStats = "tidb_skip_missing_partition_stats" - // TiDBSessionAlias indicates the alias of a session which is used for tracing. - TiDBSessionAlias = "tidb_session_alias" - // TiDBServiceScope indicates the role for tidb for distributed task framework. - TiDBServiceScope = "tidb_service_scope" - // TiDBSchemaVersionCacheLimit defines the capacity size of domain infoSchema cache. - TiDBSchemaVersionCacheLimit = "tidb_schema_version_cache_limit" - // TiDBEnableTiFlashPipelineMode means if we should use pipeline model to execute query or not in tiflash. - // It's deprecated and setting it will not have any effect. - TiDBEnableTiFlashPipelineMode = "tidb_enable_tiflash_pipeline_model" - // TiDBIdleTransactionTimeout indicates the maximum time duration a transaction could be idle, unit is second. - // Any idle transaction will be killed after being idle for `tidb_idle_transaction_timeout` seconds. - // This is similar to https://docs.percona.com/percona-server/5.7/management/innodb_kill_idle_trx.html and https://mariadb.com/kb/en/transaction-timeouts/ - TiDBIdleTransactionTimeout = "tidb_idle_transaction_timeout" - // TiDBLowResolutionTSOUpdateInterval defines how often to refresh low resolution timestamps. - TiDBLowResolutionTSOUpdateInterval = "tidb_low_resolution_tso_update_interval" - // TiDBDMLType indicates the execution type of DML in TiDB. - // The value can be STANDARD, BULK. - // Currently, the BULK mode only affects auto-committed DML. - TiDBDMLType = "tidb_dml_type" - // TiFlashHashAggPreAggMode indicates the policy of 1st hashagg. - TiFlashHashAggPreAggMode = "tiflash_hashagg_preaggregation_mode" - // TiDBEnableLazyCursorFetch defines whether to enable the lazy cursor fetch. If it's `OFF`, all results of - // of a cursor will be stored in the tidb node in `EXECUTE` command. - TiDBEnableLazyCursorFetch = "tidb_enable_lazy_cursor_fetch" - // TiDBTSOClientRPCMode controls how the TSO client performs the TSO RPC requests. It internally controls the - // concurrency of the RPC. This variable provides an approach to tune the latency of getting timestamps from PD. - TiDBTSOClientRPCMode = "tidb_tso_client_rpc_mode" -) - -// TiDB intentional limits -// Can be raised in the future. - -const ( - // MaxConfigurableConcurrency is the maximum number of "threads" (goroutines) that can be specified - // for any type of configuration item that has concurrent workers. - MaxConfigurableConcurrency = 256 - - // MaxShardRowIDBits is the maximum number of bits that can be used for row-id sharding. - MaxShardRowIDBits = 15 - - // MaxPreSplitRegions is the maximum number of regions that can be pre-split. - MaxPreSplitRegions = 15 -) - -// Default TiDB system variable values. -const ( - DefHostname = "localhost" - DefIndexLookupConcurrency = ConcurrencyUnset - DefIndexLookupJoinConcurrency = ConcurrencyUnset - DefIndexSerialScanConcurrency = 1 - DefIndexJoinBatchSize = 25000 - DefIndexLookupSize = 20000 - DefDistSQLScanConcurrency = 15 - DefAnalyzeDistSQLScanConcurrency = 4 - DefBuildStatsConcurrency = 2 - DefBuildSamplingStatsConcurrency = 2 - DefAutoAnalyzeRatio = 0.5 - DefAutoAnalyzeStartTime = "00:00 +0000" - DefAutoAnalyzeEndTime = "23:59 +0000" - DefAutoIncrementIncrement = 1 - DefAutoIncrementOffset = 1 - DefChecksumTableConcurrency = 4 - DefSkipUTF8Check = false - DefSkipASCIICheck = false - DefOptAggPushDown = false - DefOptDeriveTopN = false - DefOptCartesianBCJ = 1 - DefOptMPPOuterJoinFixedBuildSide = false - DefOptWriteRowID = false - DefOptEnableCorrelationAdjustment = true - DefOptLimitPushDownThreshold = 100 - DefOptCorrelationThreshold = 0.9 - DefOptCorrelationExpFactor = 1 - DefOptCPUFactor = 3.0 - DefOptCopCPUFactor = 3.0 - DefOptTiFlashConcurrencyFactor = 24.0 - DefOptNetworkFactor = 1.0 - DefOptScanFactor = 1.5 - DefOptDescScanFactor = 3.0 - DefOptSeekFactor = 20.0 - DefOptMemoryFactor = 0.001 - DefOptDiskFactor = 1.5 - DefOptConcurrencyFactor = 3.0 - DefOptForceInlineCTE = false - DefOptInSubqToJoinAndAgg = true - DefOptPreferRangeScan = true - DefBatchInsert = false - DefBatchDelete = false - DefBatchCommit = false - DefCurretTS = 0 - DefInitChunkSize = 32 - DefMinPagingSize = int(paging.MinPagingSize) - DefMaxPagingSize = int(paging.MaxPagingSize) - DefMaxChunkSize = 1024 - DefDMLBatchSize = 0 - DefMaxPreparedStmtCount = -1 - DefWaitTimeout = 28800 - DefTiDBMemQuotaApplyCache = 32 << 20 // 32MB. - DefTiDBMemQuotaBindingCache = 64 << 20 // 64MB. - DefTiDBGeneralLog = false - DefTiDBPProfSQLCPU = 0 - DefTiDBRetryLimit = 10 - DefTiDBDisableTxnAutoRetry = true - DefTiDBConstraintCheckInPlace = false - DefTiDBHashJoinConcurrency = ConcurrencyUnset - DefTiDBProjectionConcurrency = ConcurrencyUnset - DefBroadcastJoinThresholdSize = 100 * 1024 * 1024 - DefBroadcastJoinThresholdCount = 10 * 1024 - DefPreferBCJByExchangeDataSize = false - DefTiDBOptimizerSelectivityLevel = 0 - DefTiDBOptimizerEnableNewOFGB = false - DefTiDBEnableOuterJoinReorder = true - DefTiDBEnableNAAJ = true - DefTiDBAllowBatchCop = 1 - DefShardRowIDBits = 0 - DefPreSplitRegions = 0 - DefBlockEncryptionMode = "aes-128-ecb" - DefTiDBAllowMPPExecution = true - DefTiDBAllowTiFlashCop = false - DefTiDBHashExchangeWithNewCollation = true - DefTiDBEnforceMPPExecution = false - DefTiFlashMaxThreads = -1 - DefTiFlashMaxBytesBeforeExternalJoin = -1 - DefTiFlashMaxBytesBeforeExternalGroupBy = -1 - DefTiFlashMaxBytesBeforeExternalSort = -1 - DefTiFlashMemQuotaQueryPerNode = 0 - DefTiFlashQuerySpillRatio = 0.7 - DefTiDBEnableTiFlashPipelineMode = true - DefTiDBMPPStoreFailTTL = "60s" - DefTiDBTxnMode = PessimisticTxnMode - DefTiDBRowFormatV1 = 1 - DefTiDBRowFormatV2 = 2 - DefTiDBDDLReorgWorkerCount = 4 - DefTiDBDDLReorgBatchSize = 256 - DefTiDBDDLFlashbackConcurrency = 64 - DefTiDBDDLErrorCountLimit = 512 - DefTiDBDDLReorgMaxWriteSpeed = 0 - DefTiDBMaxDeltaSchemaCount = 1024 - DefTiDBPlacementMode = PlacementModeStrict - DefTiDBEnableAutoIncrementInGenerated = false - DefTiDBHashAggPartialConcurrency = ConcurrencyUnset - DefTiDBHashAggFinalConcurrency = ConcurrencyUnset - DefTiDBWindowConcurrency = ConcurrencyUnset - DefTiDBMergeJoinConcurrency = 1 // disable optimization by default - DefTiDBStreamAggConcurrency = 1 - DefTiDBForcePriority = mysql.NoPriority - DefEnableWindowFunction = true - DefEnablePipelinedWindowFunction = true - DefEnableStrictDoubleTypeCheck = true - DefEnableVectorizedExpression = true - DefTiDBOptJoinReorderThreshold = 0 - DefTiDBDDLSlowOprThreshold = 300 - DefTiDBUseFastAnalyze = false - DefTiDBSkipIsolationLevelCheck = false - DefTiDBExpensiveQueryTimeThreshold = 60 // 60s - DefTiDBExpensiveTxnTimeThreshold = 60 * 10 // 10 minutes - DefTiDBScatterRegion = ScatterOff - DefTiDBWaitSplitRegionFinish = true - DefWaitSplitRegionTimeout = 300 // 300s - DefTiDBEnableNoopFuncs = Off - DefTiDBEnableNoopVariables = true - DefTiDBAllowRemoveAutoInc = false - DefTiDBUsePlanBaselines = true - DefTiDBEvolvePlanBaselines = false - DefTiDBEvolvePlanTaskMaxTime = 600 // 600s - DefTiDBEvolvePlanTaskStartTime = "00:00 +0000" - DefTiDBEvolvePlanTaskEndTime = "23:59 +0000" - DefInnodbLockWaitTimeout = 50 // 50s - DefTiDBStoreLimit = 0 - DefTiDBMetricSchemaStep = 60 // 60s - DefTiDBMetricSchemaRangeDuration = 60 // 60s - DefTiDBFoundInPlanCache = false - DefTiDBFoundInBinding = false - DefTiDBEnableCollectExecutionInfo = true - DefTiDBAllowAutoRandExplicitInsert = false - DefTiDBEnableClusteredIndex = ClusteredIndexDefModeOn - DefTiDBRedactLog = Off - DefTiDBRestrictedReadOnly = false - DefTiDBSuperReadOnly = false - DefTiDBShardAllocateStep = math.MaxInt64 - DefTiDBEnableTelemetry = false - DefTiDBEnableParallelApply = false - DefTiDBPartitionPruneMode = "dynamic" - DefTiDBEnableRateLimitAction = false - DefTiDBEnableAsyncCommit = false - DefTiDBEnable1PC = false - DefTiDBGuaranteeLinearizability = true - DefTiDBAnalyzeVersion = 2 - // Deprecated: This variable is deprecated, please do not use this variable. - DefTiDBAutoAnalyzePartitionBatchSize = mysql.PartitionCountLimit - DefTiDBEnableIndexMergeJoin = false - DefTiDBTrackAggregateMemoryUsage = true - DefCTEMaxRecursionDepth = 1000 - DefTiDBTmpTableMaxSize = 64 << 20 // 64MB. - DefTiDBEnableLocalTxn = false - DefTiDBTSOClientBatchMaxWaitTime = 0.0 // 0ms - DefTiDBEnableTSOFollowerProxy = false - DefPDEnableFollowerHandleRegion = true - DefTiDBEnableOrderedResultMode = false - DefTiDBEnablePseudoForOutdatedStats = false - DefTiDBRegardNULLAsPoint = true - DefEnablePlacementCheck = true - DefTimestamp = "0" - DefTimestampFloat = 0.0 - DefTiDBEnableStmtSummary = true - DefTiDBStmtSummaryInternalQuery = false - DefTiDBStmtSummaryRefreshInterval = 1800 - DefTiDBStmtSummaryHistorySize = 24 - DefTiDBStmtSummaryMaxStmtCount = 3000 - DefTiDBStmtSummaryMaxSQLLength = 4096 - DefTiDBCapturePlanBaseline = Off - DefTiDBIgnoreInlistPlanDigest = false - DefTiDBEnableIndexMerge = true - DefEnableLegacyInstanceScope = true - DefTiDBTableCacheLease = 3 // 3s - DefTiDBPersistAnalyzeOptions = true - DefTiDBStatsLoadSyncWait = 100 - DefTiDBStatsLoadPseudoTimeout = true - DefSysdateIsNow = false - DefTiDBEnableParallelHashaggSpill = true - DefTiDBEnableMutationChecker = false - DefTiDBTxnAssertionLevel = AssertionOffStr - DefTiDBIgnorePreparedCacheCloseStmt = false - DefTiDBBatchPendingTiFlashCount = 4000 - DefRCReadCheckTS = false - DefTiDBRemoveOrderbyInSubquery = true - DefTiDBSkewDistinctAgg = false - DefTiDB3StageDistinctAgg = true - DefTiDB3StageMultiDistinctAgg = false - DefTiDBOptExplainEvaledSubquery = false - DefTiDBReadStaleness = 0 - DefTiDBGCMaxWaitTime = 24 * 60 * 60 - DefMaxAllowedPacket uint64 = 67108864 - DefTiDBEnableBatchDML = false - DefTiDBMemQuotaQuery = memory.DefMemQuotaQuery // 1GB - DefTiDBStatsCacheMemQuota = 0 - MaxTiDBStatsCacheMemQuota = 1024 * 1024 * 1024 * 1024 // 1TB - DefTiDBQueryLogMaxLen = 4096 - DefRequireSecureTransport = false - DefTiDBCommitterConcurrency = 128 - DefTiDBBatchDMLIgnoreError = false - DefTiDBMemQuotaAnalyze = -1 - DefTiDBEnableAutoAnalyze = true - DefTiDBEnableAutoAnalyzePriorityQueue = true - DefTiDBAnalyzeColumnOptions = "PREDICATE" - DefTiDBMemOOMAction = "CANCEL" - DefTiDBMaxAutoAnalyzeTime = 12 * 60 * 60 - DefTiDBAutoAnalyzeConcurrency = 1 - DefTiDBEnablePrepPlanCache = true - DefTiDBPrepPlanCacheSize = 100 - DefTiDBSessionPlanCacheSize = 100 - DefTiDBEnablePrepPlanCacheMemoryMonitor = true - DefTiDBPrepPlanCacheMemoryGuardRatio = 0.1 - DefTiDBEnableWorkloadBasedLearning = false - DefTiDBWorkloadBasedLearningInterval = 24 * time.Hour - DefTiDBEnableDistTask = true - DefTiDBEnableFastCreateTable = true - DefTiDBSimplifiedMetrics = false - DefTiDBEnablePaging = true - DefTiFlashFineGrainedShuffleStreamCount = 0 - DefStreamCountWhenMaxThreadsNotSet = 8 - DefTiFlashFineGrainedShuffleBatchSize = 8192 - DefAdaptiveClosestReadThreshold = 4096 - DefTiDBEnableAnalyzeSnapshot = false - DefTiDBGenerateBinaryPlan = true - DefEnableTiDBGCAwareMemoryTrack = false - DefTiDBDefaultStrMatchSelectivity = 0.8 - DefTiDBEnableTmpStorageOnOOM = true - DefTiDBEnableMDL = true - DefTiFlashFastScan = false - DefMemoryUsageAlarmRatio = 0.7 - DefMemoryUsageAlarmKeepRecordNum = 5 - DefTiDBEnableFastReorg = true - DefTiDBDDLDiskQuota = 100 * 1024 * 1024 * 1024 // 100GB - DefExecutorConcurrency = 5 - DefTiDBEnableNonPreparedPlanCache = false - DefTiDBEnableNonPreparedPlanCacheForDML = false - DefTiDBNonPreparedPlanCacheSize = 100 - DefTiDBPlanCacheMaxPlanSize = 2 * size.MB - DefTiDBInstancePlanCacheMaxMemSize = 100 * size.MB - MinTiDBInstancePlanCacheMemSize = 100 * size.MB - DefTiDBInstancePlanCacheReservedPercentage = 0.1 - // MaxDDLReorgBatchSize is exported for testing. - MaxDDLReorgBatchSize int32 = 10240 - MinDDLReorgBatchSize int32 = 32 - MinExpensiveQueryTimeThreshold uint64 = 10 // 10s - MinExpensiveTxnTimeThreshold uint64 = 60 // 60s - DefTiDBAutoBuildStatsConcurrency = 1 - DefTiDBSysProcScanConcurrency = 1 - DefTiDBRcWriteCheckTs = false - DefTiDBForeignKeyChecks = true - DefTiDBOptAdvancedJoinHint = true - DefTiDBAnalyzePartitionConcurrency = 2 - DefTiDBOptRangeMaxSize = 64 * int64(size.MB) // 64 MB - DefTiDBCostModelVer = 2 - DefTiDBServerMemoryLimitSessMinSize = 128 << 20 - DefTiDBMergePartitionStatsConcurrency = 1 - DefTiDBServerMemoryLimitGCTrigger = 0.7 - DefTiDBEnableGOGCTuner = true - // DefTiDBGOGCTunerThreshold is to limit TiDBGOGCTunerThreshold. - DefTiDBGOGCTunerThreshold float64 = 0.6 - DefTiDBGOGCMaxValue = 500 - DefTiDBGOGCMinValue = 100 - DefTiDBOptPrefixIndexSingleScan = true - DefTiDBEnableAsyncMergeGlobalStats = true - DefTiDBExternalTS = 0 - DefTiDBEnableExternalTSRead = false - DefTiDBEnableReusechunk = true - DefTiDBUseAlloc = false - DefTiDBEnablePlanReplayerCapture = true - DefTiDBIndexMergeIntersectionConcurrency = ConcurrencyUnset - DefTiDBTTLJobEnable = true - DefTiDBTTLScanBatchSize = 500 - DefTiDBTTLScanBatchMaxSize = 10240 - DefTiDBTTLScanBatchMinSize = 1 - DefTiDBTTLDeleteBatchSize = 100 - DefTiDBTTLDeleteBatchMaxSize = 10240 - DefTiDBTTLDeleteBatchMinSize = 1 - DefTiDBTTLDeleteRateLimit = 0 - DefTiDBTTLRunningTasks = -1 - DefPasswordReuseHistory = 0 - DefPasswordReuseTime = 0 - DefTiDBStoreBatchSize = 4 - DefTiDBHistoricalStatsDuration = 7 * 24 * time.Hour - DefTiDBEnableHistoricalStatsForCapture = false - DefTiDBTTLJobScheduleWindowStartTime = "00:00 +0000" - DefTiDBTTLJobScheduleWindowEndTime = "23:59 +0000" - DefTiDBTTLScanWorkerCount = 4 - DefTiDBTTLDeleteWorkerCount = 4 - DefaultExchangeCompressionMode = kv.ExchangeCompressionModeUnspecified - DefTiDBEnableResourceControl = true - DefTiDBResourceControlStrictMode = true - DefTiDBPessimisticTransactionFairLocking = false - DefTiDBEnablePlanCacheForParamLimit = true - DefTiDBEnableINLJoinMultiPattern = true - DefTiFlashComputeDispatchPolicy = tiflashcompute.DispatchPolicyConsistentHashStr - DefTiDBEnablePlanCacheForSubquery = true - DefTiDBLoadBasedReplicaReadThreshold = time.Second - DefTiDBOptEnableLateMaterialization = true - DefTiDBOptOrderingIdxSelThresh = 0.0 - DefTiDBOptOrderingIdxSelRatio = -1 - DefTiDBOptEnableMPPSharedCTEExecution = false - DefTiDBPlanCacheInvalidationOnFreshStats = true - DefTiDBEnableRowLevelChecksum = false - DefAuthenticationLDAPSASLAuthMethodName = "SCRAM-SHA-1" - DefAuthenticationLDAPSASLServerPort = 389 - DefAuthenticationLDAPSASLTLS = false - DefAuthenticationLDAPSASLUserSearchAttr = "uid" - DefAuthenticationLDAPSASLInitPoolSize = 10 - DefAuthenticationLDAPSASLMaxPoolSize = 1000 - DefAuthenticationLDAPSimpleAuthMethodName = "SIMPLE" - DefAuthenticationLDAPSimpleServerPort = 389 - DefAuthenticationLDAPSimpleTLS = false - DefAuthenticationLDAPSimpleUserSearchAttr = "uid" - DefAuthenticationLDAPSimpleInitPoolSize = 10 - DefAuthenticationLDAPSimpleMaxPoolSize = 1000 - DefTiFlashReplicaRead = tiflash.AllReplicaStr - DefTiDBEnableFastCheckTable = true - DefRuntimeFilterType = "IN" - DefRuntimeFilterMode = "OFF" - DefTiDBLockUnchangedKeys = true - DefTiDBEnableCheckConstraint = false - DefTiDBSkipMissingPartitionStats = true - DefTiDBOptEnableHashJoin = true - DefTiDBHashJoinVersion = joinversion.HashJoinVersionOptimized - DefTiDBOptObjective = OptObjectiveModerate - DefTiDBSchemaVersionCacheLimit = 16 - DefTiDBIdleTransactionTimeout = 0 - DefTiDBTxnEntrySizeLimit = 0 - DefTiDBSchemaCacheSize = 512 * 1024 * 1024 - DefTiDBLowResolutionTSOUpdateInterval = 2000 - DefDivPrecisionIncrement = 4 - DefTiDBDMLType = "STANDARD" - DefGroupConcatMaxLen = uint64(1024) - DefDefaultWeekFormat = "0" - DefTiFlashPreAggMode = ForcePreAggStr - DefTiDBEnableLazyCursorFetch = false - DefOptEnableProjectionPushDown = true - DefTiDBEnableSharedLockPromotion = false - DefTiDBTSOClientRPCMode = TSOClientRPCModeDefault -) - -// Process global variables. -var ( - ProcessGeneralLog = atomic.NewBool(false) - RunAutoAnalyze = atomic.NewBool(DefTiDBEnableAutoAnalyze) - EnableAutoAnalyzePriorityQueue = atomic.NewBool(DefTiDBEnableAutoAnalyzePriorityQueue) - // AnalyzeColumnOptions is a global variable that indicates the default column choice for ANALYZE. - // The value of this variable is a string that can be one of the following values: - // "PREDICATE", "ALL". - // The behavior of the analyze operation depends on the value of `tidb_persist_analyze_options`: - // 1. If `tidb_persist_analyze_options` is enabled and the column choice from the analyze options record is set to `default`, - // the value of `tidb_analyze_column_options` determines the behavior of the analyze operation. - // 2. If `tidb_persist_analyze_options` is disabled, `tidb_analyze_column_options` is used directly to decide - // whether to analyze all columns or just the predicate columns. - AnalyzeColumnOptions = atomic.NewString(DefTiDBAnalyzeColumnOptions) - GlobalLogMaxDays = atomic.NewInt32(int32(config.GetGlobalConfig().Log.File.MaxDays)) - QueryLogMaxLen = atomic.NewInt32(DefTiDBQueryLogMaxLen) - EnablePProfSQLCPU = atomic.NewBool(false) - EnableBatchDML = atomic.NewBool(false) - EnableTmpStorageOnOOM = atomic.NewBool(DefTiDBEnableTmpStorageOnOOM) - ddlReorgWorkerCounter int32 = DefTiDBDDLReorgWorkerCount - ddlReorgBatchSize int32 = DefTiDBDDLReorgBatchSize - ddlFlashbackConcurrency int32 = DefTiDBDDLFlashbackConcurrency - ddlErrorCountLimit int64 = DefTiDBDDLErrorCountLimit - ddlReorgRowFormat int64 = DefTiDBRowFormatV2 - DDLReorgMaxWriteSpeed = atomic.NewInt64(DefTiDBDDLReorgMaxWriteSpeed) - maxDeltaSchemaCount int64 = DefTiDBMaxDeltaSchemaCount - // DDLSlowOprThreshold is the threshold for ddl slow operations, uint is millisecond. - DDLSlowOprThreshold = config.GetGlobalConfig().Instance.DDLSlowOprThreshold - ForcePriority = int32(DefTiDBForcePriority) - MaxOfMaxAllowedPacket uint64 = 1073741824 - ExpensiveQueryTimeThreshold uint64 = DefTiDBExpensiveQueryTimeThreshold - ExpensiveTxnTimeThreshold uint64 = DefTiDBExpensiveTxnTimeThreshold - MemoryUsageAlarmRatio = atomic.NewFloat64(DefMemoryUsageAlarmRatio) - MemoryUsageAlarmKeepRecordNum = atomic.NewInt64(DefMemoryUsageAlarmKeepRecordNum) - EnableLocalTxn = atomic.NewBool(DefTiDBEnableLocalTxn) - MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) - EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) - EnablePDFollowerHandleRegion = atomic.NewBool(DefPDEnableFollowerHandleRegion) - RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) - VarTiDBSuperReadOnly = atomic.NewBool(DefTiDBSuperReadOnly) - PersistAnalyzeOptions = atomic.NewBool(DefTiDBPersistAnalyzeOptions) - TableCacheLease = atomic.NewInt64(DefTiDBTableCacheLease) - StatsLoadSyncWait = atomic.NewInt64(DefTiDBStatsLoadSyncWait) - StatsLoadPseudoTimeout = atomic.NewBool(DefTiDBStatsLoadPseudoTimeout) - MemQuotaBindingCache = atomic.NewInt64(DefTiDBMemQuotaBindingCache) - GCMaxWaitTime = atomic.NewInt64(DefTiDBGCMaxWaitTime) - StatsCacheMemQuota = atomic.NewInt64(DefTiDBStatsCacheMemQuota) - OOMAction = atomic.NewString(DefTiDBMemOOMAction) - MaxAutoAnalyzeTime = atomic.NewInt64(DefTiDBMaxAutoAnalyzeTime) - // variables for plan cache - PreparedPlanCacheMemoryGuardRatio = atomic.NewFloat64(DefTiDBPrepPlanCacheMemoryGuardRatio) - EnableInstancePlanCache = atomic.NewBool(false) - InstancePlanCacheReservedPercentage = atomic.NewFloat64(0.1) - InstancePlanCacheMaxMemSize = atomic.NewInt64(int64(DefTiDBInstancePlanCacheMaxMemSize)) - EnableDistTask = atomic.NewBool(DefTiDBEnableDistTask) - EnableFastCreateTable = atomic.NewBool(DefTiDBEnableFastCreateTable) - EnableNoopVariables = atomic.NewBool(DefTiDBEnableNoopVariables) - EnableMDL = atomic.NewBool(false) - AutoAnalyzePartitionBatchSize = atomic.NewInt64(DefTiDBAutoAnalyzePartitionBatchSize) - AutoAnalyzeConcurrency = atomic.NewInt32(DefTiDBAutoAnalyzeConcurrency) - // TODO: set value by session variable - EnableWorkloadBasedLearning = atomic.NewBool(DefTiDBEnableWorkloadBasedLearning) - WorkloadBasedLearningInterval = atomic.NewDuration(DefTiDBWorkloadBasedLearningInterval) - // EnableFastReorg indicates whether to use lightning to enhance DDL reorg performance. - EnableFastReorg = atomic.NewBool(DefTiDBEnableFastReorg) - // DDLDiskQuota is the temporary variable for set disk quota for lightning - DDLDiskQuota = atomic.NewUint64(DefTiDBDDLDiskQuota) - // EnableForeignKey indicates whether to enable foreign key feature. - EnableForeignKey = atomic.NewBool(true) - EnableRCReadCheckTS = atomic.NewBool(false) - // EnableRowLevelChecksum indicates whether to append checksum to row values. - EnableRowLevelChecksum = atomic.NewBool(DefTiDBEnableRowLevelChecksum) - LowResolutionTSOUpdateInterval = atomic.NewUint32(DefTiDBLowResolutionTSOUpdateInterval) - - // DefTiDBServerMemoryLimit indicates the default value of TiDBServerMemoryLimit(TotalMem * 80%). - // It should be a const and shouldn't be modified after tidb is started. - DefTiDBServerMemoryLimit = serverMemoryLimitDefaultValue() - GOGCTunerThreshold = atomic.NewFloat64(DefTiDBGOGCTunerThreshold) - PasswordValidationLength = atomic.NewInt32(8) - PasswordValidationMixedCaseCount = atomic.NewInt32(1) - PasswordValidtaionNumberCount = atomic.NewInt32(1) - PasswordValidationSpecialCharCount = atomic.NewInt32(1) - EnableTTLJob = atomic.NewBool(DefTiDBTTLJobEnable) - TTLScanBatchSize = atomic.NewInt64(DefTiDBTTLScanBatchSize) - TTLDeleteBatchSize = atomic.NewInt64(DefTiDBTTLDeleteBatchSize) - TTLDeleteRateLimit = atomic.NewInt64(DefTiDBTTLDeleteRateLimit) - TTLJobScheduleWindowStartTime = atomic.NewTime( - mustParseTime( - FullDayTimeFormat, - DefTiDBTTLJobScheduleWindowStartTime, - ), - ) - TTLJobScheduleWindowEndTime = atomic.NewTime( - mustParseTime( - FullDayTimeFormat, - DefTiDBTTLJobScheduleWindowEndTime, - ), - ) - TTLScanWorkerCount = atomic.NewInt32(DefTiDBTTLScanWorkerCount) - TTLDeleteWorkerCount = atomic.NewInt32(DefTiDBTTLDeleteWorkerCount) - PasswordHistory = atomic.NewInt64(DefPasswordReuseHistory) - PasswordReuseInterval = atomic.NewInt64(DefPasswordReuseTime) - IsSandBoxModeEnabled = atomic.NewBool(false) - MaxPreparedStmtCountValue = atomic.NewInt64(DefMaxPreparedStmtCount) - HistoricalStatsDuration = atomic.NewDuration(DefTiDBHistoricalStatsDuration) - EnableHistoricalStatsForCapture = atomic.NewBool(DefTiDBEnableHistoricalStatsForCapture) - 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) - 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.NewUint64(DefTiDBSchemaCacheSize) - SchemaCacheSizeOriginText = atomic.NewString(strconv.Itoa(DefTiDBSchemaCacheSize)) -) - var ( // SetMemQuotaAnalyze is the func registered by global/subglobal tracker to set memory quota. SetMemQuotaAnalyze func(quota int64) = nil @@ -1739,29 +63,3 @@ var ( // DisableGlobalResourceControlFunc is the function registered by tikv_driver to unset cluster resource control. DisableGlobalResourceControlFunc = func() {} ) - -func serverMemoryLimitDefaultValue() string { - total, err := memory.MemTotal() - if err == nil && total != 0 { - return "80%" - } - return "0" -} - -func mustParseDuration(str string) time.Duration { - duration, err := time.ParseDuration(str) - if err != nil { - panic(fmt.Sprintf("%s is not a duration", str)) - } - - return duration -} - -func mustParseTime(layout string, str string) time.Time { - time, err := time.ParseInLocation(layout, str, time.UTC) - if err != nil { - panic(fmt.Sprintf("%s is not in %s duration format", str, layout)) - } - - return time -} diff --git a/pkg/sessionctx/variable/variable.go b/pkg/sessionctx/variable/variable.go index aceb862f8f77b..556abca843a66 100644 --- a/pkg/sessionctx/variable/variable.go +++ b/pkg/sessionctx/variable/variable.go @@ -23,114 +23,22 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "golang.org/x/exp/maps" ) -// ScopeFlag is for system variable whether can be changed in global/session dynamically or not. -type ScopeFlag uint8 - -// TypeFlag is the SysVar type, which doesn't exactly match MySQL types. -type TypeFlag byte - -const ( - // ScopeNone means the system variable can not be changed dynamically. - ScopeNone ScopeFlag = 0 - // ScopeGlobal means the system variable can be changed globally. - ScopeGlobal ScopeFlag = 1 << 0 - // ScopeSession means the system variable can only be changed in current session. - ScopeSession ScopeFlag = 1 << 1 - // ScopeInstance means it is similar to global but doesn't propagate to other TiDB servers. - ScopeInstance ScopeFlag = 1 << 2 - - // TypeStr is the default - TypeStr TypeFlag = iota - // TypeBool for boolean - TypeBool - // TypeInt for integer - TypeInt - // TypeEnum for Enum - TypeEnum - // TypeFloat for Double - TypeFloat - // TypeUnsigned for Unsigned integer - TypeUnsigned - // TypeTime for time of day (a TiDB extension) - TypeTime - // TypeDuration for a golang duration (a TiDB extension) - TypeDuration - - // On is the canonical string for ON - On = "ON" - // Off is the canonical string for OFF - Off = "OFF" - // Warn means return warnings - Warn = "WARN" - // IntOnly means enable for int type - IntOnly = "INT_ONLY" - // Marker is a special log redact behavior - Marker = "MARKER" - - // AssertionStrictStr is a choice of variable TiDBTxnAssertionLevel that means full assertions should be performed, - // even if the performance might be slowed down. - AssertionStrictStr = "STRICT" - // AssertionFastStr is a choice of variable TiDBTxnAssertionLevel that means assertions that doesn't affect - // performance should be performed. - AssertionFastStr = "FAST" - // AssertionOffStr is a choice of variable TiDBTxnAssertionLevel that means no assertion should be performed. - AssertionOffStr = "OFF" - // OOMActionCancel constants represents the valid action configurations for OOMAction "CANCEL". - OOMActionCancel = "CANCEL" - // OOMActionLog constants represents the valid action configurations for OOMAction "LOG". - OOMActionLog = "LOG" - - // TSOClientRPCModeDefault is a choice of variable TiDBTSOClientRPCMode. In this mode, the TSO client sends batched - // TSO requests serially. - TSOClientRPCModeDefault = "DEFAULT" - // TSOClientRPCModeParallel is a choice of variable TiDBTSOClientRPCMode. In this mode, the TSO client tries to - // keep approximately 2 batched TSO requests running in parallel. This option tries to reduce the batch-waiting time - // by half, at the expense of about twice the amount of TSO RPC calls. - TSOClientRPCModeParallel = "PARALLEL" - // TSOClientRPCModeParallelFast is a choice of variable TiDBTSOClientRPCMode. In this mode, the TSO client tries to - // keep approximately 4 batched TSO requests running in parallel. This option tries to reduce the batch-waiting time - // by 3/4, at the expense of about 4 times the amount of TSO RPC calls. - TSOClientRPCModeParallelFast = "PARALLEL-FAST" -) - -// Global config name list. -const ( - GlobalConfigEnableTopSQL = "enable_resource_metering" - GlobalConfigSourceID = "source_id" -) - -func (s ScopeFlag) String() string { - var scopes []string - if s == ScopeNone { - return "NONE" - } - if s&ScopeSession != 0 { - scopes = append(scopes, "SESSION") - } - if s&ScopeGlobal != 0 { - scopes = append(scopes, "GLOBAL") - } - if s&ScopeInstance != 0 { - scopes = append(scopes, "INSTANCE") - } - return strings.Join(scopes, ",") -} - // SysVar is for system variable. // All the fields of SysVar should be READ ONLY after created. type SysVar struct { // Scope is for whether can be changed or not - Scope ScopeFlag + Scope vardef.ScopeFlag // Name is the variable name. Name string // Value is the variable value. Value string // Type is the MySQL type (optional) - Type TypeFlag + Type vardef.TypeFlag // MinValue will automatically be validated when specified (optional) MinValue int64 // MaxValue will automatically be validated when specified (optional) @@ -148,7 +56,7 @@ type SysVar struct { // AllowAutoValue means that the special value "-1" is permitted, even when outside of range. AllowAutoValue bool // Validation is a callback after the type validation has been performed, but before the Set function - Validation func(*SessionVars, string, string, ScopeFlag) (string, error) + Validation func(*SessionVars, string, string, vardef.ScopeFlag) (string, error) // SetSession is called after validation but before updating systems[]. It also doubles as an Init function // and will be called on all variables in builtinGlobalVariable, regardless of their scope. SetSession func(*SessionVars, string) error @@ -198,7 +106,7 @@ func (sv *SysVar) GetGlobalFromHook(ctx context.Context, s *SessionVars) (string } // Ensure that the results from the getter are validated // Since some are read directly from tables. - return sv.ValidateWithRelaxedValidation(s, val, ScopeGlobal), nil + return sv.ValidateWithRelaxedValidation(s, val, vardef.ScopeGlobal), nil } if sv.HasNoneScope() { return sv.Value, nil @@ -219,7 +127,7 @@ func (sv *SysVar) GetSessionFromHook(s *SessionVars) (string, error) { } // Ensure that the results from the getter are validated // Since some are read directly from tables. - return sv.ValidateWithRelaxedValidation(s, val, ScopeSession), nil + return sv.ValidateWithRelaxedValidation(s, val, vardef.ScopeSession), nil } var ( ok bool @@ -282,26 +190,26 @@ func (sv *SysVar) SetGlobalFromHook(ctx context.Context, s *SessionVars, val str // HasNoneScope returns true if the scope for the sysVar is None. func (sv *SysVar) HasNoneScope() bool { - return sv.Scope == ScopeNone + return sv.Scope == vardef.ScopeNone } // HasSessionScope returns true if the scope for the sysVar includes session. func (sv *SysVar) HasSessionScope() bool { - return sv.Scope&ScopeSession != 0 + return sv.Scope&vardef.ScopeSession != 0 } // HasGlobalScope returns true if the scope for the sysVar includes global. func (sv *SysVar) HasGlobalScope() bool { - return sv.Scope&ScopeGlobal != 0 + return sv.Scope&vardef.ScopeGlobal != 0 } // HasInstanceScope returns true if the scope for the sysVar includes instance func (sv *SysVar) HasInstanceScope() bool { - return sv.Scope&ScopeInstance != 0 + return sv.Scope&vardef.ScopeInstance != 0 } // Validate checks if system variable satisfies specific restriction. -func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (string, error) { +func (sv *SysVar) Validate(vars *SessionVars, value string, scope vardef.ScopeFlag) (string, error) { // Check that the scope is correct first. if err := sv.validateScope(scope); err != nil { return value, err @@ -320,41 +228,41 @@ func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (st } // ValidateFromType provides automatic validation based on the SysVar's type -func (sv *SysVar) ValidateFromType(vars *SessionVars, value string, scope ScopeFlag) (string, error) { +func (sv *SysVar) ValidateFromType(vars *SessionVars, value string, scope vardef.ScopeFlag) (string, error) { // Some sysvars in TiDB have a special behavior where the empty string means // "use the config file value". This needs to be cleaned up once the behavior // for instance variables is determined. - if value == "" && ((sv.AllowEmpty && scope == ScopeSession) || sv.AllowEmptyAll) { + if value == "" && ((sv.AllowEmpty && scope == vardef.ScopeSession) || sv.AllowEmptyAll) { return value, nil } // Provide validation using the SysVar struct switch sv.Type { - case TypeUnsigned: + case vardef.TypeUnsigned: return sv.checkUInt64SystemVar(value, vars) - case TypeInt: + case vardef.TypeInt: return sv.checkInt64SystemVar(value, vars) - case TypeBool: + case vardef.TypeBool: return sv.checkBoolSystemVar(value, vars) - case TypeFloat: + case vardef.TypeFloat: return sv.checkFloatSystemVar(value, vars) - case TypeEnum: + case vardef.TypeEnum: return sv.checkEnumSystemVar(value, vars) - case TypeTime: + case vardef.TypeTime: return sv.checkTimeSystemVar(value, vars) - case TypeDuration: + case vardef.TypeDuration: return sv.checkDurationSystemVar(value, vars) } return value, nil // typeString } -func (sv *SysVar) validateScope(scope ScopeFlag) error { - if sv.ReadOnly || sv.Scope == ScopeNone { +func (sv *SysVar) validateScope(scope vardef.ScopeFlag) error { + if sv.ReadOnly || sv.Scope == vardef.ScopeNone { return ErrIncorrectScope.FastGenByArgs(sv.Name, "read only") } - if scope == ScopeGlobal && !(sv.HasGlobalScope() || sv.HasInstanceScope()) { + if scope == vardef.ScopeGlobal && !(sv.HasGlobalScope() || sv.HasInstanceScope()) { return errLocalVariable.FastGenByArgs(sv.Name) } - if scope == ScopeSession && !sv.HasSessionScope() { + if scope == vardef.ScopeSession && !sv.HasSessionScope() { return errGlobalVariable.FastGenByArgs(sv.Name) } return nil @@ -364,7 +272,7 @@ func (sv *SysVar) validateScope(scope ScopeFlag) error { // Normalization+validation needs to be applied when reading values because older versions of TiDB // may be less sophisticated in normalizing values. But errors should be caught and handled, // because otherwise there will be upgrade issues. -func (sv *SysVar) ValidateWithRelaxedValidation(vars *SessionVars, value string, scope ScopeFlag) string { +func (sv *SysVar) ValidateWithRelaxedValidation(vars *SessionVars, value string, scope vardef.ScopeFlag) string { warns := vars.StmtCtx.GetWarnings() defer func() { vars.StmtCtx.SetWarnings(warns) // RelaxedValidation = trim warnings too. @@ -382,19 +290,13 @@ func (sv *SysVar) ValidateWithRelaxedValidation(vars *SessionVars, value string, return normalizedValue } -const ( - localDayTimeFormat = "15:04" - // FullDayTimeFormat is the full format of analyze start time and end time. - FullDayTimeFormat = "15:04 -0700" -) - func (sv *SysVar) checkTimeSystemVar(value string, vars *SessionVars) (string, error) { var t time.Time var err error - if len(value) <= len(localDayTimeFormat) { - t, err = time.ParseInLocation(localDayTimeFormat, value, vars.Location()) + if len(value) <= len(vardef.LocalDayTimeFormat) { + t, err = time.ParseInLocation(vardef.LocalDayTimeFormat, value, vars.Location()) } else { - t, err = time.ParseInLocation(FullDayTimeFormat, value, vars.Location()) + t, err = time.ParseInLocation(vardef.FullDayTimeFormat, value, vars.Location()) } if err != nil { return "", err @@ -403,7 +305,7 @@ func (sv *SysVar) checkTimeSystemVar(value string, vars *SessionVars) (string, e // For example, the Asia/Shanghai refers to +08:05 before 1900 now := time.Now() t = time.Date(now.Year(), now.Month(), now.Day(), t.Hour(), t.Minute(), t.Second(), t.Nanosecond(), t.Location()) - return t.Format(FullDayTimeFormat), nil + return t.Format(vardef.FullDayTimeFormat), nil } func (sv *SysVar) checkDurationSystemVar(value string, vars *SessionVars) (string, error) { @@ -507,9 +409,9 @@ func (sv *SysVar) checkFloatSystemVar(value string, vars *SessionVars) (string, func (sv *SysVar) checkBoolSystemVar(value string, vars *SessionVars) (string, error) { if strings.EqualFold(value, "ON") { - return On, nil + return vardef.On, nil } else if strings.EqualFold(value, "OFF") { - return Off, nil + return vardef.Off, nil } val, err := strconv.ParseInt(value, 10, 64) if err == nil { @@ -518,15 +420,15 @@ func (sv *SysVar) checkBoolSystemVar(value string, vars *SessionVars) (string, e // negative integer to 1. if !sv.AutoConvertNegativeBool { if val == 0 { - return Off, nil + return vardef.Off, nil } else if val == 1 { - return On, nil + return vardef.On, nil } } else { if val == 1 || val < 0 { - return On, nil + return vardef.On, nil } else if val == 0 { - return Off, nil + return vardef.Off, nil } } } @@ -537,13 +439,13 @@ func (sv *SysVar) checkBoolSystemVar(value string, vars *SessionVars) (string, e // TODO: only return 3 types now, support others like DOUBLE, TIME later func (sv *SysVar) GetNativeValType(val string) (types.Datum, byte, uint) { switch sv.Type { - case TypeUnsigned: + case vardef.TypeUnsigned: u, err := strconv.ParseUint(val, 10, 64) if err != nil { u = 0 } return types.NewUintDatum(u), mysql.TypeLonglong, mysql.UnsignedFlag | mysql.BinaryFlag - case TypeBool: + case vardef.TypeBool: optVal := int64(0) // OFF if TiDBOptOn(val) { optVal = 1 @@ -571,8 +473,8 @@ func (sv *SysVar) SkipInit() bool { // which update only these values. func (sv *SysVar) SkipSysvarCache() bool { switch sv.Name { - case TiDBGCEnable, TiDBGCRunInterval, TiDBGCLifetime, - TiDBGCConcurrency, TiDBGCScanLockMode, TiDBExternalTS: + case vardef.TiDBGCEnable, vardef.TiDBGCRunInterval, vardef.TiDBGCLifetime, + vardef.TiDBGCConcurrency, vardef.TiDBGCScanLockMode, vardef.TiDBExternalTS: return true } return false diff --git a/pkg/sessionctx/variable/variable_test.go b/pkg/sessionctx/variable/variable_test.go index 3a1057082b647..a5690bfd178d2 100644 --- a/pkg/sessionctx/variable/variable_test.go +++ b/pkg/sessionctx/variable/variable_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/stretchr/testify/require" ) @@ -87,7 +88,7 @@ func TestError(t *testing.T) { func TestRegistrationOfNewSysVar(t *testing.T) { count := len(GetSysVars()) - sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + sv := SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "mynewsysvar", Value: vardef.On, Type: vardef.TypeBool, SetSession: func(s *SessionVars, val string) error { return fmt.Errorf("set should fail") }} @@ -100,14 +101,14 @@ func TestRegistrationOfNewSysVar(t *testing.T) { vars := NewSessionVars(nil) // It is a boolean, try to set it to a bogus value - _, err := sysVar.Validate(vars, "ABCD", ScopeSession) + _, err := sysVar.Validate(vars, "ABCD", vardef.ScopeSession) require.Error(t, err) // Boolean oN or 1 converts to canonical ON or OFF - normalizedVal, err := sysVar.Validate(vars, "oN", ScopeSession) + normalizedVal, err := sysVar.Validate(vars, "oN", vardef.ScopeSession) require.Equal(t, "ON", normalizedVal) require.NoError(t, err) - normalizedVal, err = sysVar.Validate(vars, "0", ScopeSession) + normalizedVal, err = sysVar.Validate(vars, "0", vardef.ScopeSession) require.Equal(t, "OFF", normalizedVal) require.NoError(t, err) @@ -120,183 +121,183 @@ func TestRegistrationOfNewSysVar(t *testing.T) { } func TestIntValidation(t *testing.T) { - sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: "123", Type: TypeInt, MinValue: 10, MaxValue: 300, AllowAutoValue: true} + sv := SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "mynewsysvar", Value: "123", Type: vardef.TypeInt, MinValue: 10, MaxValue: 300, AllowAutoValue: true} vars := NewSessionVars(nil) - _, err := sv.Validate(vars, "oN", ScopeSession) + _, err := sv.Validate(vars, "oN", vardef.ScopeSession) require.Equal(t, "[variable:1232]Incorrect argument type to variable 'mynewsysvar'", err.Error()) - val, err := sv.Validate(vars, "301", ScopeSession) + val, err := sv.Validate(vars, "301", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "300", val) - val, err = sv.Validate(vars, "5", ScopeSession) + val, err = sv.Validate(vars, "5", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "10", val) - val, err = sv.Validate(vars, "300", ScopeSession) + val, err = sv.Validate(vars, "300", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "300", val) // out of range but permitted due to auto value - val, err = sv.Validate(vars, "-1", ScopeSession) + val, err = sv.Validate(vars, "-1", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "-1", val) } func TestUintValidation(t *testing.T) { - sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: "123", Type: TypeUnsigned, MinValue: 10, MaxValue: 300, AllowAutoValue: true} + sv := SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "mynewsysvar", Value: "123", Type: vardef.TypeUnsigned, MinValue: 10, MaxValue: 300, AllowAutoValue: true} vars := NewSessionVars(nil) - _, err := sv.Validate(vars, "oN", ScopeSession) + _, err := sv.Validate(vars, "oN", vardef.ScopeSession) require.Equal(t, "[variable:1232]Incorrect argument type to variable 'mynewsysvar'", err.Error()) - _, err = sv.Validate(vars, "", ScopeSession) + _, err = sv.Validate(vars, "", vardef.ScopeSession) require.Equal(t, "[variable:1232]Incorrect argument type to variable 'mynewsysvar'", err.Error()) - val, err := sv.Validate(vars, "301", ScopeSession) + val, err := sv.Validate(vars, "301", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "300", val) - val, err = sv.Validate(vars, "-301", ScopeSession) + val, err = sv.Validate(vars, "-301", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "10", val) - _, err = sv.Validate(vars, "-ERR", ScopeSession) + _, err = sv.Validate(vars, "-ERR", vardef.ScopeSession) require.Equal(t, "[variable:1232]Incorrect argument type to variable 'mynewsysvar'", err.Error()) - val, err = sv.Validate(vars, "5", ScopeSession) + val, err = sv.Validate(vars, "5", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "10", val) - val, err = sv.Validate(vars, "300", ScopeSession) + val, err = sv.Validate(vars, "300", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "300", val) // out of range but permitted due to auto value - val, err = sv.Validate(vars, "-1", ScopeSession) + val, err = sv.Validate(vars, "-1", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "-1", val) } func TestEnumValidation(t *testing.T) { - sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + sv := SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "mynewsysvar", Value: vardef.On, Type: vardef.TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} vars := NewSessionVars(nil) - _, err := sv.Validate(vars, "randomstring", ScopeSession) + _, err := sv.Validate(vars, "randomstring", vardef.ScopeSession) require.Equal(t, "[variable:1231]Variable 'mynewsysvar' can't be set to the value of 'randomstring'", err.Error()) - val, err := sv.Validate(vars, "oFf", ScopeSession) + val, err := sv.Validate(vars, "oFf", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "OFF", val) - val, err = sv.Validate(vars, "On", ScopeSession) + val, err = sv.Validate(vars, "On", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "ON", val) - val, err = sv.Validate(vars, "auto", ScopeSession) + val, err = sv.Validate(vars, "auto", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "AUTO", val) // Also settable by numeric offset. - val, err = sv.Validate(vars, "2", ScopeSession) + val, err = sv.Validate(vars, "2", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "AUTO", val) } func TestDurationValidation(t *testing.T) { - sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Second), MaxValue: uint64(time.Hour)} + sv := SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "mynewsysvar", Value: "10m0s", Type: vardef.TypeDuration, MinValue: int64(time.Second), MaxValue: uint64(time.Hour)} vars := NewSessionVars(nil) - _, err := sv.Validate(vars, "1hr", ScopeSession) + _, err := sv.Validate(vars, "1hr", vardef.ScopeSession) require.Equal(t, "[variable:1232]Incorrect argument type to variable 'mynewsysvar'", err.Error()) - val, err := sv.Validate(vars, "1ms", ScopeSession) + val, err := sv.Validate(vars, "1ms", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "1s", val) // truncates to min - val, err = sv.Validate(vars, "2h10m", ScopeSession) + val, err = sv.Validate(vars, "2h10m", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "1h0m0s", val) // truncates to max } func TestFloatValidation(t *testing.T) { - sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: "10m0s", Type: TypeFloat, MinValue: 2, MaxValue: 7} + sv := SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "mynewsysvar", Value: "10m0s", Type: vardef.TypeFloat, MinValue: 2, MaxValue: 7} vars := NewSessionVars(nil) - _, err := sv.Validate(vars, "stringval", ScopeSession) + _, err := sv.Validate(vars, "stringval", vardef.ScopeSession) require.Equal(t, "[variable:1232]Incorrect argument type to variable 'mynewsysvar'", err.Error()) - _, err = sv.Validate(vars, "", ScopeSession) + _, err = sv.Validate(vars, "", vardef.ScopeSession) require.Equal(t, "[variable:1232]Incorrect argument type to variable 'mynewsysvar'", err.Error()) - val, err := sv.Validate(vars, "1.1", ScopeSession) + val, err := sv.Validate(vars, "1.1", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "2", val) // truncates to min - val, err = sv.Validate(vars, "22", ScopeSession) + val, err = sv.Validate(vars, "22", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "7", val) // truncates to max } func TestBoolValidation(t *testing.T) { - sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: Off, Type: TypeBool} + sv := SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "mynewsysvar", Value: vardef.Off, Type: vardef.TypeBool} vars := NewSessionVars(nil) - _, err := sv.Validate(vars, "0.000", ScopeSession) + _, err := sv.Validate(vars, "0.000", vardef.ScopeSession) require.Equal(t, "[variable:1231]Variable 'mynewsysvar' can't be set to the value of '0.000'", err.Error()) - _, err = sv.Validate(vars, "1.000", ScopeSession) + _, err = sv.Validate(vars, "1.000", vardef.ScopeSession) require.Equal(t, "[variable:1231]Variable 'mynewsysvar' can't be set to the value of '1.000'", err.Error()) - val, err := sv.Validate(vars, "0", ScopeSession) + val, err := sv.Validate(vars, "0", vardef.ScopeSession) require.NoError(t, err) - require.Equal(t, Off, val) - val, err = sv.Validate(vars, "1", ScopeSession) + require.Equal(t, vardef.Off, val) + val, err = sv.Validate(vars, "1", vardef.ScopeSession) require.NoError(t, err) - require.Equal(t, On, val) - val, err = sv.Validate(vars, "OFF", ScopeSession) + require.Equal(t, vardef.On, val) + val, err = sv.Validate(vars, "OFF", vardef.ScopeSession) require.NoError(t, err) - require.Equal(t, Off, val) - val, err = sv.Validate(vars, "ON", ScopeSession) + require.Equal(t, vardef.Off, val) + val, err = sv.Validate(vars, "ON", vardef.ScopeSession) require.NoError(t, err) - require.Equal(t, On, val) - val, err = sv.Validate(vars, "off", ScopeSession) + require.Equal(t, vardef.On, val) + val, err = sv.Validate(vars, "off", vardef.ScopeSession) require.NoError(t, err) - require.Equal(t, Off, val) - val, err = sv.Validate(vars, "on", ScopeSession) + require.Equal(t, vardef.Off, val) + val, err = sv.Validate(vars, "on", vardef.ScopeSession) require.NoError(t, err) - require.Equal(t, On, val) + require.Equal(t, vardef.On, val) // test AutoConvertNegativeBool - sv = SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: Off, Type: TypeBool, AutoConvertNegativeBool: true} - val, err = sv.Validate(vars, "-1", ScopeSession) + sv = SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "mynewsysvar", Value: vardef.Off, Type: vardef.TypeBool, AutoConvertNegativeBool: true} + val, err = sv.Validate(vars, "-1", vardef.ScopeSession) require.NoError(t, err) - require.Equal(t, On, val) - val, err = sv.Validate(vars, "1", ScopeSession) + require.Equal(t, vardef.On, val) + val, err = sv.Validate(vars, "1", vardef.ScopeSession) require.NoError(t, err) - require.Equal(t, On, val) - val, err = sv.Validate(vars, "0", ScopeSession) + require.Equal(t, vardef.On, val) + val, err = sv.Validate(vars, "0", vardef.ScopeSession) require.NoError(t, err) - require.Equal(t, Off, val) + require.Equal(t, vardef.Off, val) } func TestTimeValidation(t *testing.T) { - sv := SysVar{Scope: ScopeSession, Name: "mynewsysvar", Value: "23:59 +0000", Type: TypeTime} + sv := SysVar{Scope: vardef.ScopeSession, Name: "mynewsysvar", Value: "23:59 +0000", Type: vardef.TypeTime} vars := NewSessionVars(nil) - val, err := sv.Validate(vars, "23:59 +0000", ScopeSession) + val, err := sv.Validate(vars, "23:59 +0000", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "23:59 +0000", val) - val, err = sv.Validate(vars, "3:00 +0000", ScopeSession) + val, err = sv.Validate(vars, "3:00 +0000", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "03:00 +0000", val) - _, err = sv.Validate(vars, "0.000", ScopeSession) + _, err = sv.Validate(vars, "0.000", vardef.ScopeSession) require.Error(t, err) } func TestGetNativeValType(t *testing.T) { - sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: Off, Type: TypeBool} + sv := SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "mynewsysvar", Value: vardef.Off, Type: vardef.TypeBool} nativeVal, nativeType, flag := sv.GetNativeValType("ON") require.Equal(t, mysql.TypeLonglong, nativeType) @@ -313,7 +314,7 @@ func TestGetNativeValType(t *testing.T) { require.Equal(t, mysql.BinaryFlag, flag) require.Equal(t, types.NewIntDatum(0), nativeVal) - sv = SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: Off, Type: TypeUnsigned} + sv = SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "mynewsysvar", Value: vardef.Off, Type: vardef.TypeUnsigned} nativeVal, nativeType, flag = sv.GetNativeValType("1234") require.Equal(t, mysql.TypeLonglong, nativeType) require.Equal(t, mysql.UnsignedFlag|mysql.BinaryFlag, flag) @@ -323,7 +324,7 @@ func TestGetNativeValType(t *testing.T) { require.Equal(t, mysql.UnsignedFlag|mysql.BinaryFlag, flag) require.Equal(t, types.NewUintDatum(0), nativeVal) // converts to zero - sv = SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: "abc"} + sv = SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "mynewsysvar", Value: "abc"} nativeVal, nativeType, flag = sv.GetNativeValType("1234") require.Equal(t, mysql.TypeVarString, nativeType) require.Equal(t, uint(0), flag) @@ -331,21 +332,21 @@ func TestGetNativeValType(t *testing.T) { } func TestSynonyms(t *testing.T) { - sysVar := GetSysVar(TxnIsolation) + sysVar := GetSysVar(vardef.TxnIsolation) require.NotNil(t, sysVar) vars := NewSessionVars(nil) // It does not permit SERIALIZABLE by default. - _, err := sysVar.Validate(vars, "SERIALIZABLE", ScopeSession) + _, err := sysVar.Validate(vars, "SERIALIZABLE", vardef.ScopeSession) require.Error(t, err) require.Equal(t, "[variable:8048]The isolation level 'SERIALIZABLE' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error", err.Error()) // Enable Skip isolation check - require.Nil(t, GetSysVar(TiDBSkipIsolationLevelCheck).SetSessionFromHook(vars, "ON")) + require.Nil(t, GetSysVar(vardef.TiDBSkipIsolationLevelCheck).SetSessionFromHook(vars, "ON")) // Serializable is now permitted. - _, err = sysVar.Validate(vars, "SERIALIZABLE", ScopeSession) + _, err = sysVar.Validate(vars, "SERIALIZABLE", vardef.ScopeSession) require.NoError(t, err) // Currently TiDB returns a warning because of SERIALIZABLE, but in future @@ -357,17 +358,17 @@ func TestSynonyms(t *testing.T) { require.Nil(t, sysVar.SetSessionFromHook(vars, "SERIALIZABLE")) // When we set TxnIsolation, it also updates TransactionIsolation. - require.Equal(t, "SERIALIZABLE", vars.systems[TxnIsolation]) - require.Equal(t, vars.systems[TxnIsolation], vars.systems[TransactionIsolation]) + require.Equal(t, "SERIALIZABLE", vars.systems[vardef.TxnIsolation]) + require.Equal(t, vars.systems[vardef.TxnIsolation], vars.systems[vardef.TransactionIsolation]) } func TestDeprecation(t *testing.T) { - sysVar := GetSysVar(TiDBIndexLookupConcurrency) + sysVar := GetSysVar(vardef.TiDBIndexLookupConcurrency) require.NotNil(t, sysVar) vars := NewSessionVars(nil) - _, err := sysVar.Validate(vars, "123", ScopeSession) + _, err := sysVar.Validate(vars, "123", vardef.ScopeSession) require.NoError(t, err) // There was no error but there is a deprecation warning. @@ -376,38 +377,38 @@ func TestDeprecation(t *testing.T) { } func TestScope(t *testing.T) { - sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + sv := SysVar{Scope: vardef.ScopeGlobal | vardef.ScopeSession, Name: "mynewsysvar", Value: vardef.On, Type: vardef.TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} require.True(t, sv.HasSessionScope()) require.True(t, sv.HasGlobalScope()) require.False(t, sv.HasInstanceScope()) require.False(t, sv.HasNoneScope()) - sv = SysVar{Scope: ScopeGlobal, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + sv = SysVar{Scope: vardef.ScopeGlobal, Name: "mynewsysvar", Value: vardef.On, Type: vardef.TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} require.False(t, sv.HasSessionScope()) require.True(t, sv.HasGlobalScope()) require.False(t, sv.HasInstanceScope()) require.False(t, sv.HasNoneScope()) - sv = SysVar{Scope: ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + sv = SysVar{Scope: vardef.ScopeSession, Name: "mynewsysvar", Value: vardef.On, Type: vardef.TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} require.True(t, sv.HasSessionScope()) require.False(t, sv.HasGlobalScope()) require.False(t, sv.HasInstanceScope()) require.False(t, sv.HasNoneScope()) - sv = SysVar{Scope: ScopeNone, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + sv = SysVar{Scope: vardef.ScopeNone, Name: "mynewsysvar", Value: vardef.On, Type: vardef.TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} require.False(t, sv.HasSessionScope()) require.False(t, sv.HasGlobalScope()) require.False(t, sv.HasInstanceScope()) require.True(t, sv.HasNoneScope()) - sv = SysVar{Scope: ScopeInstance, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + sv = SysVar{Scope: vardef.ScopeInstance, Name: "mynewsysvar", Value: vardef.On, Type: vardef.TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} require.False(t, sv.HasSessionScope()) require.False(t, sv.HasGlobalScope()) require.True(t, sv.HasInstanceScope()) require.False(t, sv.HasNoneScope()) - sv = SysVar{Scope: ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} - require.Error(t, sv.validateScope(ScopeGlobal)) + sv = SysVar{Scope: vardef.ScopeSession, Name: "mynewsysvar", Value: vardef.On, Type: vardef.TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + require.Error(t, sv.validateScope(vardef.ScopeGlobal)) } func TestBuiltInCase(t *testing.T) { @@ -420,19 +421,19 @@ func TestBuiltInCase(t *testing.T) { // TestIsNoop is used by the documentation to auto-generate docs for real sysvars. func TestIsNoop(t *testing.T) { - sv := GetSysVar(TiDBMultiStatementMode) + sv := GetSysVar(vardef.TiDBMultiStatementMode) require.False(t, sv.IsNoop) - sv = GetSysVar(InnodbLockWaitTimeout) + sv = GetSysVar(vardef.InnodbLockWaitTimeout) require.False(t, sv.IsNoop) - sv = GetSysVar(InnodbFastShutdown) + sv = GetSysVar(vardef.InnodbFastShutdown) require.True(t, sv.IsNoop) - sv = GetSysVar(ReadOnly) + sv = GetSysVar(vardef.ReadOnly) require.True(t, sv.IsNoop) - sv = GetSysVar(DefaultPasswordLifetime) + sv = GetSysVar(vardef.DefaultPasswordLifetime) require.False(t, sv.IsNoop) } @@ -444,13 +445,13 @@ func TestDefaultValuesAreSettable(t *testing.T) { vars.GlobalVarsAccessor = NewMockGlobalAccessor4Tests() for _, sv := range GetSysVars() { if sv.HasSessionScope() && !sv.ReadOnly { - val, err := sv.Validate(vars, sv.Value, ScopeSession) + val, err := sv.Validate(vars, sv.Value, vardef.ScopeSession) require.NoError(t, err) require.Equal(t, val, sv.Value) } if sv.HasGlobalScope() && !sv.ReadOnly { - val, err := sv.Validate(vars, sv.Value, ScopeGlobal) + val, err := sv.Validate(vars, sv.Value, vardef.ScopeGlobal) require.NoError(t, err) require.Equal(t, val, sv.Value) } @@ -458,7 +459,7 @@ func TestDefaultValuesAreSettable(t *testing.T) { } func TestLimitBetweenVariable(t *testing.T) { - require.Less(t, DefTiDBGOGCTunerThreshold+0.05, DefTiDBServerMemoryLimitGCTrigger) + require.Less(t, vardef.DefTiDBGOGCTunerThreshold+0.05, vardef.DefTiDBServerMemoryLimitGCTrigger) } // TestSysVarNameIsLowerCase tests that no new sysvars are added with uppercase characters. @@ -479,7 +480,7 @@ func TestSettersandGetters(t *testing.T) { } if !sv.HasGlobalScope() && !sv.HasInstanceScope() { require.Nil(t, sv.SetGlobal) - if sv.Name == Timestamp { + if sv.Name == vardef.Timestamp { // The Timestamp sysvar will have GetGlobal func even though it does not have global scope. // It's GetGlobal func will only be called when "set timestamp = default". continue @@ -510,7 +511,7 @@ func TestSkipInitIsUsed(t *testing.T) { // Test for variables that have a default of "0" or "OFF" // If it is session-only scoped there is likely no bug now. // If it is also global-scoped, then there is a bug as soon as the global changes. - if !(sv.Name == RandSeed1 || sv.Name == RandSeed2) { + if !(sv.Name == vardef.RandSeed1 || sv.Name == vardef.RandSeed2) { // The bug is because the tests might not realize the SetSession func was not called on init, // because it would initialize some session field to the empty value anyway. require.NotEqual(t, "0", sv.Value, fmt.Sprintf("default value is zero: %s", sv.Name)) @@ -522,25 +523,25 @@ func TestSkipInitIsUsed(t *testing.T) { // sure we don't introduce any new variables with skipInit, which seems // to be a problem. switch sv.Name { - case TiDBTxnScope, - TiDBSnapshot, - TiDBEnableChunkRPC, - TxnIsolationOneShot, - TiDBDDLReorgPriority, - TiDBSlowQueryFile, - TiDBWaitSplitRegionFinish, - TiDBWaitSplitRegionTimeout, - TiDBMetricSchemaStep, - TiDBMetricSchemaRangeDuration, - RandSeed1, - RandSeed2, - CollationDatabase, - CollationConnection, - CharsetDatabase, - CharacterSetConnection, - CharacterSetServer, - TiDBOptTiFlashConcurrencyFactor, - TiDBOptSeekFactor: + case vardef.TiDBTxnScope, + vardef.TiDBSnapshot, + vardef.TiDBEnableChunkRPC, + vardef.TxnIsolationOneShot, + vardef.TiDBDDLReorgPriority, + vardef.TiDBSlowQueryFile, + vardef.TiDBWaitSplitRegionFinish, + vardef.TiDBWaitSplitRegionTimeout, + vardef.TiDBMetricSchemaStep, + vardef.TiDBMetricSchemaRangeDuration, + vardef.RandSeed1, + vardef.RandSeed2, + vardef.CollationDatabase, + vardef.CollationConnection, + vardef.CharsetDatabase, + vardef.CharacterSetConnection, + vardef.CharacterSetServer, + vardef.TiDBOptTiFlashConcurrencyFactor, + vardef.TiDBOptSeekFactor: continue } require.Equal(t, false, sv.skipInit, fmt.Sprintf("skipInit should not be set on new system variables. variable %s is in violation", sv.Name)) @@ -549,36 +550,36 @@ func TestSkipInitIsUsed(t *testing.T) { } func TestScopeToString(t *testing.T) { - require.Equal(t, "GLOBAL", ScopeGlobal.String()) - require.Equal(t, "SESSION", ScopeSession.String()) - require.Equal(t, "INSTANCE", ScopeInstance.String()) - require.Equal(t, "NONE", ScopeNone.String()) - tmp := ScopeSession + ScopeGlobal + require.Equal(t, "GLOBAL", vardef.ScopeGlobal.String()) + require.Equal(t, "SESSION", vardef.ScopeSession.String()) + require.Equal(t, "INSTANCE", vardef.ScopeInstance.String()) + require.Equal(t, "NONE", vardef.ScopeNone.String()) + tmp := vardef.ScopeSession + vardef.ScopeGlobal require.Equal(t, "SESSION,GLOBAL", tmp.String()) // this is not currently possible, but might be in future. // *but* global + instance is not possible. these are mutually exclusive by design. - tmp = ScopeSession + ScopeInstance + tmp = vardef.ScopeSession + vardef.ScopeInstance require.Equal(t, "SESSION,INSTANCE", tmp.String()) } func TestValidateWithRelaxedValidation(t *testing.T) { - sv := GetSysVar(SecureAuth) + sv := GetSysVar(vardef.SecureAuth) vars := NewSessionVars(nil) - val := sv.ValidateWithRelaxedValidation(vars, "1", ScopeGlobal) + val := sv.ValidateWithRelaxedValidation(vars, "1", vardef.ScopeGlobal) require.Equal(t, "ON", val) // Relaxed validation catches the error and squashes it. // The incorrect value is returned as-is. // I am not sure this is the correct behavior, we might need to // change it to return the default instead in future. - sv = GetSysVar(DefaultAuthPlugin) - val = sv.ValidateWithRelaxedValidation(vars, "RandomText", ScopeGlobal) + sv = GetSysVar(vardef.DefaultAuthPlugin) + val = sv.ValidateWithRelaxedValidation(vars, "RandomText", vardef.ScopeGlobal) require.Equal(t, "RandomText", val) // Validation func fails, the error is also caught and squashed. // The incorrect value is returned as-is. - sv = GetSysVar(InitConnect) - val = sv.ValidateWithRelaxedValidation(vars, "RandomText - should be valid SQL", ScopeGlobal) + sv = GetSysVar(vardef.InitConnect) + val = sv.ValidateWithRelaxedValidation(vars, "RandomText - should be valid SQL", vardef.ScopeGlobal) require.Equal(t, "RandomText - should be valid SQL", val) } @@ -619,7 +620,7 @@ func TestInstanceScope(t *testing.T) { } count := len(GetSysVars()) - sv := SysVar{Scope: ScopeInstance, Name: "newinstancesysvar", Value: On, Type: TypeBool, + sv := SysVar{Scope: vardef.ScopeInstance, Name: "newinstancesysvar", Value: vardef.On, Type: vardef.TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { return fmt.Errorf("set should fail") }, @@ -637,14 +638,14 @@ func TestInstanceScope(t *testing.T) { vars := NewSessionVars(nil) // It is a boolean, try to set it to a bogus value - _, err := sysVar.Validate(vars, "ABCD", ScopeInstance) + _, err := sysVar.Validate(vars, "ABCD", vardef.ScopeInstance) require.Error(t, err) // Boolean oN or 1 converts to canonical ON or OFF - normalizedVal, err := sysVar.Validate(vars, "oN", ScopeInstance) + normalizedVal, err := sysVar.Validate(vars, "oN", vardef.ScopeInstance) require.Equal(t, "ON", normalizedVal) require.NoError(t, err) - normalizedVal, err = sysVar.Validate(vars, "0", ScopeInstance) + normalizedVal, err = sysVar.Validate(vars, "0", vardef.ScopeInstance) require.Equal(t, "OFF", normalizedVal) require.NoError(t, err) @@ -659,40 +660,40 @@ func TestInstanceScope(t *testing.T) { func TestSetSysVar(t *testing.T) { vars := NewSessionVars(nil) vars.GlobalVarsAccessor = NewMockGlobalAccessor4Tests() - originalVal := GetSysVar(SystemTimeZone).Value - SetSysVar(SystemTimeZone, "America/New_York") - require.Equal(t, "America/New_York", GetSysVar(SystemTimeZone).Value) + originalVal := GetSysVar(vardef.SystemTimeZone).Value + SetSysVar(vardef.SystemTimeZone, "America/New_York") + require.Equal(t, "America/New_York", GetSysVar(vardef.SystemTimeZone).Value) // Test alternative Get - val, err := GetSysVar(SystemTimeZone).GetGlobalFromHook(context.Background(), vars) + val, err := GetSysVar(vardef.SystemTimeZone).GetGlobalFromHook(context.Background(), vars) require.Nil(t, err) require.Equal(t, "America/New_York", val) - SetSysVar(SystemTimeZone, originalVal) // restore - require.Equal(t, originalVal, GetSysVar(SystemTimeZone).Value) + SetSysVar(vardef.SystemTimeZone, originalVal) // restore + require.Equal(t, originalVal, GetSysVar(vardef.SystemTimeZone).Value) } func TestSkipSysvarCache(t *testing.T) { - require.True(t, GetSysVar(TiDBGCEnable).SkipSysvarCache()) - require.True(t, GetSysVar(TiDBGCRunInterval).SkipSysvarCache()) - require.True(t, GetSysVar(TiDBGCLifetime).SkipSysvarCache()) - require.True(t, GetSysVar(TiDBGCConcurrency).SkipSysvarCache()) - require.True(t, GetSysVar(TiDBGCScanLockMode).SkipSysvarCache()) - require.False(t, GetSysVar(TiDBEnableAsyncCommit).SkipSysvarCache()) + require.True(t, GetSysVar(vardef.TiDBGCEnable).SkipSysvarCache()) + require.True(t, GetSysVar(vardef.TiDBGCRunInterval).SkipSysvarCache()) + require.True(t, GetSysVar(vardef.TiDBGCLifetime).SkipSysvarCache()) + require.True(t, GetSysVar(vardef.TiDBGCConcurrency).SkipSysvarCache()) + require.True(t, GetSysVar(vardef.TiDBGCScanLockMode).SkipSysvarCache()) + require.False(t, GetSysVar(vardef.TiDBEnableAsyncCommit).SkipSysvarCache()) } func TestTimeValidationWithTimezone(t *testing.T) { - sv := SysVar{Scope: ScopeSession, Name: "mynewsysvar", Value: "23:59 +0000", Type: TypeTime} + sv := SysVar{Scope: vardef.ScopeSession, Name: "mynewsysvar", Value: "23:59 +0000", Type: vardef.TypeTime} vars := NewSessionVars(nil) // In timezone UTC vars.TimeZone = time.UTC - val, err := sv.Validate(vars, "23:59", ScopeSession) + val, err := sv.Validate(vars, "23:59", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "23:59 +0000", val) // In timezone Asia/Shanghai vars.TimeZone, err = time.LoadLocation("Asia/Shanghai") require.NoError(t, err) - val, err = sv.Validate(vars, "23:59", ScopeSession) + val, err = sv.Validate(vars, "23:59", vardef.ScopeSession) require.NoError(t, err) require.Equal(t, "23:59 +0800", val) } @@ -701,23 +702,23 @@ func TestOrderByDependency(t *testing.T) { // Some other exceptions: // - tidb_snapshot and tidb_read_staleness can not be set at the same time. It doesn't affect dependency. vars := map[string]string{ - "unknown": "1", - TxReadOnly: "1", - SQLAutoIsNull: "1", - TiDBEnableNoopFuncs: "1", - TiDBEnforceMPPExecution: "1", - TiDBAllowMPPExecution: "1", - TiDBTxnScope: kv.LocalTxnScope, - TiDBEnableLocalTxn: "1", - TiDBEnablePlanReplayerContinuousCapture: "1", - TiDBEnableHistoricalStats: "1", + "unknown": "1", + vardef.TxReadOnly: "1", + vardef.SQLAutoIsNull: "1", + vardef.TiDBEnableNoopFuncs: "1", + vardef.TiDBEnforceMPPExecution: "1", + vardef.TiDBAllowMPPExecution: "1", + vardef.TiDBTxnScope: kv.LocalTxnScope, + vardef.TiDBEnableLocalTxn: "1", + vardef.TiDBEnablePlanReplayerContinuousCapture: "1", + vardef.TiDBEnableHistoricalStats: "1", } names := OrderByDependency(vars) - require.Greater(t, slices.Index(names, TxReadOnly), slices.Index(names, TiDBEnableNoopFuncs)) - require.Greater(t, slices.Index(names, SQLAutoIsNull), slices.Index(names, TiDBEnableNoopFuncs)) - require.Greater(t, slices.Index(names, TiDBEnforceMPPExecution), slices.Index(names, TiDBAllowMPPExecution)) + require.Greater(t, slices.Index(names, vardef.TxReadOnly), slices.Index(names, vardef.TiDBEnableNoopFuncs)) + require.Greater(t, slices.Index(names, vardef.SQLAutoIsNull), slices.Index(names, vardef.TiDBEnableNoopFuncs)) + require.Greater(t, slices.Index(names, vardef.TiDBEnforceMPPExecution), slices.Index(names, vardef.TiDBAllowMPPExecution)) // Depended variables below are global variables, so actually it doesn't matter. - require.Greater(t, slices.Index(names, TiDBTxnScope), slices.Index(names, TiDBEnableLocalTxn)) - require.Greater(t, slices.Index(names, TiDBEnablePlanReplayerContinuousCapture), slices.Index(names, TiDBEnableHistoricalStats)) + require.Greater(t, slices.Index(names, vardef.TiDBTxnScope), slices.Index(names, vardef.TiDBEnableLocalTxn)) + require.Greater(t, slices.Index(names, vardef.TiDBEnablePlanReplayerContinuousCapture), slices.Index(names, vardef.TiDBEnableHistoricalStats)) require.Contains(t, names, "unknown") } diff --git a/pkg/sessionctx/variable/varsutil.go b/pkg/sessionctx/variable/varsutil.go index 17146c1a95f8a..9f3b9ea26da1b 100644 --- a/pkg/sessionctx/variable/varsutil.go +++ b/pkg/sessionctx/variable/varsutil.go @@ -22,7 +22,6 @@ import ( "slices" "strconv" "strings" - "sync/atomic" "time" "github.com/docker/go-units" @@ -30,6 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/collate" "github.com/pingcap/tidb/pkg/util/memory" @@ -39,85 +39,22 @@ import ( // secondsPerYear represents seconds in a normal year. Leap year is not considered here. const secondsPerYear = 60 * 60 * 24 * 365 -// SetDDLReorgWorkerCounter sets ddlReorgWorkerCounter count. -// Sysvar validation enforces the range to already be correct. -func SetDDLReorgWorkerCounter(cnt int32) { - atomic.StoreInt32(&ddlReorgWorkerCounter, cnt) -} - -// GetDDLReorgWorkerCounter gets ddlReorgWorkerCounter. -func GetDDLReorgWorkerCounter() int32 { - return atomic.LoadInt32(&ddlReorgWorkerCounter) -} - -// SetDDLFlashbackConcurrency sets ddlFlashbackConcurrency count. -// Sysvar validation enforces the range to already be correct. -func SetDDLFlashbackConcurrency(cnt int32) { - atomic.StoreInt32(&ddlFlashbackConcurrency, cnt) -} - -// GetDDLFlashbackConcurrency gets ddlFlashbackConcurrency count. -func GetDDLFlashbackConcurrency() int32 { - return atomic.LoadInt32(&ddlFlashbackConcurrency) -} - -// SetDDLReorgBatchSize sets ddlReorgBatchSize size. -// Sysvar validation enforces the range to already be correct. -func SetDDLReorgBatchSize(cnt int32) { - atomic.StoreInt32(&ddlReorgBatchSize, cnt) -} - -// GetDDLReorgBatchSize gets ddlReorgBatchSize. -func GetDDLReorgBatchSize() int32 { - return atomic.LoadInt32(&ddlReorgBatchSize) -} - -// SetDDLErrorCountLimit sets ddlErrorCountlimit size. -func SetDDLErrorCountLimit(cnt int64) { - atomic.StoreInt64(&ddlErrorCountLimit, cnt) -} - -// GetDDLErrorCountLimit gets ddlErrorCountlimit size. -func GetDDLErrorCountLimit() int64 { - return atomic.LoadInt64(&ddlErrorCountLimit) -} - -// SetDDLReorgRowFormat sets ddlReorgRowFormat version. -func SetDDLReorgRowFormat(format int64) { - atomic.StoreInt64(&ddlReorgRowFormat, format) -} - -// GetDDLReorgRowFormat gets ddlReorgRowFormat version. -func GetDDLReorgRowFormat() int64 { - return atomic.LoadInt64(&ddlReorgRowFormat) -} - -// SetMaxDeltaSchemaCount sets maxDeltaSchemaCount size. -func SetMaxDeltaSchemaCount(cnt int64) { - atomic.StoreInt64(&maxDeltaSchemaCount, cnt) -} - -// GetMaxDeltaSchemaCount gets maxDeltaSchemaCount size. -func GetMaxDeltaSchemaCount() int64 { - return atomic.LoadInt64(&maxDeltaSchemaCount) -} - // BoolToOnOff returns the string representation of a bool, i.e. "ON/OFF" func BoolToOnOff(b bool) string { if b { - return On + return vardef.On } - return Off + return vardef.Off } func int32ToBoolStr(i int32) string { if i == 1 { - return On + return vardef.On } - return Off + return vardef.Off } -func checkCollation(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { +func checkCollation(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { coll, err := collate.GetCollationByName(normalizedValue) if err != nil { return normalizedValue, errors.Trace(err) @@ -125,7 +62,7 @@ func checkCollation(vars *SessionVars, normalizedValue string, originalValue str return coll.Name, nil } -func checkDefaultCollationForUTF8MB4(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { +func checkDefaultCollationForUTF8MB4(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { coll, err := collate.GetCollationByName(normalizedValue) if err != nil { return normalizedValue, errors.Trace(err) @@ -148,27 +85,27 @@ func checkCharacterSet(normalizedValue string, argName string) (string, error) { } // checkReadOnly requires TiDBEnableNoopFuncs=1 for the same scope otherwise an error will be returned. -func checkReadOnly(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag, offlineMode bool) (string, error) { +func checkReadOnly(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag, offlineMode bool) (string, error) { errMsg := ErrFunctionsNoopImpl.FastGenByArgs("READ ONLY") if offlineMode { errMsg = ErrFunctionsNoopImpl.FastGenByArgs("OFFLINE MODE") } if TiDBOptOn(normalizedValue) { - if scope == ScopeSession && vars.NoopFuncsMode != OnInt { + if scope == vardef.ScopeSession && vars.NoopFuncsMode != OnInt { if vars.NoopFuncsMode == OffInt { - return Off, errors.Trace(errMsg) + return vardef.Off, errors.Trace(errMsg) } vars.StmtCtx.AppendWarning(errMsg) } - if scope == ScopeGlobal { - val, err := vars.GlobalVarsAccessor.GetGlobalSysVar(TiDBEnableNoopFuncs) + if scope == vardef.ScopeGlobal { + val, err := vars.GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBEnableNoopFuncs) if err != nil { - return originalValue, errUnknownSystemVariable.GenWithStackByArgs(TiDBEnableNoopFuncs) + return originalValue, errUnknownSystemVariable.GenWithStackByArgs(vardef.TiDBEnableNoopFuncs) } - if val == Off { - return Off, errors.Trace(errMsg) + if val == vardef.Off { + return vardef.Off, errors.Trace(errMsg) } - if val == Warn { + if val == vardef.Warn { vars.StmtCtx.AppendWarning(errMsg) } } @@ -176,10 +113,10 @@ func checkReadOnly(vars *SessionVars, normalizedValue string, originalValue stri return normalizedValue, nil } -func checkIsolationLevel(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { +func checkIsolationLevel(vars *SessionVars, normalizedValue string, originalValue string, scope vardef.ScopeFlag) (string, error) { if normalizedValue == "SERIALIZABLE" || normalizedValue == "READ-UNCOMMITTED" { returnErr := ErrUnsupportedIsolationLevel.FastGenByArgs(normalizedValue) - if !TiDBOptOn(vars.systems[TiDBSkipIsolationLevelCheck]) { + if !TiDBOptOn(vars.systems[vardef.TiDBSkipIsolationLevelCheck]) { return normalizedValue, ErrUnsupportedIsolationLevel.GenWithStackByArgs(normalizedValue) } vars.StmtCtx.AppendWarning(returnErr) @@ -212,9 +149,9 @@ func setTiDBTableValue(vars *SessionVars, name, value, comment string) error { // but sysvars use the convention ON/OFF. func trueFalseToOnOff(str string) string { if strings.EqualFold("true", str) { - return On + return vardef.On } else if strings.EqualFold("false", str) { - return Off + return vardef.Off } return str } @@ -261,38 +198,14 @@ const ( // It is used for MultiStmtMode and NoopFunctionsMode func TiDBOptOnOffWarn(opt string) int { switch opt { - case Warn: + case vardef.Warn: return WarnInt - case On: + case vardef.On: return OnInt } return OffInt } -// ClusteredIndexDefMode controls the default clustered property for primary key. -type ClusteredIndexDefMode int - -const ( - // ClusteredIndexDefModeIntOnly indicates only single int primary key will default be clustered. - ClusteredIndexDefModeIntOnly ClusteredIndexDefMode = 0 - // ClusteredIndexDefModeOn indicates primary key will default be clustered. - ClusteredIndexDefModeOn ClusteredIndexDefMode = 1 - // ClusteredIndexDefModeOff indicates primary key will default be non-clustered. - ClusteredIndexDefModeOff ClusteredIndexDefMode = 2 -) - -// TiDBOptEnableClustered converts enable clustered options to ClusteredIndexDefMode. -func TiDBOptEnableClustered(opt string) ClusteredIndexDefMode { - switch opt { - case On: - return ClusteredIndexDefModeOn - case Off: - return ClusteredIndexDefModeOff - default: - return ClusteredIndexDefModeIntOnly - } -} - // AssertionLevel controls the assertion that will be performed during transactions. type AssertionLevel int @@ -307,11 +220,11 @@ const ( func tidbOptAssertionLevel(opt string) AssertionLevel { switch opt { - case AssertionStrictStr: + case vardef.AssertionStrictStr: return AssertionLevelStrict - case AssertionFastStr: + case vardef.AssertionFastStr: return AssertionLevelFast - case AssertionOffStr: + case vardef.AssertionOffStr: return AssertionLevelOff default: return AssertionLevelOff @@ -364,7 +277,7 @@ func tidbOptFloat64(opt string, defaultVal float64) float64 { func parseMemoryLimit(s *SessionVars, normalizedValue string, originalValue string) (byteSize uint64, normalizedStr string, err error) { defer func() { if err == nil && byteSize > 0 && byteSize < (512<<20) { - s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(TiDBServerMemoryLimit, originalValue)) + s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(vardef.TiDBServerMemoryLimit, originalValue)) byteSize = 512 << 20 normalizedStr = "512MB" } @@ -385,7 +298,7 @@ func parseMemoryLimit(s *SessionVars, normalizedValue string, originalValue stri return bt, str, nil } - return 0, "", ErrTruncatedWrongValue.GenWithStackByArgs(TiDBServerMemoryLimit, originalValue) + return 0, "", ErrTruncatedWrongValue.GenWithStackByArgs(vardef.TiDBServerMemoryLimit, originalValue) } func parsePercentage(s string) (percentage uint64, normalizedStr string) { @@ -531,11 +444,11 @@ func collectAllowFuncName4ExpressionIndex() string { } func updatePasswordValidationLength(s *SessionVars, length int32) error { - err := s.GlobalVarsAccessor.SetGlobalSysVarOnly(context.Background(), ValidatePasswordLength, strconv.FormatInt(int64(length), 10), false) + err := s.GlobalVarsAccessor.SetGlobalSysVarOnly(context.Background(), vardef.ValidatePasswordLength, strconv.FormatInt(int64(length), 10), false) if err != nil { return err } - PasswordValidationLength.Store(length) + vardef.PasswordValidationLength.Store(length) return nil } @@ -594,7 +507,7 @@ func ValidAnalyzeSkipColumnTypes(val string) (string, error) { for _, item := range items { columnType := strings.TrimSpace(item) if _, ok := analyzeSkipAllowedTypes[columnType]; !ok { - return val, ErrWrongValueForVar.GenWithStackByArgs(TiDBAnalyzeSkipColumnTypes, val) + return val, ErrWrongValueForVar.GenWithStackByArgs(vardef.TiDBAnalyzeSkipColumnTypes, val) } columnTypes = append(columnTypes, columnType) } @@ -624,12 +537,12 @@ var ( func parseSchemaCacheSize(s *SessionVars, normalizedValue string, originalValue string) (byteSize uint64, normalizedStr string, err error) { defer func() { if err == nil && byteSize > 0 && byteSize < SchemaCacheSizeLowerBound { - s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(TiDBSchemaCacheSize, originalValue)) + s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(vardef.TiDBSchemaCacheSize, originalValue)) byteSize = SchemaCacheSizeLowerBound normalizedStr = SchemaCacheSizeLowerBoundStr } if err == nil && byteSize > math.MaxInt64 { - s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(TiDBSchemaCacheSize, originalValue)) + s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.FastGenByArgs(vardef.TiDBSchemaCacheSize, originalValue)) byteSize = math.MaxInt64 normalizedStr = strconv.Itoa(math.MaxInt64) } @@ -640,5 +553,5 @@ func parseSchemaCacheSize(s *SessionVars, normalizedValue string, originalValue return bt, str, nil } - return 0, "", ErrTruncatedWrongValue.GenWithStackByArgs(TiDBSchemaCacheSize, originalValue) + return 0, "", ErrTruncatedWrongValue.GenWithStackByArgs(vardef.TiDBSchemaCacheSize, originalValue) } diff --git a/pkg/sessionctx/variable/varsutil_test.go b/pkg/sessionctx/variable/varsutil_test.go index dee43b3636374..8fc02823c80bc 100644 --- a/pkg/sessionctx/variable/varsutil_test.go +++ b/pkg/sessionctx/variable/varsutil_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/stretchr/testify/require" ) @@ -52,46 +53,46 @@ func TestTiDBOptOn(t *testing.T) { func TestNewSessionVars(t *testing.T) { vars := NewSessionVars(nil) - require.Equal(t, DefIndexJoinBatchSize, vars.IndexJoinBatchSize) - require.Equal(t, DefIndexLookupSize, vars.IndexLookupSize) - require.Equal(t, ConcurrencyUnset, vars.indexLookupConcurrency) - require.Equal(t, DefIndexSerialScanConcurrency, vars.indexSerialScanConcurrency) - require.Equal(t, ConcurrencyUnset, vars.indexLookupJoinConcurrency) - require.Equal(t, DefTiDBHashJoinConcurrency, vars.hashJoinConcurrency) - require.Equal(t, DefExecutorConcurrency, vars.IndexLookupConcurrency()) - require.Equal(t, DefIndexSerialScanConcurrency, vars.IndexSerialScanConcurrency()) - require.Equal(t, DefExecutorConcurrency, vars.IndexLookupJoinConcurrency()) - require.Equal(t, DefExecutorConcurrency, vars.HashJoinConcurrency()) - require.Equal(t, DefTiDBAllowBatchCop, vars.AllowBatchCop) - require.Equal(t, ConcurrencyUnset, vars.projectionConcurrency) - require.Equal(t, ConcurrencyUnset, vars.hashAggPartialConcurrency) - require.Equal(t, ConcurrencyUnset, vars.hashAggFinalConcurrency) - require.Equal(t, ConcurrencyUnset, vars.windowConcurrency) - require.Equal(t, DefTiDBMergeJoinConcurrency, vars.mergeJoinConcurrency) - require.Equal(t, DefTiDBStreamAggConcurrency, vars.streamAggConcurrency) - require.Equal(t, DefDistSQLScanConcurrency, vars.distSQLScanConcurrency) - require.Equal(t, DefExecutorConcurrency, vars.ProjectionConcurrency()) - require.Equal(t, DefExecutorConcurrency, vars.HashAggPartialConcurrency()) - require.Equal(t, DefExecutorConcurrency, vars.HashAggFinalConcurrency()) - require.Equal(t, DefExecutorConcurrency, vars.WindowConcurrency()) - require.Equal(t, DefExecutorConcurrency, vars.IndexMergeIntersectionConcurrency()) - require.Equal(t, DefTiDBMergeJoinConcurrency, vars.MergeJoinConcurrency()) - require.Equal(t, DefTiDBStreamAggConcurrency, vars.StreamAggConcurrency()) - require.Equal(t, DefDistSQLScanConcurrency, vars.DistSQLScanConcurrency()) - require.Equal(t, DefExecutorConcurrency, vars.ExecutorConcurrency) - require.Equal(t, DefMaxChunkSize, vars.MaxChunkSize) - require.Equal(t, DefDMLBatchSize, vars.DMLBatchSize) - require.Equal(t, int64(DefTiDBMemQuotaApplyCache), vars.MemQuotaApplyCache) - require.Equal(t, DefOptWriteRowID, vars.AllowWriteRowID) - require.Equal(t, DefTiDBOptJoinReorderThreshold, vars.TiDBOptJoinReorderThreshold) - require.Equal(t, DefTiDBUseFastAnalyze, vars.EnableFastAnalyze) - require.Equal(t, DefTiDBFoundInPlanCache, vars.FoundInPlanCache) - require.Equal(t, DefTiDBFoundInBinding, vars.FoundInBinding) - require.Equal(t, DefTiDBAllowAutoRandExplicitInsert, vars.AllowAutoRandExplicitInsert) - require.Equal(t, int64(DefTiDBShardAllocateStep), vars.ShardAllocateStep) - require.Equal(t, DefTiDBAnalyzeVersion, vars.AnalyzeVersion) - require.Equal(t, DefCTEMaxRecursionDepth, vars.CTEMaxRecursionDepth) - require.Equal(t, int64(DefTiDBTmpTableMaxSize), vars.TMPTableSize) + require.Equal(t, vardef.DefIndexJoinBatchSize, vars.IndexJoinBatchSize) + require.Equal(t, vardef.DefIndexLookupSize, vars.IndexLookupSize) + require.Equal(t, vardef.ConcurrencyUnset, vars.indexLookupConcurrency) + require.Equal(t, vardef.DefIndexSerialScanConcurrency, vars.indexSerialScanConcurrency) + require.Equal(t, vardef.ConcurrencyUnset, vars.indexLookupJoinConcurrency) + require.Equal(t, vardef.DefTiDBHashJoinConcurrency, vars.hashJoinConcurrency) + require.Equal(t, vardef.DefExecutorConcurrency, vars.IndexLookupConcurrency()) + require.Equal(t, vardef.DefIndexSerialScanConcurrency, vars.IndexSerialScanConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.IndexLookupJoinConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.HashJoinConcurrency()) + require.Equal(t, vardef.DefTiDBAllowBatchCop, vars.AllowBatchCop) + require.Equal(t, vardef.ConcurrencyUnset, vars.projectionConcurrency) + require.Equal(t, vardef.ConcurrencyUnset, vars.hashAggPartialConcurrency) + require.Equal(t, vardef.ConcurrencyUnset, vars.hashAggFinalConcurrency) + require.Equal(t, vardef.ConcurrencyUnset, vars.windowConcurrency) + require.Equal(t, vardef.DefTiDBMergeJoinConcurrency, vars.mergeJoinConcurrency) + require.Equal(t, vardef.DefTiDBStreamAggConcurrency, vars.streamAggConcurrency) + require.Equal(t, vardef.DefDistSQLScanConcurrency, vars.distSQLScanConcurrency) + require.Equal(t, vardef.DefExecutorConcurrency, vars.ProjectionConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.HashAggPartialConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.HashAggFinalConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.WindowConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.IndexMergeIntersectionConcurrency()) + require.Equal(t, vardef.DefTiDBMergeJoinConcurrency, vars.MergeJoinConcurrency()) + require.Equal(t, vardef.DefTiDBStreamAggConcurrency, vars.StreamAggConcurrency()) + require.Equal(t, vardef.DefDistSQLScanConcurrency, vars.DistSQLScanConcurrency()) + require.Equal(t, vardef.DefExecutorConcurrency, vars.ExecutorConcurrency) + require.Equal(t, vardef.DefMaxChunkSize, vars.MaxChunkSize) + require.Equal(t, vardef.DefDMLBatchSize, vars.DMLBatchSize) + require.Equal(t, int64(vardef.DefTiDBMemQuotaApplyCache), vars.MemQuotaApplyCache) + require.Equal(t, vardef.DefOptWriteRowID, vars.AllowWriteRowID) + require.Equal(t, vardef.DefTiDBOptJoinReorderThreshold, vars.TiDBOptJoinReorderThreshold) + require.Equal(t, vardef.DefTiDBUseFastAnalyze, vars.EnableFastAnalyze) + require.Equal(t, vardef.DefTiDBFoundInPlanCache, vars.FoundInPlanCache) + require.Equal(t, vardef.DefTiDBFoundInBinding, vars.FoundInBinding) + require.Equal(t, vardef.DefTiDBAllowAutoRandExplicitInsert, vars.AllowAutoRandExplicitInsert) + require.Equal(t, int64(vardef.DefTiDBShardAllocateStep), vars.ShardAllocateStep) + require.Equal(t, vardef.DefTiDBAnalyzeVersion, vars.AnalyzeVersion) + require.Equal(t, vardef.DefCTEMaxRecursionDepth, vars.CTEMaxRecursionDepth) + require.Equal(t, int64(vardef.DefTiDBTmpTableMaxSize), vars.TMPTableSize) assertFieldsGreaterThanZero(t, reflect.ValueOf(vars.MemQuota)) assertFieldsGreaterThanZero(t, reflect.ValueOf(vars.BatchSize)) @@ -168,7 +169,7 @@ func TestVarsutil(t *testing.T) { {"-13:00", "", false, 13 * time.Hour, ErrUnknownTimeZone.GenWithStackByArgs("-13:00")}, } for _, tc := range testCases { - err = v.SetSystemVar(TimeZone, tc.input) + err = v.SetSystemVar(vardef.TimeZone, tc.input) if tc.err != nil { require.Error(t, err) continue @@ -177,14 +178,14 @@ func TestVarsutil(t *testing.T) { require.NoError(t, err) require.Equal(t, tc.expect, v.TimeZone.String()) if tc.compareValue { - err = v.SetSystemVar(TimeZone, tc.input) + err = v.SetSystemVar(vardef.TimeZone, tc.input) require.NoError(t, err) t1 := time.Date(2000, 1, 1, 0, 0, 0, 0, time.UTC) t2 := time.Date(2000, 1, 1, 0, 0, 0, 0, v.TimeZone) require.Equal(t, tc.diff, t2.Sub(t1)) } } - err = v.SetSystemVar(TimeZone, "6:00") + err = v.SetSystemVar(vardef.TimeZone, "6:00") require.Error(t, err) require.True(t, terror.ErrorEqual(err, ErrUnknownTimeZone)) @@ -206,182 +207,182 @@ func TestVarsutil(t *testing.T) { require.Equal(t, mysql.ModeRealAsFloat|mysql.ModeANSIQuotes, v.SQLMode) // Test case for tidb_index_serial_scan_concurrency. - require.Equal(t, DefIndexSerialScanConcurrency, v.IndexSerialScanConcurrency()) - err = v.SetSystemVar(TiDBIndexSerialScanConcurrency, "4") + require.Equal(t, vardef.DefIndexSerialScanConcurrency, v.IndexSerialScanConcurrency()) + err = v.SetSystemVar(vardef.TiDBIndexSerialScanConcurrency, "4") require.NoError(t, err) require.Equal(t, 4, v.IndexSerialScanConcurrency()) // Test case for tidb_batch_insert. require.False(t, v.BatchInsert) - err = v.SetSystemVar(TiDBBatchInsert, "1") + err = v.SetSystemVar(vardef.TiDBBatchInsert, "1") require.NoError(t, err) require.True(t, v.BatchInsert) require.Equal(t, 32, v.InitChunkSize) require.Equal(t, 1024, v.MaxChunkSize) - err = v.SetSystemVar(TiDBMaxChunkSize, "2") + err = v.SetSystemVar(vardef.TiDBMaxChunkSize, "2") require.NoError(t, err) // converts to min value - err = v.SetSystemVar(TiDBInitChunkSize, "1024") + err = v.SetSystemVar(vardef.TiDBInitChunkSize, "1024") require.NoError(t, err) // converts to max value // Test case for TiDBConfig session variable. - err = v.SetSystemVar(TiDBConfig, "abc") + err = v.SetSystemVar(vardef.TiDBConfig, "abc") require.True(t, terror.ErrorEqual(err, ErrIncorrectScope)) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBConfig) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBConfig) require.NoError(t, err) jsonConfig, err := config.GetJSONConfig() require.NoError(t, err) require.Equal(t, jsonConfig, val) - require.Equal(t, DefTiDBOptimizerSelectivityLevel, v.OptimizerSelectivityLevel) - err = v.SetSystemVar(TiDBOptimizerSelectivityLevel, "1") + require.Equal(t, vardef.DefTiDBOptimizerSelectivityLevel, v.OptimizerSelectivityLevel) + err = v.SetSystemVar(vardef.TiDBOptimizerSelectivityLevel, "1") require.NoError(t, err) require.Equal(t, 1, v.OptimizerSelectivityLevel) - require.Equal(t, DefTiDBEnableOuterJoinReorder, v.EnableOuterJoinReorder) - err = v.SetSystemVar(TiDBOptimizerEnableOuterJoinReorder, "OFF") + require.Equal(t, vardef.DefTiDBEnableOuterJoinReorder, v.EnableOuterJoinReorder) + err = v.SetSystemVar(vardef.TiDBOptimizerEnableOuterJoinReorder, "OFF") require.NoError(t, err) require.Equal(t, false, v.EnableOuterJoinReorder) - err = v.SetSystemVar(TiDBOptimizerEnableOuterJoinReorder, "ON") + err = v.SetSystemVar(vardef.TiDBOptimizerEnableOuterJoinReorder, "ON") require.NoError(t, err) require.Equal(t, true, v.EnableOuterJoinReorder) - require.Equal(t, DefTiDBOptimizerEnableNewOFGB, v.OptimizerEnableNewOnlyFullGroupByCheck) - err = v.SetSystemVar(TiDBOptimizerEnableNewOnlyFullGroupByCheck, "off") + require.Equal(t, vardef.DefTiDBOptimizerEnableNewOFGB, v.OptimizerEnableNewOnlyFullGroupByCheck) + err = v.SetSystemVar(vardef.TiDBOptimizerEnableNewOnlyFullGroupByCheck, "off") require.NoError(t, err) require.Equal(t, false, v.OptimizerEnableNewOnlyFullGroupByCheck) - err = v.SetSystemVar(TiDBRetryLimit, "3") + err = v.SetSystemVar(vardef.TiDBRetryLimit, "3") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBRetryLimit) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBRetryLimit) require.NoError(t, err) require.Equal(t, "3", val) require.Equal(t, int64(3), v.RetryLimit) - require.Equal(t, DefTiDBOptJoinReorderThreshold, v.TiDBOptJoinReorderThreshold) - err = v.SetSystemVar(TiDBOptJoinReorderThreshold, "5") + require.Equal(t, vardef.DefTiDBOptJoinReorderThreshold, v.TiDBOptJoinReorderThreshold) + err = v.SetSystemVar(vardef.TiDBOptJoinReorderThreshold, "5") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBOptJoinReorderThreshold) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBOptJoinReorderThreshold) require.NoError(t, err) require.Equal(t, "5", val) require.Equal(t, 5, v.TiDBOptJoinReorderThreshold) - err = v.SetSystemVar(TiDBLowResolutionTSO, "1") + err = v.SetSystemVar(vardef.TiDBLowResolutionTSO, "1") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBLowResolutionTSO) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBLowResolutionTSO) require.NoError(t, err) require.Equal(t, "ON", val) require.True(t, v.lowResolutionTSO) - err = v.SetSystemVar(TiDBLowResolutionTSO, "0") + err = v.SetSystemVar(vardef.TiDBLowResolutionTSO, "0") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBLowResolutionTSO) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBLowResolutionTSO) require.NoError(t, err) require.Equal(t, "OFF", val) require.False(t, v.lowResolutionTSO) require.Equal(t, 0.9, v.CorrelationThreshold) - err = v.SetSystemVar(TiDBOptCorrelationThreshold, "0") + err = v.SetSystemVar(vardef.TiDBOptCorrelationThreshold, "0") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBOptCorrelationThreshold) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBOptCorrelationThreshold) require.NoError(t, err) require.Equal(t, "0", val) require.Equal(t, float64(0), v.CorrelationThreshold) require.Equal(t, 3.0, v.GetCPUFactor()) - err = v.SetSystemVar(TiDBOptCPUFactor, "5.0") + err = v.SetSystemVar(vardef.TiDBOptCPUFactor, "5.0") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBOptCPUFactor) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBOptCPUFactor) require.NoError(t, err) require.Equal(t, "5.0", val) require.Equal(t, 5.0, v.GetCPUFactor()) require.Equal(t, 3.0, v.GetCopCPUFactor()) - err = v.SetSystemVar(TiDBOptCopCPUFactor, "5.0") + err = v.SetSystemVar(vardef.TiDBOptCopCPUFactor, "5.0") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBOptCopCPUFactor) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBOptCopCPUFactor) require.NoError(t, err) require.Equal(t, "5.0", val) require.Equal(t, 5.0, v.GetCopCPUFactor()) require.Equal(t, 24.0, v.CopTiFlashConcurrencyFactor) - err = v.SetSystemVar(TiDBOptTiFlashConcurrencyFactor, "5.0") + err = v.SetSystemVar(vardef.TiDBOptTiFlashConcurrencyFactor, "5.0") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBOptTiFlashConcurrencyFactor) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBOptTiFlashConcurrencyFactor) require.NoError(t, err) require.Equal(t, "5.0", val) require.Equal(t, 5.0, v.GetCopCPUFactor()) require.Equal(t, 1.0, v.GetNetworkFactor(nil)) - err = v.SetSystemVar(TiDBOptNetworkFactor, "3.0") + err = v.SetSystemVar(vardef.TiDBOptNetworkFactor, "3.0") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBOptNetworkFactor) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBOptNetworkFactor) require.NoError(t, err) require.Equal(t, "3.0", val) require.Equal(t, 3.0, v.GetNetworkFactor(nil)) require.Equal(t, 1.5, v.GetScanFactor(nil)) - err = v.SetSystemVar(TiDBOptScanFactor, "3.0") + err = v.SetSystemVar(vardef.TiDBOptScanFactor, "3.0") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBOptScanFactor) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBOptScanFactor) require.NoError(t, err) require.Equal(t, "3.0", val) require.Equal(t, 3.0, v.GetScanFactor(nil)) require.Equal(t, 3.0, v.GetDescScanFactor(nil)) - err = v.SetSystemVar(TiDBOptDescScanFactor, "5.0") + err = v.SetSystemVar(vardef.TiDBOptDescScanFactor, "5.0") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBOptDescScanFactor) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBOptDescScanFactor) require.NoError(t, err) require.Equal(t, "5.0", val) require.Equal(t, 5.0, v.GetDescScanFactor(nil)) require.Equal(t, 20.0, v.GetSeekFactor(nil)) - err = v.SetSystemVar(TiDBOptSeekFactor, "50.0") + err = v.SetSystemVar(vardef.TiDBOptSeekFactor, "50.0") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBOptSeekFactor) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBOptSeekFactor) require.NoError(t, err) require.Equal(t, "50.0", val) require.Equal(t, 50.0, v.GetSeekFactor(nil)) require.Equal(t, 0.001, v.GetMemoryFactor()) - err = v.SetSystemVar(TiDBOptMemoryFactor, "1.0") + err = v.SetSystemVar(vardef.TiDBOptMemoryFactor, "1.0") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBOptMemoryFactor) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBOptMemoryFactor) require.NoError(t, err) require.Equal(t, "1.0", val) require.Equal(t, 1.0, v.GetMemoryFactor()) require.Equal(t, 1.5, v.GetDiskFactor()) - err = v.SetSystemVar(TiDBOptDiskFactor, "1.1") + err = v.SetSystemVar(vardef.TiDBOptDiskFactor, "1.1") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBOptDiskFactor) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBOptDiskFactor) require.NoError(t, err) require.Equal(t, "1.1", val) require.Equal(t, 1.1, v.GetDiskFactor()) require.Equal(t, 3.0, v.GetConcurrencyFactor()) - err = v.SetSystemVar(TiDBOptConcurrencyFactor, "5.0") + err = v.SetSystemVar(vardef.TiDBOptConcurrencyFactor, "5.0") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBOptConcurrencyFactor) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBOptConcurrencyFactor) require.NoError(t, err) require.Equal(t, "5.0", val) require.Equal(t, 5.0, v.GetConcurrencyFactor()) - err = v.SetSystemVar(TiDBReplicaRead, "follower") + err = v.SetSystemVar(vardef.TiDBReplicaRead, "follower") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBReplicaRead) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBReplicaRead) require.NoError(t, err) require.Equal(t, "follower", val) require.Equal(t, kv.ReplicaReadFollower, v.GetReplicaRead()) - err = v.SetSystemVar(TiDBReplicaRead, "leader") + err = v.SetSystemVar(vardef.TiDBReplicaRead, "leader") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBReplicaRead) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBReplicaRead) require.NoError(t, err) require.Equal(t, "leader", val) require.Equal(t, kv.ReplicaReadLeader, v.GetReplicaRead()) - err = v.SetSystemVar(TiDBReplicaRead, "leader-and-follower") + err = v.SetSystemVar(vardef.TiDBReplicaRead, "leader-and-follower") require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBReplicaRead) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBReplicaRead) require.NoError(t, err) require.Equal(t, "leader-and-follower", val) require.Equal(t, kv.ReplicaReadMixed, v.GetReplicaRead()) @@ -398,18 +399,18 @@ func TestVarsutil(t *testing.T) { {"marker", "MARKER"}, {"2", "MARKER"}, } { - err = v.SetSystemVar(TiDBRedactLog, c.a) + err = v.SetSystemVar(vardef.TiDBRedactLog, c.a) require.NoError(t, err) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBRedactLog) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBRedactLog) require.NoError(t, err) require.Equal(t, c.b, val) } - err = v.SetSystemVar(TiDBFoundInPlanCache, "1") + err = v.SetSystemVar(vardef.TiDBFoundInPlanCache, "1") require.Error(t, err) require.Regexp(t, "]Variable 'last_plan_from_cache' is a read only variable$", err.Error()) - err = v.SetSystemVar(TiDBFoundInBinding, "1") + err = v.SetSystemVar(vardef.TiDBFoundInBinding, "1") require.Error(t, err) require.Regexp(t, "]Variable 'last_plan_from_binding' is a read only variable$", err.Error()) @@ -421,33 +422,33 @@ func TestVarsutil(t *testing.T) { v.StmtCtx.TruncateWarnings(0) require.Len(t, v.StmtCtx.GetWarnings(), 0) - err = v.SetSystemVar(TiDBAnalyzeVersion, "4") + err = v.SetSystemVar(vardef.TiDBAnalyzeVersion, "4") require.NoError(t, err) // converts to max value warn := v.StmtCtx.GetWarnings()[0] require.Error(t, warn.Err) require.Contains(t, warn.Err.Error(), "Truncated incorrect tidb_analyze_version value") - err = v.SetSystemVar(TiDBTableCacheLease, "123") + err = v.SetSystemVar(vardef.TiDBTableCacheLease, "123") require.Error(t, err) require.Regexp(t, "'tidb_table_cache_lease' is a GLOBAL variable and should be set with SET GLOBAL", err.Error()) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBMinPagingSize) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBMinPagingSize) require.NoError(t, err) - require.Equal(t, strconv.Itoa(DefMinPagingSize), val) + require.Equal(t, strconv.Itoa(vardef.DefMinPagingSize), val) - err = v.SetSystemVar(TiDBMinPagingSize, "123") + err = v.SetSystemVar(vardef.TiDBMinPagingSize, "123") require.NoError(t, err) require.Equal(t, v.MinPagingSize, 123) - val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBMaxPagingSize) + val, err = v.GetSessionOrGlobalSystemVar(context.Background(), vardef.TiDBMaxPagingSize) require.NoError(t, err) - require.Equal(t, strconv.Itoa(DefMaxPagingSize), val) + require.Equal(t, strconv.Itoa(vardef.DefMaxPagingSize), val) - err = v.SetSystemVar(TiDBMaxPagingSize, "456") + err = v.SetSystemVar(vardef.TiDBMaxPagingSize, "456") require.NoError(t, err) require.Equal(t, v.MaxPagingSize, 456) - err = v.SetSystemVar(TiDBMaxPagingSize, "45678") + err = v.SetSystemVar(vardef.TiDBMaxPagingSize, "45678") require.NoError(t, err) require.Equal(t, v.MaxPagingSize, 45678) } @@ -462,74 +463,74 @@ func TestValidate(t *testing.T) { value string error bool }{ - {TiDBAutoAnalyzeStartTime, "15:04", false}, - {TiDBAutoAnalyzeStartTime, "15:04 -0700", false}, - {DelayKeyWrite, "ON", false}, - {DelayKeyWrite, "OFF", false}, - {DelayKeyWrite, "ALL", false}, - {DelayKeyWrite, "3", true}, - {ForeignKeyChecks, "3", true}, - {MaxSpRecursionDepth, "256", false}, - {SessionTrackGtids, "OFF", false}, - {SessionTrackGtids, "OWN_GTID", false}, - {SessionTrackGtids, "ALL_GTIDS", false}, - {SessionTrackGtids, "ON", true}, - {EnforceGtidConsistency, "OFF", false}, - {EnforceGtidConsistency, "ON", false}, - {EnforceGtidConsistency, "WARN", false}, - {SecureAuth, "1", false}, - {SecureAuth, "3", true}, - {MyISAMUseMmap, "ON", false}, - {MyISAMUseMmap, "OFF", false}, - {TiDBOptCorrelationExpFactor, "a", true}, - {TiDBOptCorrelationExpFactor, "-10", false}, - {TiDBOptCorrelationThreshold, "a", true}, - {TiDBOptCorrelationThreshold, "-2", false}, - {TiDBOptCPUFactor, "a", true}, - {TiDBOptCPUFactor, "-2", false}, - {TiDBOptTiFlashConcurrencyFactor, "-2", false}, - {TiDBOptCopCPUFactor, "a", true}, - {TiDBOptCopCPUFactor, "-2", false}, - {TiDBOptNetworkFactor, "a", true}, - {TiDBOptNetworkFactor, "-2", false}, - {TiDBOptScanFactor, "a", true}, - {TiDBOptScanFactor, "-2", false}, - {TiDBOptDescScanFactor, "a", true}, - {TiDBOptDescScanFactor, "-2", false}, - {TiDBOptSeekFactor, "a", true}, - {TiDBOptSeekFactor, "-2", false}, - {TiDBOptMemoryFactor, "a", true}, - {TiDBOptMemoryFactor, "-2", false}, - {TiDBOptDiskFactor, "a", true}, - {TiDBOptDiskFactor, "-2", false}, - {TiDBOptConcurrencyFactor, "a", true}, - {TiDBOptConcurrencyFactor, "-2", false}, - {TxnIsolation, "READ-UNCOMMITTED", true}, - {TiDBInitChunkSize, "a", true}, - {TiDBInitChunkSize, "-1", false}, - {TiDBMaxChunkSize, "a", true}, - {TiDBMaxChunkSize, "-1", false}, - {TiDBOptJoinReorderThreshold, "a", true}, - {TiDBOptJoinReorderThreshold, "-1", false}, - {TiDBReplicaRead, "invalid", true}, - {TiDBTxnMode, "invalid", true}, - {TiDBTxnMode, "pessimistic", false}, - {TiDBTxnMode, "optimistic", false}, - {TiDBTxnMode, "", false}, - {TiDBShardAllocateStep, "ad", true}, - {TiDBShardAllocateStep, "-123", false}, - {TiDBShardAllocateStep, "128", false}, - {TiDBAllowFallbackToTiKV, "", false}, - {TiDBAllowFallbackToTiKV, "tiflash", false}, - {TiDBAllowFallbackToTiKV, " tiflash ", false}, - {TiDBAllowFallbackToTiKV, "tikv", true}, - {TiDBAllowFallbackToTiKV, "tidb", true}, - {TiDBAllowFallbackToTiKV, "tiflash,tikv,tidb", true}, + {vardef.TiDBAutoAnalyzeStartTime, "15:04", false}, + {vardef.TiDBAutoAnalyzeStartTime, "15:04 -0700", false}, + {vardef.DelayKeyWrite, "ON", false}, + {vardef.DelayKeyWrite, "OFF", false}, + {vardef.DelayKeyWrite, "ALL", false}, + {vardef.DelayKeyWrite, "3", true}, + {vardef.ForeignKeyChecks, "3", true}, + {vardef.MaxSpRecursionDepth, "256", false}, + {vardef.SessionTrackGtids, "OFF", false}, + {vardef.SessionTrackGtids, "OWN_GTID", false}, + {vardef.SessionTrackGtids, "ALL_GTIDS", false}, + {vardef.SessionTrackGtids, "ON", true}, + {vardef.EnforceGtidConsistency, "OFF", false}, + {vardef.EnforceGtidConsistency, "ON", false}, + {vardef.EnforceGtidConsistency, "WARN", false}, + {vardef.SecureAuth, "1", false}, + {vardef.SecureAuth, "3", true}, + {vardef.MyISAMUseMmap, "ON", false}, + {vardef.MyISAMUseMmap, "OFF", false}, + {vardef.TiDBOptCorrelationExpFactor, "a", true}, + {vardef.TiDBOptCorrelationExpFactor, "-10", false}, + {vardef.TiDBOptCorrelationThreshold, "a", true}, + {vardef.TiDBOptCorrelationThreshold, "-2", false}, + {vardef.TiDBOptCPUFactor, "a", true}, + {vardef.TiDBOptCPUFactor, "-2", false}, + {vardef.TiDBOptTiFlashConcurrencyFactor, "-2", false}, + {vardef.TiDBOptCopCPUFactor, "a", true}, + {vardef.TiDBOptCopCPUFactor, "-2", false}, + {vardef.TiDBOptNetworkFactor, "a", true}, + {vardef.TiDBOptNetworkFactor, "-2", false}, + {vardef.TiDBOptScanFactor, "a", true}, + {vardef.TiDBOptScanFactor, "-2", false}, + {vardef.TiDBOptDescScanFactor, "a", true}, + {vardef.TiDBOptDescScanFactor, "-2", false}, + {vardef.TiDBOptSeekFactor, "a", true}, + {vardef.TiDBOptSeekFactor, "-2", false}, + {vardef.TiDBOptMemoryFactor, "a", true}, + {vardef.TiDBOptMemoryFactor, "-2", false}, + {vardef.TiDBOptDiskFactor, "a", true}, + {vardef.TiDBOptDiskFactor, "-2", false}, + {vardef.TiDBOptConcurrencyFactor, "a", true}, + {vardef.TiDBOptConcurrencyFactor, "-2", false}, + {vardef.TxnIsolation, "READ-UNCOMMITTED", true}, + {vardef.TiDBInitChunkSize, "a", true}, + {vardef.TiDBInitChunkSize, "-1", false}, + {vardef.TiDBMaxChunkSize, "a", true}, + {vardef.TiDBMaxChunkSize, "-1", false}, + {vardef.TiDBOptJoinReorderThreshold, "a", true}, + {vardef.TiDBOptJoinReorderThreshold, "-1", false}, + {vardef.TiDBReplicaRead, "invalid", true}, + {vardef.TiDBTxnMode, "invalid", true}, + {vardef.TiDBTxnMode, "pessimistic", false}, + {vardef.TiDBTxnMode, "optimistic", false}, + {vardef.TiDBTxnMode, "", false}, + {vardef.TiDBShardAllocateStep, "ad", true}, + {vardef.TiDBShardAllocateStep, "-123", false}, + {vardef.TiDBShardAllocateStep, "128", false}, + {vardef.TiDBAllowFallbackToTiKV, "", false}, + {vardef.TiDBAllowFallbackToTiKV, "tiflash", false}, + {vardef.TiDBAllowFallbackToTiKV, " tiflash ", false}, + {vardef.TiDBAllowFallbackToTiKV, "tikv", true}, + {vardef.TiDBAllowFallbackToTiKV, "tidb", true}, + {vardef.TiDBAllowFallbackToTiKV, "tiflash,tikv,tidb", true}, } for _, tc := range testCases { t.Run(tc.key, func(t *testing.T) { - _, err := GetSysVar(tc.key).Validate(v, tc.value, ScopeGlobal) + _, err := GetSysVar(tc.key).Validate(v, tc.value, vardef.ScopeGlobal) if tc.error { require.Errorf(t, err, "%v got err=%v", tc, err) } else { @@ -544,15 +545,15 @@ func TestValidate(t *testing.T) { value string error bool }{ - {TiDBIsolationReadEngines, "", true}, - {TiDBIsolationReadEngines, "tikv", false}, - {TiDBIsolationReadEngines, "TiKV,tiflash", false}, - {TiDBIsolationReadEngines, " tikv, tiflash ", false}, + {vardef.TiDBIsolationReadEngines, "", true}, + {vardef.TiDBIsolationReadEngines, "tikv", false}, + {vardef.TiDBIsolationReadEngines, "TiKV,tiflash", false}, + {vardef.TiDBIsolationReadEngines, " tikv, tiflash ", false}, } for _, tc := range testCases { t.Run(tc.key, func(t *testing.T) { - _, err := GetSysVar(tc.key).Validate(v, tc.value, ScopeSession) + _, err := GetSysVar(tc.key).Validate(v, tc.value, vardef.ScopeSession) if tc.error { require.Errorf(t, err, "%v got err=%v", tc, err) } else { @@ -572,27 +573,27 @@ func TestValidateStmtSummary(t *testing.T) { value string error bool }{ - {TiDBEnableStmtSummary, "", true}, - {TiDBStmtSummaryInternalQuery, "", true}, - {TiDBStmtSummaryRefreshInterval, "", true}, - {TiDBStmtSummaryRefreshInterval, "0", false}, - {TiDBStmtSummaryRefreshInterval, "99999999999", false}, - {TiDBStmtSummaryHistorySize, "", true}, - {TiDBStmtSummaryHistorySize, "0", false}, - {TiDBStmtSummaryHistorySize, "-1", false}, - {TiDBStmtSummaryHistorySize, "99999999", false}, - {TiDBStmtSummaryMaxStmtCount, "", true}, - {TiDBStmtSummaryMaxStmtCount, "0", false}, - {TiDBStmtSummaryMaxStmtCount, "99999999", false}, - {TiDBStmtSummaryMaxSQLLength, "", true}, - {TiDBStmtSummaryMaxSQLLength, "0", false}, - {TiDBStmtSummaryMaxSQLLength, "-1", false}, - {TiDBStmtSummaryMaxSQLLength, "99999999999", false}, + {vardef.TiDBEnableStmtSummary, "", true}, + {vardef.TiDBStmtSummaryInternalQuery, "", true}, + {vardef.TiDBStmtSummaryRefreshInterval, "", true}, + {vardef.TiDBStmtSummaryRefreshInterval, "0", false}, + {vardef.TiDBStmtSummaryRefreshInterval, "99999999999", false}, + {vardef.TiDBStmtSummaryHistorySize, "", true}, + {vardef.TiDBStmtSummaryHistorySize, "0", false}, + {vardef.TiDBStmtSummaryHistorySize, "-1", false}, + {vardef.TiDBStmtSummaryHistorySize, "99999999", false}, + {vardef.TiDBStmtSummaryMaxStmtCount, "", true}, + {vardef.TiDBStmtSummaryMaxStmtCount, "0", false}, + {vardef.TiDBStmtSummaryMaxStmtCount, "99999999", false}, + {vardef.TiDBStmtSummaryMaxSQLLength, "", true}, + {vardef.TiDBStmtSummaryMaxSQLLength, "0", false}, + {vardef.TiDBStmtSummaryMaxSQLLength, "-1", false}, + {vardef.TiDBStmtSummaryMaxSQLLength, "99999999999", false}, } for _, tc := range testCases { t.Run(tc.key, func(t *testing.T) { - _, err := GetSysVar(tc.key).Validate(v, tc.value, ScopeGlobal) + _, err := GetSysVar(tc.key).Validate(v, tc.value, vardef.ScopeGlobal) if tc.error { require.Errorf(t, err, "%v got err=%v", tc, err) } else { @@ -607,35 +608,35 @@ func TestConcurrencyVariables(t *testing.T) { vars.GlobalVarsAccessor = NewMockGlobalAccessor4Tests() wdConcurrency := 2 - require.Equal(t, ConcurrencyUnset, vars.windowConcurrency) - require.Equal(t, DefExecutorConcurrency, vars.WindowConcurrency()) - err := vars.SetSystemVar(TiDBWindowConcurrency, strconv.Itoa(wdConcurrency)) + require.Equal(t, vardef.ConcurrencyUnset, vars.windowConcurrency) + require.Equal(t, vardef.DefExecutorConcurrency, vars.WindowConcurrency()) + err := vars.SetSystemVar(vardef.TiDBWindowConcurrency, strconv.Itoa(wdConcurrency)) require.NoError(t, err) require.Equal(t, wdConcurrency, vars.windowConcurrency) require.Equal(t, wdConcurrency, vars.WindowConcurrency()) mjConcurrency := 2 - require.Equal(t, DefTiDBMergeJoinConcurrency, vars.mergeJoinConcurrency) - require.Equal(t, DefTiDBMergeJoinConcurrency, vars.MergeJoinConcurrency()) - err = vars.SetSystemVar(TiDBMergeJoinConcurrency, strconv.Itoa(mjConcurrency)) + require.Equal(t, vardef.DefTiDBMergeJoinConcurrency, vars.mergeJoinConcurrency) + require.Equal(t, vardef.DefTiDBMergeJoinConcurrency, vars.MergeJoinConcurrency()) + err = vars.SetSystemVar(vardef.TiDBMergeJoinConcurrency, strconv.Itoa(mjConcurrency)) require.NoError(t, err) require.Equal(t, mjConcurrency, vars.mergeJoinConcurrency) require.Equal(t, mjConcurrency, vars.MergeJoinConcurrency()) saConcurrency := 2 - require.Equal(t, DefTiDBStreamAggConcurrency, vars.streamAggConcurrency) - require.Equal(t, DefTiDBStreamAggConcurrency, vars.StreamAggConcurrency()) - err = vars.SetSystemVar(TiDBStreamAggConcurrency, strconv.Itoa(saConcurrency)) + require.Equal(t, vardef.DefTiDBStreamAggConcurrency, vars.streamAggConcurrency) + require.Equal(t, vardef.DefTiDBStreamAggConcurrency, vars.StreamAggConcurrency()) + err = vars.SetSystemVar(vardef.TiDBStreamAggConcurrency, strconv.Itoa(saConcurrency)) require.NoError(t, err) require.Equal(t, saConcurrency, vars.streamAggConcurrency) require.Equal(t, saConcurrency, vars.StreamAggConcurrency()) - require.Equal(t, ConcurrencyUnset, vars.indexLookupConcurrency) - require.Equal(t, DefExecutorConcurrency, vars.IndexLookupConcurrency()) - exeConcurrency := DefExecutorConcurrency + 1 - err = vars.SetSystemVar(TiDBExecutorConcurrency, strconv.Itoa(exeConcurrency)) + require.Equal(t, vardef.ConcurrencyUnset, vars.indexLookupConcurrency) + require.Equal(t, vardef.DefExecutorConcurrency, vars.IndexLookupConcurrency()) + exeConcurrency := vardef.DefExecutorConcurrency + 1 + err = vars.SetSystemVar(vardef.TiDBExecutorConcurrency, strconv.Itoa(exeConcurrency)) require.NoError(t, err) - require.Equal(t, ConcurrencyUnset, vars.indexLookupConcurrency) + require.Equal(t, vardef.ConcurrencyUnset, vars.indexLookupConcurrency) require.Equal(t, exeConcurrency, vars.IndexLookupConcurrency()) require.Equal(t, wdConcurrency, vars.WindowConcurrency()) require.Equal(t, mjConcurrency, vars.MergeJoinConcurrency()) @@ -646,9 +647,9 @@ func TestHelperFuncs(t *testing.T) { require.Equal(t, "ON", int32ToBoolStr(1)) require.Equal(t, "OFF", int32ToBoolStr(0)) - require.Equal(t, ClusteredIndexDefModeOn, TiDBOptEnableClustered("ON")) - require.Equal(t, ClusteredIndexDefModeOff, TiDBOptEnableClustered("OFF")) - require.Equal(t, ClusteredIndexDefModeIntOnly, TiDBOptEnableClustered("bogus")) // default + require.Equal(t, vardef.ClusteredIndexDefModeOn, vardef.TiDBOptEnableClustered("ON")) + require.Equal(t, vardef.ClusteredIndexDefModeOff, vardef.TiDBOptEnableClustered("OFF")) + require.Equal(t, vardef.ClusteredIndexDefModeIntOnly, vardef.TiDBOptEnableClustered("bogus")) // default require.Equal(t, 1234, tidbOptPositiveInt32("1234", 5)) require.Equal(t, 5, tidbOptPositiveInt32("-1234", 5)) @@ -667,12 +668,12 @@ func TestSessionStatesSystemVar(t *testing.T) { require.NoError(t, err) require.Equal(t, "ON", val) require.Equal(t, true, keep) - _, keep, err = vars.GetSessionStatesSystemVar(Timestamp) + _, keep, err = vars.GetSessionStatesSystemVar(vardef.Timestamp) require.NoError(t, err) require.Equal(t, false, keep) - err = vars.SetSystemVar(MaxAllowedPacket, "1024") + err = vars.SetSystemVar(vardef.MaxAllowedPacket, "1024") require.NoError(t, err) - val, keep, err = vars.GetSessionStatesSystemVar(MaxAllowedPacket) + val, keep, err = vars.GetSessionStatesSystemVar(vardef.MaxAllowedPacket) require.NoError(t, err) require.Equal(t, "1024", val) require.Equal(t, true, keep) @@ -696,8 +697,8 @@ func TestOnOffHelpers(t *testing.T) { } func TestAssertionLevel(t *testing.T) { - require.Equal(t, AssertionLevelStrict, tidbOptAssertionLevel(AssertionStrictStr)) - require.Equal(t, AssertionLevelOff, tidbOptAssertionLevel(AssertionOffStr)) - require.Equal(t, AssertionLevelFast, tidbOptAssertionLevel(AssertionFastStr)) + require.Equal(t, AssertionLevelStrict, tidbOptAssertionLevel(vardef.AssertionStrictStr)) + require.Equal(t, AssertionLevelOff, tidbOptAssertionLevel(vardef.AssertionOffStr)) + require.Equal(t, AssertionLevelFast, tidbOptAssertionLevel(vardef.AssertionFastStr)) require.Equal(t, AssertionLevelOff, tidbOptAssertionLevel("bogus")) } diff --git a/pkg/sessiontxn/isolation/BUILD.bazel b/pkg/sessiontxn/isolation/BUILD.bazel index 2f474db4727c7..dc5c8dc08da73 100644 --- a/pkg/sessiontxn/isolation/BUILD.bazel +++ b/pkg/sessiontxn/isolation/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//pkg/planner/core", "//pkg/planner/core/base", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/sessiontxn/internal", diff --git a/pkg/sessiontxn/isolation/base.go b/pkg/sessiontxn/isolation/base.go index 29207b1ef42f5..b3a2c6dd42dcc 100644 --- a/pkg/sessiontxn/isolation/base.go +++ b/pkg/sessiontxn/isolation/base.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/sessiontxn/internal" @@ -128,7 +129,7 @@ func (p *baseTxnContextProvider) OnInitialize(ctx context.Context, tp sessiontxn sessVars.TxnCtxMu.Lock() sessVars.TxnCtx = txnCtx sessVars.TxnCtxMu.Unlock() - if variable.EnableMDL.Load() { + if vardef.EnableMDL.Load() { sessVars.TxnCtx.EnableMDL = true } diff --git a/pkg/sessiontxn/isolation/readcommitted.go b/pkg/sessiontxn/isolation/readcommitted.go index 9619449858b51..db85c3656ebd7 100644 --- a/pkg/sessiontxn/isolation/readcommitted.go +++ b/pkg/sessiontxn/isolation/readcommitted.go @@ -26,6 +26,7 @@ import ( plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" isolation_metrics "github.com/pingcap/tidb/pkg/sessiontxn/isolation/metrics" @@ -106,7 +107,7 @@ func (p *PessimisticRCTxnContextProvider) OnStmtStart(ctx context.Context, node // NeedSetRCCheckTSFlag checks whether it's needed to set `RCCheckTS` flag in current stmtctx. func NeedSetRCCheckTSFlag(ctx sessionctx.Context, node ast.Node) bool { sessionVars := ctx.GetSessionVars() - if sessionVars.ConnectionID > 0 && variable.EnableRCReadCheckTS.Load() && + if sessionVars.ConnectionID > 0 && vardef.EnableRCReadCheckTS.Load() && sessionVars.InTxn() && !sessionVars.RetryInfo.Retrying && plannercore.IsReadOnly(node, sessionVars) { return true diff --git a/pkg/sessiontxn/staleread/BUILD.bazel b/pkg/sessiontxn/staleread/BUILD.bazel index cea3cf4366c09..bff1e590a5308 100644 --- a/pkg/sessiontxn/staleread/BUILD.bazel +++ b/pkg/sessiontxn/staleread/BUILD.bazel @@ -23,6 +23,7 @@ go_library( "//pkg/planner/util", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/sessiontxn/internal", diff --git a/pkg/sessiontxn/staleread/provider.go b/pkg/sessiontxn/staleread/provider.go index a67e98257c7ed..fd12013f82428 100644 --- a/pkg/sessiontxn/staleread/provider.go +++ b/pkg/sessiontxn/staleread/provider.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/sessiontxn/internal" @@ -132,7 +133,7 @@ func (p *StalenessTxnContextProvider) activateStaleTxn() error { } p.is = is - err = p.sctx.GetSessionVars().SetSystemVar(variable.TiDBSnapshot, "") + err = p.sctx.GetSessionVars().SetSystemVar(vardef.TiDBSnapshot, "") return err } diff --git a/pkg/statistics/BUILD.bazel b/pkg/statistics/BUILD.bazel index 5b388ddc2cc5f..ddccf1690cef3 100644 --- a/pkg/statistics/BUILD.bazel +++ b/pkg/statistics/BUILD.bazel @@ -35,6 +35,7 @@ go_library( "//pkg/planner/util/debugtrace", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics/asyncload", "//pkg/statistics/handle/logutil", diff --git a/pkg/statistics/handle/BUILD.bazel b/pkg/statistics/handle/BUILD.bazel index 3831a47f64388..98c86be29c90f 100644 --- a/pkg/statistics/handle/BUILD.bazel +++ b/pkg/statistics/handle/BUILD.bazel @@ -18,7 +18,7 @@ go_library( "//pkg/parser/terror", "//pkg/sessionctx", "//pkg/sessionctx/sysproctrack", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/statistics", "//pkg/statistics/handle/autoanalyze", "//pkg/statistics/handle/cache", diff --git a/pkg/statistics/handle/autoanalyze/BUILD.bazel b/pkg/statistics/handle/autoanalyze/BUILD.bazel index d6ae3b4ddd9f4..459b39830e6ec 100644 --- a/pkg/statistics/handle/autoanalyze/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/parser/terror", "//pkg/sessionctx", "//pkg/sessionctx/sysproctrack", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/statistics/handle/autoanalyze/exec", @@ -47,6 +48,7 @@ go_test( "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/statistics/handle/ddl/testutil", diff --git a/pkg/statistics/handle/autoanalyze/autoanalyze.go b/pkg/statistics/handle/autoanalyze/autoanalyze.go index 47504b5907a23..14d66dfe8d20d 100644 --- a/pkg/statistics/handle/autoanalyze/autoanalyze.go +++ b/pkg/statistics/handle/autoanalyze/autoanalyze.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sysproctrack" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" @@ -332,7 +333,7 @@ func (sa *statsAnalyze) handleAutoAnalyze(sctx sessionctx.Context) bool { ) } }() - if variable.EnableAutoAnalyzePriorityQueue.Load() { + if vardef.EnableAutoAnalyzePriorityQueue.Load() { // During the test, we need to fetch all DML changes before analyzing the highest priority tables. if intest.InTest { sa.refresher.ProcessDMLChangesForTest() @@ -347,7 +348,7 @@ func (sa *statsAnalyze) handleAutoAnalyze(sctx sessionctx.Context) bool { } parameters := exec.GetAutoAnalyzeParameters(sctx) - autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio]) + autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[vardef.TiDBAutoAnalyzeRatio]) start, end, ok := checkAutoAnalyzeWindow(parameters) if !ok { return false @@ -381,8 +382,8 @@ func CheckAutoAnalyzeWindow(sctx sessionctx.Context) bool { func checkAutoAnalyzeWindow(parameters map[string]string) (time.Time, time.Time, bool) { start, end, err := exec.ParseAutoAnalysisWindow( - parameters[variable.TiDBAutoAnalyzeStartTime], - parameters[variable.TiDBAutoAnalyzeEndTime], + parameters[vardef.TiDBAutoAnalyzeStartTime], + parameters[vardef.TiDBAutoAnalyzeEndTime], ) if err != nil { statslogutil.StatsLogger().Error( @@ -634,7 +635,7 @@ func tryAutoAnalyzePartitionTableInDynamicMode( ratio float64, ) bool { tableStatsVer := sctx.GetSessionVars().AnalyzeVersion - analyzePartitionBatchSize := int(variable.AutoAnalyzePartitionBatchSize.Load()) + analyzePartitionBatchSize := int(vardef.AutoAnalyzePartitionBatchSize.Load()) needAnalyzePartitionNames := make([]any, 0, len(partitionDefs)) for _, def := range partitionDefs { diff --git a/pkg/statistics/handle/autoanalyze/autoanalyze_test.go b/pkg/statistics/handle/autoanalyze/autoanalyze_test.go index 3ce37d1f0f64e..9af3dfad8e383 100644 --- a/pkg/statistics/handle/autoanalyze/autoanalyze_test.go +++ b/pkg/statistics/handle/autoanalyze/autoanalyze_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze" @@ -61,7 +62,7 @@ func TestEnableAutoAnalyzePriorityQueue(t *testing.T) { tk.MustExec("insert into t values (1)") // Enable auto analyze priority queue. tk.MustExec("SET GLOBAL tidb_enable_auto_analyze_priority_queue=ON") - require.True(t, variable.EnableAutoAnalyzePriorityQueue.Load()) + require.True(t, vardef.EnableAutoAnalyzePriorityQueue.Load()) h := dom.StatsHandle() err := statstestutil.HandleNextDDLEventWithTxn(h) require.NoError(t, err) diff --git a/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel b/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel index 0951755570787..003c999701cd4 100644 --- a/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel @@ -10,7 +10,7 @@ go_library( "//pkg/planner/core/resolve", "//pkg/sessionctx", "//pkg/sessionctx/sysproctrack", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/statistics", "//pkg/statistics/handle/logutil", "//pkg/statistics/handle/types", diff --git a/pkg/statistics/handle/autoanalyze/exec/exec.go b/pkg/statistics/handle/autoanalyze/exec/exec.go index 2b442b05af2d5..242392e4f25bc 100644 --- a/pkg/statistics/handle/autoanalyze/exec/exec.go +++ b/pkg/statistics/handle/autoanalyze/exec/exec.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sysproctrack" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" @@ -105,7 +105,7 @@ func RunAnalyzeStmt( // GetAutoAnalyzeParameters gets the auto analyze parameters from mysql.global_variables. func GetAutoAnalyzeParameters(sctx sessionctx.Context) map[string]string { sql := "select variable_name, variable_value from mysql.global_variables where variable_name in (%?, %?, %?)" - rows, _, err := statsutil.ExecWithOpts(sctx, nil, sql, variable.TiDBAutoAnalyzeRatio, variable.TiDBAutoAnalyzeStartTime, variable.TiDBAutoAnalyzeEndTime) + rows, _, err := statsutil.ExecWithOpts(sctx, nil, sql, vardef.TiDBAutoAnalyzeRatio, vardef.TiDBAutoAnalyzeStartTime, vardef.TiDBAutoAnalyzeEndTime) if err != nil { return map[string]string{} } @@ -120,7 +120,7 @@ func GetAutoAnalyzeParameters(sctx sessionctx.Context) map[string]string { func ParseAutoAnalyzeRatio(ratio string) float64 { autoAnalyzeRatio, err := strconv.ParseFloat(ratio, 64) if err != nil { - return variable.DefAutoAnalyzeRatio + return vardef.DefAutoAnalyzeRatio } return math.Max(autoAnalyzeRatio, 0) } @@ -129,15 +129,15 @@ func ParseAutoAnalyzeRatio(ratio string) float64 { // It parses the times in UTC location. func ParseAutoAnalysisWindow(start, end string) (time.Time, time.Time, error) { if start == "" { - start = variable.DefAutoAnalyzeStartTime + start = vardef.DefAutoAnalyzeStartTime } if end == "" { - end = variable.DefAutoAnalyzeEndTime + end = vardef.DefAutoAnalyzeEndTime } - s, err := time.ParseInLocation(variable.FullDayTimeFormat, start, time.UTC) + s, err := time.ParseInLocation(vardef.FullDayTimeFormat, start, time.UTC) if err != nil { return s, s, errors.Trace(err) } - e, err := time.ParseInLocation(variable.FullDayTimeFormat, end, time.UTC) + e, err := time.ParseInLocation(vardef.FullDayTimeFormat, end, time.UTC) return s, e, err } diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel b/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel index 6661096287a1c..2804a92d04d9f 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//pkg/meta/model", "//pkg/sessionctx", "//pkg/sessionctx/sysproctrack", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/statistics/handle/autoanalyze/exec", diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job.go b/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job.go index 3a550309119c6..f9dbcd315db8d 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sysproctrack" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util" @@ -271,7 +271,7 @@ func (j *DynamicPartitionedTableAnalysisJob) analyzePartitions( statsHandle statstypes.StatsHandle, sysProcTracker sysproctrack.Tracker, ) bool { - analyzePartitionBatchSize := int(variable.AutoAnalyzePartitionBatchSize.Load()) + analyzePartitionBatchSize := int(vardef.AutoAnalyzePartitionBatchSize.Load()) needAnalyzePartitionNames := make([]any, 0, len(j.PartitionNames)) for _, partition := range j.PartitionNames { needAnalyzePartitionNames = append(needAnalyzePartitionNames, partition) @@ -299,7 +299,7 @@ func (j *DynamicPartitionedTableAnalysisJob) analyzePartitionIndexes( statsHandle statstypes.StatsHandle, sysProcTracker sysproctrack.Tracker, ) (success bool) { - analyzePartitionBatchSize := int(variable.AutoAnalyzePartitionBatchSize.Load()) + analyzePartitionBatchSize := int(vardef.AutoAnalyzePartitionBatchSize.Load()) // For version 2, analyze one index will analyze all other indexes and columns. // For version 1, analyze one index will only analyze the specified index. analyzeVersion := sctx.GetSessionVars().AnalyzeVersion diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go b/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go index 39b75ed7af376..ac8c7fbcb7012 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/queue.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" @@ -203,7 +204,7 @@ func (pq *AnalysisPriorityQueue) rebuildWithoutLock() error { func (pq *AnalysisPriorityQueue) fetchAllTablesAndBuildAnalysisJobs() error { return statsutil.CallWithSCtx(pq.statsHandle.SPool(), func(sctx sessionctx.Context) error { parameters := exec.GetAutoAnalyzeParameters(sctx) - autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio]) + autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[vardef.TiDBAutoAnalyzeRatio]) pruneMode := variable.PartitionPruneMode(sctx.GetSessionVars().PartitionPruneMode.Load()) is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema) // Query locked tables once to minimize overhead. @@ -394,7 +395,7 @@ func (pq *AnalysisPriorityQueue) processTableStats( return nil } - autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio]) + autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[vardef.TiDBAutoAnalyzeRatio]) // Get current timestamp from the session context. currentTs, err := statsutil.GetStartTS(sctx) if err != nil { diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler.go b/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler.go index de56a1f472e9d..40be9daff063e 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" @@ -114,7 +115,7 @@ func (pq *AnalysisPriorityQueue) recreateAndPushJob( stats *statistics.Table, ) error { parameters := exec.GetAutoAnalyzeParameters(sctx) - autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio]) + autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[vardef.TiDBAutoAnalyzeRatio]) currentTs, err := statsutil.GetStartTS(sctx) if err != nil { return errors.Trace(err) @@ -168,7 +169,7 @@ func (pq *AnalysisPriorityQueue) handleAddIndexEvent( }) parameters := exec.GetAutoAnalyzeParameters(sctx) - autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio]) + autoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[vardef.TiDBAutoAnalyzeRatio]) // Get current timestamp from the session context. currentTs, err := statsutil.GetStartTS(sctx) if err != nil { diff --git a/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel b/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel index 1bf2f4d3732b3..dae0ad012c8b7 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "//pkg/ddl/notifier", "//pkg/sessionctx", "//pkg/sessionctx/sysproctrack", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics/handle/autoanalyze/exec", "//pkg/statistics/handle/autoanalyze/priorityqueue", diff --git a/pkg/statistics/handle/autoanalyze/refresher/refresher.go b/pkg/statistics/handle/autoanalyze/refresher/refresher.go index 06cf3ca0b36a9..818e17708f878 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/refresher.go +++ b/pkg/statistics/handle/autoanalyze/refresher/refresher.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/notifier" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sysproctrack" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/priorityqueue" @@ -61,7 +62,7 @@ func NewRefresher( sysProcTracker sysproctrack.Tracker, ddlNotifier *notifier.DDLNotifier, ) *Refresher { - maxConcurrency := int(variable.AutoAnalyzeConcurrency.Load()) + maxConcurrency := int(vardef.AutoAnalyzeConcurrency.Load()) r := &Refresher{ statsHandle: statsHandle, sysProcTracker: sysProcTracker, @@ -77,7 +78,7 @@ func NewRefresher( // UpdateConcurrency updates the maximum concurrency for auto-analyze jobs func (r *Refresher) UpdateConcurrency() { - newConcurrency := int(variable.AutoAnalyzeConcurrency.Load()) + newConcurrency := int(vardef.AutoAnalyzeConcurrency.Load()) r.worker.UpdateConcurrency(newConcurrency) } @@ -94,7 +95,7 @@ func (r *Refresher) AnalyzeHighestPriorityTables(sctx sessionctx.Context) bool { if !r.isWithinTimeWindow() { return false } - currentAutoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[variable.TiDBAutoAnalyzeRatio]) + currentAutoAnalyzeRatio := exec.ParseAutoAnalyzeRatio(parameters[vardef.TiDBAutoAnalyzeRatio]) currentPruneMode := variable.PartitionPruneMode(sctx.GetSessionVars().PartitionPruneMode.Load()) if !r.jobs.IsInitialized() { if err := r.jobs.Initialize(); err != nil { @@ -197,8 +198,8 @@ func (r *Refresher) setAutoAnalysisTimeWindow( parameters map[string]string, ) error { start, end, err := exec.ParseAutoAnalysisWindow( - parameters[variable.TiDBAutoAnalyzeStartTime], - parameters[variable.TiDBAutoAnalyzeEndTime], + parameters[vardef.TiDBAutoAnalyzeStartTime], + parameters[vardef.TiDBAutoAnalyzeEndTime], ) if err != nil { return errors.Wrap(err, "parse auto analyze period failed") diff --git a/pkg/statistics/handle/bootstrap.go b/pkg/statistics/handle/bootstrap.go index 4c50cd66317f5..d766c9df4f498 100644 --- a/pkg/statistics/handle/bootstrap.go +++ b/pkg/statistics/handle/bootstrap.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/cache" "github.com/pingcap/tidb/pkg/statistics/handle/initstats" @@ -820,6 +820,6 @@ func (h *Handle) InitStats(ctx context.Context, is infoschema.InfoSchema) (err e var IsFullCacheFunc func(cache statstypes.StatsCache, total uint64) bool = isFullCache func isFullCache(cache statstypes.StatsCache, total uint64) bool { - memQuota := variable.StatsCacheMemQuota.Load() + memQuota := vardef.StatsCacheMemQuota.Load() return (uint64(cache.MemConsumed()) >= total/4) || (cache.MemConsumed() >= memQuota && memQuota != 0) } diff --git a/pkg/statistics/handle/cache/BUILD.bazel b/pkg/statistics/handle/cache/BUILD.bazel index 651a7a1d38989..48e2be2f506d6 100644 --- a/pkg/statistics/handle/cache/BUILD.bazel +++ b/pkg/statistics/handle/cache/BUILD.bazel @@ -15,7 +15,7 @@ go_library( "//pkg/meta/model", "//pkg/metrics", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/statistics", "//pkg/statistics/handle/cache/internal", "//pkg/statistics/handle/cache/internal/lfu", diff --git a/pkg/statistics/handle/cache/statscacheinner.go b/pkg/statistics/handle/cache/statscacheinner.go index a405338d5ba07..ad0e1d7e00898 100644 --- a/pkg/statistics/handle/cache/statscacheinner.go +++ b/pkg/statistics/handle/cache/statscacheinner.go @@ -19,7 +19,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/config" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/cache/internal" "github.com/pingcap/tidb/pkg/statistics/handle/cache/internal/lfu" @@ -34,7 +34,7 @@ import ( func NewStatsCache() (*StatsCache, error) { enableQuota := config.GetGlobalConfig().Performance.EnableStatsCacheMemQuota if enableQuota { - capacity := variable.StatsCacheMemQuota.Load() + capacity := vardef.StatsCacheMemQuota.Load() stats, err := lfu.NewLFU(capacity) if err != nil { return nil, err diff --git a/pkg/statistics/handle/ddl/BUILD.bazel b/pkg/statistics/handle/ddl/BUILD.bazel index 5a48dfcbfecb4..b49fef529d9fc 100644 --- a/pkg/statistics/handle/ddl/BUILD.bazel +++ b/pkg/statistics/handle/ddl/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "//pkg/infoschema", "//pkg/meta/model", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics/handle/history", "//pkg/statistics/handle/lockstats", diff --git a/pkg/statistics/handle/ddl/subscriber.go b/pkg/statistics/handle/ddl/subscriber.go index 49aeb70e39760..961130527bcd6 100644 --- a/pkg/statistics/handle/ddl/subscriber.go +++ b/pkg/statistics/handle/ddl/subscriber.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics/handle/history" "github.com/pingcap/tidb/pkg/statistics/handle/lockstats" @@ -362,7 +363,7 @@ func getCurrentPruneMode( ) (variable.PartitionPruneMode, error) { pruneMode, err := sctx.GetSessionVars(). GlobalVarsAccessor. - GetGlobalSysVar(variable.TiDBPartitionPruneMode) + GetGlobalSysVar(vardef.TiDBPartitionPruneMode) return variable.PartitionPruneMode(pruneMode), errors.Trace(err) } @@ -371,7 +372,7 @@ func getEnableHistoricalStats( ) (bool, error) { val, err := sctx.GetSessionVars(). GlobalVarsAccessor. - GetGlobalSysVar(variable.TiDBEnableHistoricalStats) + GetGlobalSysVar(vardef.TiDBEnableHistoricalStats) return variable.TiDBOptOn(val), errors.Trace(err) } diff --git a/pkg/statistics/handle/storage/BUILD.bazel b/pkg/statistics/handle/storage/BUILD.bazel index 9b77ba15d5f90..3beffab153243 100644 --- a/pkg/statistics/handle/storage/BUILD.bazel +++ b/pkg/statistics/handle/storage/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/statistics/asyncload", diff --git a/pkg/statistics/handle/storage/gc.go b/pkg/statistics/handle/storage/gc.go index 0e7889b63f33e..4f17198a40667 100644 --- a/pkg/statistics/handle/storage/gc.go +++ b/pkg/statistics/handle/storage/gc.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/cache" "github.com/pingcap/tidb/pkg/statistics/handle/lockstats" @@ -127,7 +127,7 @@ func GCStats( if err := ClearOutdatedHistoryStats(sctx); err != nil { logutil.BgLogger().Warn("failed to gc outdated historical stats", - zap.Duration("duration", variable.HistoricalStatsDuration.Load()), + zap.Duration("duration", vardef.HistoricalStatsDuration.Load()), zap.Error(err)) } @@ -196,7 +196,7 @@ func forCount(total int64, batch int64) int64 { // ClearOutdatedHistoryStats clear outdated historical stats func ClearOutdatedHistoryStats(sctx sessionctx.Context) error { sql := "select count(*) from mysql.stats_meta_history use index (idx_create_time) where create_time <= NOW() - INTERVAL %? SECOND" - rs, err := util.Exec(sctx, sql, variable.HistoricalStatsDuration.Load().Seconds()) + rs, err := util.Exec(sctx, sql, vardef.HistoricalStatsDuration.Load().Seconds()) if err != nil { return err } @@ -212,14 +212,14 @@ func ClearOutdatedHistoryStats(sctx sessionctx.Context) error { if count > 0 { for n := int64(0); n < forCount(count, int64(1000)); n++ { sql = "delete from mysql.stats_meta_history use index (idx_create_time) where create_time <= NOW() - INTERVAL %? SECOND limit 1000 " - _, err = util.Exec(sctx, sql, variable.HistoricalStatsDuration.Load().Seconds()) + _, err = util.Exec(sctx, sql, vardef.HistoricalStatsDuration.Load().Seconds()) if err != nil { return err } } for n := int64(0); n < forCount(count, int64(50)); n++ { sql = "delete from mysql.stats_history use index (idx_create_time) where create_time <= NOW() - INTERVAL %? SECOND limit 50 " - _, err = util.Exec(sctx, sql, variable.HistoricalStatsDuration.Load().Seconds()) + _, err = util.Exec(sctx, sql, vardef.HistoricalStatsDuration.Load().Seconds()) return err } logutil.BgLogger().Info("clear outdated historical stats") diff --git a/pkg/statistics/handle/storage/stats_read_writer.go b/pkg/statistics/handle/storage/stats_read_writer.go index a658fd5880d1e..f7c74bd597662 100644 --- a/pkg/statistics/handle/storage/stats_read_writer.go +++ b/pkg/statistics/handle/storage/stats_read_writer.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" handle_metrics "github.com/pingcap/tidb/pkg/statistics/handle/metrics" @@ -309,10 +310,10 @@ func (s *statsReadWriter) DumpHistoricalStatsBySnapshot( ) { historicalStatsEnabled, err := s.statsHandler.CheckHistoricalStatsEnable() if err != nil { - return nil, nil, errors.Errorf("check %v failed: %v", variable.TiDBEnableHistoricalStats, err) + return nil, nil, errors.Errorf("check %v failed: %v", vardef.TiDBEnableHistoricalStats, err) } if !historicalStatsEnabled { - return nil, nil, errors.Errorf("%v should be enabled", variable.TiDBEnableHistoricalStats) + return nil, nil, errors.Errorf("%v should be enabled", vardef.TiDBEnableHistoricalStats) } defer func() { diff --git a/pkg/statistics/handle/syncload/BUILD.bazel b/pkg/statistics/handle/syncload/BUILD.bazel index 478ae548d12f3..79e3db7bdf4c7 100644 --- a/pkg/statistics/handle/syncload/BUILD.bazel +++ b/pkg/statistics/handle/syncload/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/parser/mysql", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/statistics/handle/storage", diff --git a/pkg/statistics/handle/syncload/stats_syncload.go b/pkg/statistics/handle/syncload/stats_syncload.go index a7f8fe3a204db..c4ff551a6b2a1 100644 --- a/pkg/statistics/handle/syncload/stats_syncload.go +++ b/pkg/statistics/handle/syncload/stats_syncload.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/storage" @@ -305,7 +306,7 @@ func (s *statsSyncLoad) handleOneItemTask(task *statstypes.NeededItemTask) (err } }() var skipTypes map[string]struct{} - val, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBAnalyzeSkipColumnTypes) + val, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBAnalyzeSkipColumnTypes) if err != nil { logutil.BgLogger().Warn("failed to get global variable", zap.Error(err)) } else { diff --git a/pkg/statistics/handle/util/BUILD.bazel b/pkg/statistics/handle/util/BUILD.bazel index dd5915b072e24..a0b0566ce1803 100644 --- a/pkg/statistics/handle/util/BUILD.bazel +++ b/pkg/statistics/handle/util/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/planner/core/resolve", "//pkg/sessionctx", "//pkg/sessionctx/sysproctrack", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/table", "//pkg/types", diff --git a/pkg/statistics/handle/util/util.go b/pkg/statistics/handle/util/util.go index bed79fc1a5998..7949ae04e380f 100644 --- a/pkg/statistics/handle/util/util.go +++ b/pkg/statistics/handle/util/util.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -108,14 +109,14 @@ func CallWithSCtx(pool util.SessionPool, f func(sctx sessionctx.Context) error, // UpdateSCtxVarsForStats updates all necessary variables that may affect the behavior of statistics. func UpdateSCtxVarsForStats(sctx sessionctx.Context) error { // async merge global stats - enableAsyncMergeGlobalStats, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBEnableAsyncMergeGlobalStats) + enableAsyncMergeGlobalStats, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBEnableAsyncMergeGlobalStats) if err != nil { return err } sctx.GetSessionVars().EnableAsyncMergeGlobalStats = variable.TiDBOptOn(enableAsyncMergeGlobalStats) // concurrency of save stats to storage - analyzePartitionConcurrency, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBAnalyzePartitionConcurrency) + analyzePartitionConcurrency, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBAnalyzePartitionConcurrency) if err != nil { return err } @@ -126,7 +127,7 @@ func UpdateSCtxVarsForStats(sctx sessionctx.Context) error { sctx.GetSessionVars().AnalyzePartitionConcurrency = int(c) // analyzer version - verInString, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBAnalyzeVersion) + verInString, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBAnalyzeVersion) if err != nil { return err } @@ -137,40 +138,40 @@ func UpdateSCtxVarsForStats(sctx sessionctx.Context) error { sctx.GetSessionVars().AnalyzeVersion = int(ver) // enable historical stats - val, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBEnableHistoricalStats) + val, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBEnableHistoricalStats) if err != nil { return err } sctx.GetSessionVars().EnableHistoricalStats = variable.TiDBOptOn(val) // partition mode - pruneMode, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBPartitionPruneMode) + pruneMode, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBPartitionPruneMode) if err != nil { return err } sctx.GetSessionVars().PartitionPruneMode.Store(pruneMode) // enable analyze snapshot - analyzeSnapshot, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBEnableAnalyzeSnapshot) + analyzeSnapshot, err := sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBEnableAnalyzeSnapshot) if err != nil { return err } sctx.GetSessionVars().EnableAnalyzeSnapshot = variable.TiDBOptOn(analyzeSnapshot) // enable skip column types - val, err = sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBAnalyzeSkipColumnTypes) + val, err = sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBAnalyzeSkipColumnTypes) if err != nil { return err } sctx.GetSessionVars().AnalyzeSkipColumnTypes = variable.ParseAnalyzeSkipColumnTypes(val) // skip missing partition stats - val, err = sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBSkipMissingPartitionStats) + val, err = sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBSkipMissingPartitionStats) if err != nil { return err } sctx.GetSessionVars().SkipMissingPartitionStats = variable.TiDBOptOn(val) - verInString, err = sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBMergePartitionStatsConcurrency) + verInString, err = sctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBMergePartitionStatsConcurrency) if err != nil { return err } diff --git a/pkg/statistics/histogram.go b/pkg/statistics/histogram.go index 28531936f2188..84554444e3a01 100644 --- a/pkg/statistics/histogram.go +++ b/pkg/statistics/histogram.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/planctx" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" "github.com/pingcap/tidb/pkg/tablecodec" @@ -988,7 +989,7 @@ func (hg *Histogram) OutOfRangeRowCount( return 0 } - allowUseModifyCount := sctx.GetSessionVars().GetOptObjective() != variable.OptObjectiveDeterminate + allowUseModifyCount := sctx.GetSessionVars().GetOptObjective() != vardef.OptObjectiveDeterminate // Convert the lower and upper bound of the histogram to scalar value(float64) histL := convertDatumToScalar(hg.GetLower(0), commonPrefix) diff --git a/pkg/store/copr/BUILD.bazel b/pkg/store/copr/BUILD.bazel index b10166d0362c1..4826b08beb671 100644 --- a/pkg/store/copr/BUILD.bazel +++ b/pkg/store/copr/BUILD.bazel @@ -24,7 +24,7 @@ go_library( "//pkg/metrics", "//pkg/parser/terror", "//pkg/resourcegroup", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/copr/metrics", "//pkg/store/driver/backoff", "//pkg/store/driver/error", diff --git a/pkg/store/copr/batch_coprocessor.go b/pkg/store/copr/batch_coprocessor.go index 452d5b35787e1..7749a0c49e9aa 100644 --- a/pkg/store/copr/batch_coprocessor.go +++ b/pkg/store/copr/batch_coprocessor.go @@ -36,7 +36,7 @@ import ( "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/ddl/placement" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/driver/backoff" derr "github.com/pingcap/tidb/pkg/store/driver/error" "github.com/pingcap/tidb/pkg/util/intest" @@ -1345,7 +1345,7 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *backoff.Backo } rgName := b.req.ResourceGroupName - if !variable.EnableResourceControl.Load() { + if !vardef.EnableResourceControl.Load() { rgName = "" } req := tikvrpc.NewRequest(task.cmdType, &copReq, kvrpcpb.Context{ diff --git a/pkg/store/copr/coprocessor.go b/pkg/store/copr/coprocessor.go index 4796d6822d730..419ab7d0f58df 100644 --- a/pkg/store/copr/coprocessor.go +++ b/pkg/store/copr/coprocessor.go @@ -42,7 +42,7 @@ import ( tidbmetrics "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/resourcegroup" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" copr_metrics "github.com/pingcap/tidb/pkg/store/copr/metrics" "github.com/pingcap/tidb/pkg/store/driver/backoff" derr "github.com/pingcap/tidb/pkg/store/driver/error" @@ -1357,7 +1357,7 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask) (* replicaRead := worker.req.ReplicaRead rgName := worker.req.ResourceGroupName - if task.storeType == kv.TiFlash && !variable.EnableResourceControl.Load() { + if task.storeType == kv.TiFlash && !vardef.EnableResourceControl.Load() { // By calling variable.EnableGlobalResourceControlFunc() and setting global variables, // tikv/client-go can sense whether the rg function is enabled // But for tiflash, it check if rgName is empty to decide if resource control is enabled or not. diff --git a/pkg/store/driver/BUILD.bazel b/pkg/store/driver/BUILD.bazel index 69e3e65f68dcd..1a9a3391d31a8 100644 --- a/pkg/store/driver/BUILD.bazel +++ b/pkg/store/driver/BUILD.bazel @@ -51,7 +51,6 @@ go_test( "//pkg/ddl", "//pkg/domain", "//pkg/kv", - "//pkg/meta/model", "//pkg/session", "//pkg/store/copr", "//pkg/store/mockstore", diff --git a/pkg/store/driver/client_test.go b/pkg/store/driver/client_test.go index 2576fce5426bc..b6e95ce7f6e6c 100644 --- a/pkg/store/driver/client_test.go +++ b/pkg/store/driver/client_test.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/util/tracing" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" @@ -46,7 +45,7 @@ func (c *mockTiKVClient) SendRequest(ctx context.Context, addr string, req *tikv func TestInjectTracingClient(t *testing.T) { cases := []struct { name string - trace *model.TraceInfo + trace *tracing.TraceInfo existSourceStmt *kvrpcpb.SourceStmt }{ { @@ -55,20 +54,20 @@ func TestInjectTracingClient(t *testing.T) { }, { name: "trace not nil", - trace: &model.TraceInfo{ + trace: &tracing.TraceInfo{ ConnectionID: 123, SessionAlias: "alias123", }, }, { name: "only connection id in trace valid", - trace: &model.TraceInfo{ + trace: &tracing.TraceInfo{ ConnectionID: 456, }, }, { name: "only session alias in trace valid and sourceStmt exists", - trace: &model.TraceInfo{ + trace: &tracing.TraceInfo{ SessionAlias: "alias456", }, existSourceStmt: &kvrpcpb.SourceStmt{}, diff --git a/pkg/store/gcworker/BUILD.bazel b/pkg/store/gcworker/BUILD.bazel index 3044a32492718..b0c4750faf2d9 100644 --- a/pkg/store/gcworker/BUILD.bazel +++ b/pkg/store/gcworker/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/privilege", "//pkg/session", "//pkg/session/types", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/tablecodec", "//pkg/util", diff --git a/pkg/store/gcworker/gc_worker.go b/pkg/store/gcworker/gc_worker.go index 38c24715c41fd..4893a99ea9fbb 100644 --- a/pkg/store/gcworker/gc_worker.go +++ b/pkg/store/gcworker/gc_worker.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/session" sessiontypes "github.com/pingcap/tidb/pkg/session/types" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/tablecodec" util2 "github.com/pingcap/tidb/pkg/util" @@ -243,7 +244,7 @@ func createSession(store kv.Storage) sessiontypes.Session { } // GetScope gets the status variables scope. -func (w *GCWorker) GetScope(status string) variable.ScopeFlag { +func (w *GCWorker) GetScope(status string) vardef.ScopeFlag { return variable.DefaultStatusVarScopeFlag } diff --git a/pkg/store/mockstore/mockcopr/BUILD.bazel b/pkg/store/mockstore/mockcopr/BUILD.bazel index 56811e932febf..05fa695182936 100644 --- a/pkg/store/mockstore/mockcopr/BUILD.bazel +++ b/pkg/store/mockstore/mockcopr/BUILD.bazel @@ -25,7 +25,7 @@ go_library( "//pkg/planner/core/resolve", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/statistics", "//pkg/tablecodec", "//pkg/types", diff --git a/pkg/store/mockstore/mockcopr/cop_handler_dag.go b/pkg/store/mockstore/mockcopr/cop_handler_dag.go index 5bf07633e5f7b..467f9bfc7d77c 100644 --- a/pkg/store/mockstore/mockcopr/cop_handler_dag.go +++ b/pkg/store/mockstore/mockcopr/cop_handler_dag.go @@ -34,7 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -116,7 +116,7 @@ func (h coprHandler) buildDAGExecutor(req *coprocessor.Request) (*dagContext, ex if dagReq.DivPrecisionIncrement != nil { sctx.GetSessionVars().DivPrecisionIncrement = int(*dagReq.DivPrecisionIncrement) } else { - sctx.GetSessionVars().DivPrecisionIncrement = variable.DefDivPrecisionIncrement + sctx.GetSessionVars().DivPrecisionIncrement = vardef.DefDivPrecisionIncrement } ctx := &dagContext{ diff --git a/pkg/store/mockstore/unistore/cophandler/BUILD.bazel b/pkg/store/mockstore/unistore/cophandler/BUILD.bazel index 34df6dbad4c4a..43f796f6346b7 100644 --- a/pkg/store/mockstore/unistore/cophandler/BUILD.bazel +++ b/pkg/store/mockstore/unistore/cophandler/BUILD.bazel @@ -23,7 +23,7 @@ go_library( "//pkg/planner/core/resolve", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/statistics", "//pkg/store/mockstore/unistore/client", "//pkg/store/mockstore/unistore/lockstore", diff --git a/pkg/store/mockstore/unistore/cophandler/cop_handler.go b/pkg/store/mockstore/unistore/cophandler/cop_handler.go index 0ea0e34e46882..4edf41f62e685 100644 --- a/pkg/store/mockstore/unistore/cophandler/cop_handler.go +++ b/pkg/store/mockstore/unistore/cophandler/cop_handler.go @@ -36,7 +36,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/mockstore/unistore/client" "github.com/pingcap/tidb/pkg/store/mockstore/unistore/lockstore" "github.com/pingcap/tidb/pkg/store/mockstore/unistore/tikv/dbreader" @@ -350,7 +350,7 @@ func buildDAG(reader *dbreader.DBReader, lockStore *lockstore.MemStore, req *cop if dagReq.DivPrecisionIncrement != nil { sctx.GetSessionVars().DivPrecisionIncrement = int(*dagReq.DivPrecisionIncrement) } else { - sctx.GetSessionVars().DivPrecisionIncrement = variable.DefDivPrecisionIncrement + sctx.GetSessionVars().DivPrecisionIncrement = vardef.DefDivPrecisionIncrement } ctx := &dagContext{ evalContext: &evalContext{sctx: sctx}, diff --git a/pkg/table/tables/BUILD.bazel b/pkg/table/tables/BUILD.bazel index 2f3b6a1347fba..23bb2901daa6d 100644 --- a/pkg/table/tables/BUILD.bazel +++ b/pkg/table/tables/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/table", @@ -94,6 +95,7 @@ go_test( "//pkg/session", "//pkg/session/types", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/table", diff --git a/pkg/table/tables/bench_test.go b/pkg/table/tables/bench_test.go index 0e16594c8ca26..2efc731f9e914 100644 --- a/pkg/table/tables/bench_test.go +++ b/pkg/table/tables/bench_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/testkit" @@ -47,7 +47,7 @@ func BenchmarkAddRecordInPipelinedDML(b *testing.B) { tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(b, err) - variable.EnableMDL.Store(true) + vardef.EnableMDL.Store(true) // Pre-create data to be inserted records := make([][]types.Datum, batchSize) @@ -101,7 +101,7 @@ func BenchmarkRemoveRecordInPipelinedDML(b *testing.B) { tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(b, err) - variable.EnableMDL.Store(true) + vardef.EnableMDL.Store(true) // Pre-create and add initial records records := make([][]types.Datum, batchSize) diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index 0f12063575c9f..4e873c6e3f20f 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -36,7 +36,7 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" @@ -1756,7 +1756,7 @@ func partitionedTableAddRecord(ctx table.MutateContext, txn kv.Transaction, t *p } exchangePartitionInfo := t.Meta().ExchangePartitionInfo if exchangePartitionInfo != nil && exchangePartitionInfo.ExchangePartitionDefID == pid && - variable.EnableCheckConstraint.Load() { + vardef.EnableCheckConstraint.Load() { err = checkConstraintForExchangePartition(ctx, r, pid, exchangePartitionInfo.ExchangePartitionTableID) if err != nil { return nil, errors.WithStack(err) @@ -1894,7 +1894,7 @@ func partitionedTableUpdateRecord(ctx table.MutateContext, txn kv.Transaction, t } exchangePartitionInfo := t.Meta().ExchangePartitionInfo if exchangePartitionInfo != nil && exchangePartitionInfo.ExchangePartitionDefID == to && - variable.EnableCheckConstraint.Load() { + vardef.EnableCheckConstraint.Load() { err = checkConstraintForExchangePartition(ctx, newData, to, exchangePartitionInfo.ExchangePartitionTableID) if err != nil { return errors.WithStack(err) diff --git a/pkg/timer/tablestore/BUILD.bazel b/pkg/timer/tablestore/BUILD.bazel index d329d57126083..f4d829d63ab63 100644 --- a/pkg/timer/tablestore/BUILD.bazel +++ b/pkg/timer/tablestore/BUILD.bazel @@ -13,7 +13,7 @@ go_library( "//pkg/kv", "//pkg/parser/terror", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/timer/api", "//pkg/util", "//pkg/util/chunk", diff --git a/pkg/timer/tablestore/store.go b/pkg/timer/tablestore/store.go index 1e0fdbcbc07b2..5665d4afc06eb 100644 --- a/pkg/timer/tablestore/store.go +++ b/pkg/timer/tablestore/store.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/timer/api" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/chunk" @@ -137,7 +137,7 @@ func (s *tableTimerStoreCore) List(ctx context.Context, cond api.Cond) ([]*api.T return nil, err } - tidbTimeZone, err := sctx.GetSessionVars().GetGlobalSystemVar(ctx, variable.TimeZone) + tidbTimeZone, err := sctx.GetSessionVars().GetGlobalSystemVar(ctx, vardef.TimeZone) if err != nil { return nil, err } diff --git a/pkg/ttl/session/BUILD.bazel b/pkg/ttl/session/BUILD.bazel index 2aa1fb4a2d91e..80a6dc34b6707 100644 --- a/pkg/ttl/session/BUILD.bazel +++ b/pkg/ttl/session/BUILD.bazel @@ -10,7 +10,7 @@ go_library( "//pkg/kv", "//pkg/parser/terror", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/sessiontxn", "//pkg/ttl/metrics", "//pkg/util/chunk", @@ -33,7 +33,7 @@ go_test( shard_count = 7, deps = [ ":session", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/testkit/testsetup", "//pkg/util", diff --git a/pkg/ttl/session/session.go b/pkg/ttl/session/session.go index 812108b3e4061..90919836b06cd 100644 --- a/pkg/ttl/session/session.go +++ b/pkg/ttl/session/session.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/ttl/metrics" "github.com/pingcap/tidb/pkg/util/chunk" @@ -161,12 +161,12 @@ func (s *session) RunInTxn(ctx context.Context, fn func() error, txnMode TxnMode func (s *session) ResetWithGlobalTimeZone(ctx context.Context) error { sessVar := s.GetSessionVars() if sessVar.TimeZone != nil { - globalTZ, err := sessVar.GetGlobalSystemVar(ctx, variable.TimeZone) + globalTZ, err := sessVar.GetGlobalSystemVar(ctx, vardef.TimeZone) if err != nil { return err } - tz, err := sessVar.GetSessionOrGlobalSystemVar(ctx, variable.TimeZone) + tz, err := sessVar.GetSessionOrGlobalSystemVar(ctx, vardef.TimeZone) if err != nil { return err } diff --git a/pkg/ttl/session/sysvar_test.go b/pkg/ttl/session/sysvar_test.go index a234346acd277..5683e0d64230d 100644 --- a/pkg/ttl/session/sysvar_test.go +++ b/pkg/ttl/session/sysvar_test.go @@ -19,89 +19,89 @@ import ( "strconv" "testing" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) func TestSysVarTTLJobEnable(t *testing.T) { - origEnableDDL := variable.EnableTTLJob.Load() + origEnableDDL := vardef.EnableTTLJob.Load() defer func() { - variable.EnableTTLJob.Store(origEnableDDL) + vardef.EnableTTLJob.Store(origEnableDDL) }() store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("set @@global.tidb_ttl_job_enable=0") - require.False(t, variable.EnableTTLJob.Load()) + require.False(t, vardef.EnableTTLJob.Load()) tk.MustQuery("select @@global.tidb_ttl_job_enable").Check(testkit.Rows("0")) tk.MustQuery("select @@tidb_ttl_job_enable").Check(testkit.Rows("0")) tk.MustExec("set @@global.tidb_ttl_job_enable=1") - require.True(t, variable.EnableTTLJob.Load()) + require.True(t, vardef.EnableTTLJob.Load()) tk.MustQuery("select @@global.tidb_ttl_job_enable").Check(testkit.Rows("1")) tk.MustQuery("select @@tidb_ttl_job_enable").Check(testkit.Rows("1")) tk.MustExec("set @@global.tidb_ttl_job_enable=0") - require.False(t, variable.EnableTTLJob.Load()) + require.False(t, vardef.EnableTTLJob.Load()) tk.MustQuery("select @@global.tidb_ttl_job_enable").Check(testkit.Rows("0")) tk.MustQuery("select @@tidb_ttl_job_enable").Check(testkit.Rows("0")) } func TestSysVarTTLScanBatchSize(t *testing.T) { - origScanBatchSize := variable.TTLScanBatchSize.Load() + origScanBatchSize := vardef.TTLScanBatchSize.Load() defer func() { - variable.TTLScanBatchSize.Store(origScanBatchSize) + vardef.TTLScanBatchSize.Store(origScanBatchSize) }() store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("set @@global.tidb_ttl_scan_batch_size=789") - require.Equal(t, int64(789), variable.TTLScanBatchSize.Load()) + require.Equal(t, int64(789), vardef.TTLScanBatchSize.Load()) tk.MustQuery("select @@global.tidb_ttl_scan_batch_size").Check(testkit.Rows("789")) tk.MustQuery("select @@tidb_ttl_scan_batch_size").Check(testkit.Rows("789")) tk.MustExec("set @@global.tidb_ttl_scan_batch_size=0") - require.Equal(t, int64(1), variable.TTLScanBatchSize.Load()) + require.Equal(t, int64(1), vardef.TTLScanBatchSize.Load()) tk.MustQuery("select @@global.tidb_ttl_scan_batch_size").Check(testkit.Rows("1")) tk.MustQuery("select @@tidb_ttl_scan_batch_size").Check(testkit.Rows("1")) - maxVal := int64(variable.DefTiDBTTLScanBatchMaxSize) + maxVal := int64(vardef.DefTiDBTTLScanBatchMaxSize) tk.MustExec(fmt.Sprintf("set @@global.tidb_ttl_scan_batch_size=%d", maxVal+1)) - require.Equal(t, maxVal, variable.TTLScanBatchSize.Load()) + require.Equal(t, maxVal, vardef.TTLScanBatchSize.Load()) tk.MustQuery("select @@global.tidb_ttl_scan_batch_size").Check(testkit.Rows(strconv.FormatInt(maxVal, 10))) tk.MustQuery("select @@tidb_ttl_scan_batch_size").Check(testkit.Rows(strconv.FormatInt(maxVal, 10))) } func TestSysVarTTLScanDeleteBatchSize(t *testing.T) { - origScanBatchSize := variable.TTLScanBatchSize.Load() + origScanBatchSize := vardef.TTLScanBatchSize.Load() defer func() { - variable.TTLScanBatchSize.Store(origScanBatchSize) + vardef.TTLScanBatchSize.Store(origScanBatchSize) }() store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("set @@global.tidb_ttl_delete_batch_size=789") - require.Equal(t, int64(789), variable.TTLDeleteBatchSize.Load()) + require.Equal(t, int64(789), vardef.TTLDeleteBatchSize.Load()) tk.MustQuery("select @@global.tidb_ttl_delete_batch_size").Check(testkit.Rows("789")) tk.MustQuery("select @@tidb_ttl_delete_batch_size").Check(testkit.Rows("789")) tk.MustExec("set @@global.tidb_ttl_delete_batch_size=0") - require.Equal(t, int64(1), variable.TTLDeleteBatchSize.Load()) + require.Equal(t, int64(1), vardef.TTLDeleteBatchSize.Load()) tk.MustQuery("select @@global.tidb_ttl_delete_batch_size").Check(testkit.Rows("1")) tk.MustQuery("select @@tidb_ttl_delete_batch_size").Check(testkit.Rows("1")) - maxVal := int64(variable.DefTiDBTTLDeleteBatchMaxSize) + maxVal := int64(vardef.DefTiDBTTLDeleteBatchMaxSize) tk.MustExec(fmt.Sprintf("set @@global.tidb_ttl_delete_batch_size=%d", maxVal+1)) - require.Equal(t, maxVal, variable.TTLDeleteBatchSize.Load()) + require.Equal(t, maxVal, vardef.TTLDeleteBatchSize.Load()) tk.MustQuery("select @@global.tidb_ttl_delete_batch_size").Check(testkit.Rows(strconv.FormatInt(maxVal, 10))) tk.MustQuery("select @@tidb_ttl_delete_batch_size").Check(testkit.Rows(strconv.FormatInt(maxVal, 10))) } func TestSysVarTTLScanDeleteLimit(t *testing.T) { - origDeleteLimit := variable.TTLDeleteRateLimit.Load() + origDeleteLimit := vardef.TTLDeleteRateLimit.Load() defer func() { - variable.TTLDeleteRateLimit.Store(origDeleteLimit) + vardef.TTLDeleteRateLimit.Store(origDeleteLimit) }() store := testkit.CreateMockStore(t) @@ -109,17 +109,17 @@ func TestSysVarTTLScanDeleteLimit(t *testing.T) { tk.MustQuery("select @@global.tidb_ttl_delete_rate_limit").Check(testkit.Rows("0")) tk.MustExec("set @@global.tidb_ttl_delete_rate_limit=100000") - require.Equal(t, int64(100000), variable.TTLDeleteRateLimit.Load()) + require.Equal(t, int64(100000), vardef.TTLDeleteRateLimit.Load()) tk.MustQuery("select @@global.tidb_ttl_delete_rate_limit").Check(testkit.Rows("100000")) tk.MustQuery("select @@tidb_ttl_delete_rate_limit").Check(testkit.Rows("100000")) tk.MustExec("set @@global.tidb_ttl_delete_rate_limit=0") - require.Equal(t, int64(0), variable.TTLDeleteRateLimit.Load()) + require.Equal(t, int64(0), vardef.TTLDeleteRateLimit.Load()) tk.MustQuery("select @@global.tidb_ttl_delete_rate_limit").Check(testkit.Rows("0")) tk.MustQuery("select @@tidb_ttl_delete_rate_limit").Check(testkit.Rows("0")) tk.MustExec("set @@global.tidb_ttl_delete_rate_limit=-1") - require.Equal(t, int64(0), variable.TTLDeleteRateLimit.Load()) + require.Equal(t, int64(0), vardef.TTLDeleteRateLimit.Load()) tk.MustQuery("select @@global.tidb_ttl_delete_rate_limit").Check(testkit.Rows("0")) tk.MustQuery("select @@tidb_ttl_delete_rate_limit").Check(testkit.Rows("0")) } diff --git a/pkg/ttl/ttlworker/BUILD.bazel b/pkg/ttl/ttlworker/BUILD.bazel index aa5a14bc7875f..4833634e50aa8 100644 --- a/pkg/ttl/ttlworker/BUILD.bazel +++ b/pkg/ttl/ttlworker/BUILD.bazel @@ -25,7 +25,7 @@ go_library( "//pkg/parser/ast", "//pkg/parser/terror", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/driver/error", "//pkg/timer/api", "//pkg/timer/runtime", @@ -84,6 +84,7 @@ go_test( "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/store/mockstore", diff --git a/pkg/ttl/ttlworker/del.go b/pkg/ttl/ttlworker/del.go index 4821b8790d463..ec116cdd1db83 100644 --- a/pkg/ttl/ttlworker/del.go +++ b/pkg/ttl/ttlworker/del.go @@ -22,7 +22,7 @@ import ( "sync/atomic" "time" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/ttl/cache" "github.com/pingcap/tidb/pkg/ttl/metrics" "github.com/pingcap/tidb/pkg/ttl/session" @@ -71,7 +71,7 @@ var beforeWaitLimiterForTest = &beforeWaitLimiterForTestType{} func (l *defaultDelRateLimiter) WaitDelToken(ctx context.Context) error { limit := l.limit.Load() - if variable.TTLDeleteRateLimit.Load() != limit { + if vardef.TTLDeleteRateLimit.Load() != limit { limit = l.reset() } @@ -93,7 +93,7 @@ func (l *defaultDelRateLimiter) WaitDelToken(ctx context.Context) error { func (l *defaultDelRateLimiter) reset() (newLimit int64) { l.Lock() defer l.Unlock() - newLimit = variable.TTLDeleteRateLimit.Load() + newLimit = vardef.TTLDeleteRateLimit.Load() if newLimit != l.limit.Load() { l.limit.Store(newLimit) rateLimit := rate.Inf @@ -139,7 +139,7 @@ func (t *ttlDeleteTask) doDelete(ctx context.Context, rawSe session.Session) (re se := newTableSession(rawSe, t.tbl, t.expire) for len(leftRows) > 0 && ctx.Err() == nil { - maxBatch := variable.TTLDeleteBatchSize.Load() + maxBatch := vardef.TTLDeleteBatchSize.Load() var delBatch [][]types.Datum if int64(len(leftRows)) < maxBatch { delBatch = leftRows diff --git a/pkg/ttl/ttlworker/del_test.go b/pkg/ttl/ttlworker/del_test.go index e00e4c6e62604..476bf8f08bdb7 100644 --- a/pkg/ttl/ttlworker/del_test.go +++ b/pkg/ttl/ttlworker/del_test.go @@ -26,7 +26,7 @@ import ( "testing" "time" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/ttl/cache" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -212,10 +212,10 @@ func (m *mockDelRateLimiter) WaitDelToken(ctx context.Context) error { } func TestTTLDeleteTaskDoDelete(t *testing.T) { - origBatchSize := variable.TTLDeleteBatchSize.Load() + origBatchSize := vardef.TTLDeleteBatchSize.Load() delBatch := 3 - variable.TTLDeleteBatchSize.Store(int64(delBatch)) - defer variable.TTLDeleteBatchSize.Store(origBatchSize) + vardef.TTLDeleteBatchSize.Store(int64(delBatch)) + defer vardef.TTLDeleteBatchSize.Store(origBatchSize) t1 := newMockTTLTbl(t, "t1") s := newMockSession(t) @@ -398,12 +398,12 @@ func TestTTLDeleteRateLimiter(t *testing.T) { origGlobalDelRateLimiter := globalDelRateLimiter defer func() { globalDelRateLimiter = origGlobalDelRateLimiter - variable.TTLDeleteRateLimit.Store(variable.DefTiDBTTLDeleteRateLimit) + vardef.TTLDeleteRateLimit.Store(vardef.DefTiDBTTLDeleteRateLimit) }() // The global inner limiter should have a default config - require.Equal(t, 0, variable.DefTiDBTTLDeleteRateLimit) - require.Equal(t, int64(0), variable.TTLDeleteRateLimit.Load()) + require.Equal(t, 0, vardef.DefTiDBTTLDeleteRateLimit) + require.Equal(t, int64(0), vardef.TTLDeleteRateLimit.Load()) require.Equal(t, int64(0), globalDelRateLimiter.(*defaultDelRateLimiter).limit.Load()) require.Equal(t, rate.Inf, globalDelRateLimiter.(*defaultDelRateLimiter).limiter.Limit()) // The newDelRateLimiter() should return a default config @@ -418,12 +418,12 @@ func TestTTLDeleteRateLimiter(t *testing.T) { } }() - variable.TTLDeleteRateLimit.Store(100000) + vardef.TTLDeleteRateLimit.Store(100000) require.NoError(t, globalDelRateLimiter.WaitDelToken(ctx)) require.Equal(t, rate.Limit(100000), globalDelRateLimiter.(*defaultDelRateLimiter).limiter.Limit()) require.Equal(t, int64(100000), globalDelRateLimiter.(*defaultDelRateLimiter).limit.Load()) - variable.TTLDeleteRateLimit.Store(0) + vardef.TTLDeleteRateLimit.Store(0) require.NoError(t, globalDelRateLimiter.WaitDelToken(ctx)) require.Equal(t, rate.Inf, globalDelRateLimiter.(*defaultDelRateLimiter).limiter.Limit()) require.Equal(t, int64(0), globalDelRateLimiter.(*defaultDelRateLimiter).limit.Load()) @@ -437,9 +437,9 @@ func TestTTLDeleteRateLimiter(t *testing.T) { } func TestTTLDeleteTaskWorker(t *testing.T) { - origBatchSize := variable.TTLDeleteBatchSize.Load() - variable.TTLDeleteBatchSize.Store(3) - defer variable.TTLDeleteBatchSize.Store(origBatchSize) + origBatchSize := vardef.TTLDeleteBatchSize.Load() + vardef.TTLDeleteBatchSize.Store(3) + defer vardef.TTLDeleteBatchSize.Store(origBatchSize) t1 := newMockTTLTbl(t, "t1") t2 := newMockTTLTbl(t, "t2") @@ -595,13 +595,13 @@ func TestDelRateLimiterConcurrency(t *testing.T) { origGlobalDelRateLimiter := globalDelRateLimiter defer func() { globalDelRateLimiter = origGlobalDelRateLimiter - variable.TTLDeleteRateLimit.Store(variable.DefTiDBTTLDeleteRateLimit) + vardef.TTLDeleteRateLimit.Store(vardef.DefTiDBTTLDeleteRateLimit) }() globalDelRateLimiter = newDelRateLimiter() require.NoError(t, globalDelRateLimiter.WaitDelToken(context.Background())) - variable.TTLDeleteRateLimit.Store(128) + vardef.TTLDeleteRateLimit.Store(128) var waiting atomic.Int64 continue1 := make(chan struct{}) continue2 := make(chan struct{}) @@ -627,7 +627,7 @@ func TestDelRateLimiterConcurrency(t *testing.T) { select { case <-continue1: - variable.TTLDeleteRateLimit.Store(0) + vardef.TTLDeleteRateLimit.Store(0) require.NoError(t, globalDelRateLimiter.WaitDelToken(timeCtx)) close(continue2) case <-timeCtx.Done(): diff --git a/pkg/ttl/ttlworker/job_manager.go b/pkg/ttl/ttlworker/job_manager.go index 069a2e173e5cd..5f9a44ec44cd8 100644 --- a/pkg/ttl/ttlworker/job_manager.go +++ b/pkg/ttl/ttlworker/job_manager.go @@ -27,7 +27,7 @@ import ( infoschemacontext "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/terror" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" timerapi "github.com/pingcap/tidb/pkg/timer/api" ttltablestore "github.com/pingcap/tidb/pkg/timer/tablestore" "github.com/pingcap/tidb/pkg/ttl/cache" @@ -368,12 +368,12 @@ func (m *JobManager) triggerTTLJob(requestID string, cmd *client.TriggerNewTTLJo terror.Log(m.cmdCli.ResponseCommand(m.ctx, requestID, err)) } - if !variable.EnableTTLJob.Load() { + if !vardef.EnableTTLJob.Load() { responseErr(errors.New("tidb_ttl_job_enable is disabled")) return } - if !timeutil.WithinDayTimePeriod(variable.TTLJobScheduleWindowStartTime.Load(), variable.TTLJobScheduleWindowEndTime.Load(), se.Now()) { + if !timeutil.WithinDayTimePeriod(vardef.TTLJobScheduleWindowStartTime.Load(), vardef.TTLJobScheduleWindowEndTime.Load(), se.Now()) { responseErr(errors.New("not in TTL job window")) return } @@ -602,10 +602,10 @@ func (m *JobManager) rescheduleJobs(se session.Session, now time.Time) { cancelJobs := false cancelReason := "" switch { - case !variable.EnableTTLJob.Load(): + case !vardef.EnableTTLJob.Load(): cancelJobs = true cancelReason = "tidb_ttl_job_enable turned off" - case !timeutil.WithinDayTimePeriod(variable.TTLJobScheduleWindowStartTime.Load(), variable.TTLJobScheduleWindowEndTime.Load(), now): + case !timeutil.WithinDayTimePeriod(vardef.TTLJobScheduleWindowStartTime.Load(), vardef.TTLJobScheduleWindowEndTime.Load(), now): cancelJobs = true cancelReason = "out of TTL job schedule window" } diff --git a/pkg/ttl/ttlworker/job_manager_integration_test.go b/pkg/ttl/ttlworker/job_manager_integration_test.go index 0cb045bc1098a..fa40dc2cf90ba 100644 --- a/pkg/ttl/ttlworker/job_manager_integration_test.go +++ b/pkg/ttl/ttlworker/job_manager_integration_test.go @@ -34,7 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" @@ -1426,9 +1426,9 @@ func TestFinishError(t *testing.T) { // Test the `rescheduleJobs` can tolerate the `job.finish` error // cancel job branch initializeTest() - variable.EnableTTLJob.Store(false) + vardef.EnableTTLJob.Store(false) t.Cleanup(func() { - variable.EnableTTLJob.Store(true) + vardef.EnableTTLJob.Store(true) }) for i := 0; i < 4; i++ { m.RescheduleJobs(se, now) @@ -1436,7 +1436,7 @@ func TestFinishError(t *testing.T) { } m.RescheduleJobs(se, now) tk.MustQuery("select count(*) from mysql.tidb_ttl_task").Check(testkit.Rows("0")) - variable.EnableTTLJob.Store(true) + vardef.EnableTTLJob.Store(true) // remove table branch initializeTest() tk.MustExec("drop table t") @@ -1782,11 +1782,11 @@ func TestJobManagerWithFault(t *testing.T) { go func() { defer wg.Done() - maxScanWorkerCount := variable.DefTiDBTTLScanWorkerCount * 2 - minScanWorkerCount := variable.DefTiDBTTLScanWorkerCount / 2 + maxScanWorkerCount := vardef.DefTiDBTTLScanWorkerCount * 2 + minScanWorkerCount := vardef.DefTiDBTTLScanWorkerCount / 2 - maxDelWorkerCount := variable.DefTiDBTTLDeleteWorkerCount * 2 - minDelWorkerCount := variable.DefTiDBTTLDeleteWorkerCount / 2 + maxDelWorkerCount := vardef.DefTiDBTTLDeleteWorkerCount * 2 + minDelWorkerCount := vardef.DefTiDBTTLDeleteWorkerCount / 2 faultTicker := time.NewTicker(time.Second) tk := testkit.NewTestKit(t, store) @@ -1794,8 +1794,8 @@ func TestJobManagerWithFault(t *testing.T) { select { case <-stopTestCh: // Recover to the default count - tk.MustExec("set @@global.tidb_ttl_scan_worker_count = ?", variable.DefTiDBTTLScanWorkerCount) - tk.MustExec("set @@global.tidb_ttl_delete_worker_count = ?", variable.DefTiDBTTLDeleteWorkerCount) + tk.MustExec("set @@global.tidb_ttl_scan_worker_count = ?", vardef.DefTiDBTTLScanWorkerCount) + tk.MustExec("set @@global.tidb_ttl_delete_worker_count = ?", vardef.DefTiDBTTLDeleteWorkerCount) return case <-faultTicker.C: diff --git a/pkg/ttl/ttlworker/scan.go b/pkg/ttl/ttlworker/scan.go index c9e67e10b4cf0..0a618039b6f9b 100644 --- a/pkg/ttl/ttlworker/scan.go +++ b/pkg/ttl/ttlworker/scan.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/ttl/cache" "github.com/pingcap/tidb/pkg/ttl/metrics" "github.com/pingcap/tidb/pkg/ttl/sqlbuilder" @@ -227,7 +227,7 @@ func (t *ttlScanTask) doScan(ctx context.Context, delCh chan<- *ttlDeleteTask, s sql := retrySQL if sql == "" { - limit := int(variable.TTLScanBatchSize.Load()) + limit := int(vardef.TTLScanBatchSize.Load()) if sql, err = generator.NextSQL(lastResult, limit); err != nil { return t.result(err) } diff --git a/pkg/ttl/ttlworker/scan_test.go b/pkg/ttl/ttlworker/scan_test.go index d682859ac1a0f..1fc65b0d12f70 100644 --- a/pkg/ttl/ttlworker/scan_test.go +++ b/pkg/ttl/ttlworker/scan_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/ttl/cache" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -141,9 +141,9 @@ func (w *mockScanWorker) SetExecuteSQL(fn func(ctx context.Context, sql string, } func TestScanWorkerSchedule(t *testing.T) { - origLimit := variable.TTLScanBatchSize.Load() - variable.TTLScanBatchSize.Store(5) - defer variable.TTLScanBatchSize.Store(origLimit) + origLimit := vardef.TTLScanBatchSize.Load() + vardef.TTLScanBatchSize.Store(5) + defer vardef.TTLScanBatchSize.Store(origLimit) tbl := newMockTTLTbl(t, "t1") w := NewMockScanWorker(t) @@ -191,9 +191,9 @@ func TestScanWorkerSchedule(t *testing.T) { } func TestScanWorkerScheduleWithFailedTask(t *testing.T) { - origLimit := variable.TTLScanBatchSize.Load() - variable.TTLScanBatchSize.Store(5) - defer variable.TTLScanBatchSize.Store(origLimit) + origLimit := vardef.TTLScanBatchSize.Load() + vardef.TTLScanBatchSize.Store(5) + defer vardef.TTLScanBatchSize.Store(origLimit) tbl := newMockTTLTbl(t, "t1") w := NewMockScanWorker(t) @@ -309,12 +309,12 @@ func (t *mockScanTask) selectSQL(i int) string { func (t *mockScanTask) runDoScanForTest(delTaskCnt int, errString string) *ttlScanTaskExecResult { t.ttlScanTask.statistics.Reset() - origLimit := variable.TTLScanBatchSize.Load() - variable.TTLScanBatchSize.Store(3) + origLimit := vardef.TTLScanBatchSize.Load() + vardef.TTLScanBatchSize.Store(3) origRetryInterval := scanTaskExecuteSQLRetryInterval scanTaskExecuteSQLRetryInterval = time.Millisecond defer func() { - variable.TTLScanBatchSize.Store(origLimit) + vardef.TTLScanBatchSize.Store(origLimit) scanTaskExecuteSQLRetryInterval = origRetryInterval }() diff --git a/pkg/ttl/ttlworker/session.go b/pkg/ttl/ttlworker/session.go index db0be69ae958a..ea956233f28e3 100644 --- a/pkg/ttl/ttlworker/session.go +++ b/pkg/ttl/ttlworker/session.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/ttl/cache" "github.com/pingcap/tidb/pkg/ttl/metrics" "github.com/pingcap/tidb/pkg/ttl/session" @@ -249,7 +249,7 @@ func (s *ttlTableSession) ExecuteSQLWithCheck(ctx context.Context, sql string) ( defer tracer.EnterPhase(tracer.Phase()) tracer.EnterPhase(metrics.PhaseOther) - if !variable.EnableTTLJob.Load() { + if !vardef.EnableTTLJob.Load() { return nil, false, errors.New("global TTL job is disabled") } diff --git a/pkg/ttl/ttlworker/task_manager.go b/pkg/ttl/ttlworker/task_manager.go index 2c7569cf988a1..28cc7e10a6530 100644 --- a/pkg/ttl/ttlworker/task_manager.go +++ b/pkg/ttl/ttlworker/task_manager.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" storeerr "github.com/pingcap/tidb/pkg/store/driver/error" "github.com/pingcap/tidb/pkg/ttl/cache" "github.com/pingcap/tidb/pkg/ttl/metrics" @@ -143,11 +143,11 @@ func newTaskManager(ctx context.Context, sessPool util.SessionPool, infoSchemaCa } func (m *taskManager) resizeWorkersWithSysVar() { - err := m.resizeScanWorkers(int(variable.TTLScanWorkerCount.Load())) + err := m.resizeScanWorkers(int(vardef.TTLScanWorkerCount.Load())) if err != nil { logutil.Logger(m.ctx).Warn("fail to resize scan workers", zap.Error(err)) } - err = m.resizeDelWorkers(int(variable.TTLDeleteWorkerCount.Load())) + err = m.resizeDelWorkers(int(vardef.TTLDeleteWorkerCount.Load())) if err != nil { logutil.Logger(m.ctx).Warn("fail to resize delete workers", zap.Error(err)) } @@ -727,24 +727,24 @@ func (m *taskManager) meetTTLRunningTask(count int, taskStatus cache.TaskStatus) } func getMaxRunningTasksLimit(store kv.Storage) int { - ttlRunningTask := variable.TTLRunningTasks.Load() + ttlRunningTask := vardef.TTLRunningTasks.Load() if ttlRunningTask != -1 { return int(ttlRunningTask) } tikvStore, ok := store.(tikv.Storage) if !ok { - return variable.MaxConfigurableConcurrency + return vardef.MaxConfigurableConcurrency } regionCache := tikvStore.GetRegionCache() if regionCache == nil { - return variable.MaxConfigurableConcurrency + return vardef.MaxConfigurableConcurrency } limit := len(regionCache.GetStoresByType(tikvrpc.TiKV)) - if limit > variable.MaxConfigurableConcurrency { - limit = variable.MaxConfigurableConcurrency + if limit > vardef.MaxConfigurableConcurrency { + limit = vardef.MaxConfigurableConcurrency } return limit diff --git a/pkg/ttl/ttlworker/task_manager_test.go b/pkg/ttl/ttlworker/task_manager_test.go index 45dad9491c6fd..8a8157903df36 100644 --- a/pkg/ttl/ttlworker/task_manager_test.go +++ b/pkg/ttl/ttlworker/task_manager_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/ttl/cache" "github.com/pingcap/tidb/pkg/ttl/session" "github.com/pingcap/tidb/pkg/util/logutil" @@ -292,16 +292,16 @@ func TestGetMaxRunningTasksLimit(t *testing.T) { require.NoError(t, err) }() - variable.TTLRunningTasks.Store(1) + vardef.TTLRunningTasks.Store(1) require.Equal(t, 1, getMaxRunningTasksLimit(&mockTiKVStore{})) - variable.TTLRunningTasks.Store(2) + vardef.TTLRunningTasks.Store(2) require.Equal(t, 2, getMaxRunningTasksLimit(&mockTiKVStore{})) - variable.TTLRunningTasks.Store(-1) - require.Equal(t, variable.MaxConfigurableConcurrency, getMaxRunningTasksLimit(nil)) - require.Equal(t, variable.MaxConfigurableConcurrency, getMaxRunningTasksLimit(&mockKVStore{})) - require.Equal(t, variable.MaxConfigurableConcurrency, getMaxRunningTasksLimit(&mockTiKVStore{})) + vardef.TTLRunningTasks.Store(-1) + require.Equal(t, vardef.MaxConfigurableConcurrency, getMaxRunningTasksLimit(nil)) + require.Equal(t, vardef.MaxConfigurableConcurrency, getMaxRunningTasksLimit(&mockKVStore{})) + require.Equal(t, vardef.MaxConfigurableConcurrency, getMaxRunningTasksLimit(&mockTiKVStore{})) s := &mockTiKVStore{regionCache: tikv.NewRegionCache(pdClient)} s.GetRegionCache().SetRegionCacheStore(1, "", "", tikvrpc.TiKV, 1, nil) diff --git a/pkg/ttl/ttlworker/timer.go b/pkg/ttl/ttlworker/timer.go index db27537383596..b7cc0477118d7 100644 --- a/pkg/ttl/ttlworker/timer.go +++ b/pkg/ttl/ttlworker/timer.go @@ -21,7 +21,7 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" timerapi "github.com/pingcap/tidb/pkg/timer/api" timerrt "github.com/pingcap/tidb/pkg/timer/runtime" "github.com/pingcap/tidb/pkg/util/logutil" @@ -86,7 +86,7 @@ func (t *ttlTimerHook) Stop() { } func (t *ttlTimerHook) OnPreSchedEvent(_ context.Context, event timerapi.TimerShedEvent) (r timerapi.PreSchedEventResult, err error) { - if !variable.EnableTTLJob.Load() { + if !vardef.EnableTTLJob.Load() { r.Delay = time.Minute return } @@ -96,7 +96,7 @@ func (t *ttlTimerHook) OnPreSchedEvent(_ context.Context, event timerapi.TimerSh return r, err } - windowStart, windowEnd := variable.TTLJobScheduleWindowStartTime.Load(), variable.TTLJobScheduleWindowEndTime.Load() + windowStart, windowEnd := vardef.TTLJobScheduleWindowStartTime.Load(), vardef.TTLJobScheduleWindowEndTime.Load() if !timeutil.WithinDayTimePeriod(windowStart, windowEnd, now) { r.Delay = time.Minute return diff --git a/pkg/ttl/ttlworker/timer_test.go b/pkg/ttl/ttlworker/timer_test.go index 39e56c12ce431..78e4b9ecf7a95 100644 --- a/pkg/ttl/ttlworker/timer_test.go +++ b/pkg/ttl/ttlworker/timer_test.go @@ -24,7 +24,7 @@ import ( "github.com/google/uuid" "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" timerapi "github.com/pingcap/tidb/pkg/timer/api" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/timeutil" @@ -145,9 +145,9 @@ func triggerTestTimer(t *testing.T, store *timerapi.TimerStore, timerID string) } func clearTTLWindowAndEnable() { - variable.EnableTTLJob.Store(true) - variable.TTLJobScheduleWindowStartTime.Store(time.Date(0, 0, 0, 0, 0, 0, 0, time.UTC)) - variable.TTLJobScheduleWindowEndTime.Store(time.Date(0, 0, 0, 23, 59, 0, 0, time.UTC)) + vardef.EnableTTLJob.Store(true) + vardef.TTLJobScheduleWindowStartTime.Store(time.Date(0, 0, 0, 0, 0, 0, 0, time.UTC)) + vardef.TTLJobScheduleWindowEndTime.Store(time.Date(0, 0, 0, 23, 59, 0, 0, time.UTC)) } func makeTTLSummary(t *testing.T, requestID string) (*ttlTimerSummary, []byte) { @@ -200,7 +200,7 @@ func TestTTLTimerHookPrepare(t *testing.T) { adapter.AssertExpectations(t) // global ttl job disabled - variable.EnableTTLJob.Store(false) + vardef.EnableTTLJob.Store(false) r, err = hook.OnPreSchedEvent(context.TODO(), &mockTimerSchedEvent{eventID: "event1", timer: timer}) require.NoError(t, err) require.Equal(t, timerapi.PreSchedEventResult{Delay: time.Minute}, r) @@ -210,7 +210,7 @@ func TestTTLTimerHookPrepare(t *testing.T) { now := time.Date(2023, 1, 1, 15, 10, 0, 0, time.UTC) adapter.On("Now").Return(now, nil).Once() clearTTLWindowAndEnable() - variable.TTLJobScheduleWindowStartTime.Store(time.Date(0, 0, 0, 15, 11, 0, 0, time.UTC)) + vardef.TTLJobScheduleWindowStartTime.Store(time.Date(0, 0, 0, 15, 11, 0, 0, time.UTC)) r, err = hook.OnPreSchedEvent(context.TODO(), &mockTimerSchedEvent{eventID: "event1", timer: timer}) require.NoError(t, err) require.Equal(t, timerapi.PreSchedEventResult{Delay: time.Minute}, r) @@ -218,7 +218,7 @@ func TestTTLTimerHookPrepare(t *testing.T) { clearTTLWindowAndEnable() adapter.On("Now").Return(now, nil).Once() - variable.TTLJobScheduleWindowEndTime.Store(time.Date(0, 0, 0, 15, 9, 0, 0, time.UTC)) + vardef.TTLJobScheduleWindowEndTime.Store(time.Date(0, 0, 0, 15, 9, 0, 0, time.UTC)) r, err = hook.OnPreSchedEvent(context.TODO(), &mockTimerSchedEvent{eventID: "event1", timer: timer}) require.NoError(t, err) require.Equal(t, timerapi.PreSchedEventResult{Delay: time.Minute}, r) @@ -228,8 +228,8 @@ func TestTTLTimerHookPrepare(t *testing.T) { clearTTLWindowAndEnable() adapter.On("Now").Return(now, nil).Once() adapter.On("CanSubmitJob", data.TableID, data.PhysicalID).Return(true).Once() - variable.TTLJobScheduleWindowStartTime.Store(time.Date(0, 0, 0, 15, 9, 0, 0, time.UTC)) - variable.TTLJobScheduleWindowEndTime.Store(time.Date(0, 0, 0, 15, 11, 0, 0, time.UTC)) + vardef.TTLJobScheduleWindowStartTime.Store(time.Date(0, 0, 0, 15, 9, 0, 0, time.UTC)) + vardef.TTLJobScheduleWindowEndTime.Store(time.Date(0, 0, 0, 15, 11, 0, 0, time.UTC)) r, err = hook.OnPreSchedEvent(context.TODO(), &mockTimerSchedEvent{eventID: "event1", timer: timer}) require.NoError(t, err) require.Equal(t, timerapi.PreSchedEventResult{}, r) diff --git a/pkg/util/expensivequery/BUILD.bazel b/pkg/util/expensivequery/BUILD.bazel index d35d29a6c2c7d..04a014e496c0f 100644 --- a/pkg/util/expensivequery/BUILD.bazel +++ b/pkg/util/expensivequery/BUILD.bazel @@ -7,7 +7,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/metrics", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/statistics/handle/util", "//pkg/util", "//pkg/util/logutil", diff --git a/pkg/util/expensivequery/expensivequery.go b/pkg/util/expensivequery/expensivequery.go index 4623c2c69e903..4af1efae76532 100644 --- a/pkg/util/expensivequery/expensivequery.go +++ b/pkg/util/expensivequery/expensivequery.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/metrics" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/logutil" @@ -48,8 +48,8 @@ func (eqh *Handle) SetSessionManager(sm util.SessionManager) *Handle { // Run starts a expensive query checker goroutine at the start time of the server. func (eqh *Handle) Run() { - threshold := atomic.LoadUint64(&variable.ExpensiveQueryTimeThreshold) - txnThreshold := atomic.LoadUint64(&variable.ExpensiveTxnTimeThreshold) + threshold := atomic.LoadUint64(&vardef.ExpensiveQueryTimeThreshold) + txnThreshold := atomic.LoadUint64(&vardef.ExpensiveTxnTimeThreshold) ongoingTxnDurationHistogramInternal := metrics.OngoingTxnDurationHistogram.WithLabelValues(metrics.LblInternal) ongoingTxnDurationHistogramGeneral := metrics.OngoingTxnDurationHistogram.WithLabelValues(metrics.LblGeneral) lastMetricTime := time.Time{} @@ -99,7 +99,7 @@ func (eqh *Handle) Run() { sm.Kill(info.ID, true, true, false) } if statsutil.GlobalAutoAnalyzeProcessList.Contains(info.ID) { - maxAutoAnalyzeTime := variable.MaxAutoAnalyzeTime.Load() + maxAutoAnalyzeTime := vardef.MaxAutoAnalyzeTime.Load() if maxAutoAnalyzeTime > 0 && costTime > time.Duration(maxAutoAnalyzeTime)*time.Second { logutil.BgLogger().Warn("auto analyze timeout, kill it", zap.Duration("costTime", costTime), zap.Duration("maxAutoAnalyzeTime", time.Duration(maxAutoAnalyzeTime)*time.Second), zap.String("processInfo", info.String())) @@ -114,8 +114,8 @@ func (eqh *Handle) Run() { } } } - threshold = atomic.LoadUint64(&variable.ExpensiveQueryTimeThreshold) - txnThreshold = atomic.LoadUint64(&variable.ExpensiveTxnTimeThreshold) + threshold = atomic.LoadUint64(&vardef.ExpensiveQueryTimeThreshold) + txnThreshold = atomic.LoadUint64(&vardef.ExpensiveTxnTimeThreshold) case <-eqh.exitCh: return } diff --git a/pkg/util/gcutil/BUILD.bazel b/pkg/util/gcutil/BUILD.bazel index 8a95024685b05..74c823ffa5d4a 100644 --- a/pkg/util/gcutil/BUILD.bazel +++ b/pkg/util/gcutil/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "//pkg/kv", "//pkg/meta/model", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "@com_github_pingcap_errors//:errors", "@com_github_tikv_client_go_v2//oracle", diff --git a/pkg/util/gcutil/gcutil.go b/pkg/util/gcutil/gcutil.go index 05c9e64e0d445..42bd02f2d5450 100644 --- a/pkg/util/gcutil/gcutil.go +++ b/pkg/util/gcutil/gcutil.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/util" @@ -32,7 +33,7 @@ const ( // CheckGCEnable is use to check whether GC is enable. func CheckGCEnable(ctx sessionctx.Context) (enable bool, err error) { - val, err := ctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBGCEnable) + val, err := ctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(vardef.TiDBGCEnable) if err != nil { return false, errors.Trace(err) } @@ -41,12 +42,12 @@ func CheckGCEnable(ctx sessionctx.Context) (enable bool, err error) { // DisableGC will disable GC enable variable. func DisableGC(ctx sessionctx.Context) error { - return ctx.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.TiDBGCEnable, variable.Off) + return ctx.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.TiDBGCEnable, vardef.Off) } // EnableGC will enable GC enable variable. func EnableGC(ctx sessionctx.Context) error { - return ctx.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.TiDBGCEnable, variable.On) + return ctx.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.TiDBGCEnable, vardef.On) } // ValidateSnapshot checks that the newly set snapshot time is after GC safe point time. diff --git a/pkg/util/logutil/BUILD.bazel b/pkg/util/logutil/BUILD.bazel index f073b405d8ce1..85c6357068424 100644 --- a/pkg/util/logutil/BUILD.bazel +++ b/pkg/util/logutil/BUILD.bazel @@ -11,7 +11,7 @@ go_library( importpath = "github.com/pingcap/tidb/pkg/util/logutil", visibility = ["//visibility:public"], deps = [ - "//pkg/meta/model", + "//pkg/util/tracing", "@com_github_golang_protobuf//proto", "@com_github_grpc_ecosystem_go_grpc_middleware//logging/zap", "@com_github_opentracing_opentracing_go//:opentracing-go", @@ -38,8 +38,8 @@ go_test( flaky = True, deps = [ "//pkg/kv", - "//pkg/meta/model", "//pkg/testkit/testsetup", + "//pkg/util/tracing", "@com_github_google_uuid//:uuid", "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_log//:log", diff --git a/pkg/util/logutil/log.go b/pkg/util/logutil/log.go index ba390fa787feb..99a2d6a3caafb 100644 --- a/pkg/util/logutil/log.go +++ b/pkg/util/logutil/log.go @@ -28,7 +28,7 @@ import ( tlog "github.com/opentracing/opentracing-go/log" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/util/tracing" "github.com/tikv/client-go/v2/tikv" "go.uber.org/zap" "go.uber.org/zap/zapcore" @@ -259,7 +259,7 @@ func ErrVerboseLogger() *zap.Logger { } // LoggerWithTraceInfo attaches fields from trace info to logger -func LoggerWithTraceInfo(logger *zap.Logger, info *model.TraceInfo) *zap.Logger { +func LoggerWithTraceInfo(logger *zap.Logger, info *tracing.TraceInfo) *zap.Logger { if logger == nil { logger = log.L() } @@ -287,7 +287,7 @@ func WithCategory(ctx context.Context, category string) context.Context { } // WithTraceFields attaches trace fields to context -func WithTraceFields(ctx context.Context, info *model.TraceInfo) context.Context { +func WithTraceFields(ctx context.Context, info *tracing.TraceInfo) context.Context { if info == nil { return WithFields(ctx) } @@ -297,7 +297,7 @@ func WithTraceFields(ctx context.Context, info *model.TraceInfo) context.Context ) } -func fieldsFromTraceInfo(info *model.TraceInfo) []zap.Field { +func fieldsFromTraceInfo(info *tracing.TraceInfo) []zap.Field { if info == nil { return nil } @@ -315,7 +315,7 @@ func fieldsFromTraceInfo(info *model.TraceInfo) []zap.Field { } // WithTraceLogger attaches trace identifier to context -func WithTraceLogger(ctx context.Context, info *model.TraceInfo) context.Context { +func WithTraceLogger(ctx context.Context, info *tracing.TraceInfo) context.Context { var logger *zap.Logger if ctxLogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { logger = ctxLogger @@ -325,7 +325,7 @@ func WithTraceLogger(ctx context.Context, info *model.TraceInfo) context.Context return context.WithValue(ctx, CtxLogKey, wrapTraceLogger(ctx, info, logger)) } -func wrapTraceLogger(ctx context.Context, info *model.TraceInfo, logger *zap.Logger) *zap.Logger { +func wrapTraceLogger(ctx context.Context, info *tracing.TraceInfo, logger *zap.Logger) *zap.Logger { return logger.WithOptions(zap.WrapCore(func(core zapcore.Core) zapcore.Core { tl := &traceLog{ctx: ctx} // cfg.Format == "", never return error diff --git a/pkg/util/logutil/log_test.go b/pkg/util/logutil/log_test.go index 6c8051406387f..86bb22498bf1f 100644 --- a/pkg/util/logutil/log_test.go +++ b/pkg/util/logutil/log_test.go @@ -28,7 +28,7 @@ import ( "github.com/google/uuid" "github.com/pingcap/log" - "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/util/tracing" "github.com/stretchr/testify/require" "go.uber.org/zap" "go.uber.org/zap/zapcore" @@ -38,16 +38,16 @@ func TestFieldsFromTraceInfo(t *testing.T) { fields := fieldsFromTraceInfo(nil) require.Equal(t, 0, len(fields)) - fields = fieldsFromTraceInfo(&model.TraceInfo{}) + fields = fieldsFromTraceInfo(&tracing.TraceInfo{}) require.Equal(t, 0, len(fields)) - fields = fieldsFromTraceInfo(&model.TraceInfo{ConnectionID: 1}) + fields = fieldsFromTraceInfo(&tracing.TraceInfo{ConnectionID: 1}) require.Equal(t, []zap.Field{zap.Uint64("conn", 1)}, fields) - fields = fieldsFromTraceInfo(&model.TraceInfo{SessionAlias: "alias123"}) + fields = fieldsFromTraceInfo(&tracing.TraceInfo{SessionAlias: "alias123"}) require.Equal(t, []zap.Field{zap.String("session_alias", "alias123")}, fields) - fields = fieldsFromTraceInfo(&model.TraceInfo{ConnectionID: 1, SessionAlias: "alias123"}) + fields = fieldsFromTraceInfo(&tracing.TraceInfo{ConnectionID: 1, SessionAlias: "alias123"}) require.Equal(t, []zap.Field{zap.Uint64("conn", 1), zap.String("session_alias", "alias123")}, fields) } @@ -87,14 +87,14 @@ func TestZapLoggerWithKeys(t *testing.T) { err = InitLogger(conf) require.NoError(t, err) - ctx1 = WithTraceFields(context.Background(), &model.TraceInfo{ConnectionID: 456, SessionAlias: "alias789"}) + ctx1 = WithTraceFields(context.Background(), &tracing.TraceInfo{ConnectionID: 456, SessionAlias: "alias789"}) testZapLogger(ctx1, t, fileCfg.Filename, zapLogWithTraceInfoPattern) err = os.Remove(fileCfg.Filename) require.NoError(t, err) err = InitLogger(conf) require.NoError(t, err) - newLogger := LoggerWithTraceInfo(log.L(), &model.TraceInfo{ConnectionID: 789, SessionAlias: "alias012"}) + newLogger := LoggerWithTraceInfo(log.L(), &tracing.TraceInfo{ConnectionID: 789, SessionAlias: "alias012"}) ctx1 = context.WithValue(context.Background(), CtxLogKey, newLogger) testZapLogger(ctx1, t, fileCfg.Filename, zapLogWithTraceInfoPattern) err = os.Remove(fileCfg.Filename) diff --git a/pkg/util/memoryusagealarm/BUILD.bazel b/pkg/util/memoryusagealarm/BUILD.bazel index 7c78e8fcfb5dc..9fb8d4a74323f 100644 --- a/pkg/util/memoryusagealarm/BUILD.bazel +++ b/pkg/util/memoryusagealarm/BUILD.bazel @@ -7,7 +7,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/config", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/util", "//pkg/util/disk", "//pkg/util/logutil", @@ -26,7 +26,7 @@ go_test( race = "on", deps = [ "//pkg/sessionctx/stmtctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/util", "//pkg/util/memory", "@com_github_stretchr_testify//assert", diff --git a/pkg/util/memoryusagealarm/memoryusagealarm.go b/pkg/util/memoryusagealarm/memoryusagealarm.go index 71bc8d8de4734..d0e6f048d1696 100644 --- a/pkg/util/memoryusagealarm/memoryusagealarm.go +++ b/pkg/util/memoryusagealarm/memoryusagealarm.go @@ -26,7 +26,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/config" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/disk" "github.com/pingcap/tidb/pkg/util/logutil" @@ -89,8 +89,8 @@ func (record *memoryUsageAlarm) updateVariable() { if time.Since(record.lastUpdateVariableTime) < 60*time.Second { return } - record.memoryUsageAlarmRatio = variable.MemoryUsageAlarmRatio.Load() - record.memoryUsageAlarmKeepRecordNum = variable.MemoryUsageAlarmKeepRecordNum.Load() + record.memoryUsageAlarmRatio = vardef.MemoryUsageAlarmRatio.Load() + record.memoryUsageAlarmKeepRecordNum = vardef.MemoryUsageAlarmKeepRecordNum.Load() record.serverMemoryLimit = memory.ServerMemoryLimit.Load() if record.serverMemoryLimit != 0 { record.isServerMemoryLimitSet = true @@ -268,7 +268,7 @@ func (record *memoryUsageAlarm) getTop10SqlInfo(cmp func(i, j *util.ProcessInfo) slices.SortFunc(pinfo, cmp) list := pinfo var buf strings.Builder - oomAction := variable.OOMAction.Load() + oomAction := vardef.OOMAction.Load() serverMemoryLimit := memory.ServerMemoryLimit.Load() for i, totalCnt := 0, 10; i < len(list) && totalCnt > 0; i++ { info := list[i] diff --git a/pkg/util/memoryusagealarm/memoryusagealarm_test.go b/pkg/util/memoryusagealarm/memoryusagealarm_test.go index d793d404ca278..d325790d9f8e3 100644 --- a/pkg/util/memoryusagealarm/memoryusagealarm_test.go +++ b/pkg/util/memoryusagealarm/memoryusagealarm_test.go @@ -19,7 +19,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/memory" "github.com/stretchr/testify/assert" @@ -119,8 +119,8 @@ func genMockProcessInfoList(memConsumeList []int64, startTimeList []time.Time, s } func TestUpdateVariables(t *testing.T) { - variable.MemoryUsageAlarmRatio.Store(0.3) - variable.MemoryUsageAlarmKeepRecordNum.Store(3) + vardef.MemoryUsageAlarmRatio.Store(0.3) + vardef.MemoryUsageAlarmKeepRecordNum.Store(3) memory.ServerMemoryLimit.Store(1024) record := memoryUsageAlarm{} @@ -129,8 +129,8 @@ func TestUpdateVariables(t *testing.T) { assert.Equal(t, 0.3, record.memoryUsageAlarmRatio) assert.Equal(t, int64(3), record.memoryUsageAlarmKeepRecordNum) assert.Equal(t, uint64(1024), record.serverMemoryLimit) - variable.MemoryUsageAlarmRatio.Store(0.6) - variable.MemoryUsageAlarmKeepRecordNum.Store(6) + vardef.MemoryUsageAlarmRatio.Store(0.6) + vardef.MemoryUsageAlarmKeepRecordNum.Store(6) memory.ServerMemoryLimit.Store(2048) record.updateVariable() diff --git a/pkg/util/mock/BUILD.bazel b/pkg/util/mock/BUILD.bazel index 1816beac7c6f2..197adecfaa3a5 100644 --- a/pkg/util/mock/BUILD.bazel +++ b/pkg/util/mock/BUILD.bazel @@ -27,6 +27,7 @@ go_library( "//pkg/session/cursor", "//pkg/sessionctx", "//pkg/sessionctx/sessionstates", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/statistics/handle/usage/indexusage", "//pkg/table/tblctx", diff --git a/pkg/util/mock/context.go b/pkg/util/mock/context.go index 3248c7e18bd27..65186336dc644 100644 --- a/pkg/util/mock/context.go +++ b/pkg/util/mock/context.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/pkg/session/cursor" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics/handle/usage/indexusage" "github.com/pingcap/tidb/pkg/table/tblctx" @@ -678,15 +679,15 @@ func newContext() *Context { vars.StmtCtx.MemTracker.AttachTo(vars.MemTracker) vars.StmtCtx.DiskTracker.AttachTo(vars.DiskTracker) vars.GlobalVarsAccessor = variable.NewMockGlobalAccessor() - vars.EnablePaging = variable.DefTiDBEnablePaging - vars.MinPagingSize = variable.DefMinPagingSize - vars.CostModelVersion = variable.DefTiDBCostModelVer + vars.EnablePaging = vardef.DefTiDBEnablePaging + vars.MinPagingSize = vardef.DefMinPagingSize + vars.CostModelVersion = vardef.DefTiDBCostModelVer vars.EnableChunkRPC = true - vars.DivPrecisionIncrement = variable.DefDivPrecisionIncrement - if err := sctx.GetSessionVars().SetSystemVar(variable.MaxAllowedPacket, "67108864"); err != nil { + vars.DivPrecisionIncrement = vardef.DefDivPrecisionIncrement + if err := sctx.GetSessionVars().SetSystemVar(vardef.MaxAllowedPacket, "67108864"); err != nil { panic(err) } - if err := sctx.GetSessionVars().SetSystemVar(variable.CharacterSetConnection, "utf8mb4"); err != nil { + if err := sctx.GetSessionVars().SetSystemVar(vardef.CharacterSetConnection, "utf8mb4"); err != nil { panic(err) } return sctx diff --git a/pkg/util/password-validation/BUILD.bazel b/pkg/util/password-validation/BUILD.bazel index c5a096159f55e..bf9ad67810b43 100644 --- a/pkg/util/password-validation/BUILD.bazel +++ b/pkg/util/password-validation/BUILD.bazel @@ -7,6 +7,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/parser/auth", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/util/hack", ], @@ -20,6 +21,7 @@ go_test( flaky = True, deps = [ "//pkg/parser/auth", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "@com_github_stretchr_testify//require", ], diff --git a/pkg/util/password-validation/password_validation.go b/pkg/util/password-validation/password_validation.go index b463c9ef38b71..b3931de5e6ef6 100644 --- a/pkg/util/password-validation/password_validation.go +++ b/pkg/util/password-validation/password_validation.go @@ -22,6 +22,7 @@ import ( "unicode" "github.com/pingcap/tidb/pkg/parser/auth" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util/hack" ) @@ -32,7 +33,7 @@ const minPwdValidationLength int = 4 // ValidateDictionaryPassword checks if the password contains words in the dictionary. func ValidateDictionaryPassword(pwd string, globalVars *variable.GlobalVarAccessor) (bool, error) { - dictionary, err := (*globalVars).GetGlobalSysVar(variable.ValidatePasswordDictionary) + dictionary, err := (*globalVars).GetGlobalSysVar(vardef.ValidatePasswordDictionary) if err != nil { return false, err } @@ -54,7 +55,7 @@ func ValidateDictionaryPassword(pwd string, globalVars *variable.GlobalVarAccess // ValidateUserNameInPassword checks whether pwd exists in the dictionary. func ValidateUserNameInPassword(pwd string, currentUser *auth.UserIdentity, globalVars *variable.GlobalVarAccessor) (string, error) { pwdBytes := hack.Slice(pwd) - if checkUserName, err := (*globalVars).GetGlobalSysVar(variable.ValidatePasswordCheckUserName); err != nil { + if checkUserName, err := (*globalVars).GetGlobalSysVar(vardef.ValidatePasswordCheckUserName); err != nil { return "", err } else if currentUser != nil && variable.TiDBOptOn(checkUserName) { for _, username := range []string{currentUser.AuthUsername, currentUser.Username} { @@ -80,7 +81,7 @@ func ValidateUserNameInPassword(pwd string, currentUser *auth.UserIdentity, glob // ValidatePasswordLowPolicy checks whether pwd satisfies the low policy of password validation. func ValidatePasswordLowPolicy(pwd string, globalVars *variable.GlobalVarAccessor) (string, error) { - if validateLengthStr, err := (*globalVars).GetGlobalSysVar(variable.ValidatePasswordLength); err != nil { + if validateLengthStr, err := (*globalVars).GetGlobalSysVar(vardef.ValidatePasswordLength); err != nil { return "", err } else if validateLength, err := strconv.ParseInt(validateLengthStr, 10, 64); err != nil { return "", err @@ -105,7 +106,7 @@ func ValidatePasswordMediumPolicy(pwd string, globalVars *variable.GlobalVarAcce specialCharCount++ } } - if mixedCaseCountStr, err := (*globalVars).GetGlobalSysVar(variable.ValidatePasswordMixedCaseCount); err != nil { + if mixedCaseCountStr, err := (*globalVars).GetGlobalSysVar(vardef.ValidatePasswordMixedCaseCount); err != nil { return "", err } else if mixedCaseCount, err := strconv.ParseInt(mixedCaseCountStr, 10, 64); err != nil { return "", err @@ -114,14 +115,14 @@ func ValidatePasswordMediumPolicy(pwd string, globalVars *variable.GlobalVarAcce } else if upperCaseCount < mixedCaseCount { return fmt.Sprintf("Require Password Uppercase Count: %d", mixedCaseCount), nil } - if requireNumberCountStr, err := (*globalVars).GetGlobalSysVar(variable.ValidatePasswordNumberCount); err != nil { + if requireNumberCountStr, err := (*globalVars).GetGlobalSysVar(vardef.ValidatePasswordNumberCount); err != nil { return "", err } else if requireNumberCount, err := strconv.ParseInt(requireNumberCountStr, 10, 64); err != nil { return "", err } else if numberCount < requireNumberCount { return fmt.Sprintf("Require Password Digit Count: %d", requireNumberCount), nil } - if requireSpecialCharCountStr, err := (*globalVars).GetGlobalSysVar(variable.ValidatePasswordSpecialCharCount); err != nil { + if requireSpecialCharCountStr, err := (*globalVars).GetGlobalSysVar(vardef.ValidatePasswordSpecialCharCount); err != nil { return "", err } else if requireSpecialCharCount, err := strconv.ParseInt(requireSpecialCharCountStr, 10, 64); err != nil { return "", err @@ -135,7 +136,7 @@ func ValidatePasswordMediumPolicy(pwd string, globalVars *variable.GlobalVarAcce func ValidatePassword(sessionVars *variable.SessionVars, pwd string) error { globalVars := sessionVars.GlobalVarsAccessor - validatePolicy, err := globalVars.GetGlobalSysVar(variable.ValidatePasswordPolicy) + validatePolicy, err := globalVars.GetGlobalSysVar(vardef.ValidatePasswordPolicy) if err != nil { return err } diff --git a/pkg/util/password-validation/password_validation_test.go b/pkg/util/password-validation/password_validation_test.go index e5cbe84bc1adc..725f0a5392840 100644 --- a/pkg/util/password-validation/password_validation_test.go +++ b/pkg/util/password-validation/password_validation_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/parser/auth" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/stretchr/testify/require" ) @@ -29,7 +30,7 @@ func TestValidateDictionaryPassword(t *testing.T) { mock.SessionVars = vars vars.GlobalVarsAccessor = mock - err := mock.SetGlobalSysVar(context.Background(), variable.ValidatePasswordDictionary, "abc;123;1234;5678;HIJK;中文测试;。,;!") + err := mock.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordDictionary, "abc;123;1234;5678;HIJK;中文测试;。,;!") require.NoError(t, err) testcases := []struct { pwd string @@ -70,7 +71,7 @@ func TestValidateUserNameInPassword(t *testing.T) { {"Resuhtua", ""}, } // Enable check_user_name - err := sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.ValidatePasswordCheckUserName, "ON") + err := sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordCheckUserName, "ON") require.NoError(t, err) for _, testcase := range testcases { warn, err := ValidateUserNameInPassword(testcase.pwd, sessionVars.User, &sessionVars.GlobalVarsAccessor) @@ -79,7 +80,7 @@ func TestValidateUserNameInPassword(t *testing.T) { } // Disable check_user_name - err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.ValidatePasswordCheckUserName, "OFF") + err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordCheckUserName, "OFF") require.NoError(t, err) for _, testcase := range testcases { warn, err := ValidateUserNameInPassword(testcase.pwd, sessionVars.User, &sessionVars.GlobalVarsAccessor) @@ -92,7 +93,7 @@ func TestValidatePasswordLowPolicy(t *testing.T) { sessionVars := variable.NewSessionVars(nil) sessionVars.GlobalVarsAccessor = variable.NewMockGlobalAccessor4Tests() sessionVars.GlobalVarsAccessor.(*variable.MockGlobalAccessor).SessionVars = sessionVars - err := sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.ValidatePasswordLength, "8") + err := sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordLength, "8") require.NoError(t, err) warn, err := ValidatePasswordLowPolicy("1234", &sessionVars.GlobalVarsAccessor) @@ -102,7 +103,7 @@ func TestValidatePasswordLowPolicy(t *testing.T) { require.NoError(t, err) require.Equal(t, "", warn) - err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.ValidatePasswordLength, "12") + err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordLength, "12") require.NoError(t, err) warn, err = ValidatePasswordLowPolicy("12345678", &sessionVars.GlobalVarsAccessor) require.NoError(t, err) @@ -114,11 +115,11 @@ func TestValidatePasswordMediumPolicy(t *testing.T) { sessionVars.GlobalVarsAccessor = variable.NewMockGlobalAccessor4Tests() sessionVars.GlobalVarsAccessor.(*variable.MockGlobalAccessor).SessionVars = sessionVars - err := sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.ValidatePasswordMixedCaseCount, "1") + err := sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordMixedCaseCount, "1") require.NoError(t, err) - err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.ValidatePasswordSpecialCharCount, "2") + err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordSpecialCharCount, "2") require.NoError(t, err) - err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.ValidatePasswordNumberCount, "3") + err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordNumberCount, "3") require.NoError(t, err) warn, err := ValidatePasswordMediumPolicy("!@A123", &sessionVars.GlobalVarsAccessor) @@ -144,7 +145,7 @@ func TestValidatePassword(t *testing.T) { sessionVars.GlobalVarsAccessor.(*variable.MockGlobalAccessor).SessionVars = sessionVars sessionVars.User = &auth.UserIdentity{Username: "user", AuthUsername: "authuser"} - err := sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.ValidatePasswordPolicy, "LOW") + err := sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordPolicy, "LOW") require.NoError(t, err) err = ValidatePassword(sessionVars, "1234") require.Error(t, err) @@ -155,7 +156,7 @@ func TestValidatePassword(t *testing.T) { err = ValidatePassword(sessionVars, "User1234") require.NoError(t, err) - err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.ValidatePasswordPolicy, "MEDIUM") + err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordPolicy, "MEDIUM") require.NoError(t, err) err = ValidatePassword(sessionVars, "User1234") require.Error(t, err) @@ -164,9 +165,9 @@ func TestValidatePassword(t *testing.T) { err = ValidatePassword(sessionVars, "!User1234") require.NoError(t, err) - err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.ValidatePasswordPolicy, "STRONG") + err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordPolicy, "STRONG") require.NoError(t, err) - err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.ValidatePasswordDictionary, "User") + err = sessionVars.GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.ValidatePasswordDictionary, "User") require.NoError(t, err) err = ValidatePassword(sessionVars, "!User1234") require.Error(t, err) diff --git a/pkg/util/sem/BUILD.bazel b/pkg/util/sem/BUILD.bazel index 7f9c79450f76a..e61ec4d1d03d1 100644 --- a/pkg/util/sem/BUILD.bazel +++ b/pkg/util/sem/BUILD.bazel @@ -7,6 +7,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/parser/mysql", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/util/logutil", ], @@ -23,7 +24,7 @@ go_test( flaky = True, deps = [ "//pkg/parser/mysql", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit/testsetup", "@com_github_stretchr_testify//assert", "@org_uber_go_goleak//:goleak", diff --git a/pkg/util/sem/sem.go b/pkg/util/sem/sem.go index 25ae2a9413450..2a5677b4d2725 100644 --- a/pkg/util/sem/sem.go +++ b/pkg/util/sem/sem.go @@ -20,6 +20,7 @@ import ( "sync/atomic" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util/logutil" ) @@ -72,8 +73,8 @@ var ( // Dynamic configuration by users may be a security risk. func Enable() { atomic.StoreInt32(&semEnabled, 1) - variable.SetSysVar(variable.TiDBEnableEnhancedSecurity, variable.On) - variable.SetSysVar(variable.Hostname, variable.DefHostname) + variable.SetSysVar(vardef.TiDBEnableEnhancedSecurity, vardef.On) + variable.SetSysVar(vardef.Hostname, vardef.DefHostname) // write to log so users understand why some operations are weird. logutil.BgLogger().Info("tidb-server is operating with security enhanced mode (SEM) enabled") } @@ -82,9 +83,9 @@ func Enable() { // Dynamic configuration by users may be a security risk. func Disable() { atomic.StoreInt32(&semEnabled, 0) - variable.SetSysVar(variable.TiDBEnableEnhancedSecurity, variable.Off) + variable.SetSysVar(vardef.TiDBEnableEnhancedSecurity, vardef.Off) if hostname, err := os.Hostname(); err == nil { - variable.SetSysVar(variable.Hostname, hostname) + variable.SetSysVar(vardef.Hostname, hostname) } } @@ -135,29 +136,29 @@ func IsInvisibleStatusVar(varName string) bool { // IsInvisibleSysVar returns true if the sysvar needs to be hidden func IsInvisibleSysVar(varNameInLower string) bool { switch varNameInLower { - case variable.TiDBDDLSlowOprThreshold, // ddl_slow_threshold - variable.TiDBCheckMb4ValueInUTF8, - variable.TiDBConfig, - variable.TiDBEnableSlowLog, - variable.TiDBEnableTelemetry, - variable.TiDBExpensiveQueryTimeThreshold, - variable.TiDBForcePriority, - variable.TiDBGeneralLog, - variable.TiDBMetricSchemaRangeDuration, - variable.TiDBMetricSchemaStep, - variable.TiDBOptWriteRowID, - variable.TiDBPProfSQLCPU, - variable.TiDBRecordPlanInSlowLog, - variable.TiDBRowFormatVersion, - variable.TiDBSlowQueryFile, - variable.TiDBSlowLogThreshold, - variable.TiDBSlowTxnLogThreshold, - variable.TiDBEnableCollectExecutionInfo, - variable.TiDBMemoryUsageAlarmRatio, - variable.TiDBRedactLog, - variable.TiDBRestrictedReadOnly, - variable.TiDBTopSQLMaxTimeSeriesCount, - variable.TiDBTopSQLMaxMetaCount, + case vardef.TiDBDDLSlowOprThreshold, // ddl_slow_threshold + vardef.TiDBCheckMb4ValueInUTF8, + vardef.TiDBConfig, + vardef.TiDBEnableSlowLog, + vardef.TiDBEnableTelemetry, + vardef.TiDBExpensiveQueryTimeThreshold, + vardef.TiDBForcePriority, + vardef.TiDBGeneralLog, + vardef.TiDBMetricSchemaRangeDuration, + vardef.TiDBMetricSchemaStep, + vardef.TiDBOptWriteRowID, + vardef.TiDBPProfSQLCPU, + vardef.TiDBRecordPlanInSlowLog, + vardef.TiDBRowFormatVersion, + vardef.TiDBSlowQueryFile, + vardef.TiDBSlowLogThreshold, + vardef.TiDBSlowTxnLogThreshold, + vardef.TiDBEnableCollectExecutionInfo, + vardef.TiDBMemoryUsageAlarmRatio, + vardef.TiDBRedactLog, + vardef.TiDBRestrictedReadOnly, + vardef.TiDBTopSQLMaxTimeSeriesCount, + vardef.TiDBTopSQLMaxMetaCount, tidbAuditRetractLog: return true } diff --git a/pkg/util/sem/sem_test.go b/pkg/util/sem/sem_test.go index 2645cb79b21de..399ce9d79030b 100644 --- a/pkg/util/sem/sem_test.go +++ b/pkg/util/sem/sem_test.go @@ -18,7 +18,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/stretchr/testify/assert" ) @@ -79,29 +79,29 @@ func TestIsInvisibleStatusVar(t *testing.T) { func TestIsInvisibleSysVar(t *testing.T) { assert := assert.New(t) - assert.False(IsInvisibleSysVar(variable.Hostname)) // changes the value to default, but is not invisible - assert.False(IsInvisibleSysVar(variable.TiDBEnableEnhancedSecurity)) // should be able to see the mode is on. - assert.False(IsInvisibleSysVar(variable.TiDBAllowRemoveAutoInc)) - - assert.True(IsInvisibleSysVar(variable.TiDBCheckMb4ValueInUTF8)) - assert.True(IsInvisibleSysVar(variable.TiDBConfig)) - assert.True(IsInvisibleSysVar(variable.TiDBEnableSlowLog)) - assert.True(IsInvisibleSysVar(variable.TiDBExpensiveQueryTimeThreshold)) - assert.True(IsInvisibleSysVar(variable.TiDBForcePriority)) - assert.True(IsInvisibleSysVar(variable.TiDBGeneralLog)) - assert.True(IsInvisibleSysVar(variable.TiDBMetricSchemaRangeDuration)) - assert.True(IsInvisibleSysVar(variable.TiDBMetricSchemaStep)) - assert.True(IsInvisibleSysVar(variable.TiDBOptWriteRowID)) - assert.True(IsInvisibleSysVar(variable.TiDBPProfSQLCPU)) - assert.True(IsInvisibleSysVar(variable.TiDBRecordPlanInSlowLog)) - assert.True(IsInvisibleSysVar(variable.TiDBSlowQueryFile)) - assert.True(IsInvisibleSysVar(variable.TiDBSlowLogThreshold)) - assert.True(IsInvisibleSysVar(variable.TiDBEnableCollectExecutionInfo)) - assert.True(IsInvisibleSysVar(variable.TiDBMemoryUsageAlarmRatio)) - assert.True(IsInvisibleSysVar(variable.TiDBEnableTelemetry)) - assert.True(IsInvisibleSysVar(variable.TiDBRowFormatVersion)) - assert.True(IsInvisibleSysVar(variable.TiDBRedactLog)) - assert.True(IsInvisibleSysVar(variable.TiDBTopSQLMaxTimeSeriesCount)) - assert.True(IsInvisibleSysVar(variable.TiDBTopSQLMaxTimeSeriesCount)) + assert.False(IsInvisibleSysVar(vardef.Hostname)) // changes the value to default, but is not invisible + assert.False(IsInvisibleSysVar(vardef.TiDBEnableEnhancedSecurity)) // should be able to see the mode is on. + assert.False(IsInvisibleSysVar(vardef.TiDBAllowRemoveAutoInc)) + + assert.True(IsInvisibleSysVar(vardef.TiDBCheckMb4ValueInUTF8)) + assert.True(IsInvisibleSysVar(vardef.TiDBConfig)) + assert.True(IsInvisibleSysVar(vardef.TiDBEnableSlowLog)) + assert.True(IsInvisibleSysVar(vardef.TiDBExpensiveQueryTimeThreshold)) + assert.True(IsInvisibleSysVar(vardef.TiDBForcePriority)) + assert.True(IsInvisibleSysVar(vardef.TiDBGeneralLog)) + assert.True(IsInvisibleSysVar(vardef.TiDBMetricSchemaRangeDuration)) + assert.True(IsInvisibleSysVar(vardef.TiDBMetricSchemaStep)) + assert.True(IsInvisibleSysVar(vardef.TiDBOptWriteRowID)) + assert.True(IsInvisibleSysVar(vardef.TiDBPProfSQLCPU)) + assert.True(IsInvisibleSysVar(vardef.TiDBRecordPlanInSlowLog)) + assert.True(IsInvisibleSysVar(vardef.TiDBSlowQueryFile)) + assert.True(IsInvisibleSysVar(vardef.TiDBSlowLogThreshold)) + assert.True(IsInvisibleSysVar(vardef.TiDBEnableCollectExecutionInfo)) + assert.True(IsInvisibleSysVar(vardef.TiDBMemoryUsageAlarmRatio)) + assert.True(IsInvisibleSysVar(vardef.TiDBEnableTelemetry)) + assert.True(IsInvisibleSysVar(vardef.TiDBRowFormatVersion)) + assert.True(IsInvisibleSysVar(vardef.TiDBRedactLog)) + assert.True(IsInvisibleSysVar(vardef.TiDBTopSQLMaxTimeSeriesCount)) + assert.True(IsInvisibleSysVar(vardef.TiDBTopSQLMaxTimeSeriesCount)) assert.True(IsInvisibleSysVar(tidbAuditRetractLog)) } diff --git a/pkg/util/tiflash/BUILD.bazel b/pkg/util/tiflash/BUILD.bazel index 065ed59873641..c273bc1473492 100644 --- a/pkg/util/tiflash/BUILD.bazel +++ b/pkg/util/tiflash/BUILD.bazel @@ -5,4 +5,5 @@ go_library( srcs = ["tiflash_replica_read.go"], importpath = "github.com/pingcap/tidb/pkg/util/tiflash", visibility = ["//visibility:public"], + deps = ["//pkg/sessionctx/vardef"], ) diff --git a/pkg/util/tiflash/tiflash_replica_read.go b/pkg/util/tiflash/tiflash_replica_read.go index f2b7a3f9c1111..c7c6e7b7cad67 100644 --- a/pkg/util/tiflash/tiflash_replica_read.go +++ b/pkg/util/tiflash/tiflash_replica_read.go @@ -14,6 +14,8 @@ package tiflash +import "github.com/pingcap/tidb/pkg/sessionctx/vardef" + // ReplicaRead is the policy to select TiFlash nodes. type ReplicaRead int @@ -26,15 +28,6 @@ const ( ClosestReplicas ) -const ( - // AllReplicaStr is the string value of AllReplicas. - AllReplicaStr = "all_replicas" - // ClosestAdaptiveStr is the string value of ClosestAdaptive. - ClosestAdaptiveStr = "closest_adaptive" - // ClosestReplicasStr is the string value of ClosestReplicas. - ClosestReplicasStr = "closest_replicas" -) - // IsAllReplicas return whether the policy is AllReplicas. func (policy ReplicaRead) IsAllReplicas() bool { return policy == AllReplicas @@ -49,24 +42,24 @@ func (policy ReplicaRead) IsClosestReplicas() bool { func GetTiFlashReplicaRead(policy ReplicaRead) string { switch policy { case AllReplicas: - return AllReplicaStr + return vardef.AllReplicaStr case ClosestAdaptive: - return ClosestAdaptiveStr + return vardef.ClosestAdaptiveStr case ClosestReplicas: - return ClosestReplicasStr + return vardef.ClosestReplicasStr default: - return AllReplicaStr + return vardef.AllReplicaStr } } // GetTiFlashReplicaReadByStr return corresponding policy in string. func GetTiFlashReplicaReadByStr(str string) ReplicaRead { switch str { - case AllReplicaStr: + case vardef.AllReplicaStr: return AllReplicas - case ClosestAdaptiveStr: + case vardef.ClosestAdaptiveStr: return ClosestAdaptive - case ClosestReplicasStr: + case vardef.ClosestReplicasStr: return ClosestReplicas default: return AllReplicas diff --git a/pkg/util/tiflashcompute/BUILD.bazel b/pkg/util/tiflashcompute/BUILD.bazel index 2d84cba472216..6ed9fc2fd9de5 100644 --- a/pkg/util/tiflashcompute/BUILD.bazel +++ b/pkg/util/tiflashcompute/BUILD.bazel @@ -9,7 +9,9 @@ go_library( importpath = "github.com/pingcap/tidb/pkg/util/tiflashcompute", visibility = ["//visibility:public"], deps = [ + "//pkg/config", "//pkg/errno", + "//pkg/sessionctx/vardef", "//pkg/util/dbterror", "//pkg/util/logutil", "@com_github_pingcap_errors//:errors", diff --git a/pkg/util/tiflashcompute/dispatch_policy.go b/pkg/util/tiflashcompute/dispatch_policy.go index e3fc329036c48..b49c5281f483d 100644 --- a/pkg/util/tiflashcompute/dispatch_policy.go +++ b/pkg/util/tiflashcompute/dispatch_policy.go @@ -16,6 +16,7 @@ package tiflashcompute import ( "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" ) // DispatchPolicy means different policy to dispatching task to tiflash_compute nods. @@ -30,26 +31,17 @@ const ( DispatchPolicyInvalid ) -const ( - // DispatchPolicyRRStr is string value for DispatchPolicyRR. - DispatchPolicyRRStr = "round_robin" - // DispatchPolicyConsistentHashStr is string value for DispatchPolicyConsistentHash. - DispatchPolicyConsistentHashStr = "consistent_hash" - // DispatchPolicyInvalidStr is string value for DispatchPolicyInvalid. - DispatchPolicyInvalidStr = "invalid" -) - // GetValidDispatchPolicy return all valid policy string. func GetValidDispatchPolicy() []string { - return []string{DispatchPolicyConsistentHashStr, DispatchPolicyRRStr} + return []string{vardef.DispatchPolicyConsistentHashStr, vardef.DispatchPolicyRRStr} } // GetDispatchPolicyByStr return corresponding policy. func GetDispatchPolicyByStr(str string) (DispatchPolicy, error) { switch str { - case DispatchPolicyConsistentHashStr: + case vardef.DispatchPolicyConsistentHashStr: return DispatchPolicyConsistentHash, nil - case DispatchPolicyRRStr: + case vardef.DispatchPolicyRRStr: return DispatchPolicyRR, nil default: return DispatchPolicyInvalid, @@ -61,10 +53,10 @@ func GetDispatchPolicyByStr(str string) (DispatchPolicy, error) { func GetDispatchPolicy(p DispatchPolicy) string { switch p { case DispatchPolicyConsistentHash: - return DispatchPolicyConsistentHashStr + return vardef.DispatchPolicyConsistentHashStr case DispatchPolicyRR: - return DispatchPolicyRRStr + return vardef.DispatchPolicyRRStr default: - return DispatchPolicyInvalidStr + return vardef.DispatchPolicyInvalidStr } } diff --git a/pkg/util/tiflashcompute/topo_fetcher.go b/pkg/util/tiflashcompute/topo_fetcher.go index 51b23c3304c41..e4b6203a453c8 100644 --- a/pkg/util/tiflashcompute/topo_fetcher.go +++ b/pkg/util/tiflashcompute/topo_fetcher.go @@ -24,6 +24,7 @@ import ( "sync" "github.com/pingcap/errors" + "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/logutil" @@ -35,37 +36,10 @@ var _ TopoFetcher = &MockTopoFetcher{} var _ TopoFetcher = &AWSTopoFetcher{} var _ TopoFetcher = &TestTopoFetcher{} -const ( - // MockASStr is string value for mock AutoScaler. - MockASStr = "mock" - // AWSASStr is string value for aws AutoScaler. - AWSASStr = "aws" - // GCPASStr is string value for gcp AutoScaler. - GCPASStr = "gcp" - // TestASStr is string value for test AutoScaler. - TestASStr = "test" - // InvalidASStr is string value for invalid AutoScaler. - InvalidASStr = "invalid" -) - -const ( - // MockASType is int value for mock AutoScaler. - MockASType int = iota - // AWSASType is int value for aws AutoScaler. - AWSASType - // GCPASType is int value for gcp AutoScaler. - GCPASType - // TestASType is for local tidb test AutoScaler. - TestASType - // InvalidASType is int value for invalid check. - InvalidASType -) - const ( // DefAWSAutoScalerAddr is the default address for AWS AutoScaler. - DefAWSAutoScalerAddr = "tiflash-autoscale-lb.tiflash-autoscale.svc.cluster.local:8081" + // DefASStr default AutoScaler. - DefASStr = AWSASStr awsFixedPoolHTTPPath = "sharedfixedpool" awsFetchHTTPPath = "resume-and-get-topology" @@ -109,28 +83,6 @@ type TopoFetcher interface { RecoveryAndGetTopo(recovery RecoveryType, oriCNCnt int) ([]string, error) } -// IsValidAutoScalerConfig return true if user config of autoscaler type is valid. -func IsValidAutoScalerConfig(typ string) bool { - t := getAutoScalerType(typ) - return t == MockASType || t == AWSASType || t == GCPASType -} - -// getAutoScalerType return topo fetcher type. -func getAutoScalerType(typ string) int { - switch typ { - case MockASStr: - return MockASType - case AWSASStr: - return AWSASType - case GCPASStr: - return GCPASType - case TestASStr: - return TestASType - default: - return InvalidASType - } -} - // InitGlobalTopoFetcher init globalTopoFetcher if is in disaggregated-tiflash mode. It's not thread-safe. func InitGlobalTopoFetcher(typ string, addr string, clusterID string, isFixedPool bool) (err error) { logutil.BgLogger().Info("init globalTopoFetcher", zap.String("type", typ), zap.String("addr", addr), @@ -139,20 +91,20 @@ func InitGlobalTopoFetcher(typ string, addr string, clusterID string, isFixedPoo return errors.Errorf("ClusterID(%s) or AutoScaler(%s) addr is empty", clusterID, addr) } - ft := getAutoScalerType(typ) + ft := config.GetAutoScalerType(typ) switch ft { - case MockASType: + case config.MockASType: globalTopoFetcher = NewMockAutoScalerFetcher(addr) - case AWSASType: + case config.AWSASType: globalTopoFetcher = NewAWSAutoScalerFetcher(addr, clusterID, isFixedPool) - case GCPASType: + case config.GCPASType: err = errors.Errorf("topo fetch not implemented yet(%s)", typ) - case TestASType: + case config.TestASType: globalTopoFetcher = NewTestAutoScalerFetcher() default: globalTopoFetcher = nil err = errors.Errorf("unexpected topo fetch type. expect: %s or %s or %s, got %s", - MockASStr, AWSASStr, GCPASStr, typ) + config.MockASStr, config.AWSASStr, config.GCPASStr, typ) } return err } diff --git a/pkg/util/tracing/BUILD.bazel b/pkg/util/tracing/BUILD.bazel index 0b88f052dd209..ffa559987510b 100644 --- a/pkg/util/tracing/BUILD.bazel +++ b/pkg/util/tracing/BUILD.bazel @@ -9,7 +9,6 @@ go_library( importpath = "github.com/pingcap/tidb/pkg/util/tracing", visibility = ["//visibility:public"], deps = [ - "//pkg/meta/model", "@com_github_opentracing_basictracer_go//:basictracer-go", "@com_github_opentracing_opentracing_go//:opentracing-go", ], @@ -27,7 +26,6 @@ go_test( embed = [":tracing"], flaky = True, deps = [ - "//pkg/meta/model", "//pkg/testkit/testsetup", "@com_github_opentracing_basictracer_go//:basictracer-go", "@com_github_opentracing_opentracing_go//:opentracing-go", diff --git a/pkg/util/tracing/util.go b/pkg/util/tracing/util.go index 4e5346a503a78..79e8a3b1fe160 100644 --- a/pkg/util/tracing/util.go +++ b/pkg/util/tracing/util.go @@ -20,7 +20,6 @@ import ( "github.com/opentracing/basictracer-go" "github.com/opentracing/opentracing-go" - "github.com/pingcap/tidb/pkg/meta/model" ) // TiDBTrace is set as Baggage on traces which are used for tidb tracing. @@ -38,6 +37,14 @@ func (cr CallbackRecorder) RecordSpan(sp basictracer.RawSpan) { cr(sp) } +// TraceInfo is the information for trace. +type TraceInfo struct { + // SessionAlias is the alias of session + SessionAlias string `json:"session_alias"` + // ConnectionID is the id of the connection + ConnectionID uint64 `json:"connection_id"` +} + // NewRecordedTrace returns a Span which records directly via the specified // callback. func NewRecordedTrace(opName string, callback func(sp basictracer.RawSpan)) opentracing.Span { @@ -129,16 +136,16 @@ func (r Region) End() { } // TraceInfoFromContext returns the `model.TraceInfo` in context -func TraceInfoFromContext(ctx context.Context) *model.TraceInfo { +func TraceInfoFromContext(ctx context.Context) *TraceInfo { val := ctx.Value(sqlTracingCtxKey) - if info, ok := val.(*model.TraceInfo); ok { + if info, ok := val.(*TraceInfo); ok { return info } return nil } // ContextWithTraceInfo creates a new `model.TraceInfo` for context -func ContextWithTraceInfo(ctx context.Context, info *model.TraceInfo) context.Context { +func ContextWithTraceInfo(ctx context.Context, info *TraceInfo) context.Context { if info == nil { return ctx } diff --git a/pkg/util/tracing/util_test.go b/pkg/util/tracing/util_test.go index 7b4e4a1a8fc5a..1d0746ca65efe 100644 --- a/pkg/util/tracing/util_test.go +++ b/pkg/util/tracing/util_test.go @@ -20,7 +20,6 @@ import ( "github.com/opentracing/basictracer-go" "github.com/opentracing/opentracing-go" - "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/util/tracing" "github.com/stretchr/testify/require" ) @@ -140,7 +139,7 @@ func TestTraceInfoFromContext(t *testing.T) { require.Equal(t, ctx, tracing.ContextWithTraceInfo(ctx, nil)) // create a context with trace info ctx, cancel := context.WithCancel(context.WithValue(ctx, "val1", "a")) - ctx = tracing.ContextWithTraceInfo(ctx, &model.TraceInfo{ConnectionID: 12345, SessionAlias: "alias1"}) + ctx = tracing.ContextWithTraceInfo(ctx, &tracing.TraceInfo{ConnectionID: 12345, SessionAlias: "alias1"}) // new context should have the same value as the original one info := tracing.TraceInfoFromContext(ctx) require.Equal(t, uint64(12345), info.ConnectionID) diff --git a/pkg/util/workloadrepo/BUILD.bazel b/pkg/util/workloadrepo/BUILD.bazel index 4ca08689831e5..b885213d5662a 100644 --- a/pkg/util/workloadrepo/BUILD.bazel +++ b/pkg/util/workloadrepo/BUILD.bazel @@ -25,6 +25,7 @@ go_library( "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/sessionctx", + "//pkg/sessionctx/vardef", "//pkg/sessionctx/variable", "//pkg/sessiontxn", "//pkg/util", diff --git a/pkg/util/workloadrepo/worker.go b/pkg/util/workloadrepo/worker.go index fac28374ca5a2..3705ff67466cd 100644 --- a/pkg/util/workloadrepo/worker.go +++ b/pkg/util/workloadrepo/worker.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/owner" "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/chunk" @@ -138,21 +139,21 @@ func init() { executor.TakeSnapshot = takeSnapshot variable.RegisterSysVar(&variable.SysVar{ - Scope: variable.ScopeGlobal, + Scope: vardef.ScopeGlobal, Name: repositoryDest, - Type: variable.TypeStr, + Type: vardef.TypeStr, Value: "", SetGlobal: func(ctx context.Context, _ *variable.SessionVars, val string) error { return workerCtx.setRepositoryDest(ctx, val) }, - Validation: func(_ *variable.SessionVars, norm, _ string, _ variable.ScopeFlag) (string, error) { + Validation: func(_ *variable.SessionVars, norm, _ string, _ vardef.ScopeFlag) (string, error) { return validateDest(norm) }, }) variable.RegisterSysVar(&variable.SysVar{ - Scope: variable.ScopeGlobal, + Scope: vardef.ScopeGlobal, Name: repositoryRetentionDays, - Type: variable.TypeInt, + Type: vardef.TypeInt, Value: strconv.Itoa(defRententionDays), MinValue: 0, MaxValue: 365, @@ -161,9 +162,9 @@ func init() { }, }) variable.RegisterSysVar(&variable.SysVar{ - Scope: variable.ScopeGlobal, + Scope: vardef.ScopeGlobal, Name: repositorySamplingInterval, - Type: variable.TypeInt, + Type: vardef.TypeInt, Value: strconv.Itoa(defSamplingInterval), MinValue: 0, MaxValue: 600, @@ -172,9 +173,9 @@ func init() { }, }) variable.RegisterSysVar(&variable.SysVar{ - Scope: variable.ScopeGlobal, + Scope: vardef.ScopeGlobal, Name: repositorySnapshotInterval, - Type: variable.TypeInt, + Type: vardef.TypeInt, Value: strconv.Itoa(defSnapshotInterval), MinValue: 900, MaxValue: 7200, diff --git a/tests/realtikvtest/BUILD.bazel b/tests/realtikvtest/BUILD.bazel index cf7046233daba..83b34842d0892 100644 --- a/tests/realtikvtest/BUILD.bazel +++ b/tests/realtikvtest/BUILD.bazel @@ -12,7 +12,7 @@ go_library( "//pkg/domain", "//pkg/kv", "//pkg/session", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/driver", "//pkg/store/mockstore", "//pkg/testkit", diff --git a/tests/realtikvtest/addindextest1/BUILD.bazel b/tests/realtikvtest/addindextest1/BUILD.bazel index e5d7b0219b485..bcbcf5a53529b 100644 --- a/tests/realtikvtest/addindextest1/BUILD.bazel +++ b/tests/realtikvtest/addindextest1/BUILD.bazel @@ -15,7 +15,7 @@ go_test( "//pkg/disttask/framework/taskexecutor", "//pkg/errno", "//pkg/meta/model", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//pkg/testkit/testfailpoint", "//tests/realtikvtest", diff --git a/tests/realtikvtest/addindextest1/disttask_test.go b/tests/realtikvtest/addindextest1/disttask_test.go index 74421c7a54183..3681002c51c9c 100644 --- a/tests/realtikvtest/addindextest1/disttask_test.go +++ b/tests/realtikvtest/addindextest1/disttask_test.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor" "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" "github.com/pingcap/tidb/tests/realtikvtest" @@ -64,10 +64,10 @@ func TestAddIndexDistBasic(t *testing.T) { tk.MustExec("use test;") tk.MustExec(`set global tidb_enable_dist_task=1;`) - bak := variable.GetDDLReorgWorkerCounter() + bak := vardef.GetDDLReorgWorkerCounter() tk.MustExec("set global tidb_ddl_reorg_worker_cnt = 111") tk.MustExec("set @@tidb_ddl_reorg_worker_cnt = 111") - require.Equal(t, int32(111), variable.GetDDLReorgWorkerCounter()) + require.Equal(t, int32(111), vardef.GetDDLReorgWorkerCounter()) tk.MustExec("create table t(a bigint auto_random primary key) partition by hash(a) partitions 20;") tk.MustExec("insert into t values (), (), (), (), (), ()") tk.MustExec("insert into t values (), (), (), (), (), ()") @@ -86,7 +86,7 @@ func TestAddIndexDistBasic(t *testing.T) { tk.MustExec(fmt.Sprintf("set global tidb_ddl_reorg_worker_cnt = %d", bak)) tk.MustExec(fmt.Sprintf("set @@tidb_ddl_reorg_worker_cnt = %d", bak)) - require.Equal(t, bak, variable.GetDDLReorgWorkerCounter()) + require.Equal(t, bak, vardef.GetDDLReorgWorkerCounter()) tk.MustExec("create table t1(a bigint auto_random primary key);") tk.MustExec("insert into t1 values (), (), (), (), (), ()") @@ -423,12 +423,12 @@ func TestAddIndexScheduleAway(t *testing.T) { tk1 := testkit.NewTestKit(t, store) tk1.MustExec("use test") updateExecID := fmt.Sprintf(` - update mysql.tidb_background_subtask set exec_id = 'other' where task_key in + update mysql.tidb_background_subtask set exec_id = 'other' where task_key in (select id from mysql.tidb_global_task where task_key like '%%%d')`, jobID.Load()) tk1.MustExec(updateExecID) <-afterCancel updateExecID = fmt.Sprintf(` - update mysql.tidb_background_subtask set exec_id = ':4000' where task_key in + update mysql.tidb_background_subtask set exec_id = ':4000' where task_key in (select id from mysql.tidb_global_task where task_key like '%%%d')`, jobID.Load()) tk1.MustExec(updateExecID) }) diff --git a/tests/realtikvtest/addindextest2/BUILD.bazel b/tests/realtikvtest/addindextest2/BUILD.bazel index 4bba0c9c929c9..ee034b4d696cb 100644 --- a/tests/realtikvtest/addindextest2/BUILD.bazel +++ b/tests/realtikvtest/addindextest2/BUILD.bazel @@ -14,7 +14,7 @@ go_test( "//pkg/kv", "//pkg/lightning/backend/external", "//pkg/meta/model", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/helper", "//pkg/tablecodec", "//pkg/testkit", diff --git a/tests/realtikvtest/addindextest2/global_sort_test.go b/tests/realtikvtest/addindextest2/global_sort_test.go index f7c212b24a9f8..cea17a1c5d289 100644 --- a/tests/realtikvtest/addindextest2/global_sort_test.go +++ b/tests/realtikvtest/addindextest2/global_sort_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/lightning/backend/external" "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/helper" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/testkit" @@ -98,7 +98,7 @@ func TestGlobalSortBasic(t *testing.T) { tk.MustExec(fmt.Sprintf(`set @@global.tidb_cloud_storage_uri = "%s"`, cloudStorageURI)) defer func() { tk.MustExec("set @@global.tidb_enable_dist_task = 0;") - variable.CloudStorageURI.Store("") + vardef.CloudStorageURI.Store("") }() tk.MustExec("create table t (a int, b int, c int);") @@ -272,7 +272,7 @@ func TestGlobalSortDuplicateErrMsg(t *testing.T) { tk.MustExec(fmt.Sprintf(`set @@global.tidb_cloud_storage_uri = "%s"`, cloudStorageURI)) defer func() { tk.MustExec("set @@global.tidb_enable_dist_task = 0;") - variable.CloudStorageURI.Store("") + vardef.CloudStorageURI.Store("") }() tk.MustExec("create table t (a int, b int, c int);") diff --git a/tests/realtikvtest/addindextest3/BUILD.bazel b/tests/realtikvtest/addindextest3/BUILD.bazel index df54d061be734..71e515772a13d 100644 --- a/tests/realtikvtest/addindextest3/BUILD.bazel +++ b/tests/realtikvtest/addindextest3/BUILD.bazel @@ -24,7 +24,7 @@ go_test( "//pkg/meta/model", "//pkg/parser/ast", "//pkg/sessionctx", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/table", "//pkg/table/tables", "//pkg/tablecodec", diff --git a/tests/realtikvtest/addindextest3/ingest_test.go b/tests/realtikvtest/addindextest3/ingest_test.go index cb1433ace2380..ff064ddc05604 100644 --- a/tests/realtikvtest/addindextest3/ingest_test.go +++ b/tests/realtikvtest/addindextest3/ingest_test.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/lightning/backend/local" "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" "github.com/pingcap/tidb/tests/realtikvtest" @@ -213,7 +213,7 @@ func TestIngestMVIndexOnPartitionTable(t *testing.T) { } func TestAddIndexIngestAdjustBackfillWorker(t *testing.T) { - if variable.EnableDistTask.Load() { + if vardef.EnableDistTask.Load() { t.Skip("dist reorg didn't support checkBackfillWorkerNum, skip this test") } store := realtikvtest.CreateMockStoreAndSetup(t) diff --git a/tests/realtikvtest/pessimistictest/BUILD.bazel b/tests/realtikvtest/pessimistictest/BUILD.bazel index 76a00174025a8..465fc74c4c1ea 100644 --- a/tests/realtikvtest/pessimistictest/BUILD.bazel +++ b/tests/realtikvtest/pessimistictest/BUILD.bazel @@ -20,7 +20,7 @@ go_test( "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/session", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/sessiontxn", "//pkg/store/driver/error", "//pkg/store/gcworker", diff --git a/tests/realtikvtest/pessimistictest/pessimistic_test.go b/tests/realtikvtest/pessimistictest/pessimistic_test.go index d4bb4bcb94b0a..477d1f336da4f 100644 --- a/tests/realtikvtest/pessimistictest/pessimistic_test.go +++ b/tests/realtikvtest/pessimistictest/pessimistic_test.go @@ -37,7 +37,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/sessiontxn" storeerr "github.com/pingcap/tidb/pkg/store/driver/error" "github.com/pingcap/tidb/pkg/store/gcworker" @@ -1615,9 +1615,9 @@ func TestGenerateColPointGet(t *testing.T) { tk.MustExec("use test") defer func() { - tk.MustExec(fmt.Sprintf("set global tidb_row_format_version = %d", variable.DefTiDBRowFormatV2)) + tk.MustExec(fmt.Sprintf("set global tidb_row_format_version = %d", vardef.DefTiDBRowFormatV2)) }() - tests2 := []int{variable.DefTiDBRowFormatV1, variable.DefTiDBRowFormatV2} + tests2 := []int{vardef.DefTiDBRowFormatV1, vardef.DefTiDBRowFormatV2} for _, rowFormat := range tests2 { tk.MustExec(fmt.Sprintf("set global tidb_row_format_version = %d", rowFormat)) tk.MustExec("drop table if exists tu") diff --git a/tests/realtikvtest/pipelineddmltest/BUILD.bazel b/tests/realtikvtest/pipelineddmltest/BUILD.bazel index c3fa442b04e22..741cce8ee817c 100644 --- a/tests/realtikvtest/pipelineddmltest/BUILD.bazel +++ b/tests/realtikvtest/pipelineddmltest/BUILD.bazel @@ -12,7 +12,7 @@ go_test( deps = [ "//pkg/config", "//pkg/kv", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/testkit", "//tests/realtikvtest", "@com_github_pingcap_failpoint//:failpoint", diff --git a/tests/realtikvtest/pipelineddmltest/pipelineddml_test.go b/tests/realtikvtest/pipelineddmltest/pipelineddml_test.go index ee79204c49567..6d578d9d5878a 100644 --- a/tests/realtikvtest/pipelineddmltest/pipelineddml_test.go +++ b/tests/realtikvtest/pipelineddmltest/pipelineddml_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/tests/realtikvtest" "github.com/stretchr/testify/require" @@ -182,12 +182,12 @@ func TestPipelinedDMLNegative(t *testing.T) { // for deprecated batch-dml tk.Session().GetSessionVars().BatchDelete = true tk.Session().GetSessionVars().DMLBatchSize = 1 - variable.EnableBatchDML.Store(true) + vardef.EnableBatchDML.Store(true) tk.MustExec("insert into t values(7, 7)") tk.MustQuery("show warnings").CheckContain("Pipelined DML can not be used with the deprecated Batch DML. Fallback to standard mode") tk.Session().GetSessionVars().BatchDelete = false tk.Session().GetSessionVars().DMLBatchSize = 0 - variable.EnableBatchDML.Store(false) + vardef.EnableBatchDML.Store(false) // for explain and explain analyze tk.MustExec("explain insert into t values(8, 8)") diff --git a/tests/realtikvtest/sessiontest/BUILD.bazel b/tests/realtikvtest/sessiontest/BUILD.bazel index e401af7c3a27d..832ddd6a77bf2 100644 --- a/tests/realtikvtest/sessiontest/BUILD.bazel +++ b/tests/realtikvtest/sessiontest/BUILD.bazel @@ -17,7 +17,7 @@ go_test( "//pkg/meta", "//pkg/parser/ast", "//pkg/session", - "//pkg/sessionctx/variable", + "//pkg/sessionctx/vardef", "//pkg/store/helper", "//pkg/testkit", "//pkg/util/sqlkiller", diff --git a/tests/realtikvtest/sessiontest/session_fail_test.go b/tests/realtikvtest/sessiontest/session_fail_test.go index 6aa93bf456c27..f3382e7bea9e8 100644 --- a/tests/realtikvtest/sessiontest/session_fail_test.go +++ b/tests/realtikvtest/sessiontest/session_fail_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/sqlkiller" "github.com/pingcap/tidb/tests/realtikvtest" @@ -36,7 +36,7 @@ func TestFailStatementCommitInRetry(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("create table t (id int)") tk.MustExec("begin") @@ -58,7 +58,7 @@ func TestGetTSFailDirtyState(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("create table t (id int)") require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/session/mockGetTSFail", "return")) @@ -92,7 +92,7 @@ func TestGetTSFailDirtyStateInretry(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("create table t (id int)") require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/session/mockCommitError", `return(true)`)) @@ -111,7 +111,7 @@ func TestKillFlagInBackoff(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("create table kill_backoff (id int)") // Inject 1 time timeout. If `Killed` is not successfully passed, it will retry and complete query. require.NoError(t, failpoint.Enable("tikvclient/tikvStoreSendReqResult", `sleep(1000)->return("timeout")->return("")`)) @@ -134,7 +134,7 @@ func TestClusterTableSendError(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn require.NoError(t, failpoint.Enable("tikvclient/tikvStoreSendReqResult", `return("requestTiDBStoreError")`)) defer func() { require.NoError(t, failpoint.Disable("tikvclient/tikvStoreSendReqResult")) }() tk.MustQuery("select * from information_schema.cluster_slow_query") @@ -147,7 +147,7 @@ func TestAutoCommitNeedNotLinearizability(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn tk.MustExec("drop table if exists t1;") defer tk.MustExec("drop table if exists t1") tk.MustExec(`create table t1 (c int)`) diff --git a/tests/realtikvtest/testkit.go b/tests/realtikvtest/testkit.go index 802205e94795e..5c74a69a061db 100644 --- a/tests/realtikvtest/testkit.go +++ b/tests/realtikvtest/testkit.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/session" - "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/vardef" "github.com/pingcap/tidb/pkg/store/driver" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" @@ -143,7 +143,7 @@ func CreateMockStoreAndDomainAndSetup(t *testing.T, opts ...mockstore.MockTiKVSt dom.InfoSyncer().SetSessionManager(&sm) tk := testkit.NewTestKit(t, store) // set it to default value. - tk.MustExec(fmt.Sprintf("set global innodb_lock_wait_timeout = %d", variable.DefInnodbLockWaitTimeout)) + tk.MustExec(fmt.Sprintf("set global innodb_lock_wait_timeout = %d", vardef.DefInnodbLockWaitTimeout)) tk.MustExec("use test") if !RetainOldData { rs := tk.MustQuery("show tables")